Add external indexes
This change include the following:
1. additional data parser for external data that parse hdfs records using Hive serdes.
2. allow users to create external data. this includes:
a) changes in metadata external dataset details.
b) addition of a new metadata index to store external file's statuses.
c) the pipeline for building the B-Tree and R-Tree indexes.
d) hyracks operators to fetch records with their RIDs using different formats.
e) hyracks operators to lookup and parse external records.
f) test cases for indexing and index access of different hdfs file formats.
g) exposing the secondary indexes over external data to the compiler.
3. adding a new aql command to refresh external datasets. this includes
a) global recovery on system startup.
b) changes in the aql parser.
c) construction of bulk modify pipelines and additional operators to perform local commit and abort operations (using 2PC protocol).
4. Added copyright header to all new files
5. Added additional test cases to test left outer join on external data
Change-Id: I1065a473299f6027eb073aeeba3a56d137f6f98e
Reviewed-on: http://fulliautomatix.ics.uci.edu:8443/70
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Ian Maxon <imaxon@uci.edu>
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 1003bc9..78e6f74 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
@@ -39,6 +39,7 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExternalDataLookupOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -446,4 +447,10 @@
public Map<LogicalVariable, LogicalVariable> getVariableMapping() {
return outVarMapping;
}
+
+ @Override
+ public ILogicalOperator visitExternalDataLookupOperator(ExternalDataLookupOperator op, ILogicalOperator arg)
+ throws AlgebricksException {
+ throw new UnsupportedOperationException();
+ }
}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java
new file mode 100644
index 0000000..4411869
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java
@@ -0,0 +1,180 @@
+/*
+ * 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.algebra.operators.physical;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.asterix.external.indexing.dataflow.HDFSLookupAdapterFactory;
+import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
+import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.declared.AqlSourceId;
+import edu.uci.ics.asterix.metadata.declared.DatasetDataSource;
+import edu.uci.ics.asterix.metadata.declared.AqlDataSource.AqlDataSourceType;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.Index;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSourcePropertiesProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExternalDataLookupOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AbstractScanPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.BroadcastPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+
+public class ExternalDataLookupPOperator extends AbstractScanPOperator {
+
+ private final List<LogicalVariable> ridVarList;
+ private AqlSourceId datasetId;
+ private Dataset dataset;
+ private ARecordType recordType;
+ private Index secondaryIndex;
+ private boolean requiresBroadcast;
+ private boolean retainInput;
+ private boolean retainNull;
+
+ public ExternalDataLookupPOperator(AqlSourceId datasetId, Dataset dataset, ARecordType recordType,
+ Index secondaryIndex, List<LogicalVariable> ridVarList, boolean requiresBroadcast, boolean retainInput, boolean retainNull) {
+ this.datasetId = datasetId;
+ this.dataset = dataset;
+ this.recordType = recordType;
+ this.secondaryIndex = secondaryIndex;
+ this.ridVarList = ridVarList;
+ this.requiresBroadcast = requiresBroadcast;
+ this.retainInput = retainInput;
+ this.retainNull = retainNull;
+ }
+
+ public Dataset getDataset() {
+ return dataset;
+ }
+
+ public void setDataset(Dataset dataset) {
+ this.dataset = dataset;
+ }
+
+ public ARecordType getRecordType() {
+ return recordType;
+ }
+
+ public void setRecordType(ARecordType recordType) {
+ this.recordType = recordType;
+ }
+
+ public AqlSourceId getDatasetId() {
+ return datasetId;
+ }
+
+ public void setDatasetId(AqlSourceId datasetId) {
+ this.datasetId = datasetId;
+ }
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.EXTERNAL_LOOKUP;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context)
+ throws AlgebricksException {
+ AqlDataSource ds = new DatasetDataSource(datasetId, datasetId.getDataverseName(), datasetId.getDatasetName(),
+ recordType, AqlDataSourceType.EXTERNAL_DATASET);
+ IDataSourcePropertiesProvider dspp = ds.getPropertiesProvider();
+ AbstractScanOperator as = (AbstractScanOperator) op;
+ deliveredProperties = dspp.computePropertiesVector(as.getVariables());
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ ExternalDataLookupOperator edabro = (ExternalDataLookupOperator) op;
+ ILogicalExpression expr = edabro.getExpressionRef().getValue();
+ if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ throw new IllegalStateException();
+ }
+ AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+ FunctionIdentifier funcIdent = funcExpr.getFunctionIdentifier();
+ if (!funcIdent.equals(AsterixBuiltinFunctions.EXTERNAL_LOOKUP)) {
+ return;
+ }
+ int[] ridIndexes = getKeyIndexes(ridVarList, inputSchemas);
+ IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(op);
+ List<LogicalVariable> outputVars = new ArrayList<LogicalVariable>();
+ if (retainInput) {
+ VariableUtilities.getLiveVariables(edabro, outputVars);
+ } else {
+ VariableUtilities.getProducedVariables(edabro, outputVars);
+ }
+
+ AqlMetadataProvider metadataProvider = (AqlMetadataProvider) context.getMetadataProvider();
+ Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> externalLoopup = HDFSLookupAdapterFactory
+ .buildExternalDataLookupRuntime(builder.getJobSpec(), dataset, secondaryIndex, ridIndexes, retainInput,
+ typeEnv, outputVars, opSchema, context, metadataProvider, retainNull);
+ builder.contributeHyracksOperator(edabro, externalLoopup.first);
+ builder.contributeAlgebricksPartitionConstraint(externalLoopup.first, externalLoopup.second);
+ ILogicalOperator srcExchange = edabro.getInputs().get(0).getValue();
+ builder.contributeGraphEdge(srcExchange, 0, edabro, 0);
+ }
+
+ protected int[] getKeyIndexes(List<LogicalVariable> keyVarList, IOperatorSchema[] inputSchemas) {
+ if (keyVarList == null) {
+ return null;
+ }
+ int[] keyIndexes = new int[keyVarList.size()];
+ for (int i = 0; i < keyVarList.size(); i++) {
+ keyIndexes[i] = inputSchemas[0].findVariable(keyVarList.get(i));
+ }
+ return keyIndexes;
+ }
+
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ if (requiresBroadcast) {
+ StructuralPropertiesVector[] pv = new StructuralPropertiesVector[1];
+ pv[0] = new StructuralPropertiesVector(new BroadcastPartitioningProperty(null), null);
+ return new PhysicalRequirements(pv, IPartitioningRequirementsCoordinator.NO_COORDINATION);
+
+ } else {
+ return super.getRequiredPropertiesForChildren(op, reqdByParent);
+ }
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return false;
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
index db070e7..706c734 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
@@ -207,7 +207,7 @@
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadataProvider
- .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataset.getDataverseName(),
+ .splitProviderAndPartitionConstraintsForDataset(dataset.getDataverseName(),
datasetName, indexName);
// TODO: Here we assume there is only one search key field.
int queryField = keyFields[0];
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 96e5c4b..d7e3ceb 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
@@ -20,6 +20,7 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExternalDataLookupOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -259,6 +260,11 @@
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/optimizer/rules/am/AccessMethodUtils.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodUtils.java
index 1691f0c..d385b82 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodUtils.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodUtils.java
@@ -22,10 +22,15 @@
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.commons.lang3.mutable.MutableObject;
+import edu.uci.ics.asterix.algebra.operators.physical.ExternalDataLookupPOperator;
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.AqlSourceId;
import edu.uci.ics.asterix.metadata.entities.Dataset;
import edu.uci.ics.asterix.metadata.entities.Index;
+import edu.uci.ics.asterix.metadata.external.IndexingConstants;
import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
import edu.uci.ics.asterix.om.base.ABoolean;
import edu.uci.ics.asterix.om.base.AInt32;
@@ -55,6 +60,7 @@
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.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExternalDataLookupOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
@@ -213,20 +219,34 @@
}
}
// Primary keys.
- List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
- for (String partitioningKey : partitioningKeys) {
+ if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+ //add primary keys
try {
- dest.add(recordType.getFieldType(partitioningKey));
- } catch (IOException e) {
+ appendExternalRecPrimaryKeys(dataset, dest);
+ } catch (AsterixException e) {
throw new AlgebricksException(e);
}
+ } else {
+ List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+ for (String partitioningKey : partitioningKeys) {
+ try {
+ dest.add(recordType.getFieldType(partitioningKey));
+ } catch (IOException e) {
+ throw new AlgebricksException(e);
+ }
+ }
}
}
public static void appendSecondaryIndexOutputVars(Dataset dataset, ARecordType recordType, Index index,
boolean primaryKeysOnly, IOptimizationContext context, List<LogicalVariable> dest)
throws AlgebricksException {
- int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
+ int numPrimaryKeys = 0;
+ if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+ numPrimaryKeys = IndexingConstants.getRIDSize(dataset);
+ } else {
+ numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
+ }
int numSecondaryKeys = getNumSecondaryKeys(index, recordType);
int numVars = (primaryKeysOnly) ? numPrimaryKeys : numPrimaryKeys + numSecondaryKeys;
for (int i = 0; i < numVars; i++) {
@@ -236,7 +256,12 @@
public static List<LogicalVariable> getPrimaryKeyVarsFromSecondaryUnnestMap(Dataset dataset,
ILogicalOperator unnestMapOp) {
- int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
+ int numPrimaryKeys;
+ if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+ numPrimaryKeys = IndexingConstants.getRIDSize(dataset);
+ } else {
+ numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
+ }
List<LogicalVariable> primaryKeyVars = new ArrayList<LogicalVariable>();
List<LogicalVariable> sourceVars = ((UnnestMapOperator) unnestMapOp).getVariables();
// Assumes the primary keys are located at the end.
@@ -431,4 +456,83 @@
//recompute type environment.
OperatorPropertiesUtil.typeOpRec(analysisCtx.getLOJGroupbyOpRef(), context);
}
+
+ // New < For external datasets indexing>
+ private static void appendExternalRecTypes(Dataset dataset, IAType itemType, List<Object> target) {
+ target.add(itemType);
+ }
+
+ private static void appendExternalRecPrimaryKeys(Dataset dataset, List<Object> target) throws AsterixException {
+ int numPrimaryKeys = IndexingConstants.getRIDSize(dataset);
+ for (int i = 0; i < numPrimaryKeys; i++) {
+ target.add(IndexingConstants.getFieldType(i));
+ }
+ }
+
+ private static void writeVarList(List<LogicalVariable> varList, List<Mutable<ILogicalExpression>> funcArgs) {
+ Mutable<ILogicalExpression> numKeysRef = new MutableObject<ILogicalExpression>(new ConstantExpression(
+ new AsterixConstantValue(new AInt32(varList.size()))));
+ funcArgs.add(numKeysRef);
+ for (LogicalVariable keyVar : varList) {
+ Mutable<ILogicalExpression> keyVarRef = new MutableObject<ILogicalExpression>(
+ new VariableReferenceExpression(keyVar));
+ funcArgs.add(keyVarRef);
+ }
+ }
+
+ private static void addStringArg(String argument, List<Mutable<ILogicalExpression>> funcArgs) {
+ Mutable<ILogicalExpression> stringRef = new MutableObject<ILogicalExpression>(new ConstantExpression(
+ new AsterixConstantValue(new AString(argument))));
+ funcArgs.add(stringRef);
+ }
+
+ public static ExternalDataLookupOperator createExternalDataLookupUnnestMap(DataSourceScanOperator dataSourceScan,
+ Dataset dataset, ARecordType recordType, ILogicalOperator inputOp, IOptimizationContext context,
+ Index secondaryIndex, boolean retainInput, boolean retainNull) throws AlgebricksException {
+ List<LogicalVariable> primaryKeyVars = AccessMethodUtils.getPrimaryKeyVarsFromSecondaryUnnestMap(dataset,
+ inputOp);
+
+ // add a sort on the RID fields before fetching external data.
+ OrderOperator order = new OrderOperator();
+ for (LogicalVariable pkVar : primaryKeyVars) {
+ Mutable<ILogicalExpression> vRef = new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
+ pkVar));
+ order.getOrderExpressions().add(
+ new Pair<IOrder, Mutable<ILogicalExpression>>(OrderOperator.ASC_ORDER, vRef));
+ }
+ // The secondary-index search feeds into the sort.
+ order.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
+ order.setExecutionMode(ExecutionMode.LOCAL);
+ context.computeAndSetTypeEnvironmentForOperator(order);
+ List<Mutable<ILogicalExpression>> externalRIDAccessFuncArgs = new ArrayList<Mutable<ILogicalExpression>>();
+ //Add dataverse and dataset to the arguments
+ AccessMethodUtils.addStringArg(dataset.getDataverseName(), externalRIDAccessFuncArgs);
+ AccessMethodUtils.addStringArg(dataset.getDatasetName(), externalRIDAccessFuncArgs);
+ AccessMethodUtils.writeVarList(primaryKeyVars, externalRIDAccessFuncArgs);
+
+ // Variables and types coming out of the external access.
+ List<LogicalVariable> externalAccessByRIDVars = new ArrayList<LogicalVariable>();
+ List<Object> externalAccessOutputTypes = new ArrayList<Object>();
+ // Append output variables/types generated by the data scan (not forwarded from input).
+ externalAccessByRIDVars.addAll(dataSourceScan.getVariables());
+ appendExternalRecTypes(dataset, recordType, externalAccessOutputTypes);
+
+ IFunctionInfo externalAccessByRID = FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.EXTERNAL_LOOKUP);
+ AbstractFunctionCallExpression externalAccessFunc = new ScalarFunctionCallExpression(externalAccessByRID,
+ externalRIDAccessFuncArgs);
+
+ ExternalDataLookupOperator externalLookupOp = new ExternalDataLookupOperator(externalAccessByRIDVars,
+ new MutableObject<ILogicalExpression>(externalAccessFunc), externalAccessOutputTypes, retainInput);
+ // Fed by the order operator or the secondaryIndexUnnestOp.
+ externalLookupOp.getInputs().add(new MutableObject<ILogicalOperator>(order));
+
+ context.computeAndSetTypeEnvironmentForOperator(externalLookupOp);
+ externalLookupOp.setExecutionMode(ExecutionMode.PARTITIONED);
+
+ //set the physical operator
+ AqlSourceId dataSourceId = new AqlSourceId(dataset.getDataverseName(), dataset.getDatasetName());
+ externalLookupOp.setPhysicalOperator(new ExternalDataLookupPOperator(dataSourceId, dataset, recordType,
+ secondaryIndex, primaryKeyVars, false, retainInput, retainNull));
+ return externalLookupOp;
+ }
}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeAccessMethod.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeAccessMethod.java
index 9fadef7..96c1b02 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeAccessMethod.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeAccessMethod.java
@@ -29,6 +29,7 @@
import edu.uci.ics.asterix.aql.util.FunctionUtils;
import edu.uci.ics.asterix.common.annotations.SkipSecondaryIndexSearchExpressionAnnotation;
+import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
import edu.uci.ics.asterix.metadata.entities.Dataset;
import edu.uci.ics.asterix.metadata.entities.Index;
@@ -52,6 +53,7 @@
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.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExternalDataLookupOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
@@ -440,7 +442,13 @@
// Generate the rest of the upstream plan which feeds the search results into the primary index.
UnnestMapOperator primaryIndexUnnestOp = null;
boolean isPrimaryIndex = chosenIndex.isPrimaryIndex();
- if (!isPrimaryIndex) {
+ if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+ // External dataset
+ ExternalDataLookupOperator externalDataAccessOp = AccessMethodUtils.createExternalDataLookupUnnestMap(
+ dataSourceScan, dataset, recordType, secondaryIndexUnnestOp, context, chosenIndex, retainInput, retainNull);
+ indexSubTree.dataSourceRef.setValue(externalDataAccessOp);
+ return externalDataAccessOp;
+ } else if (!isPrimaryIndex) {
primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceScan, dataset, recordType,
secondaryIndexUnnestOp, context, true, retainInput, retainNull, false);
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
index 5c2daa7..392f9b1 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
@@ -19,7 +19,6 @@
import org.apache.commons.lang3.mutable.Mutable;
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
import edu.uci.ics.asterix.metadata.entities.Dataset;
import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
@@ -151,9 +150,6 @@
if (dataset == null) {
throw new AlgebricksException("No metadata for dataset " + datasetName);
}
- if (dataset.getDatasetType() != DatasetType.INTERNAL) {
- return false;
- }
// Get the record type for that dataset.
IAType itemType = metadataProvider.findType(dataverseName, dataset.getItemTypeName());
if (itemType.getTypeTag() != ATypeTag.RECORD) {
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/RTreeAccessMethod.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/RTreeAccessMethod.java
index 9103178..66d06f6 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/RTreeAccessMethod.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/RTreeAccessMethod.java
@@ -22,6 +22,7 @@
import edu.uci.ics.asterix.aql.util.FunctionUtils;
import edu.uci.ics.asterix.common.annotations.SkipSecondaryIndexSearchExpressionAnnotation;
+import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
import edu.uci.ics.asterix.metadata.entities.Dataset;
import edu.uci.ics.asterix.metadata.entities.Index;
@@ -46,6 +47,7 @@
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.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExternalDataLookupOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
@@ -220,10 +222,17 @@
chosenIndex, assignSearchKeys, jobGenParams, context, false, retainInput);
// Generate the rest of the upstream plan which feeds the search results into the primary index.
- UnnestMapOperator primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceScan, dataset,
- recordType, secondaryIndexUnnestOp, context, true, retainInput, false, false);
+ if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+ ExternalDataLookupOperator externalDataAccessOp = AccessMethodUtils.createExternalDataLookupUnnestMap(
+ dataSourceScan, dataset, recordType, secondaryIndexUnnestOp, context, chosenIndex, retainInput,
+ retainNull);
+ return externalDataAccessOp;
+ } else {
+ UnnestMapOperator primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceScan,
+ dataset, recordType, secondaryIndexUnnestOp, context, true, retainInput, false, false);
- return primaryIndexUnnestOp;
+ return primaryIndexUnnestOp;
+ }
}
@Override
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AbstractAqlTranslator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AbstractAqlTranslator.java
index 1d6b1c3..f06784a 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AbstractAqlTranslator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AbstractAqlTranslator.java
@@ -48,7 +48,8 @@
public void validateOperation(Dataverse defaultDataverse, Statement stmt) throws AsterixException {
- if (!AsterixClusterProperties.INSTANCE.getState().equals(AsterixClusterProperties.State.ACTIVE)) {
+ if (!(AsterixClusterProperties.INSTANCE.getState().equals(AsterixClusterProperties.State.ACTIVE) && AsterixClusterProperties.INSTANCE
+ .isGlobalRecoveryCompleted())) {
int maxWaitCycles = AsterixAppContextInfo.getInstance().getExternalProperties().getMaxWaitClusterActive();
int waitCycleCount = 0;
try {
@@ -73,12 +74,30 @@
}
}
-
if (AsterixClusterProperties.INSTANCE.getState().equals(AsterixClusterProperties.State.UNUSABLE)) {
throw new AsterixException(" Asterix Cluster is in " + AsterixClusterProperties.State.UNUSABLE + " state."
+ "\n One or more Node Controllers have left.\n");
}
+ if (!AsterixClusterProperties.INSTANCE.isGlobalRecoveryCompleted()) {
+ int maxWaitCycles = AsterixAppContextInfo.getInstance().getExternalProperties().getMaxWaitClusterActive();
+ int waitCycleCount = 0;
+ try {
+ while (!AsterixClusterProperties.INSTANCE.isGlobalRecoveryCompleted() && waitCycleCount < maxWaitCycles) {
+ Thread.sleep(1000);
+ waitCycleCount++;
+ }
+ } catch (InterruptedException e) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Thread interrupted while waiting for cluster to complete global recovery ");
+ }
+ }
+ if (!AsterixClusterProperties.INSTANCE.isGlobalRecoveryCompleted()) {
+ throw new AsterixException(" Asterix Cluster Global recovery is not yet complete and The system is in "
+ + AsterixClusterProperties.State.ACTIVE + " state");
+ }
+ }
+
boolean invalidOperation = false;
String message = null;
String dataverse = defaultDataverse != null ? defaultDataverse.getDataverseName() : null;
diff --git a/asterix-app/data/hdfs/external-indexing-test.rc b/asterix-app/data/hdfs/external-indexing-test.rc
new file mode 100644
index 0000000..4801881
--- /dev/null
+++ b/asterix-app/data/hdfs/external-indexing-test.rc
Binary files differ
diff --git a/asterix-app/data/hdfs/external-indexing-test.seq b/asterix-app/data/hdfs/external-indexing-test.seq
new file mode 100644
index 0000000..0146c83
--- /dev/null
+++ b/asterix-app/data/hdfs/external-indexing-test.seq
Binary files differ
diff --git a/asterix-app/data/hdfs/external-indexing-test.txt b/asterix-app/data/hdfs/external-indexing-test.txt
new file mode 100644
index 0000000..555272e
--- /dev/null
+++ b/asterix-app/data/hdfs/external-indexing-test.txt
@@ -0,0 +1,11 @@
+1|Steve|50
+2|John|23
+3|Samuel|22
+4|Mary|29
+5|William|75
+6|Sarah|16
+7|Noel|33
+8|Carlos|40
+9|Joseph|45
+10|David|22
+11|Nadine|10
diff --git a/asterix-app/data/hdfs/spatialData.json b/asterix-app/data/hdfs/spatialData.json
new file mode 100644
index 0000000..9c78064
--- /dev/null
+++ b/asterix-app/data/hdfs/spatialData.json
@@ -0,0 +1,21 @@
+{"id": 1, "point": point("4.1,7.0"), "kwds": "sign ahead", "line1": line("4.0,7.0 9.0,7.0"), "line2": line("5.0,8.0 5.0,1.0"), "poly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0"), "poly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "rec": rectangle("0.0,0.0 4.2,7.0"), "circle" : circle("1.0,1.0 10.0")}
+{"id": 2, "point": point("40.2152,-75.0449"), "kwds": "factory hosedan", "line1": line("-4.0,2.0 2.0,2.0"), "line2": line("4.0,7.0 2.0,17.0"), "poly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0"), "poly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "rec": rectangle("0.0,0.0 4.0,4.0"), "circle" : circle("2.0,3.0 2.0")}
+{"id": 3, "point": point("43.5083,-79.3007"), "kwds": "enterprisecamp torcamp", "line1": line("3.0,0.0 0.0,4.0"), "line2": line("4.0,7.0 2.0,17.0"), "poly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0"), "poly2": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0"), "rec": rectangle("0.0,0.0 4.0,4.0"), "circle" : circle("5.5,1.0 10.0")}
+{"id": 4, "point": point("43.5083,-79.3007"), "kwds": "enterprisecamp torcamp", "line1": line("4.0,7.0 2.0,17.0"), "line2": line("4.0,7.0 2.0,17.0"), "poly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0 2.0,1.0 1.0,0.0"), "poly2": polygon("2.0,1.0 2.0,2.0 3.0,2.0 3.0,1.0"), "rec": rectangle("0.0,0.0 4.0,4.0"), "circle" : circle("77.0,4.0 30.0")}
+{"id": 5, "point": point("43.5083,-79.3007"), "kwds": "enterprisecamp torcamp", "line1": line("4.0,7.0 2.0,17.0"), "line2": line("4.0,7.0 2.0,17.0"), "poly1": polygon("100.0,100.0 100.0,400.0 300.0,400.0 300.0,100.0"), "poly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "rec": rectangle("0.0,0.0 4.0,4.0"), "circle" : circle("88.0,1.0 10.0")}
+{"id": 6, "point": point("43.5083,-79.3007"), "kwds": "enterprisecamp torcamp", "line1": line("0.0,5.0 1.0,7.0"), "line2": line("4.0,7.0 2.0,-17.0"), "poly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0"), "poly2": polygon("3.1,1.0 2.9,4.0 7.0,4.0 7.0,1.0"), "rec": rectangle("0.0,0.0 4.0,4.0"), "circle" : circle("1.0,1.0 10.0")}
+{"id": 7, "point": point("43.5083,-79.3007"), "kwds": "enterprisecamp torcamp", "line1": line("0.0,5.0 4.0,7.1"), "line2": line("4.0,7.0 2.0,-17.0"), "poly1": polygon("-5.0,-2.0 -4.0,-1.0 -3.0,-1.0 -2.0,-2.0 -4.0,-4.0 -5.0,-3.0"), "poly2": polygon("3.0,1.0 3.0,4.0 7.0,4.0 7.0,1.0"), "rec": rectangle("3.0,6.0 5.0,7.0"), "circle" : circle("13.0,75.0 1.0")}
+{"id": 8, "point": point("43.5083,-79.3007"), "kwds": "enterprisecamp torcamp", "line1": line("4.0,7.0 2.0,17.0"), "line2": line("4.0,7.0 2.0,17.0"), "poly1": polygon("-5.0,-2.0 -4.0,-1.0 -3.0,-1.0 -2.0,-2.0 -4.0,-4.0 -5.0,-3.0"), "poly2": polygon("-3.0,-3.0 -1.0,-3.0 -3.0,-5.0"), "rec": rectangle("0.0,0.0 4.0,4.0"), "circle" : circle("76.0,87.0 50.0")}
+{"id": 9, "point": point("5.0,1.0"), "kwds": "sign ahead", "line1": line("5.0,1.0 5.0,4.0"), "line2": line("5.0,8.0 5.0,1.0"), "poly1": polygon("1.0,1.0 1.0,4.0 3.0,4.0 3.0,1.0"), "poly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "rec": rectangle("0.0,0.0 4.0,4.0"), "circle" : circle("11.0,14.0 15.0")}
+{"id": 10, "point": point("2.0,3.0"), "kwds": "sign ahead", "line1": line("1.0,2.0 3.0,4.0"), "line2": line("5.0,8.0 5.0,1.0"), "poly1": polygon("6.01,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "poly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "rec": rectangle("0.0,0.0 4.0,4.0"), "circle" : circle("1.0,76.0 17.0")}
+{"id": 11, "point": point("4.9,0.0"), "kwds": "sign ahead", "line1": line("1.0,2.0 3.0,4.0"), "line2": line("5.0,8.0 5.0,1.0"), "poly1": polygon("4.9,0.1 4.9,4.0 12.0,4.0 12.0,1.0"), "poly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "rec": rectangle("0.0,0.0 4.0,4.0"), "circle" : circle("22.0,35.0 144.0")}
+{"id": 12, "point": point("6.0,3.0"), "kwds": "sign ahead", "line1": line("1.0,2.0 3.0,4.0"), "line2": line("5.0,8.0 5.0,1.0"), "poly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "poly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "rec": rectangle("0.0,0.0 4.0,4.0"), "circle" : circle("1.0,23.0 12.0")}
+{"id": 13, "point": point("5.0,5.0"), "kwds": "sign ahead", "line1": line("1.0,2.0 3.0,4.0"), "line2": line("5.0,8.0 5.0,1.0"), "poly1": polygon("6.0,1.0 6.0,4.0 12.0,4.0 12.0,1.0"), "poly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "rec": rectangle("0.0,0.0 4.0,4.0"), "circle" : circle("30.0,11.0 11.0")}
+{"id": 14, "point": point("5.1,5.1"), "kwds": "sign ahead", "line1": line("1.0,2.0 3.0,4.0"), "line2": line("5.0,8.0 5.0,1.0"), "poly1": polygon("5.0,1.0 5.0,4.0 12.0,4.0 12.0,1.0"), "poly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "rec": rectangle("0.0,0.0 4.0,4.0"), "circle" : circle("1.0,66.0 17.0")}
+{"id": 15, "point": point("-2.0,3.0"), "kwds": "sign ahead", "line1": line("1.0,2.0 3.0,4.0"), "line2": line("5.0,8.0 5.0,1.0"), "poly1": polygon("5.1,1.0 5.1,4.0 12.0,4.0 12.0,1.0"), "poly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "rec": rectangle("0.0,0.0 4.0,4.0"), "circle" : circle("12.0,87.0 10.0")}
+{"id": 16, "point": point("-2.0,3.0"), "kwds": "sign ahead", "line1": line("1.0,2.0 3.0,4.0"), "line2": line("5.0,8.0 5.0,1.0"), "poly1": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "poly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "rec": rectangle("0.0,0.0 4.0,4.0"), "circle" : circle("1.0,35.0 10.0")}
+{"id": 17, "point": point("4.1,7.0"), "kwds": "sign ahead", "line1": line("4.0,7.0 9.0,7.0"), "line2": line("5.0,8.0 5.0,1.0"), "poly1": polygon("0.0,6.0 0.0,0.0 3.0,0.0 4.0,1.0 6.0,1.0 8.0,0.0 12.0,0.0 13.0,2.0 8.0,2.0 8.0,4.0 11.0,4.0 11.0,6.0 6.0,6.0 4.0,3.0 2.0,6.0"), "poly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "rec": rectangle("0.0,0.0 4.0,4.0"), "circle" : circle("1.0,51.0 10.0")}
+{"id": 18, "point": point("-2.0,3.0"), "kwds": "sign ahead", "line1": line("1.0,2.0 3.0,4.0"), "line2": line("5.0,8.0 5.0,1.0"), "poly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "poly2": polygon("6.0,3.0 7.0,5.0 6.0,7.0 5.0,5.0"), "rec": rectangle("0.0,0.0 4.0,4.0"), "circle" : circle("43.0,45.0 12.0")}
+{"id": 19, "point": point("-2.0,3.0"), "kwds": "sign ahead", "line1": line("1.0,2.0 3.0,4.0"), "line2": line("5.0,8.0 5.0,1.0"), "poly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "poly2": polygon("6.0,1.0 7.0,5.0 6.0,7.0 5.0,5.0"), "rec": rectangle("0.0,0.0 4.0,4.0"), "circle" : circle("65.0,2.0 13.0")}
+{"id": 20, "point": point("4.0,3.0"), "kwds": "sign ahead", "line1": line("20.0,20.0 30.0,40.0"), "line2": line("5.0,8.0 0.0,1.0"), "poly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "poly2": polygon("50.0,10.0 50.0,40.0 70.0,40.0 70.0,10.0"), "rec": rectangle("0.0,0.0 4.0,4.0"), "circle" : circle("1.0,23.0 12.0")}
+{"id": 21, "point": point("0.0,5.0"), "kwds": "sign ahead", "line1": line("0.0,5.0 0.0,40.0"), "line2": line("5.0,8.0 0.0,1.0"), "poly1": polygon("5.1,5.1 14.0,14.0 22.0,14.0 22.0,10.0"), "poly2": polygon("50.0,10.0 50.0,40.0 70.0,40.0 70.0,10.0"), "rec": rectangle("0.0,0.0 5.1,5.1"), "circle" : circle("1.0,23.0 12.0")}
\ No newline at end of file
diff --git a/asterix-app/data/hdfs/tw_for_indexleftouterjoin.adm b/asterix-app/data/hdfs/tw_for_indexleftouterjoin.adm
new file mode 100644
index 0000000..405dd08
--- /dev/null
+++ b/asterix-app/data/hdfs/tw_for_indexleftouterjoin.adm
@@ -0,0 +1,250 @@
+{ "tweetid": 1i64, "user": { "screen-name": "WardLoewentsein@340", "lang": "en", "friends-count": 11, "statuses-count": 388, "name": "Ward Loewentsein", "followers-count": 129 }, "sender-location": point("42.83,72.44"), "send-time": datetime("2009-10-21T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "speed" }}, "message-text": " love t-mobile the speed is mind-blowing:)", "countA": 1, "countB": 26 }
+{ "tweetid": 2i64, "user": { "screen-name": "KyleGraham_120", "lang": "en", "friends-count": 55, "statuses-count": 231, "name": "Kyle Graham", "followers-count": 42 }, "sender-location": point("34.81,72.44"), "send-time": datetime("2011-09-23T10:10:00.000Z"), "referred-topics": {{ "samsung", "3G" }}, "message-text": " hate samsung the 3G is horrible", "countA": 2, "countB": 131 }
+{ "tweetid": 3i64, "user": { "screen-name": "TateGarneys@542", "lang": "en", "friends-count": 74, "statuses-count": 370, "name": "Tate Garneys", "followers-count": 111 }, "sender-location": point("24.54,82.66"), "send-time": datetime("2009-12-07T10:10:00.000Z"), "referred-topics": {{ "iphone", "shortcut-menu" }}, "message-text": " hate iphone the shortcut-menu is bad", "countA": 3, "countB": 187 }
+{ "tweetid": 4i64, "user": { "screen-name": "BuckFields@708", "lang": "en", "friends-count": 20, "statuses-count": 469, "name": "Buck Fields", "followers-count": 181 }, "sender-location": point("38.14,68.1"), "send-time": datetime("2008-10-24T10:10:00.000Z"), "referred-topics": {{ "samsung", "speed" }}, "message-text": " dislike samsung the speed is OMG", "countA": 4, "countB": 52 }
+{ "tweetid": 5i64, "user": { "screen-name": "NoreenBaldwin_373", "lang": "en", "friends-count": 89, "statuses-count": 144, "name": "Noreen Baldwin", "followers-count": 187 }, "sender-location": point("35.4,68.89"), "send-time": datetime("2008-10-05T10:10:00.000Z"), "referred-topics": {{ "motorola", "3G" }}, "message-text": " hate motorola its 3G is OMG:(", "countA": 5, "countB": 35 }
+{ "tweetid": 6i64, "user": { "screen-name": "IselaHatcher_237", "lang": "en", "friends-count": 85, "statuses-count": 333, "name": "Isela Hatcher", "followers-count": 148 }, "sender-location": point("42.75,78.5"), "send-time": datetime("2011-10-15T10:10:00.000Z"), "referred-topics": {{ "sprint", "wireless" }}, "message-text": " hate sprint the wireless is terrible:(", "countA": 6, "countB": 61 }
+{ "tweetid": 7i64, "user": { "screen-name": "NicolaJolce$660", "lang": "en", "friends-count": 45, "statuses-count": 420, "name": "Nicola Jolce", "followers-count": 12 }, "sender-location": point("48.16,71.59"), "send-time": datetime("2005-11-23T10:10:00.000Z"), "referred-topics": {{ "motorola", "voice-command" }}, "message-text": " like motorola its voice-command is amazing", "countA": 7, "countB": 47 }
+{ "tweetid": 8i64, "user": { "screen-name": "MorganKeppel_176", "lang": "en", "friends-count": 74, "statuses-count": 190, "name": "Morgan Keppel", "followers-count": 2 }, "sender-location": point("36.17,72.56"), "send-time": datetime("2011-12-02T10:10:00.000Z"), "referred-topics": {{ "verizon", "3G" }}, "message-text": " hate verizon the 3G is OMG:(", "countA": 8, "countB": 98 }
+{ "tweetid": 9i64, "user": { "screen-name": "GerardMcdonald$43", "lang": "en", "friends-count": 72, "statuses-count": 151, "name": "Gerard Mcdonald", "followers-count": 96 }, "sender-location": point("38.02,70.38"), "send-time": datetime("2005-10-01T10:10:00.000Z"), "referred-topics": {{ "sprint", "voice-clarity" }}, "message-text": " love sprint its voice-clarity is amazing", "countA": 9, "countB": 69 }
+{ "tweetid": 10i64, "user": { "screen-name": "WynonnaButler_286", "lang": "en", "friends-count": 30, "statuses-count": 375, "name": "Wynonna Butler", "followers-count": 78 }, "sender-location": point("38.71,90.05"), "send-time": datetime("2008-09-21T10:10:00.000Z"), "referred-topics": {{ "motorola", "wireless" }}, "message-text": " love motorola its wireless is good:)", "countA": 10, "countB": 75 }
+{ "tweetid": 11i64, "user": { "screen-name": "BrodyKing@977", "lang": "en", "friends-count": 3, "statuses-count": 62, "name": "Brody King", "followers-count": 106 }, "sender-location": point("32.26,73.48"), "send-time": datetime("2007-05-20T10:10:00.000Z"), "referred-topics": {{ "sprint", "shortcut-menu" }}, "message-text": " can't stand sprint the shortcut-menu is OMG", "countA": 11, "countB": 28 }
+{ "tweetid": 12i64, "user": { "screen-name": "ValentineSchofield@448", "lang": "en", "friends-count": 16, "statuses-count": 260, "name": "Valentine Schofield", "followers-count": 136 }, "sender-location": point("24.99,70.66"), "send-time": datetime("2009-07-26T10:10:00.000Z"), "referred-topics": {{ "sprint", "shortcut-menu" }}, "message-text": " can't stand sprint its shortcut-menu is terrible", "countA": 12, "countB": 159 }
+{ "tweetid": 13i64, "user": { "screen-name": "MaryroseBennett#483", "lang": "en", "friends-count": 99, "statuses-count": 496, "name": "Maryrose Bennett", "followers-count": 57 }, "sender-location": point("38.94,93.31"), "send-time": datetime("2005-01-06T10:10:00.000Z"), "referred-topics": {{ "at&t", "platform" }}, "message-text": " love at&t its platform is awesome", "countA": 13, "countB": 13 }
+{ "tweetid": 14i64, "user": { "screen-name": "GarlandAlliman$490", "lang": "en", "friends-count": 23, "statuses-count": 146, "name": "Garland Alliman", "followers-count": 51 }, "sender-location": point("29.96,75.0"), "send-time": datetime("2009-09-26T10:10:00.000Z"), "referred-topics": {{ "motorola", "platform" }}, "message-text": " dislike motorola the platform is bad:(", "countA": 14, "countB": 74 }
+{ "tweetid": 15i64, "user": { "screen-name": "AnnabelPirl$171", "lang": "en", "friends-count": 43, "statuses-count": 402, "name": "Annabel Pirl", "followers-count": 137 }, "sender-location": point("27.22,86.32"), "send-time": datetime("2008-09-08T10:10:00.000Z"), "referred-topics": {{ "at&t", "network" }}, "message-text": " like at&t the network is amazing:)", "countA": 15, "countB": 19 }
+{ "tweetid": 16i64, "user": { "screen-name": "JarvisPickering@42", "lang": "en", "friends-count": 98, "statuses-count": 498, "name": "Jarvis Pickering", "followers-count": 20 }, "sender-location": point("30.54,81.6"), "send-time": datetime("2005-11-05T10:10:00.000Z"), "referred-topics": {{ "iphone", "wireless" }}, "message-text": " love iphone the wireless is awesome", "countA": 16, "countB": 27 }
+{ "tweetid": 17i64, "user": { "screen-name": "LillyHoffhants@595", "lang": "en", "friends-count": 35, "statuses-count": 391, "name": "Lilly Hoffhants", "followers-count": 129 }, "sender-location": point("47.96,95.21"), "send-time": datetime("2007-02-14T10:10:00.000Z"), "referred-topics": {{ "verizon", "customer-service" }}, "message-text": " like verizon its customer-service is amazing", "countA": 17, "countB": 55 }
+{ "tweetid": 18i64, "user": { "screen-name": "AllanPolson_455", "lang": "en", "friends-count": 36, "statuses-count": 227, "name": "Allan Polson", "followers-count": 113 }, "sender-location": point("46.39,73.85"), "send-time": datetime("2009-12-14T10:10:00.000Z"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " hate iphone the platform is bad", "countA": 18, "countB": 199 }
+{ "tweetid": 19i64, "user": { "screen-name": "DonnieWentzel#857", "lang": "en", "friends-count": 76, "statuses-count": 23, "name": "Donnie Wentzel", "followers-count": 78 }, "sender-location": point("37.21,95.76"), "send-time": datetime("2012-04-10T10:10:00.000Z"), "referred-topics": {{ "iphone", "reachability" }}, "message-text": " like iphone the reachability is mind-blowing", "countA": 19, "countB": 68 }
+{ "tweetid": 20i64, "user": { "screen-name": "TraversFaast@428", "lang": "en", "friends-count": 42, "statuses-count": 70, "name": "Travers Faast", "followers-count": 116 }, "sender-location": point("26.54,74.71"), "send-time": datetime("2010-06-14T10:10:00.000Z"), "referred-topics": {{ "iphone", "shortcut-menu" }}, "message-text": " like iphone the shortcut-menu is mind-blowing", "countA": 20, "countB": 18 }
+{ "tweetid": 21i64, "user": { "screen-name": "KameronSandford#555", "lang": "en", "friends-count": 22, "statuses-count": 104, "name": "Kameron Sandford", "followers-count": 17 }, "sender-location": point("25.8,88.76"), "send-time": datetime("2007-11-20T10:10:00.000Z"), "referred-topics": {{ "at&t", "network" }}, "message-text": " can't stand at&t the network is bad:(", "countA": 21, "countB": 64 }
+{ "tweetid": 22i64, "user": { "screen-name": "TiannaArmitage_372", "lang": "en", "friends-count": 32, "statuses-count": 307, "name": "Tianna Armitage", "followers-count": 126 }, "sender-location": point("45.79,80.75"), "send-time": datetime("2005-04-14T10:10:00.000Z"), "referred-topics": {{ "motorola", "reachability" }}, "message-text": " can't stand motorola its reachability is terrible:(", "countA": 22, "countB": 126 }
+{ "tweetid": 23i64, "user": { "screen-name": "NevadaCattley#858", "lang": "en", "friends-count": 74, "statuses-count": 389, "name": "Nevada Cattley", "followers-count": 17 }, "sender-location": point("43.6,93.24"), "send-time": datetime("2007-03-11T10:10:00.000Z"), "referred-topics": {{ "at&t", "network" }}, "message-text": " hate at&t the network is horrible", "countA": 23, "countB": 74 }
+{ "tweetid": 24i64, "user": { "screen-name": "ReannonEisenhart#637", "lang": "en", "friends-count": 34, "statuses-count": 235, "name": "Reannon Eisenhart", "followers-count": 145 }, "sender-location": point("47.56,76.49"), "send-time": datetime("2012-03-20T10:10:00.000Z"), "referred-topics": {{ "samsung", "voice-clarity" }}, "message-text": " can't stand samsung its voice-clarity is terrible:(", "countA": 24, "countB": 70 }
+{ "tweetid": 25i64, "user": { "screen-name": "LillyMang#928", "lang": "en", "friends-count": 20, "statuses-count": 40, "name": "Lilly Mang", "followers-count": 47 }, "sender-location": point("38.68,94.93"), "send-time": datetime("2006-04-21T10:10:00.000Z"), "referred-topics": {{ "sprint", "customization" }}, "message-text": " love sprint its customization is awesome", "countA": 25, "countB": 86 }
+{ "tweetid": 26i64, "user": { "screen-name": "MicaBusk$903", "lang": "en", "friends-count": 87, "statuses-count": 164, "name": "Mica Busk", "followers-count": 92 }, "sender-location": point("45.47,90.97"), "send-time": datetime("2008-01-24T10:10:00.000Z"), "referred-topics": {{ "motorola", "touch-screen" }}, "message-text": " hate motorola the touch-screen is terrible", "countA": 26, "countB": 11 }
+{ "tweetid": 27i64, "user": { "screen-name": "PiaHildyard_915", "lang": "en", "friends-count": 92, "statuses-count": 302, "name": "Pia Hildyard", "followers-count": 16 }, "sender-location": point("43.76,68.58"), "send-time": datetime("2007-08-10T10:10:00.000Z"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " dislike iphone its platform is bad:(", "countA": 27, "countB": 125 }
+{ "tweetid": 28i64, "user": { "screen-name": "CamelliaSiegrist_676", "lang": "en", "friends-count": 73, "statuses-count": 392, "name": "Camellia Siegrist", "followers-count": 193 }, "sender-location": point("24.94,77.95"), "send-time": datetime("2007-12-26T10:10:00.000Z"), "referred-topics": {{ "verizon", "voice-command" }}, "message-text": " can't stand verizon its voice-command is bad", "countA": 28, "countB": 123 }
+{ "tweetid": 29i64, "user": { "screen-name": "BurtTaggart_922", "lang": "en", "friends-count": 49, "statuses-count": 62, "name": "Burt Taggart", "followers-count": 134 }, "sender-location": point("35.67,97.43"), "send-time": datetime("2011-07-21T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "signal" }}, "message-text": " love t-mobile the signal is awesome", "countA": 29, "countB": 115 }
+{ "tweetid": 30i64, "user": { "screen-name": "MarlaHill@215", "lang": "en", "friends-count": 84, "statuses-count": 305, "name": "Marla Hill", "followers-count": 71 }, "sender-location": point("24.29,84.28"), "send-time": datetime("2012-07-03T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "3G" }}, "message-text": " like t-mobile the 3G is awesome", "countA": 30, "countB": 36 }
+{ "tweetid": 31i64, "user": { "screen-name": "CaitlynChristman@452", "lang": "en", "friends-count": 57, "statuses-count": 414, "name": "Caitlyn Christman", "followers-count": 67 }, "sender-location": point("41.04,85.13"), "send-time": datetime("2005-11-05T10:10:00.000Z"), "referred-topics": {{ "samsung", "voicemail-service" }}, "message-text": " love samsung the voicemail-service is mind-blowing:)", "countA": 31, "countB": 59 }
+{ "tweetid": 32i64, "user": { "screen-name": "BraxtonBonner#527", "lang": "en", "friends-count": 21, "statuses-count": 427, "name": "Braxton Bonner", "followers-count": 168 }, "sender-location": point("34.25,86.09"), "send-time": datetime("2011-10-07T10:10:00.000Z"), "referred-topics": {{ "motorola", "3G" }}, "message-text": " dislike motorola its 3G is horrible:(", "countA": 32, "countB": 38 }
+{ "tweetid": 33i64, "user": { "screen-name": "WilmaSouthern@238", "lang": "en", "friends-count": 83, "statuses-count": 413, "name": "Wilma Southern", "followers-count": 24 }, "sender-location": point("34.71,69.57"), "send-time": datetime("2005-02-19T10:10:00.000Z"), "referred-topics": {{ "sprint", "wireless" }}, "message-text": " dislike sprint the wireless is OMG:(", "countA": 33, "countB": 105 }
+{ "tweetid": 34i64, "user": { "screen-name": "MaxNash$802", "lang": "en", "friends-count": 13, "statuses-count": 189, "name": "Max Nash", "followers-count": 39 }, "sender-location": point("48.12,89.23"), "send-time": datetime("2012-02-17T10:10:00.000Z"), "referred-topics": {{ "motorola", "touch-screen" }}, "message-text": " dislike motorola its touch-screen is horrible", "countA": 34, "countB": 185 }
+{ "tweetid": 35i64, "user": { "screen-name": "HannahWarrick_843", "lang": "en", "friends-count": 14, "statuses-count": 10, "name": "Hannah Warrick", "followers-count": 2 }, "sender-location": point("32.75,69.94"), "send-time": datetime("2007-09-14T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "plan" }}, "message-text": " like t-mobile the plan is amazing:)", "countA": 35, "countB": 176 }
+{ "tweetid": 36i64, "user": { "screen-name": "SherikaBarth#732", "lang": "en", "friends-count": 80, "statuses-count": 277, "name": "Sherika Barth", "followers-count": 138 }, "sender-location": point("34.85,66.87"), "send-time": datetime("2011-10-13T10:10:00.000Z"), "referred-topics": {{ "at&t", "plan" }}, "message-text": " can't stand at&t the plan is OMG", "countA": 36, "countB": 147 }
+{ "tweetid": 37i64, "user": { "screen-name": "SabinaCattley$355", "lang": "en", "friends-count": 67, "statuses-count": 20, "name": "Sabina Cattley", "followers-count": 104 }, "sender-location": point("40.22,71.18"), "send-time": datetime("2007-04-06T10:10:00.000Z"), "referred-topics": {{ "samsung", "voice-clarity" }}, "message-text": " can't stand samsung its voice-clarity is bad:(", "countA": 37, "countB": 55 }
+{ "tweetid": 38i64, "user": { "screen-name": "KimberlyVeith$848", "lang": "en", "friends-count": 43, "statuses-count": 274, "name": "Kimberly Veith", "followers-count": 14 }, "sender-location": point("34.94,83.17"), "send-time": datetime("2011-07-15T10:10:00.000Z"), "referred-topics": {{ "samsung", "speed" }}, "message-text": " hate samsung the speed is horrible:(", "countA": 38, "countB": 89 }
+{ "tweetid": 39i64, "user": { "screen-name": "AdrianneMackendoerfer_478", "lang": "en", "friends-count": 2, "statuses-count": 125, "name": "Adrianne Mackendoerfer", "followers-count": 113 }, "sender-location": point("40.14,78.49"), "send-time": datetime("2010-10-19T10:10:00.000Z"), "referred-topics": {{ "motorola", "voice-command" }}, "message-text": " hate motorola its voice-command is OMG", "countA": 39, "countB": 97 }
+{ "tweetid": 40i64, "user": { "screen-name": "MunroWire@995", "lang": "en", "friends-count": 89, "statuses-count": 336, "name": "Munro Wire", "followers-count": 181 }, "sender-location": point("30.94,80.83"), "send-time": datetime("2009-05-07T10:10:00.000Z"), "referred-topics": {{ "verizon", "network" }}, "message-text": " love verizon the network is good", "countA": 40, "countB": 193 }
+{ "tweetid": 41i64, "user": { "screen-name": "AmadaAft@648", "lang": "en", "friends-count": 50, "statuses-count": 127, "name": "Amada Aft", "followers-count": 20 }, "sender-location": point("32.88,81.46"), "send-time": datetime("2010-04-16T10:10:00.000Z"), "referred-topics": {{ "iphone", "customer-service" }}, "message-text": " can't stand iphone its customer-service is OMG:(", "countA": 41, "countB": 169 }
+{ "tweetid": 42i64, "user": { "screen-name": "SalenaMcfall_717", "lang": "en", "friends-count": 30, "statuses-count": 93, "name": "Salena Mcfall", "followers-count": 184 }, "sender-location": point("47.86,71.93"), "send-time": datetime("2010-02-28T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "network" }}, "message-text": " can't stand t-mobile its network is bad", "countA": 42, "countB": 130 }
+{ "tweetid": 43i64, "user": { "screen-name": "JeniferCanham$317", "lang": "en", "friends-count": 63, "statuses-count": 344, "name": "Jenifer Canham", "followers-count": 132 }, "sender-location": point("25.68,81.87"), "send-time": datetime("2010-04-22T10:10:00.000Z"), "referred-topics": {{ "motorola", "3G" }}, "message-text": " hate motorola the 3G is OMG:(", "countA": 43, "countB": 153 }
+{ "tweetid": 44i64, "user": { "screen-name": "NannieBender$656", "lang": "en", "friends-count": 26, "statuses-count": 84, "name": "Nannie Bender", "followers-count": 184 }, "sender-location": point("47.46,85.04"), "send-time": datetime("2007-06-08T10:10:00.000Z"), "referred-topics": {{ "samsung", "voice-command" }}, "message-text": " dislike samsung its voice-command is bad", "countA": 44, "countB": 12 }
+{ "tweetid": 45i64, "user": { "screen-name": "ThaoKooser@875", "lang": "en", "friends-count": 60, "statuses-count": 289, "name": "Thao Kooser", "followers-count": 8 }, "sender-location": point("37.02,87.94"), "send-time": datetime("2005-11-28T10:10:00.000Z"), "referred-topics": {{ "verizon", "network" }}, "message-text": " like verizon its network is amazing:)", "countA": 45, "countB": 151 }
+{ "tweetid": 46i64, "user": { "screen-name": "AugustaBaumgartner_385", "lang": "en", "friends-count": 17, "statuses-count": 70, "name": "Augusta Baumgartner", "followers-count": 162 }, "sender-location": point("24.83,73.16"), "send-time": datetime("2008-09-23T10:10:00.000Z"), "referred-topics": {{ "verizon", "network" }}, "message-text": " like verizon its network is mind-blowing", "countA": 46, "countB": 37 }
+{ "tweetid": 47i64, "user": { "screen-name": "OtisHill_124", "lang": "en", "friends-count": 46, "statuses-count": 68, "name": "Otis Hill", "followers-count": 29 }, "sender-location": point("36.01,86.76"), "send-time": datetime("2011-05-16T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "touch-screen" }}, "message-text": " dislike t-mobile the touch-screen is horrible:(", "countA": 47, "countB": 27 }
+{ "tweetid": 48i64, "user": { "screen-name": "ZolaJudge@572", "lang": "en", "friends-count": 8, "statuses-count": 39, "name": "Zola Judge", "followers-count": 36 }, "sender-location": point("42.67,91.8"), "send-time": datetime("2009-03-06T10:10:00.000Z"), "referred-topics": {{ "motorola", "network" }}, "message-text": " like motorola the network is awesome", "countA": 48, "countB": 16 }
+{ "tweetid": 49i64, "user": { "screen-name": "ChristianaWeisgarber$35", "lang": "en", "friends-count": 40, "statuses-count": 427, "name": "Christiana Weisgarber", "followers-count": 20 }, "sender-location": point("36.91,86.0"), "send-time": datetime("2010-04-18T10:10:00.000Z"), "referred-topics": {{ "sprint", "plan" }}, "message-text": " hate sprint its plan is terrible", "countA": 49, "countB": 28 }
+{ "tweetid": 50i64, "user": { "screen-name": "MollyGarneis_210", "lang": "en", "friends-count": 6, "statuses-count": 453, "name": "Molly Garneis", "followers-count": 185 }, "sender-location": point("44.42,87.86"), "send-time": datetime("2012-04-28T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "wireless" }}, "message-text": " love t-mobile its wireless is good:)", "countA": 50, "countB": 123 }
+{ "tweetid": 51i64, "user": { "screen-name": "HarlanHanseu#420", "lang": "en", "friends-count": 66, "statuses-count": 295, "name": "Harlan Hanseu", "followers-count": 99 }, "sender-location": point("37.65,70.54"), "send-time": datetime("2006-05-11T10:10:00.000Z"), "referred-topics": {{ "samsung", "speed" }}, "message-text": " can't stand samsung the speed is terrible:(", "countA": 51, "countB": 94 }
+{ "tweetid": 52i64, "user": { "screen-name": "DelorseSloan#229", "lang": "en", "friends-count": 84, "statuses-count": 287, "name": "Delorse Sloan", "followers-count": 20 }, "sender-location": point("27.12,78.69"), "send-time": datetime("2011-02-27T10:10:00.000Z"), "referred-topics": {{ "motorola", "touch-screen" }}, "message-text": " dislike motorola its touch-screen is horrible:(", "countA": 52, "countB": 156 }
+{ "tweetid": 53i64, "user": { "screen-name": "MylesEwing@54", "lang": "en", "friends-count": 45, "statuses-count": 411, "name": "Myles Ewing", "followers-count": 23 }, "sender-location": point("25.82,97.9"), "send-time": datetime("2007-11-10T10:10:00.000Z"), "referred-topics": {{ "at&t", "network" }}, "message-text": " like at&t its network is mind-blowing:)", "countA": 53, "countB": 174 }
+{ "tweetid": 54i64, "user": { "screen-name": "OprahClark_160", "lang": "en", "friends-count": 26, "statuses-count": 299, "name": "Oprah Clark", "followers-count": 161 }, "sender-location": point("36.1,87.24"), "send-time": datetime("2010-04-22T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile its shortcut-menu is good", "countA": 54, "countB": 128 }
+{ "tweetid": 55i64, "user": { "screen-name": "CoreyRichards#130", "lang": "en", "friends-count": 45, "statuses-count": 420, "name": "Corey Richards", "followers-count": 102 }, "sender-location": point("42.77,72.16"), "send-time": datetime("2012-07-20T10:10:00.000Z"), "referred-topics": {{ "at&t", "customer-service" }}, "message-text": " can't stand at&t its customer-service is horrible", "countA": 55, "countB": 30 }
+{ "tweetid": 56i64, "user": { "screen-name": "GwendolenHahn#673", "lang": "en", "friends-count": 62, "statuses-count": 426, "name": "Gwendolen Hahn", "followers-count": 158 }, "sender-location": point("39.76,90.94"), "send-time": datetime("2009-05-10T10:10:00.000Z"), "referred-topics": {{ "samsung", "3G" }}, "message-text": " dislike samsung its 3G is bad:(", "countA": 56, "countB": 50 }
+{ "tweetid": 57i64, "user": { "screen-name": "DewayneBallou@258", "lang": "en", "friends-count": 42, "statuses-count": 215, "name": "Dewayne Ballou", "followers-count": 85 }, "sender-location": point("28.45,75.02"), "send-time": datetime("2006-08-07T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "speed" }}, "message-text": " love t-mobile the speed is mind-blowing:)", "countA": 57, "countB": 61 }
+{ "tweetid": 58i64, "user": { "screen-name": "RenayReese@543", "lang": "en", "friends-count": 80, "statuses-count": 459, "name": "Renay Reese", "followers-count": 102 }, "sender-location": point("38.09,77.66"), "send-time": datetime("2007-12-07T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " love motorola the speed is awesome:)", "countA": 58, "countB": 35 }
+{ "tweetid": 59i64, "user": { "screen-name": "SamuelHoffhants@740", "lang": "en", "friends-count": 53, "statuses-count": 64, "name": "Samuel Hoffhants", "followers-count": 150 }, "sender-location": point("48.86,97.19"), "send-time": datetime("2009-11-27T10:10:00.000Z"), "referred-topics": {{ "sprint", "speed" }}, "message-text": " dislike sprint its speed is horrible", "countA": 59, "countB": 95 }
+{ "tweetid": 60i64, "user": { "screen-name": "GarlandOneal@886", "lang": "en", "friends-count": 12, "statuses-count": 481, "name": "Garland Oneal", "followers-count": 13 }, "sender-location": point("37.6,75.74"), "send-time": datetime("2005-05-27T10:10:00.000Z"), "referred-topics": {{ "sprint", "touch-screen" }}, "message-text": " hate sprint its touch-screen is OMG:(", "countA": 60, "countB": 2 }
+{ "tweetid": 61i64, "user": { "screen-name": "LukeHoopengarner@327", "lang": "en", "friends-count": 77, "statuses-count": 224, "name": "Luke Hoopengarner", "followers-count": 107 }, "sender-location": point("46.38,80.88"), "send-time": datetime("2006-11-13T10:10:00.000Z"), "referred-topics": {{ "sprint", "voicemail-service" }}, "message-text": " can't stand sprint its voicemail-service is bad:(", "countA": 61, "countB": 12 }
+{ "tweetid": 62i64, "user": { "screen-name": "AudieStahl@296", "lang": "en", "friends-count": 89, "statuses-count": 90, "name": "Audie Stahl", "followers-count": 74 }, "sender-location": point("32.48,96.01"), "send-time": datetime("2010-04-23T10:10:00.000Z"), "referred-topics": {{ "at&t", "platform" }}, "message-text": " can't stand at&t its platform is terrible:(", "countA": 62, "countB": 47 }
+{ "tweetid": 63i64, "user": { "screen-name": "ArielleErrett_963", "lang": "en", "friends-count": 15, "statuses-count": 385, "name": "Arielle Errett", "followers-count": 34 }, "sender-location": point("25.39,82.02"), "send-time": datetime("2012-02-20T10:10:00.000Z"), "referred-topics": {{ "verizon", "signal" }}, "message-text": " love verizon its signal is awesome:)", "countA": 63, "countB": 149 }
+{ "tweetid": 64i64, "user": { "screen-name": "MiltonWeldi#571", "lang": "en", "friends-count": 72, "statuses-count": 236, "name": "Milton Weldi", "followers-count": 128 }, "sender-location": point("26.08,85.94"), "send-time": datetime("2007-09-22T10:10:00.000Z"), "referred-topics": {{ "sprint", "reachability" }}, "message-text": " like sprint the reachability is awesome", "countA": 64, "countB": 158 }
+{ "tweetid": 65i64, "user": { "screen-name": "CarlineAft_666", "lang": "en", "friends-count": 25, "statuses-count": 352, "name": "Carline Aft", "followers-count": 59 }, "sender-location": point("29.33,78.49"), "send-time": datetime("2012-01-18T10:10:00.000Z"), "referred-topics": {{ "samsung", "customization" }}, "message-text": " can't stand samsung the customization is terrible:(", "countA": 65, "countB": 77 }
+{ "tweetid": 66i64, "user": { "screen-name": "TenaGronko#55", "lang": "en", "friends-count": 91, "statuses-count": 2, "name": "Tena Gronko", "followers-count": 19 }, "sender-location": point("40.14,73.21"), "send-time": datetime("2011-03-18T10:10:00.000Z"), "referred-topics": {{ "at&t", "voice-command" }}, "message-text": " hate at&t the voice-command is horrible", "countA": 66, "countB": 73 }
+{ "tweetid": 67i64, "user": { "screen-name": "DanialBrinigh#499", "lang": "en", "friends-count": 67, "statuses-count": 413, "name": "Danial Brinigh", "followers-count": 4 }, "sender-location": point("41.26,97.09"), "send-time": datetime("2008-02-28T10:10:00.000Z"), "referred-topics": {{ "motorola", "3G" }}, "message-text": " can't stand motorola the 3G is OMG:(", "countA": 67, "countB": 47 }
+{ "tweetid": 68i64, "user": { "screen-name": "GretaBusk#270", "lang": "en", "friends-count": 30, "statuses-count": 150, "name": "Greta Busk", "followers-count": 124 }, "sender-location": point("30.35,86.51"), "send-time": datetime("2008-03-13T10:10:00.000Z"), "referred-topics": {{ "motorola", "reachability" }}, "message-text": " can't stand motorola the reachability is bad:(", "countA": 68, "countB": 21 }
+{ "tweetid": 69i64, "user": { "screen-name": "DanielBurch@155", "lang": "en", "friends-count": 5, "statuses-count": 268, "name": "Daniel Burch", "followers-count": 178 }, "sender-location": point("45.31,66.89"), "send-time": datetime("2009-06-27T10:10:00.000Z"), "referred-topics": {{ "samsung", "network" }}, "message-text": " hate samsung its network is horrible", "countA": 69, "countB": 153 }
+{ "tweetid": 70i64, "user": { "screen-name": "JaynaBash@532", "lang": "en", "friends-count": 90, "statuses-count": 244, "name": "Jayna Bash", "followers-count": 184 }, "sender-location": point("43.92,69.28"), "send-time": datetime("2012-08-06T10:10:00.000Z"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " can't stand samsung the platform is bad", "countA": 70, "countB": 133 }
+{ "tweetid": 71i64, "user": { "screen-name": "PatriciaCason#475", "lang": "en", "friends-count": 50, "statuses-count": 149, "name": "Patricia Cason", "followers-count": 114 }, "sender-location": point("43.74,69.29"), "send-time": datetime("2009-08-28T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is amazing:)", "countA": 71, "countB": 185 }
+{ "tweetid": 72i64, "user": { "screen-name": "KatharineElsas_215", "lang": "en", "friends-count": 69, "statuses-count": 128, "name": "Katharine Elsas", "followers-count": 114 }, "sender-location": point("29.05,94.41"), "send-time": datetime("2010-09-25T10:10:00.000Z"), "referred-topics": {{ "samsung", "signal" }}, "message-text": " can't stand samsung the signal is OMG:(", "countA": 72, "countB": 31 }
+{ "tweetid": 73i64, "user": { "screen-name": "YorkSanborn_951", "lang": "en", "friends-count": 69, "statuses-count": 375, "name": "York Sanborn", "followers-count": 15 }, "sender-location": point("43.92,94.49"), "send-time": datetime("2010-09-19T10:10:00.000Z"), "referred-topics": {{ "motorola", "reachability" }}, "message-text": " dislike motorola its reachability is bad:(", "countA": 73, "countB": 61 }
+{ "tweetid": 74i64, "user": { "screen-name": "AlbertoDull$598", "lang": "en", "friends-count": 29, "statuses-count": 181, "name": "Alberto Dull", "followers-count": 192 }, "sender-location": point("25.6,85.23"), "send-time": datetime("2005-09-22T10:10:00.000Z"), "referred-topics": {{ "samsung", "wireless" }}, "message-text": " dislike samsung the wireless is OMG:(", "countA": 74, "countB": 32 }
+{ "tweetid": 75i64, "user": { "screen-name": "EnriqueFaast$123", "lang": "en", "friends-count": 9, "statuses-count": 24, "name": "Enrique Faast", "followers-count": 24 }, "sender-location": point("30.09,72.93"), "send-time": datetime("2009-10-17T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "voicemail-service" }}, "message-text": " dislike t-mobile its voicemail-service is horrible", "countA": 75, "countB": 185 }
+{ "tweetid": 76i64, "user": { "screen-name": "AndreaBruxner$43", "lang": "en", "friends-count": 37, "statuses-count": 279, "name": "Andrea Bruxner", "followers-count": 118 }, "sender-location": point("30.39,92.92"), "send-time": datetime("2011-04-18T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " like t-mobile its customization is awesome", "countA": 76, "countB": 146 }
+{ "tweetid": 77i64, "user": { "screen-name": "LashawnaKemble$318", "lang": "en", "friends-count": 53, "statuses-count": 44, "name": "Lashawna Kemble", "followers-count": 102 }, "sender-location": point("46.29,93.16"), "send-time": datetime("2010-12-09T10:10:00.000Z"), "referred-topics": {{ "samsung", "voice-command" }}, "message-text": " dislike samsung the voice-command is horrible", "countA": 77, "countB": 0 }
+{ "tweetid": 78i64, "user": { "screen-name": "RodolfoWoodworth#419", "lang": "en", "friends-count": 2, "statuses-count": 82, "name": "Rodolfo Woodworth", "followers-count": 16 }, "sender-location": point("44.92,70.03"), "send-time": datetime("2008-12-18T10:10:00.000Z"), "referred-topics": {{ "motorola", "touch-screen" }}, "message-text": " like motorola the touch-screen is awesome", "countA": 78, "countB": 140 }
+{ "tweetid": 79i64, "user": { "screen-name": "AbramCourtney_384", "lang": "en", "friends-count": 10, "statuses-count": 33, "name": "Abram Courtney", "followers-count": 138 }, "sender-location": point("34.9,96.91"), "send-time": datetime("2007-03-15T10:10:00.000Z"), "referred-topics": {{ "at&t", "plan" }}, "message-text": " hate at&t the plan is bad:(", "countA": 79, "countB": 193 }
+{ "tweetid": 80i64, "user": { "screen-name": "LaurindaRosensteel@202", "lang": "en", "friends-count": 19, "statuses-count": 222, "name": "Laurinda Rosensteel", "followers-count": 20 }, "sender-location": point("47.86,92.66"), "send-time": datetime("2008-09-24T10:10:00.000Z"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " hate samsung its platform is horrible:(", "countA": 80, "countB": 39 }
+{ "tweetid": 81i64, "user": { "screen-name": "JarrettBratton@573", "lang": "en", "friends-count": 90, "statuses-count": 287, "name": "Jarrett Bratton", "followers-count": 18 }, "sender-location": point("33.85,75.61"), "send-time": datetime("2006-05-06T10:10:00.000Z"), "referred-topics": {{ "sprint", "network" }}, "message-text": " like sprint its network is amazing:)", "countA": 81, "countB": 105 }
+{ "tweetid": 82i64, "user": { "screen-name": "EleanorBicknell$880", "lang": "en", "friends-count": 35, "statuses-count": 444, "name": "Eleanor Bicknell", "followers-count": 199 }, "sender-location": point("42.31,95.69"), "send-time": datetime("2008-04-19T10:10:00.000Z"), "referred-topics": {{ "samsung", "plan" }}, "message-text": " can't stand samsung its plan is terrible", "countA": 82, "countB": 140 }
+{ "tweetid": 83i64, "user": { "screen-name": "MannyWerner#209", "lang": "en", "friends-count": 85, "statuses-count": 476, "name": "Manny Werner", "followers-count": 78 }, "sender-location": point("35.76,80.06"), "send-time": datetime("2007-11-17T10:10:00.000Z"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " dislike samsung the platform is terrible", "countA": 83, "countB": 163 }
+{ "tweetid": 84i64, "user": { "screen-name": "TeshaReade_713", "lang": "en", "friends-count": 74, "statuses-count": 99, "name": "Tesha Reade", "followers-count": 112 }, "sender-location": point("26.69,82.05"), "send-time": datetime("2007-01-04T10:10:00.000Z"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon the shortcut-menu is mind-blowing", "countA": 84, "countB": 172 }
+{ "tweetid": 85i64, "user": { "screen-name": "WanCrissman$283", "lang": "en", "friends-count": 37, "statuses-count": 384, "name": "Wan Crissman", "followers-count": 176 }, "sender-location": point("42.3,69.18"), "send-time": datetime("2006-10-27T10:10:00.000Z"), "referred-topics": {{ "at&t", "network" }}, "message-text": " love at&t its network is awesome:)", "countA": 85, "countB": 77 }
+{ "tweetid": 86i64, "user": { "screen-name": "DeedeeMccallum#158", "lang": "en", "friends-count": 81, "statuses-count": 370, "name": "Deedee Mccallum", "followers-count": 104 }, "sender-location": point("46.35,92.36"), "send-time": datetime("2011-09-22T10:10:00.000Z"), "referred-topics": {{ "verizon", "voice-command" }}, "message-text": " like verizon the voice-command is amazing:)", "countA": 86, "countB": 156 }
+{ "tweetid": 87i64, "user": { "screen-name": "AileenAft@340", "lang": "en", "friends-count": 87, "statuses-count": 476, "name": "Aileen Aft", "followers-count": 14 }, "sender-location": point("41.44,95.97"), "send-time": datetime("2012-07-16T10:10:00.000Z"), "referred-topics": {{ "motorola", "network" }}, "message-text": " love motorola its network is mind-blowing", "countA": 87, "countB": 155 }
+{ "tweetid": 88i64, "user": { "screen-name": "BurtonLinton_390", "lang": "en", "friends-count": 13, "statuses-count": 462, "name": "Burton Linton", "followers-count": 34 }, "sender-location": point("45.22,88.29"), "send-time": datetime("2011-12-15T10:10:00.000Z"), "referred-topics": {{ "iphone", "reachability" }}, "message-text": " love iphone the reachability is awesome", "countA": 88, "countB": 95 }
+{ "tweetid": 89i64, "user": { "screen-name": "DamionJoghs_943", "lang": "en", "friends-count": 18, "statuses-count": 388, "name": "Damion Joghs", "followers-count": 111 }, "sender-location": point("36.32,83.38"), "send-time": datetime("2011-06-13T10:10:00.000Z"), "referred-topics": {{ "motorola", "touch-screen" }}, "message-text": " love motorola its touch-screen is awesome", "countA": 89, "countB": 85 }
+{ "tweetid": 90i64, "user": { "screen-name": "LatoshaCowart_858", "lang": "en", "friends-count": 14, "statuses-count": 318, "name": "Latosha Cowart", "followers-count": 27 }, "sender-location": point("26.63,82.77"), "send-time": datetime("2011-10-22T10:10:00.000Z"), "referred-topics": {{ "motorola", "plan" }}, "message-text": " love motorola the plan is good:)", "countA": 90, "countB": 139 }
+{ "tweetid": 91i64, "user": { "screen-name": "LoganPowers$336", "lang": "en", "friends-count": 52, "statuses-count": 154, "name": "Logan Powers", "followers-count": 28 }, "sender-location": point("30.66,96.22"), "send-time": datetime("2011-02-25T10:10:00.000Z"), "referred-topics": {{ "verizon", "reachability" }}, "message-text": " hate verizon the reachability is OMG:(", "countA": 91, "countB": 198 }
+{ "tweetid": 92i64, "user": { "screen-name": "NeilParkinson#794", "lang": "en", "friends-count": 18, "statuses-count": 365, "name": "Neil Parkinson", "followers-count": 27 }, "sender-location": point("31.25,71.75"), "send-time": datetime("2009-12-22T10:10:00.000Z"), "referred-topics": {{ "at&t", "platform" }}, "message-text": " can't stand at&t the platform is terrible:(", "countA": 92, "countB": 59 }
+{ "tweetid": 93i64, "user": { "screen-name": "GoddardFiscina$655", "lang": "en", "friends-count": 10, "statuses-count": 388, "name": "Goddard Fiscina", "followers-count": 142 }, "sender-location": point("33.4,72.55"), "send-time": datetime("2009-04-15T10:10:00.000Z"), "referred-topics": {{ "samsung", "speed" }}, "message-text": " like samsung the speed is mind-blowing", "countA": 93, "countB": 55 }
+{ "tweetid": 94i64, "user": { "screen-name": "JacindaCressman_698", "lang": "en", "friends-count": 50, "statuses-count": 380, "name": "Jacinda Cressman", "followers-count": 112 }, "sender-location": point("33.68,85.33"), "send-time": datetime("2010-09-07T10:10:00.000Z"), "referred-topics": {{ "sprint", "network" }}, "message-text": " like sprint its network is amazing:)", "countA": 94, "countB": 128 }
+{ "tweetid": 95i64, "user": { "screen-name": "NelsonWilks_476", "lang": "en", "friends-count": 43, "statuses-count": 249, "name": "Nelson Wilks", "followers-count": 47 }, "sender-location": point("26.2,74.63"), "send-time": datetime("2010-10-28T10:10:00.000Z"), "referred-topics": {{ "iphone", "shortcut-menu" }}, "message-text": " love iphone the shortcut-menu is mind-blowing", "countA": 95, "countB": 144 }
+{ "tweetid": 96i64, "user": { "screen-name": "FelipeBeach_761", "lang": "en", "friends-count": 70, "statuses-count": 191, "name": "Felipe Beach", "followers-count": 56 }, "sender-location": point("38.59,75.94"), "send-time": datetime("2012-08-06T10:10:00.000Z"), "referred-topics": {{ "sprint", "customization" }}, "message-text": " like sprint the customization is awesome:)", "countA": 96, "countB": 66 }
+{ "tweetid": 97i64, "user": { "screen-name": "MaximaPoehl$770", "lang": "en", "friends-count": 9, "statuses-count": 99, "name": "Maxima Poehl", "followers-count": 198 }, "sender-location": point("46.94,66.2"), "send-time": datetime("2008-03-16T10:10:00.000Z"), "referred-topics": {{ "iphone", "customization" }}, "message-text": " like iphone the customization is good", "countA": 97, "countB": 83 }
+{ "tweetid": 98i64, "user": { "screen-name": "IraLombardi#278", "lang": "en", "friends-count": 10, "statuses-count": 282, "name": "Ira Lombardi", "followers-count": 26 }, "sender-location": point("44.99,93.61"), "send-time": datetime("2011-02-11T10:10:00.000Z"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " love iphone its platform is awesome:)", "countA": 98, "countB": 78 }
+{ "tweetid": 99i64, "user": { "screen-name": "RexHincken_917", "lang": "en", "friends-count": 88, "statuses-count": 292, "name": "Rex Hincken", "followers-count": 74 }, "sender-location": point("42.0,81.22"), "send-time": datetime("2008-09-01T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "voicemail-service" }}, "message-text": " love t-mobile the voicemail-service is amazing:)", "countA": 99, "countB": 51 }
+{ "tweetid": 100i64, "user": { "screen-name": "DakotaTeagarden_163", "lang": "en", "friends-count": 54, "statuses-count": 391, "name": "Dakota Teagarden", "followers-count": 160 }, "sender-location": point("43.59,92.49"), "send-time": datetime("2010-11-23T10:10:00.000Z"), "referred-topics": {{ "samsung", "shortcut-menu" }}, "message-text": " can't stand samsung its shortcut-menu is OMG:(", "countA": 100, "countB": 184 }
+{ "tweetid": 101i64, "user": { "screen-name": "ChetMilliron_934", "lang": "en", "friends-count": 62, "statuses-count": 453, "name": "Chet Milliron", "followers-count": 53 }, "sender-location": point("47.95,77.58"), "send-time": datetime("2012-08-06T10:10:00.000Z"), "referred-topics": {{ "samsung", "3G" }}, "message-text": " like samsung the 3G is mind-blowing:)", "countA": 101, "countB": 142 }
+{ "tweetid": 102i64, "user": { "screen-name": "ZackLosey_956", "lang": "en", "friends-count": 90, "statuses-count": 6, "name": "Zack Losey", "followers-count": 116 }, "sender-location": point("26.53,80.6"), "send-time": datetime("2005-01-11T10:10:00.000Z"), "referred-topics": {{ "verizon", "platform" }}, "message-text": " like verizon its platform is amazing", "countA": 102, "countB": 190 }
+{ "tweetid": 103i64, "user": { "screen-name": "BrionyLafortune$483", "lang": "en", "friends-count": 87, "statuses-count": 496, "name": "Briony Lafortune", "followers-count": 4 }, "sender-location": point("42.2,73.96"), "send-time": datetime("2010-12-05T10:10:00.000Z"), "referred-topics": {{ "samsung", "voice-command" }}, "message-text": " can't stand samsung its voice-command is OMG", "countA": 103, "countB": 27 }
+{ "tweetid": 104i64, "user": { "screen-name": "SaraGraham@726", "lang": "en", "friends-count": 38, "statuses-count": 398, "name": "Sara Graham", "followers-count": 68 }, "sender-location": point("32.83,81.29"), "send-time": datetime("2009-10-25T10:10:00.000Z"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " like iphone its platform is awesome", "countA": 104, "countB": 55 }
+{ "tweetid": 105i64, "user": { "screen-name": "EvanBarnes_217", "lang": "en", "friends-count": 42, "statuses-count": 239, "name": "Evan Barnes", "followers-count": 108 }, "sender-location": point("44.7,90.98"), "send-time": datetime("2008-07-27T10:10:00.000Z"), "referred-topics": {{ "sprint", "reachability" }}, "message-text": " like sprint its reachability is mind-blowing", "countA": 105, "countB": 3 }
+{ "tweetid": 106i64, "user": { "screen-name": "JulianeNorthey#34", "lang": "en", "friends-count": 69, "statuses-count": 94, "name": "Juliane Northey", "followers-count": 187 }, "sender-location": point("26.32,67.64"), "send-time": datetime("2007-06-26T10:10:00.000Z"), "referred-topics": {{ "sprint", "voice-clarity" }}, "message-text": " can't stand sprint the voice-clarity is terrible", "countA": 106, "countB": 127 }
+{ "tweetid": 107i64, "user": { "screen-name": "ShannahBailey$196", "lang": "en", "friends-count": 47, "statuses-count": 215, "name": "Shannah Bailey", "followers-count": 67 }, "sender-location": point("40.85,87.01"), "send-time": datetime("2009-10-07T10:10:00.000Z"), "referred-topics": {{ "verizon", "reachability" }}, "message-text": " love verizon the reachability is awesome:)", "countA": 107, "countB": 31 }
+{ "tweetid": 108i64, "user": { "screen-name": "GranvilleKnisely$497", "lang": "en", "friends-count": 57, "statuses-count": 117, "name": "Granville Knisely", "followers-count": 52 }, "sender-location": point("35.46,78.27"), "send-time": datetime("2006-06-06T10:10:00.000Z"), "referred-topics": {{ "samsung", "customization" }}, "message-text": " hate samsung its customization is horrible", "countA": 108, "countB": 148 }
+{ "tweetid": 109i64, "user": { "screen-name": "LeonardoJardine@763", "lang": "en", "friends-count": 48, "statuses-count": 415, "name": "Leonardo Jardine", "followers-count": 96 }, "sender-location": point("27.7,92.32"), "send-time": datetime("2010-12-15T10:10:00.000Z"), "referred-topics": {{ "verizon", "signal" }}, "message-text": " dislike verizon the signal is bad", "countA": 109, "countB": 29 }
+{ "tweetid": 110i64, "user": { "screen-name": "AuroraMcelroy@927", "lang": "en", "friends-count": 79, "statuses-count": 297, "name": "Aurora Mcelroy", "followers-count": 119 }, "sender-location": point("48.56,85.12"), "send-time": datetime("2005-06-14T10:10:00.000Z"), "referred-topics": {{ "motorola", "shortcut-menu" }}, "message-text": " dislike motorola the shortcut-menu is bad:(", "countA": 110, "countB": 157 }
+{ "tweetid": 111i64, "user": { "screen-name": "NoleneLeslie#166", "lang": "en", "friends-count": 30, "statuses-count": 3, "name": "Nolene Leslie", "followers-count": 18 }, "sender-location": point("31.07,78.53"), "send-time": datetime("2005-10-03T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "platform" }}, "message-text": " like t-mobile its platform is good", "countA": 111, "countB": 50 }
+{ "tweetid": 112i64, "user": { "screen-name": "EusebioBeedell@329", "lang": "en", "friends-count": 94, "statuses-count": 341, "name": "Eusebio Beedell", "followers-count": 89 }, "sender-location": point("32.75,68.79"), "send-time": datetime("2007-08-15T10:10:00.000Z"), "referred-topics": {{ "verizon", "network" }}, "message-text": " can't stand verizon its network is terrible:(", "countA": 112, "countB": 30 }
+{ "tweetid": 113i64, "user": { "screen-name": "WoodySaltser$873", "lang": "en", "friends-count": 68, "statuses-count": 365, "name": "Woody Saltser", "followers-count": 132 }, "sender-location": point("37.57,88.05"), "send-time": datetime("2012-01-21T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "voice-clarity" }}, "message-text": " love t-mobile its voice-clarity is awesome", "countA": 113, "countB": 177 }
+{ "tweetid": 114i64, "user": { "screen-name": "ReannaSeelig#553", "lang": "en", "friends-count": 31, "statuses-count": 291, "name": "Reanna Seelig", "followers-count": 175 }, "sender-location": point("42.87,72.38"), "send-time": datetime("2006-03-20T10:10:00.000Z"), "referred-topics": {{ "motorola", "signal" }}, "message-text": " dislike motorola the signal is OMG:(", "countA": 114, "countB": 59 }
+{ "tweetid": 115i64, "user": { "screen-name": "AllannahNapier@336", "lang": "en", "friends-count": 34, "statuses-count": 359, "name": "Allannah Napier", "followers-count": 50 }, "sender-location": point("31.29,88.73"), "send-time": datetime("2010-07-28T10:10:00.000Z"), "referred-topics": {{ "samsung", "wireless" }}, "message-text": " hate samsung the wireless is bad:(", "countA": 115, "countB": 175 }
+{ "tweetid": 116i64, "user": { "screen-name": "AlaynaOsteen_327", "lang": "en", "friends-count": 59, "statuses-count": 237, "name": "Alayna Osteen", "followers-count": 12 }, "sender-location": point("30.6,71.97"), "send-time": datetime("2007-07-25T10:10:00.000Z"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is amazing:)", "countA": 116, "countB": 70 }
+{ "tweetid": 117i64, "user": { "screen-name": "LeticiaMillard#139", "lang": "en", "friends-count": 95, "statuses-count": 46, "name": "Leticia Millard", "followers-count": 72 }, "sender-location": point("26.53,73.37"), "send-time": datetime("2005-06-22T10:10:00.000Z"), "referred-topics": {{ "iphone", "3G" }}, "message-text": " dislike iphone its 3G is horrible", "countA": 117, "countB": 168 }
+{ "tweetid": 118i64, "user": { "screen-name": "WinifredMckee_639", "lang": "en", "friends-count": 48, "statuses-count": 442, "name": "Winifred Mckee", "followers-count": 199 }, "sender-location": point("27.51,76.65"), "send-time": datetime("2012-06-19T10:10:00.000Z"), "referred-topics": {{ "sprint", "reachability" }}, "message-text": " can't stand sprint the reachability is bad", "countA": 118, "countB": 70 }
+{ "tweetid": 119i64, "user": { "screen-name": "SungShea#585", "lang": "en", "friends-count": 38, "statuses-count": 193, "name": "Sung Shea", "followers-count": 149 }, "sender-location": point("28.86,83.73"), "send-time": datetime("2009-04-22T10:10:00.000Z"), "referred-topics": {{ "samsung", "customer-service" }}, "message-text": " can't stand samsung its customer-service is horrible:(", "countA": 119, "countB": 155 }
+{ "tweetid": 120i64, "user": { "screen-name": "BernadineSutton@199", "lang": "en", "friends-count": 72, "statuses-count": 46, "name": "Bernadine Sutton", "followers-count": 105 }, "sender-location": point("40.19,77.94"), "send-time": datetime("2007-11-12T10:10:00.000Z"), "referred-topics": {{ "sprint", "voice-clarity" }}, "message-text": " like sprint the voice-clarity is good:)", "countA": 120, "countB": 63 }
+{ "tweetid": 121i64, "user": { "screen-name": "DeedeeJerome#182", "lang": "en", "friends-count": 74, "statuses-count": 342, "name": "Deedee Jerome", "followers-count": 170 }, "sender-location": point("42.66,73.84"), "send-time": datetime("2012-07-03T10:10:00.000Z"), "referred-topics": {{ "sprint", "reachability" }}, "message-text": " can't stand sprint its reachability is horrible", "countA": 121, "countB": 163 }
+{ "tweetid": 122i64, "user": { "screen-name": "NigelPrechtl$759", "lang": "en", "friends-count": 10, "statuses-count": 133, "name": "Nigel Prechtl", "followers-count": 137 }, "sender-location": point("37.22,80.92"), "send-time": datetime("2012-01-05T10:10:00.000Z"), "referred-topics": {{ "at&t", "speed" }}, "message-text": " dislike at&t its speed is terrible", "countA": 122, "countB": 23 }
+{ "tweetid": 123i64, "user": { "screen-name": "KimmyWynne$198", "lang": "en", "friends-count": 16, "statuses-count": 40, "name": "Kimmy Wynne", "followers-count": 6 }, "sender-location": point("26.49,70.55"), "send-time": datetime("2008-10-22T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "signal" }}, "message-text": " dislike t-mobile the signal is terrible", "countA": 123, "countB": 44 }
+{ "tweetid": 124i64, "user": { "screen-name": "ByronHarshman$352", "lang": "en", "friends-count": 26, "statuses-count": 133, "name": "Byron Harshman", "followers-count": 144 }, "sender-location": point("26.4,88.43"), "send-time": datetime("2012-03-21T10:10:00.000Z"), "referred-topics": {{ "samsung", "touch-screen" }}, "message-text": " dislike samsung the touch-screen is terrible", "countA": 124, "countB": 44 }
+{ "tweetid": 125i64, "user": { "screen-name": "PlacidPrevatt#865", "lang": "en", "friends-count": 3, "statuses-count": 493, "name": "Placid Prevatt", "followers-count": 10 }, "sender-location": point("43.09,84.0"), "send-time": datetime("2010-07-07T10:10:00.000Z"), "referred-topics": {{ "motorola", "voicemail-service" }}, "message-text": " love motorola its voicemail-service is amazing:)", "countA": 125, "countB": 7 }
+{ "tweetid": 126i64, "user": { "screen-name": "TranterGarneis_456", "lang": "en", "friends-count": 89, "statuses-count": 151, "name": "Tranter Garneis", "followers-count": 166 }, "sender-location": point("41.6,93.6"), "send-time": datetime("2007-08-11T10:10:00.000Z"), "referred-topics": {{ "iphone", "shortcut-menu" }}, "message-text": " like iphone its shortcut-menu is amazing:)", "countA": 126, "countB": 76 }
+{ "tweetid": 127i64, "user": { "screen-name": "DonyaWilliamson$23", "lang": "en", "friends-count": 62, "statuses-count": 325, "name": "Donya Williamson", "followers-count": 101 }, "sender-location": point("27.75,66.01"), "send-time": datetime("2005-11-02T10:10:00.000Z"), "referred-topics": {{ "verizon", "reachability" }}, "message-text": " can't stand verizon its reachability is OMG", "countA": 127, "countB": 184 }
+{ "tweetid": 128i64, "user": { "screen-name": "GalinaJoghs$90", "lang": "en", "friends-count": 61, "statuses-count": 86, "name": "Galina Joghs", "followers-count": 169 }, "sender-location": point("30.95,71.04"), "send-time": datetime("2010-06-01T10:10:00.000Z"), "referred-topics": {{ "motorola", "signal" }}, "message-text": " can't stand motorola its signal is horrible", "countA": 128, "countB": 24 }
+{ "tweetid": 129i64, "user": { "screen-name": "SamsonWerner#683", "lang": "en", "friends-count": 92, "statuses-count": 171, "name": "Samson Werner", "followers-count": 108 }, "sender-location": point("36.53,92.04"), "send-time": datetime("2009-08-18T10:10:00.000Z"), "referred-topics": {{ "motorola", "network" }}, "message-text": " hate motorola its network is terrible:(", "countA": 129, "countB": 80 }
+{ "tweetid": 130i64, "user": { "screen-name": "GabrielleMang#424", "lang": "en", "friends-count": 66, "statuses-count": 8, "name": "Gabrielle Mang", "followers-count": 80 }, "sender-location": point("36.74,96.64"), "send-time": datetime("2006-04-18T10:10:00.000Z"), "referred-topics": {{ "sprint", "plan" }}, "message-text": " love sprint its plan is amazing:)", "countA": 130, "countB": 157 }
+{ "tweetid": 131i64, "user": { "screen-name": "ZachariasBaldwin#74", "lang": "en", "friends-count": 5, "statuses-count": 205, "name": "Zacharias Baldwin", "followers-count": 87 }, "sender-location": point("25.44,72.7"), "send-time": datetime("2006-01-21T10:10:00.000Z"), "referred-topics": {{ "sprint", "platform" }}, "message-text": " like sprint the platform is amazing", "countA": 131, "countB": 192 }
+{ "tweetid": 132i64, "user": { "screen-name": "FanniePoorbaugh@315", "lang": "en", "friends-count": 59, "statuses-count": 441, "name": "Fannie Poorbaugh", "followers-count": 114 }, "sender-location": point("48.89,68.65"), "send-time": datetime("2005-05-07T10:10:00.000Z"), "referred-topics": {{ "verizon", "reachability" }}, "message-text": " hate verizon the reachability is terrible", "countA": 132, "countB": 142 }
+{ "tweetid": 133i64, "user": { "screen-name": "SandraTeagarden$747", "lang": "en", "friends-count": 60, "statuses-count": 353, "name": "Sandra Teagarden", "followers-count": 141 }, "sender-location": point("44.37,76.54"), "send-time": datetime("2008-05-25T10:10:00.000Z"), "referred-topics": {{ "at&t", "platform" }}, "message-text": " can't stand at&t its platform is bad", "countA": 133, "countB": 17 }
+{ "tweetid": 134i64, "user": { "screen-name": "SteveMayers_702", "lang": "en", "friends-count": 70, "statuses-count": 196, "name": "Steve Mayers", "followers-count": 22 }, "sender-location": point("41.52,91.39"), "send-time": datetime("2012-08-11T10:10:00.000Z"), "referred-topics": {{ "iphone", "speed" }}, "message-text": " love iphone its speed is amazing", "countA": 134, "countB": 86 }
+{ "tweetid": 135i64, "user": { "screen-name": "RosalynPullman@789", "lang": "en", "friends-count": 17, "statuses-count": 470, "name": "Rosalyn Pullman", "followers-count": 123 }, "sender-location": point("48.33,86.41"), "send-time": datetime("2009-12-08T10:10:00.000Z"), "referred-topics": {{ "motorola", "customization" }}, "message-text": " hate motorola its customization is bad:(", "countA": 135, "countB": 171 }
+{ "tweetid": 136i64, "user": { "screen-name": "LamarChauvin$832", "lang": "en", "friends-count": 21, "statuses-count": 234, "name": "Lamar Chauvin", "followers-count": 184 }, "sender-location": point("36.83,89.48"), "send-time": datetime("2011-06-26T10:10:00.000Z"), "referred-topics": {{ "samsung", "reachability" }}, "message-text": " love samsung its reachability is awesome:)", "countA": 136, "countB": 77 }
+{ "tweetid": 137i64, "user": { "screen-name": "EleaseReade#477", "lang": "en", "friends-count": 24, "statuses-count": 299, "name": "Elease Reade", "followers-count": 24 }, "sender-location": point("45.55,93.09"), "send-time": datetime("2012-07-19T10:10:00.000Z"), "referred-topics": {{ "at&t", "network" }}, "message-text": " hate at&t the network is bad:(", "countA": 137, "countB": 25 }
+{ "tweetid": 138i64, "user": { "screen-name": "LeviPhilbrick$328", "lang": "en", "friends-count": 73, "statuses-count": 77, "name": "Levi Philbrick", "followers-count": 179 }, "sender-location": point("33.51,68.88"), "send-time": datetime("2009-08-28T10:10:00.000Z"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " dislike verizon its shortcut-menu is horrible", "countA": 138, "countB": 5 }
+{ "tweetid": 139i64, "user": { "screen-name": "RaeburnNickolson_295", "lang": "en", "friends-count": 72, "statuses-count": 176, "name": "Raeburn Nickolson", "followers-count": 103 }, "sender-location": point("38.42,74.16"), "send-time": datetime("2008-05-28T10:10:00.000Z"), "referred-topics": {{ "at&t", "voicemail-service" }}, "message-text": " like at&t its voicemail-service is good:)", "countA": 139, "countB": 175 }
+{ "tweetid": 140i64, "user": { "screen-name": "NerissaBallou@177", "lang": "en", "friends-count": 17, "statuses-count": 447, "name": "Nerissa Ballou", "followers-count": 183 }, "sender-location": point("34.65,81.44"), "send-time": datetime("2008-07-27T10:10:00.000Z"), "referred-topics": {{ "at&t", "plan" }}, "message-text": " hate at&t its plan is OMG", "countA": 140, "countB": 192 }
+{ "tweetid": 141i64, "user": { "screen-name": "DanyelWalker#602", "lang": "en", "friends-count": 22, "statuses-count": 397, "name": "Danyel Walker", "followers-count": 154 }, "sender-location": point("40.38,78.39"), "send-time": datetime("2012-01-05T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "3G" }}, "message-text": " hate t-mobile its 3G is OMG", "countA": 141, "countB": 9 }
+{ "tweetid": 142i64, "user": { "screen-name": "YaronLeichter_45", "lang": "en", "friends-count": 80, "statuses-count": 103, "name": "Yaron Leichter", "followers-count": 115 }, "sender-location": point("42.59,71.72"), "send-time": datetime("2010-04-24T10:10:00.000Z"), "referred-topics": {{ "verizon", "signal" }}, "message-text": " like verizon the signal is mind-blowing:)", "countA": 142, "countB": 130 }
+{ "tweetid": 143i64, "user": { "screen-name": "DeemerCable_599", "lang": "en", "friends-count": 26, "statuses-count": 371, "name": "Deemer Cable", "followers-count": 34 }, "sender-location": point("29.89,70.29"), "send-time": datetime("2012-08-18T10:10:00.000Z"), "referred-topics": {{ "motorola", "wireless" }}, "message-text": " like motorola the wireless is awesome", "countA": 143, "countB": 50 }
+{ "tweetid": 144i64, "user": { "screen-name": "AliaHay_860", "lang": "en", "friends-count": 34, "statuses-count": 470, "name": "Alia Hay", "followers-count": 111 }, "sender-location": point("46.39,96.22"), "send-time": datetime("2010-12-18T10:10:00.000Z"), "referred-topics": {{ "at&t", "shortcut-menu" }}, "message-text": " like at&t its shortcut-menu is amazing:)", "countA": 144, "countB": 184 }
+{ "tweetid": 145i64, "user": { "screen-name": "AdamMoore$384", "lang": "en", "friends-count": 42, "statuses-count": 198, "name": "Adam Moore", "followers-count": 17 }, "sender-location": point("27.99,84.55"), "send-time": datetime("2007-03-16T10:10:00.000Z"), "referred-topics": {{ "sprint", "network" }}, "message-text": " love sprint its network is good", "countA": 145, "countB": 162 }
+{ "tweetid": 146i64, "user": { "screen-name": "AileenSouthern$868", "lang": "en", "friends-count": 62, "statuses-count": 388, "name": "Aileen Southern", "followers-count": 131 }, "sender-location": point("30.85,66.62"), "send-time": datetime("2005-07-14T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "signal" }}, "message-text": " like t-mobile the signal is amazing", "countA": 146, "countB": 102 }
+{ "tweetid": 147i64, "user": { "screen-name": "JewellWise_154", "lang": "en", "friends-count": 2, "statuses-count": 279, "name": "Jewell Wise", "followers-count": 107 }, "sender-location": point("46.72,83.98"), "send-time": datetime("2006-01-09T10:10:00.000Z"), "referred-topics": {{ "motorola", "plan" }}, "message-text": " hate motorola the plan is terrible:(", "countA": 147, "countB": 174 }
+{ "tweetid": 148i64, "user": { "screen-name": "DanielJowers#519", "lang": "en", "friends-count": 23, "statuses-count": 22, "name": "Daniel Jowers", "followers-count": 131 }, "sender-location": point("34.26,72.22"), "send-time": datetime("2008-07-22T10:10:00.000Z"), "referred-topics": {{ "motorola", "voice-clarity" }}, "message-text": " like motorola its voice-clarity is mind-blowing:)", "countA": 148, "countB": 6 }
+{ "tweetid": 149i64, "user": { "screen-name": "DillonWilliams_557", "lang": "en", "friends-count": 18, "statuses-count": 136, "name": "Dillon Williams", "followers-count": 35 }, "sender-location": point("46.63,97.38"), "send-time": datetime("2011-05-09T10:10:00.000Z"), "referred-topics": {{ "motorola", "network" }}, "message-text": " love motorola the network is good", "countA": 149, "countB": 20 }
+{ "tweetid": 150i64, "user": { "screen-name": "DerrickBullard$202", "lang": "en", "friends-count": 16, "statuses-count": 9, "name": "Derrick Bullard", "followers-count": 100 }, "sender-location": point("41.89,90.62"), "send-time": datetime("2012-08-05T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "touch-screen" }}, "message-text": " like t-mobile its touch-screen is mind-blowing", "countA": 150, "countB": 145 }
+{ "tweetid": 151i64, "user": { "screen-name": "LuigiMcfall_976", "lang": "en", "friends-count": 31, "statuses-count": 215, "name": "Luigi Mcfall", "followers-count": 79 }, "sender-location": point("45.38,70.52"), "send-time": datetime("2005-11-27T10:10:00.000Z"), "referred-topics": {{ "motorola", "reachability" }}, "message-text": " hate motorola its reachability is OMG:(", "countA": 151, "countB": 43 }
+{ "tweetid": 152i64, "user": { "screen-name": "MartinPinney_858", "lang": "en", "friends-count": 21, "statuses-count": 465, "name": "Martin Pinney", "followers-count": 16 }, "sender-location": point("32.87,75.66"), "send-time": datetime("2007-10-12T10:10:00.000Z"), "referred-topics": {{ "at&t", "reachability" }}, "message-text": " hate at&t its reachability is terrible", "countA": 152, "countB": 56 }
+{ "tweetid": 153i64, "user": { "screen-name": "JackieAft_623", "lang": "en", "friends-count": 70, "statuses-count": 413, "name": "Jackie Aft", "followers-count": 138 }, "sender-location": point("29.9,73.29"), "send-time": datetime("2010-07-23T10:10:00.000Z"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " like verizon the voice-clarity is amazing", "countA": 153, "countB": 154 }
+{ "tweetid": 154i64, "user": { "screen-name": "SherriWickes#118", "lang": "en", "friends-count": 20, "statuses-count": 31, "name": "Sherri Wickes", "followers-count": 59 }, "sender-location": point("39.2,79.2"), "send-time": datetime("2006-07-13T10:10:00.000Z"), "referred-topics": {{ "sprint", "network" }}, "message-text": " hate sprint the network is OMG:(", "countA": 154, "countB": 124 }
+{ "tweetid": 155i64, "user": { "screen-name": "CarlieCowher@103", "lang": "en", "friends-count": 81, "statuses-count": 127, "name": "Carlie Cowher", "followers-count": 184 }, "sender-location": point("30.3,76.43"), "send-time": datetime("2010-10-04T10:10:00.000Z"), "referred-topics": {{ "sprint", "voice-clarity" }}, "message-text": " can't stand sprint the voice-clarity is bad", "countA": 155, "countB": 39 }
+{ "tweetid": 156i64, "user": { "screen-name": "AndraWardle@74", "lang": "en", "friends-count": 41, "statuses-count": 35, "name": "Andra Wardle", "followers-count": 168 }, "sender-location": point("45.49,93.97"), "send-time": datetime("2009-02-18T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "wireless" }}, "message-text": " love t-mobile its wireless is amazing:)", "countA": 156, "countB": 23 }
+{ "tweetid": 157i64, "user": { "screen-name": "KanishaPinney@150", "lang": "en", "friends-count": 89, "statuses-count": 315, "name": "Kanisha Pinney", "followers-count": 173 }, "sender-location": point("24.72,77.36"), "send-time": datetime("2005-06-10T10:10:00.000Z"), "referred-topics": {{ "verizon", "touch-screen" }}, "message-text": " like verizon the touch-screen is amazing", "countA": 157, "countB": 153 }
+{ "tweetid": 158i64, "user": { "screen-name": "GlyndaSchere@104", "lang": "en", "friends-count": 6, "statuses-count": 111, "name": "Glynda Schere", "followers-count": 120 }, "sender-location": point("33.86,67.49"), "send-time": datetime("2010-11-26T10:10:00.000Z"), "referred-topics": {{ "verizon", "reachability" }}, "message-text": " like verizon its reachability is amazing:)", "countA": 158, "countB": 96 }
+{ "tweetid": 159i64, "user": { "screen-name": "JenelleNehling@461", "lang": "en", "friends-count": 4, "statuses-count": 384, "name": "Jenelle Nehling", "followers-count": 57 }, "sender-location": point("32.65,89.38"), "send-time": datetime("2010-12-06T10:10:00.000Z"), "referred-topics": {{ "iphone", "shortcut-menu" }}, "message-text": " like iphone its shortcut-menu is good", "countA": 159, "countB": 113 }
+{ "tweetid": 160i64, "user": { "screen-name": "DelWheeler@286", "lang": "en", "friends-count": 23, "statuses-count": 403, "name": "Del Wheeler", "followers-count": 177 }, "sender-location": point("39.39,78.05"), "send-time": datetime("2011-07-23T10:10:00.000Z"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " love samsung the platform is mind-blowing", "countA": 160, "countB": 163 }
+{ "tweetid": 161i64, "user": { "screen-name": "TrinityCowart@360", "lang": "en", "friends-count": 85, "statuses-count": 204, "name": "Trinity Cowart", "followers-count": 145 }, "sender-location": point("32.11,76.64"), "send-time": datetime("2010-10-03T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "wireless" }}, "message-text": " like t-mobile its wireless is amazing:)", "countA": 161, "countB": 62 }
+{ "tweetid": 162i64, "user": { "screen-name": "HudsonBasmanoff_348", "lang": "en", "friends-count": 3, "statuses-count": 394, "name": "Hudson Basmanoff", "followers-count": 114 }, "sender-location": point("39.24,78.13"), "send-time": datetime("2007-09-05T10:10:00.000Z"), "referred-topics": {{ "motorola", "wireless" }}, "message-text": " can't stand motorola its wireless is OMG:(", "countA": 162, "countB": 57 }
+{ "tweetid": 163i64, "user": { "screen-name": "MatthewPowers_801", "lang": "en", "friends-count": 2, "statuses-count": 203, "name": "Matthew Powers", "followers-count": 199 }, "sender-location": point("33.79,69.57"), "send-time": datetime("2012-04-05T10:10:00.000Z"), "referred-topics": {{ "at&t", "customer-service" }}, "message-text": " love at&t its customer-service is awesome", "countA": 163, "countB": 115 }
+{ "tweetid": 164i64, "user": { "screen-name": "TitaniaKern$100", "lang": "en", "friends-count": 98, "statuses-count": 300, "name": "Titania Kern", "followers-count": 118 }, "sender-location": point("45.86,67.64"), "send-time": datetime("2005-11-04T10:10:00.000Z"), "referred-topics": {{ "at&t", "shortcut-menu" }}, "message-text": " love at&t the shortcut-menu is amazing", "countA": 164, "countB": 148 }
+{ "tweetid": 165i64, "user": { "screen-name": "EhtelCrissman#778", "lang": "en", "friends-count": 33, "statuses-count": 286, "name": "Ehtel Crissman", "followers-count": 63 }, "sender-location": point("32.98,82.49"), "send-time": datetime("2011-08-22T10:10:00.000Z"), "referred-topics": {{ "at&t", "platform" }}, "message-text": " like at&t its platform is good:)", "countA": 165, "countB": 127 }
+{ "tweetid": 166i64, "user": { "screen-name": "WilletteLeslie@682", "lang": "en", "friends-count": 38, "statuses-count": 491, "name": "Willette Leslie", "followers-count": 75 }, "sender-location": point("32.57,84.97"), "send-time": datetime("2012-04-18T10:10:00.000Z"), "referred-topics": {{ "verizon", "reachability" }}, "message-text": " like verizon its reachability is mind-blowing", "countA": 166, "countB": 162 }
+{ "tweetid": 167i64, "user": { "screen-name": "DarellHincken_722", "lang": "en", "friends-count": 33, "statuses-count": 111, "name": "Darell Hincken", "followers-count": 74 }, "sender-location": point("48.57,77.77"), "send-time": datetime("2008-06-27T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "speed" }}, "message-text": " love t-mobile its speed is good:)", "countA": 167, "countB": 93 }
+{ "tweetid": 168i64, "user": { "screen-name": "DuaneKing@956", "lang": "en", "friends-count": 5, "statuses-count": 44, "name": "Duane King", "followers-count": 169 }, "sender-location": point("31.26,68.61"), "send-time": datetime("2011-03-24T10:10:00.000Z"), "referred-topics": {{ "sprint", "touch-screen" }}, "message-text": " like sprint its touch-screen is mind-blowing:)", "countA": 168, "countB": 174 }
+{ "tweetid": 169i64, "user": { "screen-name": "AmbroseKeilbach$300", "lang": "en", "friends-count": 76, "statuses-count": 278, "name": "Ambrose Keilbach", "followers-count": 54 }, "sender-location": point("29.75,71.35"), "send-time": datetime("2012-02-01T10:10:00.000Z"), "referred-topics": {{ "sprint", "signal" }}, "message-text": " can't stand sprint its signal is horrible", "countA": 169, "countB": 129 }
+{ "tweetid": 170i64, "user": { "screen-name": "KarlBrooks#97", "lang": "en", "friends-count": 76, "statuses-count": 150, "name": "Karl Brooks", "followers-count": 117 }, "sender-location": point("30.77,85.78"), "send-time": datetime("2006-09-16T10:10:00.000Z"), "referred-topics": {{ "motorola", "plan" }}, "message-text": " hate motorola the plan is horrible", "countA": 170, "countB": 174 }
+{ "tweetid": 171i64, "user": { "screen-name": "ShainaMayers$261", "lang": "en", "friends-count": 76, "statuses-count": 240, "name": "Shaina Mayers", "followers-count": 194 }, "sender-location": point("26.11,78.33"), "send-time": datetime("2005-06-22T10:10:00.000Z"), "referred-topics": {{ "sprint", "voicemail-service" }}, "message-text": " love sprint its voicemail-service is mind-blowing:)", "countA": 171, "countB": 69 }
+{ "tweetid": 172i64, "user": { "screen-name": "LakeshaPery_35", "lang": "en", "friends-count": 58, "statuses-count": 300, "name": "Lakesha Pery", "followers-count": 51 }, "sender-location": point("38.45,75.31"), "send-time": datetime("2009-12-20T10:10:00.000Z"), "referred-topics": {{ "motorola", "3G" }}, "message-text": " like motorola its 3G is good:)", "countA": 172, "countB": 127 }
+{ "tweetid": 173i64, "user": { "screen-name": "DoranMingle#901", "lang": "en", "friends-count": 3, "statuses-count": 302, "name": "Doran Mingle", "followers-count": 152 }, "sender-location": point("47.76,91.28"), "send-time": datetime("2009-06-07T10:10:00.000Z"), "referred-topics": {{ "at&t", "customization" }}, "message-text": " dislike at&t the customization is OMG", "countA": 173, "countB": 41 }
+{ "tweetid": 174i64, "user": { "screen-name": "AmadaHatcher#710", "lang": "en", "friends-count": 3, "statuses-count": 12, "name": "Amada Hatcher", "followers-count": 193 }, "sender-location": point("28.64,89.42"), "send-time": datetime("2011-09-16T10:10:00.000Z"), "referred-topics": {{ "samsung", "network" }}, "message-text": " hate samsung the network is OMG", "countA": 174, "countB": 0 }
+{ "tweetid": 175i64, "user": { "screen-name": "CorianderMoon@658", "lang": "en", "friends-count": 28, "statuses-count": 117, "name": "Coriander Moon", "followers-count": 77 }, "sender-location": point("43.82,87.23"), "send-time": datetime("2011-03-26T10:10:00.000Z"), "referred-topics": {{ "samsung", "customization" }}, "message-text": " dislike samsung the customization is bad:(", "countA": 175, "countB": 5 }
+{ "tweetid": 176i64, "user": { "screen-name": "PhilomenaEiford#608", "lang": "en", "friends-count": 53, "statuses-count": 467, "name": "Philomena Eiford", "followers-count": 80 }, "sender-location": point("30.06,71.08"), "send-time": datetime("2006-05-05T10:10:00.000Z"), "referred-topics": {{ "motorola", "signal" }}, "message-text": " like motorola the signal is amazing", "countA": 176, "countB": 89 }
+{ "tweetid": 177i64, "user": { "screen-name": "JacobCongdon$162", "lang": "en", "friends-count": 61, "statuses-count": 161, "name": "Jacob Congdon", "followers-count": 19 }, "sender-location": point("36.36,78.43"), "send-time": datetime("2006-12-10T10:10:00.000Z"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " dislike samsung its platform is horrible", "countA": 177, "countB": 15 }
+{ "tweetid": 178i64, "user": { "screen-name": "GeorgeRichards$777", "lang": "en", "friends-count": 12, "statuses-count": 213, "name": "George Richards", "followers-count": 72 }, "sender-location": point("44.78,90.69"), "send-time": datetime("2006-03-17T10:10:00.000Z"), "referred-topics": {{ "verizon", "speed" }}, "message-text": " can't stand verizon its speed is horrible", "countA": 178, "countB": 64 }
+{ "tweetid": 179i64, "user": { "screen-name": "BridgerHamilton@431", "lang": "en", "friends-count": 51, "statuses-count": 396, "name": "Bridger Hamilton", "followers-count": 110 }, "sender-location": point("32.82,81.54"), "send-time": datetime("2008-12-23T10:10:00.000Z"), "referred-topics": {{ "verizon", "plan" }}, "message-text": " love verizon the plan is awesome", "countA": 179, "countB": 141 }
+{ "tweetid": 180i64, "user": { "screen-name": "FemieLucy@34", "lang": "en", "friends-count": 68, "statuses-count": 221, "name": "Femie Lucy", "followers-count": 11 }, "sender-location": point("31.58,82.78"), "send-time": datetime("2010-02-07T10:10:00.000Z"), "referred-topics": {{ "iphone", "customization" }}, "message-text": " hate iphone its customization is OMG", "countA": 180, "countB": 191 }
+{ "tweetid": 181i64, "user": { "screen-name": "JodiNapier@338", "lang": "en", "friends-count": 96, "statuses-count": 467, "name": "Jodi Napier", "followers-count": 69 }, "sender-location": point("26.14,78.5"), "send-time": datetime("2007-01-06T10:10:00.000Z"), "referred-topics": {{ "verizon", "plan" }}, "message-text": " love verizon the plan is mind-blowing", "countA": 181, "countB": 13 }
+{ "tweetid": 182i64, "user": { "screen-name": "MitsueRawls_424", "lang": "en", "friends-count": 91, "statuses-count": 70, "name": "Mitsue Rawls", "followers-count": 193 }, "sender-location": point("47.77,70.41"), "send-time": datetime("2007-04-12T10:10:00.000Z"), "referred-topics": {{ "iphone", "customer-service" }}, "message-text": " dislike iphone its customer-service is OMG", "countA": 182, "countB": 59 }
+{ "tweetid": 183i64, "user": { "screen-name": "DeshawnAultman_690", "lang": "en", "friends-count": 49, "statuses-count": 330, "name": "Deshawn Aultman", "followers-count": 39 }, "sender-location": point("40.65,79.37"), "send-time": datetime("2005-08-11T10:10:00.000Z"), "referred-topics": {{ "sprint", "shortcut-menu" }}, "message-text": " like sprint the shortcut-menu is amazing:)", "countA": 183, "countB": 199 }
+{ "tweetid": 184i64, "user": { "screen-name": "BradfordEiford#127", "lang": "en", "friends-count": 90, "statuses-count": 425, "name": "Bradford Eiford", "followers-count": 10 }, "sender-location": point("40.85,91.17"), "send-time": datetime("2005-09-16T10:10:00.000Z"), "referred-topics": {{ "iphone", "voicemail-service" }}, "message-text": " dislike iphone its voicemail-service is horrible:(", "countA": 184, "countB": 141 }
+{ "tweetid": 185i64, "user": { "screen-name": "MadelynGaskins_356", "lang": "en", "friends-count": 48, "statuses-count": 455, "name": "Madelyn Gaskins", "followers-count": 66 }, "sender-location": point("33.81,88.32"), "send-time": datetime("2011-10-22T10:10:00.000Z"), "referred-topics": {{ "at&t", "voicemail-service" }}, "message-text": " love at&t its voicemail-service is good:)", "countA": 185, "countB": 189 }
+{ "tweetid": 186i64, "user": { "screen-name": "SophiaMang@768", "lang": "en", "friends-count": 28, "statuses-count": 86, "name": "Sophia Mang", "followers-count": 125 }, "sender-location": point("37.45,68.47"), "send-time": datetime("2010-08-16T10:10:00.000Z"), "referred-topics": {{ "iphone", "touch-screen" }}, "message-text": " love iphone its touch-screen is awesome:)", "countA": 186, "countB": 149 }
+{ "tweetid": 187i64, "user": { "screen-name": "VernonKnisely#170", "lang": "en", "friends-count": 63, "statuses-count": 406, "name": "Vernon Knisely", "followers-count": 31 }, "sender-location": point("32.93,94.65"), "send-time": datetime("2006-03-22T10:10:00.000Z"), "referred-topics": {{ "sprint", "reachability" }}, "message-text": " love sprint the reachability is awesome", "countA": 187, "countB": 98 }
+{ "tweetid": 188i64, "user": { "screen-name": "AmyEndsley@85", "lang": "en", "friends-count": 83, "statuses-count": 43, "name": "Amy Endsley", "followers-count": 7 }, "sender-location": point("27.33,82.34"), "send-time": datetime("2005-01-11T10:10:00.000Z"), "referred-topics": {{ "at&t", "customization" }}, "message-text": " love at&t its customization is good", "countA": 188, "countB": 48 }
+{ "tweetid": 189i64, "user": { "screen-name": "AntonChristner#166", "lang": "en", "friends-count": 12, "statuses-count": 10, "name": "Anton Christner", "followers-count": 66 }, "sender-location": point("25.29,89.55"), "send-time": datetime("2006-11-22T10:10:00.000Z"), "referred-topics": {{ "motorola", "wireless" }}, "message-text": " can't stand motorola its wireless is terrible:(", "countA": 189, "countB": 160 }
+{ "tweetid": 190i64, "user": { "screen-name": "DeshawnHarris#34", "lang": "en", "friends-count": 32, "statuses-count": 488, "name": "Deshawn Harris", "followers-count": 178 }, "sender-location": point("45.46,76.04"), "send-time": datetime("2007-05-13T10:10:00.000Z"), "referred-topics": {{ "samsung", "customization" }}, "message-text": " like samsung the customization is awesome:)", "countA": 190, "countB": 86 }
+{ "tweetid": 191i64, "user": { "screen-name": "MarioHolts_870", "lang": "en", "friends-count": 20, "statuses-count": 192, "name": "Mario Holts", "followers-count": 71 }, "sender-location": point("29.69,71.42"), "send-time": datetime("2005-09-15T10:10:00.000Z"), "referred-topics": {{ "iphone", "speed" }}, "message-text": " hate iphone the speed is horrible:(", "countA": 191, "countB": 150 }
+{ "tweetid": 192i64, "user": { "screen-name": "DeanHall#220", "lang": "en", "friends-count": 55, "statuses-count": 236, "name": "Dean Hall", "followers-count": 68 }, "sender-location": point("48.12,72.0"), "send-time": datetime("2006-11-27T10:10:00.000Z"), "referred-topics": {{ "sprint", "platform" }}, "message-text": " love sprint the platform is awesome", "countA": 192, "countB": 199 }
+{ "tweetid": 193i64, "user": { "screen-name": "LoanClarke_206", "lang": "en", "friends-count": 32, "statuses-count": 173, "name": "Loan Clarke", "followers-count": 186 }, "sender-location": point("45.39,96.01"), "send-time": datetime("2009-05-15T10:10:00.000Z"), "referred-topics": {{ "verizon", "customization" }}, "message-text": " can't stand verizon its customization is horrible:(", "countA": 193, "countB": 147 }
+{ "tweetid": 194i64, "user": { "screen-name": "DonnetteGoodman@627", "lang": "en", "friends-count": 61, "statuses-count": 202, "name": "Donnette Goodman", "followers-count": 106 }, "sender-location": point("44.72,73.13"), "send-time": datetime("2005-04-27T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "plan" }}, "message-text": " love t-mobile its plan is good", "countA": 194, "countB": 142 }
+{ "tweetid": 195i64, "user": { "screen-name": "TamekaPorter#315", "lang": "en", "friends-count": 79, "statuses-count": 63, "name": "Tameka Porter", "followers-count": 7 }, "sender-location": point("37.68,81.78"), "send-time": datetime("2006-06-24T10:10:00.000Z"), "referred-topics": {{ "samsung", "wireless" }}, "message-text": " like samsung its wireless is good", "countA": 195, "countB": 126 }
+{ "tweetid": 196i64, "user": { "screen-name": "GarlandClark@425", "lang": "en", "friends-count": 15, "statuses-count": 375, "name": "Garland Clark", "followers-count": 24 }, "sender-location": point("44.9,70.1"), "send-time": datetime("2006-01-08T10:10:00.000Z"), "referred-topics": {{ "sprint", "touch-screen" }}, "message-text": " hate sprint the touch-screen is OMG:(", "countA": 196, "countB": 197 }
+{ "tweetid": 197i64, "user": { "screen-name": "RupertSanner$868", "lang": "en", "friends-count": 49, "statuses-count": 414, "name": "Rupert Sanner", "followers-count": 189 }, "sender-location": point("40.45,94.94"), "send-time": datetime("2012-06-20T10:10:00.000Z"), "referred-topics": {{ "iphone", "customer-service" }}, "message-text": " can't stand iphone the customer-service is terrible:(", "countA": 197, "countB": 195 }
+{ "tweetid": 198i64, "user": { "screen-name": "JenZoucks#841", "lang": "en", "friends-count": 99, "statuses-count": 73, "name": "Jen Zoucks", "followers-count": 71 }, "sender-location": point("38.77,70.33"), "send-time": datetime("2010-07-07T10:10:00.000Z"), "referred-topics": {{ "iphone", "signal" }}, "message-text": " like iphone its signal is good", "countA": 198, "countB": 94 }
+{ "tweetid": 199i64, "user": { "screen-name": "IsmaelLlora@529", "lang": "en", "friends-count": 63, "statuses-count": 239, "name": "Ismael Llora", "followers-count": 135 }, "sender-location": point("35.31,82.71"), "send-time": datetime("2007-03-19T10:10:00.000Z"), "referred-topics": {{ "at&t", "platform" }}, "message-text": " like at&t its platform is amazing:)", "countA": 199, "countB": 172 }
+{ "tweetid": 200i64, "user": { "screen-name": "JenniferHoltzer_459", "lang": "en", "friends-count": 93, "statuses-count": 172, "name": "Jennifer Holtzer", "followers-count": 51 }, "sender-location": point("30.44,81.57"), "send-time": datetime("2007-11-24T10:10:00.000Z"), "referred-topics": {{ "at&t", "wireless" }}, "message-text": " love at&t the wireless is amazing", "countA": 200, "countB": 20 }
+{ "tweetid": 201i64, "user": { "screen-name": "EvanJardine$78", "lang": "en", "friends-count": 80, "statuses-count": 349, "name": "Evan Jardine", "followers-count": 155 }, "sender-location": point("39.12,68.0"), "send-time": datetime("2010-01-13T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "signal" }}, "message-text": " like t-mobile its signal is awesome:)", "countA": 201, "countB": 65 }
+{ "tweetid": 202i64, "user": { "screen-name": "KerenBard_268", "lang": "en", "friends-count": 28, "statuses-count": 348, "name": "Keren Bard", "followers-count": 158 }, "sender-location": point("32.72,77.86"), "send-time": datetime("2008-07-18T10:10:00.000Z"), "referred-topics": {{ "sprint", "platform" }}, "message-text": " love sprint its platform is awesome:)", "countA": 202, "countB": 117 }
+{ "tweetid": 203i64, "user": { "screen-name": "GilbertLosey#710", "lang": "en", "friends-count": 67, "statuses-count": 141, "name": "Gilbert Losey", "followers-count": 192 }, "sender-location": point("44.66,84.32"), "send-time": datetime("2011-09-27T10:10:00.000Z"), "referred-topics": {{ "at&t", "signal" }}, "message-text": " dislike at&t its signal is horrible", "countA": 203, "countB": 182 }
+{ "tweetid": 204i64, "user": { "screen-name": "TawnyGraham_816", "lang": "en", "friends-count": 63, "statuses-count": 13, "name": "Tawny Graham", "followers-count": 155 }, "sender-location": point("43.08,86.89"), "send-time": datetime("2012-05-10T10:10:00.000Z"), "referred-topics": {{ "verizon", "touch-screen" }}, "message-text": " dislike verizon its touch-screen is terrible", "countA": 204, "countB": 104 }
+{ "tweetid": 205i64, "user": { "screen-name": "LamontWilkins_977", "lang": "en", "friends-count": 19, "statuses-count": 102, "name": "Lamont Wilkins", "followers-count": 127 }, "sender-location": point("31.77,87.68"), "send-time": datetime("2005-04-12T10:10:00.000Z"), "referred-topics": {{ "samsung", "3G" }}, "message-text": " dislike samsung its 3G is bad", "countA": 205, "countB": 57 }
+{ "tweetid": 206i64, "user": { "screen-name": "VincentRiggle$122", "lang": "en", "friends-count": 99, "statuses-count": 105, "name": "Vincent Riggle", "followers-count": 173 }, "sender-location": point("34.28,74.71"), "send-time": datetime("2007-05-06T10:10:00.000Z"), "referred-topics": {{ "motorola", "wireless" }}, "message-text": " like motorola the wireless is amazing:)", "countA": 206, "countB": 3 }
+{ "tweetid": 207i64, "user": { "screen-name": "AdelaJones_400", "lang": "en", "friends-count": 74, "statuses-count": 121, "name": "Adela Jones", "followers-count": 161 }, "sender-location": point("28.5,94.52"), "send-time": datetime("2010-01-20T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "platform" }}, "message-text": " like t-mobile the platform is good:)", "countA": 207, "countB": 110 }
+{ "tweetid": 208i64, "user": { "screen-name": "PrestonLittle$360", "lang": "en", "friends-count": 61, "statuses-count": 430, "name": "Preston Little", "followers-count": 115 }, "sender-location": point("35.79,78.52"), "send-time": datetime("2010-03-01T10:10:00.000Z"), "referred-topics": {{ "sprint", "speed" }}, "message-text": " dislike sprint the speed is terrible:(", "countA": 208, "countB": 149 }
+{ "tweetid": 209i64, "user": { "screen-name": "TaraAnderson#214", "lang": "en", "friends-count": 60, "statuses-count": 219, "name": "Tara Anderson", "followers-count": 110 }, "sender-location": point("34.04,79.5"), "send-time": datetime("2006-12-05T10:10:00.000Z"), "referred-topics": {{ "motorola", "plan" }}, "message-text": " dislike motorola its plan is OMG", "countA": 209, "countB": 164 }
+{ "tweetid": 210i64, "user": { "screen-name": "DeshawnWallace$305", "lang": "en", "friends-count": 29, "statuses-count": 279, "name": "Deshawn Wallace", "followers-count": 27 }, "sender-location": point("39.49,76.58"), "send-time": datetime("2011-01-05T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " dislike t-mobile the customization is OMG", "countA": 210, "countB": 70 }
+{ "tweetid": 211i64, "user": { "screen-name": "OliFisher$694", "lang": "en", "friends-count": 39, "statuses-count": 131, "name": "Oli Fisher", "followers-count": 101 }, "sender-location": point("44.91,94.72"), "send-time": datetime("2010-04-10T10:10:00.000Z"), "referred-topics": {{ "iphone", "3G" }}, "message-text": " love iphone the 3G is mind-blowing", "countA": 211, "countB": 95 }
+{ "tweetid": 212i64, "user": { "screen-name": "LizaMathews#376", "lang": "en", "friends-count": 40, "statuses-count": 107, "name": "Liza Mathews", "followers-count": 70 }, "sender-location": point("46.01,77.85"), "send-time": datetime("2009-07-20T10:10:00.000Z"), "referred-topics": {{ "verizon", "plan" }}, "message-text": " like verizon the plan is mind-blowing:)", "countA": 212, "countB": 189 }
+{ "tweetid": 213i64, "user": { "screen-name": "MylesRahl#433", "lang": "en", "friends-count": 51, "statuses-count": 144, "name": "Myles Rahl", "followers-count": 90 }, "sender-location": point("45.41,95.69"), "send-time": datetime("2006-04-03T10:10:00.000Z"), "referred-topics": {{ "verizon", "3G" }}, "message-text": " love verizon its 3G is mind-blowing:)", "countA": 213, "countB": 190 }
+{ "tweetid": 214i64, "user": { "screen-name": "BertEve@968", "lang": "en", "friends-count": 84, "statuses-count": 110, "name": "Bert Eve", "followers-count": 122 }, "sender-location": point("43.25,87.82"), "send-time": datetime("2009-11-07T10:10:00.000Z"), "referred-topics": {{ "at&t", "voice-command" }}, "message-text": " dislike at&t the voice-command is terrible:(", "countA": 214, "countB": 142 }
+{ "tweetid": 215i64, "user": { "screen-name": "MelissaLaurenzi_383", "lang": "en", "friends-count": 78, "statuses-count": 318, "name": "Melissa Laurenzi", "followers-count": 19 }, "sender-location": point("37.82,86.96"), "send-time": datetime("2010-05-14T10:10:00.000Z"), "referred-topics": {{ "iphone", "customer-service" }}, "message-text": " like iphone its customer-service is mind-blowing", "countA": 215, "countB": 67 }
+{ "tweetid": 216i64, "user": { "screen-name": "ReneaPennington#175", "lang": "en", "friends-count": 42, "statuses-count": 16, "name": "Renea Pennington", "followers-count": 52 }, "sender-location": point("39.06,78.62"), "send-time": datetime("2006-05-23T10:10:00.000Z"), "referred-topics": {{ "iphone", "3G" }}, "message-text": " dislike iphone its 3G is bad:(", "countA": 216, "countB": 50 }
+{ "tweetid": 217i64, "user": { "screen-name": "UrsulaMitchell@26", "lang": "en", "friends-count": 50, "statuses-count": 150, "name": "Ursula Mitchell", "followers-count": 191 }, "sender-location": point("34.02,71.3"), "send-time": datetime("2005-03-26T10:10:00.000Z"), "referred-topics": {{ "iphone", "customer-service" }}, "message-text": " like iphone its customer-service is awesome", "countA": 217, "countB": 154 }
+{ "tweetid": 218i64, "user": { "screen-name": "WillyLambert_669", "lang": "en", "friends-count": 6, "statuses-count": 44, "name": "Willy Lambert", "followers-count": 60 }, "sender-location": point("32.84,73.25"), "send-time": datetime("2011-10-04T10:10:00.000Z"), "referred-topics": {{ "verizon", "customization" }}, "message-text": " like verizon the customization is amazing", "countA": 218, "countB": 135 }
+{ "tweetid": 219i64, "user": { "screen-name": "ChristineLaurence_912", "lang": "en", "friends-count": 77, "statuses-count": 458, "name": "Christine Laurence", "followers-count": 166 }, "sender-location": point("35.39,90.54"), "send-time": datetime("2009-12-01T10:10:00.000Z"), "referred-topics": {{ "at&t", "customization" }}, "message-text": " like at&t the customization is awesome:)", "countA": 219, "countB": 170 }
+{ "tweetid": 220i64, "user": { "screen-name": "JedFiddler_540", "lang": "en", "friends-count": 11, "statuses-count": 264, "name": "Jed Fiddler", "followers-count": 48 }, "sender-location": point("44.11,91.45"), "send-time": datetime("2009-10-10T10:10:00.000Z"), "referred-topics": {{ "at&t", "voicemail-service" }}, "message-text": " like at&t its voicemail-service is awesome:)", "countA": 220, "countB": 188 }
+{ "tweetid": 221i64, "user": { "screen-name": "RubinMueller_263", "lang": "en", "friends-count": 7, "statuses-count": 374, "name": "Rubin Mueller", "followers-count": 77 }, "sender-location": point("25.65,78.68"), "send-time": datetime("2006-09-20T10:10:00.000Z"), "referred-topics": {{ "sprint", "signal" }}, "message-text": " love sprint its signal is amazing", "countA": 221, "countB": 185 }
+{ "tweetid": 222i64, "user": { "screen-name": "JeniWeldi#255", "lang": "en", "friends-count": 71, "statuses-count": 259, "name": "Jeni Weldi", "followers-count": 63 }, "sender-location": point("27.21,67.74"), "send-time": datetime("2009-11-24T10:10:00.000Z"), "referred-topics": {{ "iphone", "customization" }}, "message-text": " like iphone its customization is mind-blowing:)", "countA": 222, "countB": 51 }
+{ "tweetid": 223i64, "user": { "screen-name": "RochelleSaline@265", "lang": "en", "friends-count": 18, "statuses-count": 319, "name": "Rochelle Saline", "followers-count": 11 }, "sender-location": point("33.44,67.08"), "send-time": datetime("2007-10-23T10:10:00.000Z"), "referred-topics": {{ "iphone", "voice-command" }}, "message-text": " can't stand iphone its voice-command is OMG", "countA": 223, "countB": 84 }
+{ "tweetid": 224i64, "user": { "screen-name": "TabbyEckhardstein@204", "lang": "en", "friends-count": 5, "statuses-count": 207, "name": "Tabby Eckhardstein", "followers-count": 5 }, "sender-location": point("24.33,84.9"), "send-time": datetime("2005-02-01T10:10:00.000Z"), "referred-topics": {{ "samsung", "speed" }}, "message-text": " hate samsung its speed is horrible", "countA": 224, "countB": 192 }
+{ "tweetid": 225i64, "user": { "screen-name": "FranklynBurns$23", "lang": "en", "friends-count": 73, "statuses-count": 475, "name": "Franklyn Burns", "followers-count": 131 }, "sender-location": point("26.5,94.14"), "send-time": datetime("2007-08-19T10:10:00.000Z"), "referred-topics": {{ "motorola", "reachability" }}, "message-text": " love motorola its reachability is good", "countA": 225, "countB": 65 }
+{ "tweetid": 226i64, "user": { "screen-name": "CorianderFischer#204", "lang": "en", "friends-count": 4, "statuses-count": 205, "name": "Coriander Fischer", "followers-count": 30 }, "sender-location": point("46.59,73.39"), "send-time": datetime("2010-03-13T10:10:00.000Z"), "referred-topics": {{ "motorola", "reachability" }}, "message-text": " can't stand motorola its reachability is horrible", "countA": 226, "countB": 73 }
+{ "tweetid": 227i64, "user": { "screen-name": "EmikoBarth$856", "lang": "en", "friends-count": 28, "statuses-count": 7, "name": "Emiko Barth", "followers-count": 98 }, "sender-location": point("42.43,97.13"), "send-time": datetime("2005-03-23T10:10:00.000Z"), "referred-topics": {{ "verizon", "reachability" }}, "message-text": " can't stand verizon the reachability is bad:(", "countA": 227, "countB": 184 }
+{ "tweetid": 228i64, "user": { "screen-name": "LavoneStroh$831", "lang": "en", "friends-count": 53, "statuses-count": 2, "name": "Lavone Stroh", "followers-count": 44 }, "sender-location": point("32.04,66.17"), "send-time": datetime("2011-04-25T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "touch-screen" }}, "message-text": " love t-mobile its touch-screen is awesome:)", "countA": 228, "countB": 38 }
+{ "tweetid": 229i64, "user": { "screen-name": "ShantelLaurence@745", "lang": "en", "friends-count": 78, "statuses-count": 195, "name": "Shantel Laurence", "followers-count": 124 }, "sender-location": point("46.58,97.58"), "send-time": datetime("2007-12-02T10:10:00.000Z"), "referred-topics": {{ "iphone", "wireless" }}, "message-text": " love iphone its wireless is awesome:)", "countA": 229, "countB": 26 }
+{ "tweetid": 230i64, "user": { "screen-name": "PiedadMosser@971", "lang": "en", "friends-count": 84, "statuses-count": 80, "name": "Piedad Mosser", "followers-count": 198 }, "sender-location": point("40.44,70.59"), "send-time": datetime("2005-06-27T10:10:00.000Z"), "referred-topics": {{ "motorola", "wireless" }}, "message-text": " hate motorola the wireless is bad", "countA": 230, "countB": 46 }
+{ "tweetid": 231i64, "user": { "screen-name": "CorrieHindman$963", "lang": "en", "friends-count": 54, "statuses-count": 78, "name": "Corrie Hindman", "followers-count": 180 }, "sender-location": point("26.04,77.36"), "send-time": datetime("2012-03-07T10:10:00.000Z"), "referred-topics": {{ "sprint", "wireless" }}, "message-text": " like sprint its wireless is mind-blowing:)", "countA": 231, "countB": 77 }
+{ "tweetid": 232i64, "user": { "screen-name": "DignaPorter_932", "lang": "en", "friends-count": 63, "statuses-count": 278, "name": "Digna Porter", "followers-count": 139 }, "sender-location": point("41.76,97.51"), "send-time": datetime("2009-12-01T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "signal" }}, "message-text": " love t-mobile the signal is mind-blowing:)", "countA": 232, "countB": 85 }
+{ "tweetid": 233i64, "user": { "screen-name": "TerrellStoddard@17", "lang": "en", "friends-count": 74, "statuses-count": 3, "name": "Terrell Stoddard", "followers-count": 35 }, "sender-location": point("34.89,73.83"), "send-time": datetime("2007-01-28T10:10:00.000Z"), "referred-topics": {{ "at&t", "touch-screen" }}, "message-text": " can't stand at&t the touch-screen is terrible", "countA": 233, "countB": 28 }
+{ "tweetid": 234i64, "user": { "screen-name": "EzraField#602", "lang": "en", "friends-count": 7, "statuses-count": 41, "name": "Ezra Field", "followers-count": 169 }, "sender-location": point("26.51,74.01"), "send-time": datetime("2011-03-06T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " love motorola the speed is good", "countA": 234, "countB": 94 }
+{ "tweetid": 235i64, "user": { "screen-name": "NathanielWentzel$505", "lang": "en", "friends-count": 40, "statuses-count": 178, "name": "Nathaniel Wentzel", "followers-count": 83 }, "sender-location": point("47.03,84.5"), "send-time": datetime("2007-07-02T10:10:00.000Z"), "referred-topics": {{ "motorola", "voice-clarity" }}, "message-text": " hate motorola its voice-clarity is bad", "countA": 235, "countB": 53 }
+{ "tweetid": 236i64, "user": { "screen-name": "AlanePycroft$112", "lang": "en", "friends-count": 61, "statuses-count": 222, "name": "Alane Pycroft", "followers-count": 125 }, "sender-location": point("31.82,81.86"), "send-time": datetime("2010-09-09T10:10:00.000Z"), "referred-topics": {{ "at&t", "platform" }}, "message-text": " like at&t its platform is amazing:)", "countA": 236, "countB": 133 }
+{ "tweetid": 237i64, "user": { "screen-name": "EliseoMunson$584", "lang": "en", "friends-count": 21, "statuses-count": 353, "name": "Eliseo Munson", "followers-count": 9 }, "sender-location": point("39.12,74.86"), "send-time": datetime("2011-05-02T10:10:00.000Z"), "referred-topics": {{ "sprint", "speed" }}, "message-text": " hate sprint its speed is terrible", "countA": 237, "countB": 100 }
+{ "tweetid": 238i64, "user": { "screen-name": "LupeCram@152", "lang": "en", "friends-count": 89, "statuses-count": 57, "name": "Lupe Cram", "followers-count": 51 }, "sender-location": point("40.34,71.58"), "send-time": datetime("2010-12-13T10:10:00.000Z"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is amazing", "countA": 238, "countB": 47 }
+{ "tweetid": 239i64, "user": { "screen-name": "AuroraChristman#544", "lang": "en", "friends-count": 96, "statuses-count": 435, "name": "Aurora Christman", "followers-count": 88 }, "sender-location": point("35.44,67.79"), "send-time": datetime("2009-08-11T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "signal" }}, "message-text": " love t-mobile the signal is mind-blowing", "countA": 239, "countB": 114 }
+{ "tweetid": 240i64, "user": { "screen-name": "JermainePotter_329", "lang": "en", "friends-count": 20, "statuses-count": 85, "name": "Jermaine Potter", "followers-count": 177 }, "sender-location": point("35.26,90.47"), "send-time": datetime("2012-04-17T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "network" }}, "message-text": " like t-mobile its network is mind-blowing:)", "countA": 240, "countB": 104 }
+{ "tweetid": 241i64, "user": { "screen-name": "KaetyHall$972", "lang": "en", "friends-count": 23, "statuses-count": 416, "name": "Kaety Hall", "followers-count": 87 }, "sender-location": point("25.06,93.28"), "send-time": datetime("2011-02-13T10:10:00.000Z"), "referred-topics": {{ "verizon", "network" }}, "message-text": " can't stand verizon its network is bad:(", "countA": 241, "countB": 45 }
+{ "tweetid": 242i64, "user": { "screen-name": "LaurenChristopher#195", "lang": "en", "friends-count": 12, "statuses-count": 149, "name": "Lauren Christopher", "followers-count": 183 }, "sender-location": point("24.03,79.65"), "send-time": datetime("2012-05-22T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "touch-screen" }}, "message-text": " love t-mobile the touch-screen is amazing", "countA": 242, "countB": 179 }
+{ "tweetid": 243i64, "user": { "screen-name": "KoreyBonner$296", "lang": "en", "friends-count": 56, "statuses-count": 412, "name": "Korey Bonner", "followers-count": 190 }, "sender-location": point("42.54,97.81"), "send-time": datetime("2006-10-07T10:10:00.000Z"), "referred-topics": {{ "iphone", "touch-screen" }}, "message-text": " like iphone its touch-screen is amazing:)", "countA": 243, "countB": 176 }
+{ "tweetid": 244i64, "user": { "screen-name": "LaetitiaWise@568", "lang": "en", "friends-count": 87, "statuses-count": 363, "name": "Laetitia Wise", "followers-count": 49 }, "sender-location": point("39.34,70.46"), "send-time": datetime("2010-11-25T10:10:00.000Z"), "referred-topics": {{ "iphone", "voicemail-service" }}, "message-text": " hate iphone its voicemail-service is terrible", "countA": 244, "countB": 150 }
+{ "tweetid": 245i64, "user": { "screen-name": "PatricaKellogg@123", "lang": "en", "friends-count": 4, "statuses-count": 80, "name": "Patrica Kellogg", "followers-count": 47 }, "sender-location": point("34.06,84.02"), "send-time": datetime("2005-05-27T10:10:00.000Z"), "referred-topics": {{ "sprint", "touch-screen" }}, "message-text": " hate sprint its touch-screen is bad:(", "countA": 245, "countB": 83 }
+{ "tweetid": 246i64, "user": { "screen-name": "AdelleSwink$50", "lang": "en", "friends-count": 64, "statuses-count": 411, "name": "Adelle Swink", "followers-count": 165 }, "sender-location": point("28.66,71.42"), "send-time": datetime("2007-04-11T10:10:00.000Z"), "referred-topics": {{ "sprint", "plan" }}, "message-text": " can't stand sprint the plan is horrible", "countA": 246, "countB": 128 }
+{ "tweetid": 247i64, "user": { "screen-name": "MckennaAlbright$534", "lang": "en", "friends-count": 94, "statuses-count": 374, "name": "Mckenna Albright", "followers-count": 61 }, "sender-location": point("45.26,93.62"), "send-time": datetime("2011-11-12T10:10:00.000Z"), "referred-topics": {{ "sprint", "speed" }}, "message-text": " can't stand sprint the speed is OMG", "countA": 247, "countB": 194 }
+{ "tweetid": 248i64, "user": { "screen-name": "MerryReade$689", "lang": "en", "friends-count": 41, "statuses-count": 172, "name": "Merry Reade", "followers-count": 105 }, "sender-location": point("42.21,81.81"), "send-time": datetime("2009-08-18T10:10:00.000Z"), "referred-topics": {{ "verizon", "wireless" }}, "message-text": " like verizon its wireless is amazing", "countA": 248, "countB": 123 }
+{ "tweetid": 249i64, "user": { "screen-name": "CoryGoldvogel$187", "lang": "en", "friends-count": 91, "statuses-count": 359, "name": "Cory Goldvogel", "followers-count": 20 }, "sender-location": point("39.94,83.42"), "send-time": datetime("2005-09-11T10:10:00.000Z"), "referred-topics": {{ "verizon", "plan" }}, "message-text": " dislike verizon its plan is bad:(", "countA": 249, "countB": 163 }
+{ "tweetid": 250i64, "user": { "screen-name": "RoystonRummel@500", "lang": "en", "friends-count": 34, "statuses-count": 154, "name": "Royston Rummel", "followers-count": 146 }, "sender-location": point("40.73,72.93"), "send-time": datetime("2005-11-20T10:10:00.000Z"), "referred-topics": {{ "samsung", "touch-screen" }}, "message-text": " love samsung its touch-screen is amazing:)", "countA": 250, "countB": 125 }
\ No newline at end of file
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/UpdateAPIServlet.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/UpdateAPIServlet.java
index 50adb57..523c59c 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/UpdateAPIServlet.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/UpdateAPIServlet.java
@@ -31,7 +31,7 @@
protected List<Statement.Kind> getAllowedStatements() {
Kind[] statementsArray = { Kind.DATAVERSE_DECL, Kind.DELETE, Kind.INSERT, Kind.UPDATE, Kind.DML_CMD_LIST,
- Kind.LOAD, Kind.CONNECT_FEED, Kind.DISCONNECT_FEED, Kind.SET, Kind.COMPACT };
+ Kind.LOAD, Kind.CONNECT_FEED, Kind.DISCONNECT_FEED, Kind.SET, Kind.COMPACT, Kind.EXTERNAL_DATASET_REFRESH };
return Arrays.asList(statementsArray);
}
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 35518a6..ef823ff 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
@@ -19,8 +19,10 @@
import java.nio.ByteBuffer;
import java.rmi.RemoteException;
import java.util.ArrayList;
+import java.util.Date;
import java.util.HashMap;
import java.util.HashSet;
+import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Random;
@@ -61,12 +63,16 @@
import edu.uci.ics.asterix.aql.expression.NodeGroupDropStatement;
import edu.uci.ics.asterix.aql.expression.NodegroupDecl;
import edu.uci.ics.asterix.aql.expression.Query;
+import edu.uci.ics.asterix.aql.expression.RefreshExternalDatasetStatement;
import edu.uci.ics.asterix.aql.expression.SetStatement;
import edu.uci.ics.asterix.aql.expression.TypeDecl;
import edu.uci.ics.asterix.aql.expression.TypeDropStatement;
import edu.uci.ics.asterix.aql.expression.WriteStatement;
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.ExternalDatasetTransactionState;
+import edu.uci.ics.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
+import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
import edu.uci.ics.asterix.common.config.GlobalConfig;
import edu.uci.ics.asterix.common.config.OptimizationConfUtil;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
@@ -75,6 +81,7 @@
import edu.uci.ics.asterix.common.functions.FunctionSignature;
import edu.uci.ics.asterix.file.DatasetOperations;
import edu.uci.ics.asterix.file.DataverseOperations;
+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;
@@ -92,6 +99,7 @@
import edu.uci.ics.asterix.metadata.entities.Datatype;
import edu.uci.ics.asterix.metadata.entities.Dataverse;
import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
+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.FeedPolicy;
@@ -101,6 +109,7 @@
import edu.uci.ics.asterix.metadata.entities.NodeGroup;
import edu.uci.ics.asterix.metadata.feeds.BuiltinFeedPolicies;
import edu.uci.ics.asterix.metadata.feeds.FeedUtil;
+import edu.uci.ics.asterix.metadata.utils.ExternalDatasetsRegistry;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.ATypeTag;
import edu.uci.ics.asterix.om.types.IAType;
@@ -308,6 +317,11 @@
break;
}
+ case EXTERNAL_DATASET_REFRESH: {
+ handleExternalDatasetRefreshStatement(metadataProvider, stmt, hcc);
+ break;
+ }
+
case WRITE: {
Pair<IAWriterFactory, FileSplit> result = handleWriteStatement(metadataProvider, stmt);
if (result.first != null) {
@@ -480,7 +494,20 @@
case EXTERNAL: {
String adapter = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl()).getAdapter();
Map<String, String> properties = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl()).getProperties();
- datasetDetails = new ExternalDatasetDetails(adapter, properties);
+ Identifier ngNameId = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl()).getNodegroupName();
+ String ngName = ngNameId != null ? ngNameId.getValue() : configureNodegroupForDataset(dd,
+ dataverseName, mdTxnCtx);
+ String compactionPolicy = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl()).getCompactionPolicy();
+ Map<String, String> compactionPolicyProperties = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl())
+ .getCompactionPolicyProperties();
+ if (compactionPolicy == null) {
+ compactionPolicy = GlobalConfig.DEFAULT_COMPACTION_POLICY_NAME;
+ compactionPolicyProperties = GlobalConfig.DEFAULT_COMPACTION_POLICY_PROPERTIES;
+ } else {
+ validateCompactionPolicy(compactionPolicy, compactionPolicyProperties, mdTxnCtx);
+ }
+ datasetDetails = new ExternalDatasetDetails(adapter, properties, ngName, new Date(),
+ ExternalDatasetTransactionState.COMMIT, compactionPolicy, compactionPolicyProperties);
break;
}
@@ -625,7 +652,6 @@
private void handleCreateIndexStatement(AqlMetadataProvider metadataProvider, Statement stmt,
IHyracksClientConnection hcc) throws Exception {
-
ProgressState progress = ProgressState.NO_PROGRESS;
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
boolean bActiveTxn = true;
@@ -637,6 +663,12 @@
String indexName = null;
JobSpecification spec = null;
Dataset ds = null;
+ // For external datasets
+ ArrayList<ExternalFile> externalFilesSnapshot = null;
+ boolean firstExternalDatasetIndex = false;
+ boolean filesIndexReplicated = false;
+ Index filesIndex = null;
+ boolean datasetLocked = false;
try {
CreateIndexStatement stmtCreateIndex = (CreateIndexStatement) stmt;
dataverseName = getActiveDataverseName(stmtCreateIndex.getDataverseName());
@@ -669,17 +701,61 @@
}
}
- List<FeedActivity> feedActivities = MetadataManager.INSTANCE.getActiveFeeds(mdTxnCtx, dataverseName,
- datasetName);
- if (feedActivities != null && !feedActivities.isEmpty()) {
- StringBuilder builder = new StringBuilder();
+ if (ds.getDatasetType() == DatasetType.INTERNAL) {
+ List<FeedActivity> feedActivities = MetadataManager.INSTANCE.getActiveFeeds(mdTxnCtx, dataverseName,
+ datasetName);
+ if (feedActivities != null && !feedActivities.isEmpty()) {
+ StringBuilder builder = new StringBuilder();
- for (FeedActivity fa : feedActivities) {
- builder.append(fa + "\n");
+ for (FeedActivity fa : feedActivities) {
+ builder.append(fa + "\n");
+ }
+ throw new AsterixException("Dataset" + datasetName
+ + " is currently being fed into by the following feeds " + "." + builder.toString()
+ + "\nOperation not supported.");
}
- throw new AsterixException("Dataset" + datasetName
- + " is currently being fed into by the following feeds " + "." + builder.toString()
- + "\nOperation not supported.");
+
+ } else {
+ // External dataset
+ // Check if the dataset is indexible
+ if (!ExternalIndexingOperations.isIndexible((ExternalDatasetDetails) ds.getDatasetDetails())) {
+ throw new AlgebricksException("dataset using "
+ + ((ExternalDatasetDetails) ds.getDatasetDetails()).getAdapter()
+ + " Adapter can't be indexed");
+ }
+ // check if the name of the index is valid
+ if (!ExternalIndexingOperations.isValidIndexName(datasetName, indexName)) {
+ throw new AlgebricksException("external dataset index name is invalid");
+ }
+ // lock external dataset
+ ExternalDatasetsRegistry.INSTANCE.buildIndexBegin(ds);
+ datasetLocked = true;
+ // Check if the files index exist
+ filesIndex = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(), dataverseName,
+ datasetName, ExternalIndexingOperations.getFilesIndexName(datasetName));
+ firstExternalDatasetIndex = (filesIndex == null);
+ if (firstExternalDatasetIndex) {
+ // Get snapshot from External File System
+ externalFilesSnapshot = ExternalIndexingOperations.getSnapshotFromExternalFileSystem(ds);
+ // Add an entry for the files index
+ filesIndex = new Index(dataverseName, datasetName,
+ ExternalIndexingOperations.getFilesIndexName(datasetName), IndexType.BTREE,
+ ExternalIndexingOperations.FILE_INDEX_FIELDS, false, IMetadataEntity.PENDING_ADD_OP);
+ MetadataManager.INSTANCE.addIndex(metadataProvider.getMetadataTxnContext(), filesIndex);
+ // Add files to the external files index
+ for (ExternalFile file : externalFilesSnapshot) {
+ MetadataManager.INSTANCE.addExternalFile(mdTxnCtx, file);
+ }
+ // This is the first index for the external dataset, replicate the files index
+ spec = ExternalIndexingOperations.buildFilesIndexReplicationJobSpec(ds, externalFilesSnapshot,
+ metadataProvider, true);
+ if (spec == null) {
+ throw new AsterixException(
+ "Failed to create job spec for replicating Files Index For external dataset");
+ }
+ filesIndexReplicated = true;
+ runJob(hcc, spec, true);
+ }
}
//#. add a new index with PendingAddOp
@@ -726,12 +802,41 @@
indexName);
index.setPendingOp(IMetadataEntity.PENDING_NO_OP);
MetadataManager.INSTANCE.addIndex(metadataProvider.getMetadataTxnContext(), index);
+ // add another new files index with PendingNoOp after deleting the index with PendingAddOp
+ if (firstExternalDatasetIndex) {
+ MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(), dataverseName,
+ datasetName, filesIndex.getIndexName());
+ filesIndex.setPendingOp(IMetadataEntity.PENDING_NO_OP);
+ MetadataManager.INSTANCE.addIndex(metadataProvider.getMetadataTxnContext(), filesIndex);
+ // update transaction timestamp
+ ((ExternalDatasetDetails) ds.getDatasetDetails()).setRefreshTimestamp(new Date());
+ MetadataManager.INSTANCE.updateDataset(mdTxnCtx, ds);
+ }
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
} catch (Exception e) {
if (bActiveTxn) {
abort(e, e, mdTxnCtx);
}
+ // If files index was replicated for external dataset, it should be cleaned up on NC side
+ if (filesIndexReplicated) {
+ mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ bActiveTxn = true;
+ CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName,
+ ExternalIndexingOperations.getFilesIndexName(datasetName));
+ try {
+ JobSpecification jobSpec = ExternalIndexingOperations.buildDropFilesIndexJobSpec(cds,
+ metadataProvider, ds);
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ bActiveTxn = false;
+ runJob(hcc, jobSpec, true);
+ } catch (Exception e2) {
+ e.addSuppressed(e2);
+ if (bActiveTxn) {
+ abort(e, e2, mdTxnCtx);
+ }
+ }
+ }
if (progress == ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA) {
//#. execute compensation operations
@@ -754,7 +859,35 @@
}
}
- // remove the record from the metadata.
+ if (firstExternalDatasetIndex) {
+ mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+ try {
+ // Drop External Files from metadata
+ MetadataManager.INSTANCE.dropDatasetExternalFiles(mdTxnCtx, ds);
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ } catch (Exception e2) {
+ e.addSuppressed(e2);
+ abort(e, e2, mdTxnCtx);
+ throw new IllegalStateException("System is inconsistent state: pending files for("
+ + dataverseName + "." + datasetName + ") couldn't be removed from the metadata", e);
+ }
+ mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+ try {
+ // Drop the files index from metadata
+ MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(), dataverseName,
+ datasetName, ExternalIndexingOperations.getFilesIndexName(datasetName));
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ } catch (Exception e2) {
+ e.addSuppressed(e2);
+ abort(e, e2, mdTxnCtx);
+ throw new IllegalStateException("System is inconsistent state: pending index(" + dataverseName
+ + "." + datasetName + "." + ExternalIndexingOperations.getFilesIndexName(datasetName)
+ + ") couldn't be removed from the metadata", e);
+ }
+ }
+ // remove the record from the metadata.
mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
metadataProvider.setMetadataTxnContext(mdTxnCtx);
try {
@@ -764,13 +897,16 @@
} catch (Exception e2) {
e.addSuppressed(e2);
abort(e, e2, mdTxnCtx);
- throw new IllegalStateException("System is inconsistent state: pending index(" + dataverseName
+ throw new IllegalStateException("System is in inconsistent state: pending index(" + dataverseName
+ "." + datasetName + "." + indexName + ") couldn't be removed from the metadata", e);
}
}
throw e;
} finally {
releaseWriteLatch();
+ if (datasetLocked) {
+ ExternalDatasetsRegistry.INSTANCE.buildIndexEnd(ds);
+ }
}
}
@@ -864,7 +1000,6 @@
String datasetName = datasets.get(j).getDatasetName();
DatasetType dsType = datasets.get(j).getDatasetType();
if (dsType == DatasetType.INTERNAL) {
-
List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName,
datasetName);
for (int k = 0; k < indexes.size(); k++) {
@@ -878,6 +1013,23 @@
CompiledDatasetDropStatement cds = new CompiledDatasetDropStatement(dataverseName, datasetName);
jobsToExecute.add(DatasetOperations.createDropDatasetJobSpec(cds, metadataProvider));
+ } else {
+ // External dataset
+ List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName,
+ datasetName);
+ for (int k = 0; k < indexes.size(); k++) {
+ if (ExternalIndexingOperations.isFileIndex(indexes.get(k))) {
+ CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName,
+ indexes.get(k).getIndexName());
+ jobsToExecute.add(ExternalIndexingOperations.buildDropFilesIndexJobSpec(cds,
+ metadataProvider, datasets.get(j)));
+ } else {
+ CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName,
+ indexes.get(k).getIndexName());
+ jobsToExecute.add(IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider,
+ datasets.get(j)));
+ }
+ }
}
}
jobsToExecute.add(DataverseOperations.createDropDataverseJobSpec(dv, metadataProvider));
@@ -906,7 +1058,7 @@
if (activeDefaultDataverse != null && activeDefaultDataverse.getDataverseName() == dataverseName) {
activeDefaultDataverse = null;
}
-
+ ExternalDatasetsRegistry.INSTANCE.removeDataverse(dv);
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
} catch (Exception e) {
if (bActiveTxn) {
@@ -1029,16 +1181,52 @@
mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
bActiveTxn = true;
metadataProvider.setMetadataTxnContext(mdTxnCtx);
+ } else {
+ // External dataset
+ //#. prepare jobs to drop the datatset and the indexes in NC
+ List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
+ for (int j = 0; j < indexes.size(); j++) {
+ if (ExternalIndexingOperations.isFileIndex(indexes.get(j))) {
+ CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName,
+ indexes.get(j).getIndexName());
+ jobsToExecute.add(IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider, ds));
+ } else {
+ CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName,
+ indexes.get(j).getIndexName());
+ jobsToExecute.add(ExternalIndexingOperations.buildDropFilesIndexJobSpec(cds, metadataProvider,
+ ds));
+ }
+ }
+
+ //#. mark the existing dataset as PendingDropOp
+ MetadataManager.INSTANCE.dropDataset(mdTxnCtx, dataverseName, datasetName);
+ MetadataManager.INSTANCE.addDataset(
+ mdTxnCtx,
+ new Dataset(dataverseName, datasetName, ds.getItemTypeName(), ds.getDatasetDetails(), ds
+ .getHints(), ds.getDatasetType(), ds.getDatasetId(), IMetadataEntity.PENDING_DROP_OP));
+
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ bActiveTxn = false;
+ progress = ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA;
+
+ //#. run the jobs
+ for (JobSpecification jobSpec : jobsToExecute) {
+ runJob(hcc, jobSpec, true);
+ }
+ if (indexes.size() > 0) {
+ ExternalDatasetsRegistry.INSTANCE.removeDatasetInfo(ds);
+ }
+ mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ bActiveTxn = true;
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
}
//#. finally, delete the dataset.
MetadataManager.INSTANCE.dropDataset(mdTxnCtx, dataverseName, datasetName);
// Drop the associated nodegroup
- if (ds.getDatasetType() == DatasetType.INTERNAL) {
- String nodegroup = ((InternalDatasetDetails) ds.getDatasetDetails()).getNodeGroupName();
- if (!nodegroup.equalsIgnoreCase(MetadataConstants.METADATA_DEFAULT_NODEGROUP_NAME)) {
- MetadataManager.INSTANCE.dropNodegroup(mdTxnCtx, dataverseName + ":" + datasetName);
- }
+ String nodegroup = ds.getDatasetDetails().getNodeGroupName();
+ if (!nodegroup.equalsIgnoreCase(MetadataConstants.METADATA_DEFAULT_NODEGROUP_NAME)) {
+ MetadataManager.INSTANCE.dropNodegroup(mdTxnCtx, dataverseName + ":" + datasetName);
}
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -1092,6 +1280,7 @@
String dataverseName = null;
String datasetName = null;
String indexName = null;
+ boolean dropFilesIndex = false;
List<JobSpecification> jobsToExecute = new ArrayList<JobSpecification>();
try {
IndexDropStatement stmtIndexDrop = (IndexDropStatement) stmt;
@@ -1155,8 +1344,74 @@
//#. finally, delete the existing index
MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName);
} else {
- throw new AlgebricksException(datasetName
- + " is an external dataset. Indexes are not maintained for external datasets.");
+ // External dataset
+ indexName = stmtIndexDrop.getIndexName().getValue();
+ Index index = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataverseName, datasetName, indexName);
+ if (index == null) {
+ if (stmtIndexDrop.getIfExists()) {
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ return;
+ } else {
+ throw new AlgebricksException("There is no index with this name " + indexName + ".");
+ }
+ } else if (ExternalIndexingOperations.isFileIndex(index)) {
+ throw new AlgebricksException("Dropping a dataset's files index is not allowed.");
+ }
+ //#. prepare a job to drop the index in NC.
+ CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName, indexName);
+ jobsToExecute.add(IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider, ds));
+ List<Index> datasetIndexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName,
+ datasetName);
+ if (datasetIndexes.size() == 2) {
+ dropFilesIndex = true;
+ // only one index + the files index, we need to delete both of the indexes
+ for (Index externalIndex : datasetIndexes) {
+ if (ExternalIndexingOperations.isFileIndex(externalIndex)) {
+ cds = new CompiledIndexDropStatement(dataverseName, datasetName,
+ externalIndex.getIndexName());
+ jobsToExecute.add(ExternalIndexingOperations.buildDropFilesIndexJobSpec(cds,
+ metadataProvider, ds));
+ //#. mark PendingDropOp on the existing files index
+ MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName,
+ externalIndex.getIndexName());
+ MetadataManager.INSTANCE.addIndex(
+ mdTxnCtx,
+ new Index(dataverseName, datasetName, externalIndex.getIndexName(), externalIndex
+ .getIndexType(), externalIndex.getKeyFieldNames(), externalIndex
+ .isPrimaryIndex(), IMetadataEntity.PENDING_DROP_OP));
+ }
+ }
+ }
+
+ //#. mark PendingDropOp on the existing index
+ MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName);
+ MetadataManager.INSTANCE.addIndex(mdTxnCtx,
+ new Index(dataverseName, datasetName, indexName, index.getIndexType(),
+ index.getKeyFieldNames(), index.isPrimaryIndex(), IMetadataEntity.PENDING_DROP_OP));
+
+ //#. commit the existing transaction before calling runJob.
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ bActiveTxn = false;
+ progress = ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA;
+
+ for (JobSpecification jobSpec : jobsToExecute) {
+ runJob(hcc, jobSpec, true);
+ }
+
+ //#. begin a new transaction
+ mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ bActiveTxn = true;
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+
+ //#. finally, delete the existing index
+ MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName);
+ if (dropFilesIndex) {
+ // delete the files index too
+ MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName,
+ ExternalIndexingOperations.getFilesIndexName(datasetName));
+ MetadataManager.INSTANCE.dropDatasetExternalFiles(mdTxnCtx, ds);
+ ExternalDatasetsRegistry.INSTANCE.removeDatasetInfo(ds);
+ }
}
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -1183,6 +1438,10 @@
try {
MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(), dataverseName,
datasetName, indexName);
+ if (dropFilesIndex) {
+ MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(), dataverseName,
+ datasetName, ExternalIndexingOperations.getFilesIndexName(datasetName));
+ }
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
} catch (Exception e2) {
e.addSuppressed(e2);
@@ -1673,29 +1932,41 @@
CompactStatement compactStatement = (CompactStatement) stmt;
dataverseName = getActiveDataverseName(compactStatement.getDataverseName());
datasetName = compactStatement.getDatasetName().getValue();
-
Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverseName, datasetName);
if (ds == null) {
throw new AlgebricksException("There is no dataset with this name " + datasetName + " in dataverse "
+ dataverseName + ".");
- } else if (ds.getDatasetType() != DatasetType.INTERNAL) {
- throw new AlgebricksException("Cannot compact the extrenal dataset " + datasetName + ".");
}
-
- // Prepare jobs to compact the datatset and its indexes
List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
- for (int j = 0; j < indexes.size(); j++) {
- if (indexes.get(j).isSecondaryIndex()) {
- CompiledIndexCompactStatement cics = new CompiledIndexCompactStatement(dataverseName, datasetName,
- indexes.get(j).getIndexName(), indexes.get(j).getKeyFieldNames(), indexes.get(j)
- .getGramLength(), indexes.get(j).getIndexType());
- jobsToExecute.add(IndexOperations.buildSecondaryIndexCompactJobSpec(cics, metadataProvider, ds));
- }
+ if (indexes.size() == 0) {
+ throw new AlgebricksException("Cannot compact the extrenal dataset " + datasetName
+ + " because it has no indexes");
}
- Dataverse dataverse = MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(),
- dataverseName);
- jobsToExecute.add(DatasetOperations.compactDatasetJobSpec(dataverse, datasetName, metadataProvider));
-
+ if (ds.getDatasetType() == DatasetType.INTERNAL) {
+ for (int j = 0; j < indexes.size(); j++) {
+ if (indexes.get(j).isSecondaryIndex()) {
+ CompiledIndexCompactStatement cics = new CompiledIndexCompactStatement(dataverseName,
+ datasetName, indexes.get(j).getIndexName(), indexes.get(j).getKeyFieldNames(), indexes
+ .get(j).getGramLength(), indexes.get(j).getIndexType());
+ jobsToExecute
+ .add(IndexOperations.buildSecondaryIndexCompactJobSpec(cics, metadataProvider, ds));
+ }
+ }
+ Dataverse dataverse = MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(),
+ dataverseName);
+ jobsToExecute.add(DatasetOperations.compactDatasetJobSpec(dataverse, datasetName, metadataProvider));
+ } else {
+ for (int j = 0; j < indexes.size(); j++) {
+ if (!ExternalIndexingOperations.isFileIndex(indexes.get(j))) {
+ CompiledIndexCompactStatement cics = new CompiledIndexCompactStatement(dataverseName,
+ datasetName, indexes.get(j).getIndexName(), indexes.get(j).getKeyFieldNames(), indexes
+ .get(j).getGramLength(), indexes.get(j).getIndexType());
+ jobsToExecute
+ .add(IndexOperations.buildSecondaryIndexCompactJobSpec(cics, metadataProvider, ds));
+ }
+ }
+ jobsToExecute.add(ExternalIndexingOperations.compactFilesIndexJobSpec(ds, metadataProvider));
+ }
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
bActiveTxn = false;
@@ -1720,9 +1991,9 @@
boolean bActiveTxn = true;
metadataProvider.setMetadataTxnContext(mdTxnCtx);
acquireReadLatch();
-
+ JobSpecification compiled = null;
try {
- JobSpecification compiled = rewriteCompileQuery(metadataProvider, query, null);
+ compiled = rewriteCompileQuery(metadataProvider, query, null);
QueryResult queryResult = new QueryResult(query, metadataProvider.getResultSetId());
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -1786,6 +2057,8 @@
throw e;
} finally {
releaseReadLatch();
+ // release locks aquired during compilation of the query
+ ExternalDatasetsRegistry.INSTANCE.releaseAcquiredLocks(metadataProvider);
}
}
@@ -1819,6 +2092,232 @@
}
}
+ private void handleExternalDatasetRefreshStatement(AqlMetadataProvider metadataProvider, Statement stmt,
+ IHyracksClientConnection hcc) throws Exception {
+ RefreshExternalDatasetStatement stmtRefresh = (RefreshExternalDatasetStatement) stmt;
+ ExternalDatasetTransactionState transactionState = ExternalDatasetTransactionState.COMMIT;
+ MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ acquireWriteLatch();
+ boolean bActiveTxn = true;
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+ String dataverseName = null;
+ String datasetName = null;
+ JobSpecification spec = null;
+ Dataset ds = null;
+ List<ExternalFile> metadataFiles = null;
+ List<ExternalFile> deletedFiles = null;
+ List<ExternalFile> addedFiles = null;
+ List<ExternalFile> appendedFiles = null;
+ List<Index> indexes = null;
+ Dataset transactionDataset = null;
+ boolean lockAquired = false;
+ boolean success = false;
+ try {
+ dataverseName = getActiveDataverseName(stmtRefresh.getDataverseName());
+ datasetName = stmtRefresh.getDatasetName().getValue();
+ ds = MetadataManager.INSTANCE.getDataset(metadataProvider.getMetadataTxnContext(), dataverseName,
+ datasetName);
+
+ // Dataset exists ?
+ if (ds == null) {
+ throw new AlgebricksException("There is no dataset with this name " + datasetName + " in dataverse "
+ + dataverseName);
+ }
+ // Dataset external ?
+ if (ds.getDatasetType() != DatasetType.EXTERNAL) {
+ throw new AlgebricksException("dataset " + datasetName + " in dataverse " + dataverseName
+ + " is not an external dataset");
+ }
+ // Dataset has indexes ?
+ indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
+ if (indexes.size() == 0) {
+ throw new AlgebricksException("External dataset " + datasetName + " in dataverse " + dataverseName
+ + " doesn't have any index");
+ }
+
+ // Record transaction time
+ Date txnTime = new Date();
+
+ // refresh lock here
+ ExternalDatasetsRegistry.INSTANCE.refreshBegin(ds);
+ lockAquired = true;
+
+ // Get internal files
+ metadataFiles = MetadataManager.INSTANCE.getDatasetExternalFiles(mdTxnCtx, ds);
+ deletedFiles = new ArrayList<ExternalFile>();
+ addedFiles = new ArrayList<ExternalFile>();
+ appendedFiles = new ArrayList<ExternalFile>();
+
+ // Compute delta
+ // Now we compare snapshot with external file system
+ if (ExternalIndexingOperations
+ .isDatasetUptodate(ds, metadataFiles, addedFiles, deletedFiles, appendedFiles)) {
+ ((ExternalDatasetDetails) ds.getDatasetDetails()).setRefreshTimestamp(txnTime);
+ MetadataManager.INSTANCE.updateDataset(mdTxnCtx, ds);
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ // latch will be released in the finally clause
+ return;
+ }
+
+ // 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
+ */
+ MetadataManager.INSTANCE.updateDataset(mdTxnCtx, transactionDataset);
+
+ // Add delta files to the metadata
+ for (ExternalFile file : addedFiles) {
+ MetadataManager.INSTANCE.addExternalFile(mdTxnCtx, file);
+ }
+ for (ExternalFile file : appendedFiles) {
+ MetadataManager.INSTANCE.addExternalFile(mdTxnCtx, file);
+ }
+ for (ExternalFile file : deletedFiles) {
+ MetadataManager.INSTANCE.addExternalFile(mdTxnCtx, file);
+ }
+
+ // Create the files index update job
+ spec = ExternalIndexingOperations.buildFilesIndexUpdateOp(ds, metadataFiles, deletedFiles, addedFiles,
+ appendedFiles, metadataProvider);
+
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ bActiveTxn = false;
+ transactionState = ExternalDatasetTransactionState.BEGIN;
+
+ //run the files update job
+ runJob(hcc, spec, true);
+
+ for (Index index : indexes) {
+ if (!ExternalIndexingOperations.isFileIndex(index)) {
+ spec = ExternalIndexingOperations.buildIndexUpdateOp(ds, index, metadataFiles, deletedFiles,
+ addedFiles, appendedFiles, metadataProvider);
+ //run the files update job
+ runJob(hcc, spec, true);
+ }
+ }
+
+ // all index updates has completed successfully, record transaction state
+ spec = ExternalIndexingOperations.buildCommitJob(ds, indexes, metadataProvider);
+
+ // Aquire write latch again -> start a transaction and record the decision to commit
+ mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+ bActiveTxn = true;
+ ((ExternalDatasetDetails) transactionDataset.getDatasetDetails())
+ .setState(ExternalDatasetTransactionState.READY_TO_COMMIT);
+ ((ExternalDatasetDetails) transactionDataset.getDatasetDetails()).setRefreshTimestamp(txnTime);
+ MetadataManager.INSTANCE.updateDataset(mdTxnCtx, transactionDataset);
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ bActiveTxn = false;
+ transactionState = ExternalDatasetTransactionState.READY_TO_COMMIT;
+ // We don't release the latch since this job is expected to be quick
+ runJob(hcc, spec, true);
+ // Start a new metadata transaction to record the final state of the transaction
+ mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+ bActiveTxn = true;
+
+ for (ExternalFile file : metadataFiles) {
+ if (file.getPendingOp() == ExternalFilePendingOp.PENDING_DROP_OP) {
+ MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, file);
+ } else if (file.getPendingOp() == ExternalFilePendingOp.PENDING_NO_OP) {
+ Iterator<ExternalFile> iterator = appendedFiles.iterator();
+ while (iterator.hasNext()) {
+ ExternalFile appendedFile = iterator.next();
+ if (file.getFileName().equals(appendedFile.getFileName())) {
+ // delete existing file
+ MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, file);
+ // delete existing appended file
+ MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, appendedFile);
+ // add the original file with appended information
+ appendedFile.setFileNumber(file.getFileNumber());
+ appendedFile.setPendingOp(ExternalFilePendingOp.PENDING_NO_OP);
+ MetadataManager.INSTANCE.addExternalFile(mdTxnCtx, appendedFile);
+ iterator.remove();
+ }
+ }
+ }
+ }
+
+ // remove the deleted files delta
+ for (ExternalFile file : deletedFiles) {
+ MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, file);
+ }
+
+ // insert new files
+ for (ExternalFile file : addedFiles) {
+ MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, file);
+ file.setPendingOp(ExternalFilePendingOp.PENDING_NO_OP);
+ MetadataManager.INSTANCE.addExternalFile(mdTxnCtx, file);
+ }
+
+ // mark the transaction as complete
+ ((ExternalDatasetDetails) transactionDataset.getDatasetDetails())
+ .setState(ExternalDatasetTransactionState.COMMIT);
+ MetadataManager.INSTANCE.updateDataset(mdTxnCtx, transactionDataset);
+
+ // commit metadata transaction
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ success = true;
+ } catch (Exception e) {
+ if (bActiveTxn) {
+ abort(e, e, mdTxnCtx);
+ }
+ if (transactionState == ExternalDatasetTransactionState.READY_TO_COMMIT) {
+ throw new IllegalStateException("System is inconsistent state: commit of (" + dataverseName + "."
+ + datasetName + ") refresh couldn't carry out the commit phase", e);
+ }
+ if (transactionState == ExternalDatasetTransactionState.COMMIT) {
+ // Nothing to do , everything should be clean
+ throw e;
+ }
+ if (transactionState == ExternalDatasetTransactionState.BEGIN) {
+ // transaction failed, need to do the following
+ // clean NCs removing transaction components
+ mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ bActiveTxn = true;
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+ spec = ExternalIndexingOperations.buildAbortOp(ds, indexes, metadataProvider);
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ bActiveTxn = false;
+ try {
+ runJob(hcc, spec, true);
+ } catch (Exception e2) {
+ // This should never happen -- fix throw illegal
+ e.addSuppressed(e2);
+ throw new IllegalStateException("System is in inconsistent state. Failed to abort refresh", e);
+ }
+ // remove the delta of files
+ // return the state of the dataset to committed
+ try {
+ mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ for (ExternalFile file : deletedFiles) {
+ MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, file);
+ }
+ for (ExternalFile file : addedFiles) {
+ MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, file);
+ }
+ for (ExternalFile file : appendedFiles) {
+ MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, file);
+ }
+ MetadataManager.INSTANCE.updateDataset(mdTxnCtx, ds);
+ // commit metadata transaction
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ } catch (Exception e2) {
+ abort(e, e2, mdTxnCtx);
+ e.addSuppressed(e2);
+ throw new IllegalStateException("System is in inconsistent state. Failed to drop delta files", e);
+ }
+ }
+ } finally {
+ releaseWriteLatch();
+ if (lockAquired) {
+ ExternalDatasetsRegistry.INSTANCE.refreshEnd(ds, success);
+ }
+ }
+ }
+
private JobId runJob(IHyracksClientConnection hcc, JobSpecification spec, boolean waitForCompletion)
throws Exception {
spec.setFrameSize(OptimizationConfUtil.getPhysicalOptimizationConfig().getFrameSize());
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 ac0b919..19283ff 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
@@ -26,6 +26,7 @@
import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
+import edu.uci.ics.asterix.common.config.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;
@@ -123,7 +124,7 @@
JobSpecification specPrimary = JobSpecificationUtils.createJobSpecification();
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadataProvider
- .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataset.getDataverseName(), datasetName,
+ .splitProviderAndPartitionConstraintsForDataset(dataset.getDataverseName(), datasetName,
datasetName);
AsterixStorageProperties storageProperties = AsterixAppContextInfo.getInstance().getStorageProperties();
Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils.getMergePolicyFactory(dataset,
@@ -164,7 +165,7 @@
int[] blooFilterKeyFields = DatasetUtils.createBloomFilterKeyFields(dataset);
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
- .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverseName, datasetName, datasetName);
+ .splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, datasetName);
FileSplit[] fs = splitsAndConstraint.first.getFileSplits();
StringBuilder sb = new StringBuilder();
for (int i = 0; i < fs.length; i++) {
@@ -225,10 +226,10 @@
int[] blooFilterKeyFields = DatasetUtils.createBloomFilterKeyFields(dataset);
ExternalDatasetDetails externalDatasetDetails = new ExternalDatasetDetails(loadStmt.getAdapter(),
- loadStmt.getProperties());
+ loadStmt.getProperties(), null, null, ExternalDatasetTransactionState.COMMIT, null, null);
Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = metadataProvider.buildExternalDataScannerRuntime(
- spec, itemType, externalDatasetDetails, format);
+ spec, itemType, externalDatasetDetails, format, dataset);
IOperatorDescriptor scanner = p.first;
AlgebricksPartitionConstraint scannerPc = p.second;
RecordDescriptor recDesc = computePayloadKeyRecordDescriptor(dataset, itemType, payloadSerde, format);
@@ -253,7 +254,7 @@
fieldPermutation[numKeys] = 0;
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadataProvider
- .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverseName, datasetName, datasetName);
+ .splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, datasetName);
FileSplit[] fs = splitsAndConstraint.first.getFileSplits();
StringBuilder sb = new StringBuilder();
@@ -389,7 +390,7 @@
int[] blooFilterKeyFields = DatasetUtils.createBloomFilterKeyFields(dataset);
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
- .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverseName, datasetName, datasetName);
+ .splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, datasetName);
AsterixStorageProperties storageProperties = AsterixAppContextInfo.getInstance().getStorageProperties();
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
new file mode 100644
index 0000000..bcd9ff9
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/ExternalIndexingOperations.java
@@ -0,0 +1,750 @@
+/*
+ * 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.file;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Date;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
+import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
+import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
+import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
+import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
+import edu.uci.ics.asterix.common.config.DatasetConfig.ExternalDatasetTransactionState;
+import edu.uci.ics.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
+import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeWithBuddyIOOperationCallbackFactory;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
+import edu.uci.ics.asterix.dataflow.data.nontagged.valueproviders.AqlPrimitiveValueProviderFactory;
+import edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory;
+import edu.uci.ics.asterix.external.adapter.factory.HDFSIndexingAdapterFactory;
+import edu.uci.ics.asterix.external.adapter.factory.HiveAdapterFactory;
+import edu.uci.ics.asterix.external.indexing.operators.ExternalDatasetIndexesAbortOperatorDescriptor;
+import edu.uci.ics.asterix.external.indexing.operators.ExternalDatasetIndexesCommitOperatorDescriptor;
+import edu.uci.ics.asterix.external.indexing.operators.ExternalDatasetIndexesRecoverOperatorDescriptor;
+import edu.uci.ics.asterix.external.indexing.operators.IndexInfoOperatorDescriptor;
+import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
+import edu.uci.ics.asterix.metadata.MetadataException;
+import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
+import edu.uci.ics.asterix.metadata.entities.ExternalFile;
+import edu.uci.ics.asterix.metadata.entities.Index;
+import edu.uci.ics.asterix.metadata.external.FilesIndexDescription;
+import edu.uci.ics.asterix.metadata.external.IndexingConstants;
+import edu.uci.ics.asterix.metadata.feeds.ExternalDataScanOperatorDescriptor;
+import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
+import edu.uci.ics.asterix.metadata.utils.ExternalDatasetsRegistry;
+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.AsterixAppContextInfo;
+import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
+import edu.uci.ics.asterix.tools.external.data.ExternalFilesIndexOperatorDescriptor;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
+import edu.uci.ics.asterix.transaction.management.resource.ExternalBTreeLocalResourceMetadata;
+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.CompiledCreateIndexStatement;
+import edu.uci.ics.asterix.translator.CompiledStatements.CompiledIndexDropStatement;
+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.core.jobgen.impl.ConnectorPolicyAssignmentPolicy;
+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;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeWithBuddyDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.dataflow.LSMTreeIndexCompactOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.dataflow.ExternalRTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+import edu.uci.ics.hyracks.storage.common.file.LocalResource;
+
+public class ExternalIndexingOperations {
+
+ public static final ArrayList<String> FILE_INDEX_FIELDS = new ArrayList<String>();
+ static {
+ FILE_INDEX_FIELDS.add("");
+ }
+
+ public static boolean isIndexible(ExternalDatasetDetails ds) {
+ String adapter = ds.getAdapter();
+ if (adapter.equalsIgnoreCase("hdfs") || adapter.equalsIgnoreCase("hive")
+ || adapter.equalsIgnoreCase("edu.uci.ics.asterix.external.dataset.adapter.HDFSAdapter")
+ || adapter.equalsIgnoreCase("edu.uci.ics.asterix.external.dataset.adapter.HIVEAdapter")) {
+ return true;
+ }
+ return false;
+ }
+
+ public static boolean isRefereshActive(ExternalDatasetDetails ds) {
+ return ds.getState() != ExternalDatasetTransactionState.COMMIT;
+ }
+
+ public static boolean datasetUsesHiveAdapter(ExternalDatasetDetails ds) {
+ String adapter = ds.getAdapter();
+ return (adapter.equalsIgnoreCase("hive") || adapter
+ .equalsIgnoreCase("edu.uci.ics.asterix.external.dataset.adapter.HIVEAdapter"));
+ }
+
+ public static boolean isValidIndexName(String datasetName, String indexName) {
+ return (!datasetName.concat(IndexingConstants.EXTERNAL_FILE_INDEX_NAME_SUFFIX).equals(indexName));
+ }
+
+ public static String getFilesIndexName(String datasetName) {
+ return datasetName.concat(IndexingConstants.EXTERNAL_FILE_INDEX_NAME_SUFFIX);
+ }
+
+ public static int getRIDSize(Dataset dataset) {
+ ExternalDatasetDetails dsd = ((ExternalDatasetDetails) dataset.getDatasetDetails());
+ return IndexingConstants.getRIDSize(dsd.getProperties().get(IndexingConstants.KEY_INPUT_FORMAT));
+ }
+
+ public static IBinaryComparatorFactory[] getComparatorFactories(Dataset dataset) {
+ ExternalDatasetDetails dsd = ((ExternalDatasetDetails) dataset.getDatasetDetails());
+ return IndexingConstants.getComparatorFactories((dsd.getProperties().get(IndexingConstants.KEY_INPUT_FORMAT)));
+ }
+
+ public static IBinaryComparatorFactory[] getBuddyBtreeComparatorFactories() {
+ return IndexingConstants.getBuddyBtreeComparatorFactories();
+ }
+
+ public static ArrayList<ExternalFile> getSnapshotFromExternalFileSystem(Dataset dataset) throws AlgebricksException {
+ ArrayList<ExternalFile> files = new ArrayList<ExternalFile>();
+ ExternalDatasetDetails datasetDetails = (ExternalDatasetDetails) dataset.getDatasetDetails();
+ try {
+ // Create the file system object
+ FileSystem fs = getFileSystemObject(datasetDetails.getProperties());
+ // If dataset uses hive adapter, add path to the dataset properties
+ if (datasetUsesHiveAdapter(datasetDetails)) {
+ HiveAdapterFactory.populateConfiguration(datasetDetails.getProperties());
+ }
+ // Get paths of dataset
+ String path = datasetDetails.getProperties().get(HDFSAdapterFactory.KEY_PATH);
+ String[] paths = path.split(",");
+
+ // Add fileStatuses to files
+ for (String aPath : paths) {
+ FileStatus[] fileStatuses = fs.listStatus(new Path(aPath));
+ for (int i = 0; i < fileStatuses.length; i++) {
+ int nextFileNumber = files.size();
+ if (fileStatuses[i].isDir()) {
+ listSubFiles(dataset, fs, fileStatuses[i], files);
+ } else {
+ files.add(new ExternalFile(dataset.getDataverseName(), dataset.getDatasetName(),
+ nextFileNumber, fileStatuses[i].getPath().toUri().getPath(), new Date(fileStatuses[i]
+ .getModificationTime()), fileStatuses[i].getLen(),
+ ExternalFilePendingOp.PENDING_NO_OP));
+ }
+ }
+ }
+ // Close file system
+ fs.close();
+ if (files.size() == 0) {
+ throw new AlgebricksException("File Snapshot retrieved from external file system is empty");
+ }
+ return files;
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new AlgebricksException("Unable to get list of HDFS files " + e);
+ }
+ }
+
+ /* list all files under the directory
+ * src is expected to be a folder
+ */
+ private static void listSubFiles(Dataset dataset, FileSystem srcFs, FileStatus src, ArrayList<ExternalFile> files)
+ throws IOException {
+ Path path = src.getPath();
+ FileStatus[] fileStatuses = srcFs.listStatus(path);
+ for (int i = 0; i < fileStatuses.length; i++) {
+ int nextFileNumber = files.size();
+ if (fileStatuses[i].isDir()) {
+ listSubFiles(dataset, srcFs, fileStatuses[i], files);
+ } else {
+ files.add(new ExternalFile(dataset.getDataverseName(), dataset.getDatasetName(), nextFileNumber,
+ fileStatuses[i].getPath().toUri().getPath(), new Date(fileStatuses[i].getModificationTime()),
+ fileStatuses[i].getLen(), ExternalFilePendingOp.PENDING_NO_OP));
+ }
+ }
+ }
+
+ public static FileSystem getFileSystemObject(Map<String, String> map) throws IOException {
+ Configuration conf = new Configuration();
+ conf.set("fs.default.name", ((String) map.get(HDFSAdapterFactory.KEY_HDFS_URL)).trim());
+ conf.set("fs.hdfs.impl", "org.apache.hadoop.hdfs.DistributedFileSystem");
+ return FileSystem.get(conf);
+ }
+
+ public static JobSpecification buildFilesIndexReplicationJobSpec(Dataset dataset,
+ ArrayList<ExternalFile> externalFilesSnapshot, AqlMetadataProvider metadataProvider, boolean createIndex)
+ throws MetadataException, AlgebricksException {
+ JobSpecification spec = JobSpecificationUtils.createJobSpecification();
+ IAsterixPropertiesProvider asterixPropertiesProvider = AsterixAppContextInfo.getInstance();
+ AsterixStorageProperties storageProperties = asterixPropertiesProvider.getStorageProperties();
+ Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils.getMergePolicyFactory(dataset,
+ metadataProvider.getMetadataTxnContext());
+ ILSMMergePolicyFactory mergePolicyFactory = compactionInfo.first;
+ Map<String, String> mergePolicyFactoryProperties = compactionInfo.second;
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadataProvider
+ .splitProviderAndPartitionConstraintsForFilesIndex(dataset.getDataverseName(),
+ dataset.getDatasetName(), getFilesIndexName(dataset.getDatasetName()), true);
+ IFileSplitProvider secondaryFileSplitProvider = secondarySplitsAndConstraint.first;
+ ILocalResourceMetadata localResourceMetadata = new ExternalBTreeLocalResourceMetadata(
+ FilesIndexDescription.EXTERNAL_FILE_INDEX_TYPE_TRAITS,
+ FilesIndexDescription.FILES_INDEX_COMP_FACTORIES, new int[] { 0 }, false, dataset.getDatasetId(),
+ mergePolicyFactory, mergePolicyFactoryProperties);
+ PersistentLocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
+ localResourceMetadata, LocalResource.ExternalBTreeResource);
+ ExternalBTreeDataflowHelperFactory indexDataflowHelperFactory = new ExternalBTreeDataflowHelperFactory(
+ mergePolicyFactory, mergePolicyFactoryProperties, new SecondaryIndexOperationTrackerProvider(
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties.getBloomFilterFalsePositiveRate(),
+ ExternalDatasetsRegistry.INSTANCE.getDatasetVersion(dataset));
+ ExternalFilesIndexOperatorDescriptor externalFilesOp = new ExternalFilesIndexOperatorDescriptor(spec,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ secondaryFileSplitProvider, indexDataflowHelperFactory, localResourceFactoryProvider,
+ externalFilesSnapshot, createIndex);
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, externalFilesOp,
+ secondarySplitsAndConstraint.second);
+ spec.addRoot(externalFilesOp);
+ spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+ return spec;
+ }
+
+ /**
+ * This method create an indexing operator that index records in HDFS
+ *
+ * @param jobSpec
+ * @param itemType
+ * @param dataset
+ * @param format
+ * @param files
+ * @param indexerDesc
+ * @return
+ * @throws Exception
+ */
+ private static Pair<ExternalDataScanOperatorDescriptor, AlgebricksPartitionConstraint> getExternalDataIndexingOperator(
+ JobSpecification jobSpec, IAType itemType, Dataset dataset, List<ExternalFile> files,
+ RecordDescriptor indexerDesc, AqlMetadataProvider metadataProvider) throws Exception {
+ HDFSIndexingAdapterFactory adapterFactory = new HDFSIndexingAdapterFactory();
+ adapterFactory.setFiles(files);
+ adapterFactory.configure(((ExternalDatasetDetails) dataset.getDatasetDetails()).getProperties(),
+ (ARecordType) itemType);
+ return new Pair<ExternalDataScanOperatorDescriptor, AlgebricksPartitionConstraint>(
+ new ExternalDataScanOperatorDescriptor(jobSpec, indexerDesc, adapterFactory),
+ adapterFactory.getPartitionConstraint());
+ }
+
+ public static Pair<ExternalDataScanOperatorDescriptor, AlgebricksPartitionConstraint> createExternalIndexingOp(
+ JobSpecification spec, AqlMetadataProvider metadataProvider, Dataset dataset, ARecordType itemType,
+ RecordDescriptor indexerDesc, List<ExternalFile> files) throws Exception {
+ if (files == null) {
+ files = MetadataManager.INSTANCE.getDatasetExternalFiles(metadataProvider.getMetadataTxnContext(), dataset);
+ }
+ return getExternalDataIndexingOperator(spec, itemType, dataset, files, indexerDesc, metadataProvider);
+ }
+
+ /**
+ * At the end of this method, we expect to have 4 sets as follows:
+ * metadataFiles should contain only the files that are appended in their original state
+ * addedFiles should contain new files that has number assigned starting after the max original file number
+ * deleteedFiles should contain files that are no longer there in the file system
+ * appendedFiles should have the new file information of existing files
+ * The method should return false in case of zero delta
+ *
+ * @param dataset
+ * @param metadataFiles
+ * @param addedFiles
+ * @param deletedFiles
+ * @param appendedFiles
+ * @return
+ * @throws MetadataException
+ * @throws AlgebricksException
+ */
+ public static boolean isDatasetUptodate(Dataset dataset, List<ExternalFile> metadataFiles,
+ List<ExternalFile> addedFiles, List<ExternalFile> deletedFiles, List<ExternalFile> appendedFiles)
+ throws MetadataException, AlgebricksException {
+ boolean uptodate = true;
+ int newFileNumber = metadataFiles.get(metadataFiles.size() - 1).getFileNumber() + 1;
+
+ ArrayList<ExternalFile> fileSystemFiles = getSnapshotFromExternalFileSystem(dataset);
+
+ // Loop over file system files < taking care of added files >
+ for (ExternalFile fileSystemFile : fileSystemFiles) {
+ boolean fileFound = false;
+ Iterator<ExternalFile> mdFilesIterator = metadataFiles.iterator();
+ while (mdFilesIterator.hasNext()) {
+ ExternalFile metadataFile = mdFilesIterator.next();
+ if (fileSystemFile.getFileName().equals(metadataFile.getFileName())) {
+ // Same file name
+ if (fileSystemFile.getLastModefiedTime().equals(metadataFile.getLastModefiedTime())) {
+ // Same timestamp
+ if (fileSystemFile.getSize() == metadataFile.getSize()) {
+ // Same size -> no op
+ mdFilesIterator.remove();
+ fileFound = true;
+ } else {
+ // Different size -> append op
+ metadataFile.setPendingOp(ExternalFilePendingOp.PENDING_APPEND_OP);
+ fileSystemFile.setPendingOp(ExternalFilePendingOp.PENDING_APPEND_OP);
+ appendedFiles.add(fileSystemFile);
+ fileFound = true;
+ uptodate = false;
+ }
+ } else {
+ // Same file name, Different file mod date -> delete and add
+ metadataFile.setPendingOp(ExternalFilePendingOp.PENDING_DROP_OP);
+ deletedFiles.add(new ExternalFile(metadataFile.getDataverseName(), metadataFile
+ .getDatasetName(), 0, metadataFile.getFileName(), metadataFile.getLastModefiedTime(),
+ metadataFile.getSize(), ExternalFilePendingOp.PENDING_DROP_OP));
+ fileSystemFile.setPendingOp(ExternalFilePendingOp.PENDING_ADD_OP);
+ fileSystemFile.setFileNumber(newFileNumber);
+ addedFiles.add(fileSystemFile);
+ newFileNumber++;
+ fileFound = true;
+ uptodate = false;
+ }
+ }
+ if (fileFound)
+ break;
+ }
+ if (!fileFound) {
+ // File not stored previously in metadata -> pending add op
+ fileSystemFile.setPendingOp(ExternalFilePendingOp.PENDING_ADD_OP);
+ fileSystemFile.setFileNumber(newFileNumber);
+ addedFiles.add(fileSystemFile);
+ newFileNumber++;
+ uptodate = false;
+ }
+ }
+
+ // Done with files from external file system -> metadata files now contain both deleted files and appended ones
+ // first, correct number assignment to deleted and updated files
+ for (ExternalFile deletedFile : deletedFiles) {
+ deletedFile.setFileNumber(newFileNumber);
+ newFileNumber++;
+ }
+ for (ExternalFile appendedFile : appendedFiles) {
+ appendedFile.setFileNumber(newFileNumber);
+ newFileNumber++;
+ }
+
+ // include the remaining deleted files
+ Iterator<ExternalFile> mdFilesIterator = metadataFiles.iterator();
+ while (mdFilesIterator.hasNext()) {
+ ExternalFile metadataFile = mdFilesIterator.next();
+ if (metadataFile.getPendingOp() == ExternalFilePendingOp.PENDING_NO_OP) {
+ metadataFile.setPendingOp(ExternalFilePendingOp.PENDING_DROP_OP);
+ deletedFiles.add(new ExternalFile(metadataFile.getDataverseName(), metadataFile.getDatasetName(),
+ newFileNumber, metadataFile.getFileName(), metadataFile.getLastModefiedTime(), metadataFile
+ .getSize(), metadataFile.getPendingOp()));
+ newFileNumber++;
+ uptodate = false;
+ }
+ }
+ return uptodate;
+ }
+
+ public static Dataset createTransactionDataset(Dataset dataset) {
+ ExternalDatasetDetails originalDsd = (ExternalDatasetDetails) dataset.getDatasetDetails();
+ ExternalDatasetDetails dsd = new ExternalDatasetDetails(originalDsd.getAdapter(), originalDsd.getProperties(),
+ originalDsd.getNodeGroupName(), originalDsd.getTimestamp(), ExternalDatasetTransactionState.BEGIN,
+ originalDsd.getCompactionPolicy(), originalDsd.getCompactionPolicyProperties());
+ Dataset transactionDatset = new Dataset(dataset.getDataverseName(), dataset.getDatasetName(),
+ dataset.getItemTypeName(), dsd, dataset.getHints(), DatasetType.EXTERNAL, dataset.getDatasetId(),
+ dataset.getPendingOp());
+ return transactionDatset;
+ }
+
+ public static boolean isFileIndex(Index index) {
+ return (index.getIndexName().equals(getFilesIndexName(index.getDatasetName())));
+ }
+
+ public static JobSpecification buildDropFilesIndexJobSpec(CompiledIndexDropStatement indexDropStmt,
+ AqlMetadataProvider metadataProvider, Dataset dataset) throws AlgebricksException, MetadataException {
+ String dataverseName = indexDropStmt.getDataverseName() == null ? metadataProvider.getDefaultDataverseName()
+ : indexDropStmt.getDataverseName();
+ String datasetName = indexDropStmt.getDatasetName();
+ String indexName = indexDropStmt.getIndexName();
+ JobSpecification spec = JobSpecificationUtils.createJobSpecification();
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadataProvider
+ .splitProviderAndPartitionConstraintsForFilesIndex(dataverseName, datasetName, indexName, true);
+ AsterixStorageProperties storageProperties = AsterixAppContextInfo.getInstance().getStorageProperties();
+ Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils.getMergePolicyFactory(dataset,
+ metadataProvider.getMetadataTxnContext());
+ IndexDropOperatorDescriptor btreeDrop = new IndexDropOperatorDescriptor(spec,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ splitsAndConstraint.first, new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
+ dataset.getDatasetId()), compactionInfo.first, compactionInfo.second,
+ new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
+ storageProperties.getBloomFilterFalsePositiveRate(), false));
+ AlgebricksPartitionConstraintHelper
+ .setPartitionConstraintInJobSpec(spec, btreeDrop, splitsAndConstraint.second);
+ spec.addRoot(btreeDrop);
+
+ return spec;
+ }
+
+ public static JobSpecification buildFilesIndexUpdateOp(Dataset ds, List<ExternalFile> metadataFiles,
+ List<ExternalFile> deletedFiles, List<ExternalFile> addedFiles, List<ExternalFile> appendedFiles,
+ AqlMetadataProvider metadataProvider) throws MetadataException, AlgebricksException {
+ ArrayList<ExternalFile> files = new ArrayList<ExternalFile>();
+ for (ExternalFile file : metadataFiles) {
+ if (file.getPendingOp() == ExternalFilePendingOp.PENDING_DROP_OP)
+ files.add(file);
+ else if (file.getPendingOp() == ExternalFilePendingOp.PENDING_APPEND_OP) {
+ for (ExternalFile appendedFile : appendedFiles) {
+ if (appendedFile.getFileName().equals(file.getFileName())) {
+ files.add(new ExternalFile(file.getDataverseName(), file.getDatasetName(),
+ file.getFileNumber(), file.getFileName(), file.getLastModefiedTime(), appendedFile
+ .getSize(), ExternalFilePendingOp.PENDING_NO_OP));
+ }
+ }
+ }
+ }
+ for (ExternalFile file : addedFiles) {
+ files.add(file);
+ }
+ Collections.sort(files);
+ return buildFilesIndexReplicationJobSpec(ds, files, metadataProvider, false);
+ }
+
+ public static JobSpecification buildIndexUpdateOp(Dataset ds, Index index, List<ExternalFile> metadataFiles,
+ List<ExternalFile> deletedFiles, List<ExternalFile> addedFiles, List<ExternalFile> appendedFiles,
+ AqlMetadataProvider metadataProvider) throws AsterixException, AlgebricksException {
+ // Create files list
+ ArrayList<ExternalFile> files = new ArrayList<ExternalFile>();
+
+ for (ExternalFile metadataFile : metadataFiles) {
+ if (metadataFile.getPendingOp() != ExternalFilePendingOp.PENDING_APPEND_OP) {
+ files.add(metadataFile);
+ } else {
+ metadataFile.setPendingOp(ExternalFilePendingOp.PENDING_NO_OP);
+ files.add(metadataFile);
+ }
+ }
+ // add new files
+ for (ExternalFile file : addedFiles) {
+ files.add(file);
+ }
+ // add appended files
+ for (ExternalFile file : appendedFiles) {
+ files.add(file);
+ }
+
+ CompiledCreateIndexStatement ccis = new CompiledCreateIndexStatement(index.getIndexName(),
+ index.getDataverseName(), index.getDatasetName(), index.getKeyFieldNames(), index.getGramLength(),
+ index.getIndexType());
+ return IndexOperations.buildSecondaryIndexLoadingJobSpec(ccis, metadataProvider, files);
+ }
+
+ public static JobSpecification buildCommitJob(Dataset ds, List<Index> indexes, AqlMetadataProvider metadataProvider)
+ throws AlgebricksException, AsterixException {
+ JobSpecification spec = JobSpecificationUtils.createJobSpecification();
+ IAsterixPropertiesProvider asterixPropertiesProvider = AsterixAppContextInfo.getInstance();
+ AsterixStorageProperties storageProperties = asterixPropertiesProvider.getStorageProperties();
+ Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils.getMergePolicyFactory(ds,
+ metadataProvider.getMetadataTxnContext());
+ ILSMMergePolicyFactory mergePolicyFactory = compactionInfo.first;
+ Map<String, String> mergePolicyFactoryProperties = compactionInfo.second;
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> filesIndexSplitsAndConstraint = metadataProvider
+ .splitProviderAndPartitionConstraintsForDataset(ds.getDataverseName(), ds.getDatasetName(),
+ getFilesIndexName(ds.getDatasetName()));
+ IFileSplitProvider filesIndexSplitProvider = filesIndexSplitsAndConstraint.first;
+ ExternalBTreeDataflowHelperFactory filesIndexDataflowHelperFactory = getFilesIndexDataflowHelperFactory(ds,
+ mergePolicyFactory, mergePolicyFactoryProperties, storageProperties, spec);
+ IndexInfoOperatorDescriptor filesIndexInfo = new IndexInfoOperatorDescriptor(filesIndexSplitProvider,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER);
+
+ ArrayList<ExternalBTreeWithBuddyDataflowHelperFactory> btreeDataflowHelperFactories = new ArrayList<ExternalBTreeWithBuddyDataflowHelperFactory>();
+ ArrayList<IndexInfoOperatorDescriptor> btreeInfos = new ArrayList<IndexInfoOperatorDescriptor>();
+ ArrayList<ExternalRTreeDataflowHelperFactory> rtreeDataflowHelperFactories = new ArrayList<ExternalRTreeDataflowHelperFactory>();
+ ArrayList<IndexInfoOperatorDescriptor> rtreeInfos = new ArrayList<IndexInfoOperatorDescriptor>();
+
+ for (Index index : indexes) {
+ if (isValidIndexName(index.getDatasetName(), index.getIndexName())) {
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> indexSplitsAndConstraint = metadataProvider
+ .splitProviderAndPartitionConstraintsForDataset(ds.getDataverseName(), ds.getDatasetName(),
+ index.getIndexName());
+ if (index.getIndexType() == IndexType.BTREE) {
+ btreeDataflowHelperFactories.add(getBTreeDataflowHelperFactory(ds, index, mergePolicyFactory,
+ mergePolicyFactoryProperties, storageProperties, spec));
+ btreeInfos.add(new IndexInfoOperatorDescriptor(indexSplitsAndConstraint.first,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER));
+ } else if (index.getIndexType() == IndexType.RTREE) {
+ rtreeDataflowHelperFactories.add(getRTreeDataflowHelperFactory(ds, index, mergePolicyFactory,
+ mergePolicyFactoryProperties, storageProperties, metadataProvider, spec));
+ rtreeInfos.add(new IndexInfoOperatorDescriptor(indexSplitsAndConstraint.first,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER));
+ }
+ }
+ }
+
+ ExternalDatasetIndexesCommitOperatorDescriptor op = new ExternalDatasetIndexesCommitOperatorDescriptor(spec,
+ filesIndexDataflowHelperFactory, filesIndexInfo, btreeDataflowHelperFactories, btreeInfos,
+ rtreeDataflowHelperFactories, rtreeInfos);
+
+ spec.addRoot(op);
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, op,
+ filesIndexSplitsAndConstraint.second);
+ spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+ return spec;
+ }
+
+ private static ExternalBTreeDataflowHelperFactory getFilesIndexDataflowHelperFactory(Dataset ds,
+ ILSMMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyFactoryProperties,
+ AsterixStorageProperties storageProperties, JobSpecification spec) {
+ return new ExternalBTreeDataflowHelperFactory(mergePolicyFactory, mergePolicyFactoryProperties,
+ new SecondaryIndexOperationTrackerProvider(ds.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
+ storageProperties.getBloomFilterFalsePositiveRate(),
+ ExternalDatasetsRegistry.INSTANCE.getDatasetVersion(ds));
+ }
+
+ private static ExternalBTreeWithBuddyDataflowHelperFactory getBTreeDataflowHelperFactory(Dataset ds, Index index,
+ ILSMMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyFactoryProperties,
+ AsterixStorageProperties storageProperties, JobSpecification spec) {
+ return new ExternalBTreeWithBuddyDataflowHelperFactory(mergePolicyFactory, mergePolicyFactoryProperties,
+ new SecondaryIndexOperationTrackerProvider(ds.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMBTreeWithBuddyIOOperationCallbackFactory.INSTANCE,
+ storageProperties.getBloomFilterFalsePositiveRate(), new int[] { index.getKeyFieldNames().size() },
+ ExternalDatasetsRegistry.INSTANCE.getDatasetVersion(ds));
+ }
+
+ @SuppressWarnings("rawtypes")
+ private static ExternalRTreeDataflowHelperFactory getRTreeDataflowHelperFactory(Dataset ds, Index index,
+ ILSMMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyFactoryProperties,
+ AsterixStorageProperties storageProperties, AqlMetadataProvider metadataProvider, JobSpecification spec)
+ throws AlgebricksException, AsterixException {
+ int numPrimaryKeys = getRIDSize(ds);
+ List<String> secondaryKeyFields = index.getKeyFieldNames();
+ secondaryKeyFields.size();
+ ARecordType itemType = (ARecordType) metadataProvider.findType(ds.getDataverseName(), ds.getItemTypeName());
+ Pair<IAType, Boolean> spatialTypePair = Index.getNonNullableKeyFieldType(secondaryKeyFields.get(0), itemType);
+ IAType spatialType = spatialTypePair.first;
+ if (spatialType == null) {
+ throw new AsterixException("Could not find field " + secondaryKeyFields.get(0) + " in the schema.");
+ }
+ int numDimensions = NonTaggedFormatUtil.getNumDimensions(spatialType.getTypeTag());
+ int numNestedSecondaryKeyFields = numDimensions * 2;
+ IPrimitiveValueProviderFactory[] valueProviderFactories = new IPrimitiveValueProviderFactory[numNestedSecondaryKeyFields];
+ IBinaryComparatorFactory[] secondaryComparatorFactories = new IBinaryComparatorFactory[numNestedSecondaryKeyFields];
+
+ ISerializerDeserializer[] secondaryRecFields = new ISerializerDeserializer[numPrimaryKeys
+ + numNestedSecondaryKeyFields];
+ ITypeTraits[] secondaryTypeTraits = new ITypeTraits[numNestedSecondaryKeyFields + numPrimaryKeys];
+ IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(spatialType.getTypeTag());
+ ATypeTag keyType = nestedKeyType.getTypeTag();
+
+ keyType = nestedKeyType.getTypeTag();
+ for (int i = 0; i < numNestedSecondaryKeyFields; i++) {
+ ISerializerDeserializer keySerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(nestedKeyType);
+ secondaryRecFields[i] = keySerde;
+
+ secondaryComparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(
+ nestedKeyType, true);
+ secondaryTypeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(nestedKeyType);
+ valueProviderFactories[i] = AqlPrimitiveValueProviderFactory.INSTANCE;
+ }
+ // Add serializers and comparators for primary index fields.
+ for (int i = 0; i < numPrimaryKeys; i++) {
+ secondaryRecFields[numNestedSecondaryKeyFields + i] = IndexingConstants.getSerializerDeserializer(i);
+ secondaryTypeTraits[numNestedSecondaryKeyFields + i] = IndexingConstants.getTypeTraits(i);
+ }
+ int[] primaryKeyFields = new int[numPrimaryKeys];
+ for (int i = 0; i < primaryKeyFields.length; i++) {
+ primaryKeyFields[i] = i + numNestedSecondaryKeyFields;
+ }
+
+ return new ExternalRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
+ getBuddyBtreeComparatorFactories(), mergePolicyFactory, mergePolicyFactoryProperties,
+ new SecondaryIndexOperationTrackerProvider(ds.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMRTreeIOOperationCallbackFactory.INSTANCE,
+ AqlMetadataProvider.proposeLinearizer(keyType, secondaryComparatorFactories.length),
+ storageProperties.getBloomFilterFalsePositiveRate(), new int[] { index.getKeyFieldNames().size() },
+ ExternalDatasetsRegistry.INSTANCE.getDatasetVersion(ds));
+ }
+
+ public static JobSpecification buildAbortOp(Dataset ds, List<Index> indexes, AqlMetadataProvider metadataProvider)
+ throws AlgebricksException, AsterixException {
+ JobSpecification spec = JobSpecificationUtils.createJobSpecification();
+ IAsterixPropertiesProvider asterixPropertiesProvider = AsterixAppContextInfo.getInstance();
+ AsterixStorageProperties storageProperties = asterixPropertiesProvider.getStorageProperties();
+ Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils.getMergePolicyFactory(ds,
+ metadataProvider.getMetadataTxnContext());
+ ILSMMergePolicyFactory mergePolicyFactory = compactionInfo.first;
+ Map<String, String> mergePolicyFactoryProperties = compactionInfo.second;
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> filesIndexSplitsAndConstraint = metadataProvider
+ .splitProviderAndPartitionConstraintsForDataset(ds.getDataverseName(), ds.getDatasetName(),
+ getFilesIndexName(ds.getDatasetName()));
+ IFileSplitProvider filesIndexSplitProvider = filesIndexSplitsAndConstraint.first;
+ ExternalBTreeDataflowHelperFactory filesIndexDataflowHelperFactory = getFilesIndexDataflowHelperFactory(ds,
+ mergePolicyFactory, mergePolicyFactoryProperties, storageProperties, spec);
+ IndexInfoOperatorDescriptor filesIndexInfo = new IndexInfoOperatorDescriptor(filesIndexSplitProvider,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER);
+
+ ArrayList<ExternalBTreeWithBuddyDataflowHelperFactory> btreeDataflowHelperFactories = new ArrayList<ExternalBTreeWithBuddyDataflowHelperFactory>();
+ ArrayList<IndexInfoOperatorDescriptor> btreeInfos = new ArrayList<IndexInfoOperatorDescriptor>();
+ ArrayList<ExternalRTreeDataflowHelperFactory> rtreeDataflowHelperFactories = new ArrayList<ExternalRTreeDataflowHelperFactory>();
+ ArrayList<IndexInfoOperatorDescriptor> rtreeInfos = new ArrayList<IndexInfoOperatorDescriptor>();
+
+ for (Index index : indexes) {
+ if (isValidIndexName(index.getDatasetName(), index.getIndexName())) {
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> indexSplitsAndConstraint = metadataProvider
+ .splitProviderAndPartitionConstraintsForDataset(ds.getDataverseName(), ds.getDatasetName(),
+ index.getIndexName());
+ if (index.getIndexType() == IndexType.BTREE) {
+ btreeDataflowHelperFactories.add(getBTreeDataflowHelperFactory(ds, index, mergePolicyFactory,
+ mergePolicyFactoryProperties, storageProperties, spec));
+ btreeInfos.add(new IndexInfoOperatorDescriptor(indexSplitsAndConstraint.first,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER));
+ } else if (index.getIndexType() == IndexType.RTREE) {
+ rtreeDataflowHelperFactories.add(getRTreeDataflowHelperFactory(ds, index, mergePolicyFactory,
+ mergePolicyFactoryProperties, storageProperties, metadataProvider, spec));
+ rtreeInfos.add(new IndexInfoOperatorDescriptor(indexSplitsAndConstraint.first,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER));
+ }
+ }
+ }
+
+ ExternalDatasetIndexesAbortOperatorDescriptor op = new ExternalDatasetIndexesAbortOperatorDescriptor(spec,
+ filesIndexDataflowHelperFactory, filesIndexInfo, btreeDataflowHelperFactories, btreeInfos,
+ rtreeDataflowHelperFactories, rtreeInfos);
+
+ spec.addRoot(op);
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, op,
+ filesIndexSplitsAndConstraint.second);
+ spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+ return spec;
+
+ }
+
+ public static JobSpecification buildRecoverOp(Dataset ds, List<Index> indexes, AqlMetadataProvider metadataProvider)
+ throws AlgebricksException, AsterixException {
+ JobSpecification spec = JobSpecificationUtils.createJobSpecification();
+ IAsterixPropertiesProvider asterixPropertiesProvider = AsterixAppContextInfo.getInstance();
+ AsterixStorageProperties storageProperties = asterixPropertiesProvider.getStorageProperties();
+ Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils.getMergePolicyFactory(ds,
+ metadataProvider.getMetadataTxnContext());
+ ILSMMergePolicyFactory mergePolicyFactory = compactionInfo.first;
+ Map<String, String> mergePolicyFactoryProperties = compactionInfo.second;
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> filesIndexSplitsAndConstraint = metadataProvider
+ .splitProviderAndPartitionConstraintsForDataset(ds.getDataverseName(), ds.getDatasetName(),
+ getFilesIndexName(ds.getDatasetName()));
+ IFileSplitProvider filesIndexSplitProvider = filesIndexSplitsAndConstraint.first;
+ ExternalBTreeDataflowHelperFactory filesIndexDataflowHelperFactory = getFilesIndexDataflowHelperFactory(ds,
+ mergePolicyFactory, mergePolicyFactoryProperties, storageProperties, spec);
+ IndexInfoOperatorDescriptor filesIndexInfo = new IndexInfoOperatorDescriptor(filesIndexSplitProvider,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER);
+
+ ArrayList<ExternalBTreeWithBuddyDataflowHelperFactory> btreeDataflowHelperFactories = new ArrayList<ExternalBTreeWithBuddyDataflowHelperFactory>();
+ ArrayList<IndexInfoOperatorDescriptor> btreeInfos = new ArrayList<IndexInfoOperatorDescriptor>();
+ ArrayList<ExternalRTreeDataflowHelperFactory> rtreeDataflowHelperFactories = new ArrayList<ExternalRTreeDataflowHelperFactory>();
+ ArrayList<IndexInfoOperatorDescriptor> rtreeInfos = new ArrayList<IndexInfoOperatorDescriptor>();
+
+ for (Index index : indexes) {
+ if (isValidIndexName(index.getDatasetName(), index.getIndexName())) {
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> indexSplitsAndConstraint = metadataProvider
+ .splitProviderAndPartitionConstraintsForDataset(ds.getDataverseName(), ds.getDatasetName(),
+ index.getIndexName());
+ if (index.getIndexType() == IndexType.BTREE) {
+ btreeDataflowHelperFactories.add(getBTreeDataflowHelperFactory(ds, index, mergePolicyFactory,
+ mergePolicyFactoryProperties, storageProperties, spec));
+ btreeInfos.add(new IndexInfoOperatorDescriptor(indexSplitsAndConstraint.first,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER));
+ } else if (index.getIndexType() == IndexType.RTREE) {
+ rtreeDataflowHelperFactories.add(getRTreeDataflowHelperFactory(ds, index, mergePolicyFactory,
+ mergePolicyFactoryProperties, storageProperties, metadataProvider, spec));
+ rtreeInfos.add(new IndexInfoOperatorDescriptor(indexSplitsAndConstraint.first,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER));
+ }
+ }
+ }
+
+ ExternalDatasetIndexesRecoverOperatorDescriptor op = new ExternalDatasetIndexesRecoverOperatorDescriptor(spec,
+ filesIndexDataflowHelperFactory, filesIndexInfo, btreeDataflowHelperFactories, btreeInfos,
+ rtreeDataflowHelperFactories, rtreeInfos);
+
+ spec.addRoot(op);
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, op,
+ filesIndexSplitsAndConstraint.second);
+ spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+ return spec;
+ }
+
+ public static JobSpecification compactFilesIndexJobSpec(Dataset dataset, AqlMetadataProvider metadataProvider)
+ throws MetadataException, AlgebricksException {
+ JobSpecification spec = JobSpecificationUtils.createJobSpecification();
+ IAsterixPropertiesProvider asterixPropertiesProvider = AsterixAppContextInfo.getInstance();
+ AsterixStorageProperties storageProperties = asterixPropertiesProvider.getStorageProperties();
+ Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils.getMergePolicyFactory(dataset,
+ metadataProvider.getMetadataTxnContext());
+ ILSMMergePolicyFactory mergePolicyFactory = compactionInfo.first;
+ Map<String, String> mergePolicyFactoryProperties = compactionInfo.second;
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadataProvider
+ .splitProviderAndPartitionConstraintsForFilesIndex(dataset.getDataverseName(),
+ dataset.getDatasetName(), getFilesIndexName(dataset.getDatasetName()), true);
+ IFileSplitProvider secondaryFileSplitProvider = secondarySplitsAndConstraint.first;
+ ExternalBTreeDataflowHelperFactory indexDataflowHelperFactory = new ExternalBTreeDataflowHelperFactory(
+ mergePolicyFactory, mergePolicyFactoryProperties, new SecondaryIndexOperationTrackerProvider(
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties.getBloomFilterFalsePositiveRate(),
+ ExternalDatasetsRegistry.INSTANCE.getDatasetVersion(dataset));
+ LSMTreeIndexCompactOperatorDescriptor compactOp = new LSMTreeIndexCompactOperatorDescriptor(spec,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ secondaryFileSplitProvider, FilesIndexDescription.EXTERNAL_FILE_INDEX_TYPE_TRAITS,
+ FilesIndexDescription.FILES_INDEX_COMP_FACTORIES, new int[] { 0 }, indexDataflowHelperFactory,
+ NoOpOperationCallbackFactory.INSTANCE);
+ spec.addRoot(compactOp);
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, compactOp,
+ secondarySplitsAndConstraint.second);
+ spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+ return spec;
+ }
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java
index 5e3f7de..859e39c 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java
@@ -14,6 +14,7 @@
*/
package edu.uci.ics.asterix.file;
+import java.util.List;
import java.util.Map;
import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
@@ -24,6 +25,7 @@
import edu.uci.ics.asterix.metadata.MetadataException;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.ExternalFile;
import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
@@ -66,6 +68,17 @@
physicalOptimizationConfig);
return secondaryIndexHelper.buildLoadingJobSpec();
}
+
+ public static JobSpecification buildSecondaryIndexLoadingJobSpec(CompiledCreateIndexStatement createIndexStmt,
+ AqlMetadataProvider metadataProvider, List<ExternalFile> files) throws AsterixException, AlgebricksException {
+ SecondaryIndexOperationsHelper secondaryIndexHelper = SecondaryIndexOperationsHelper
+ .createIndexOperationsHelper(createIndexStmt.getIndexType(), createIndexStmt.getDataverseName(),
+ createIndexStmt.getDatasetName(), createIndexStmt.getIndexName(),
+ createIndexStmt.getKeyFields(), createIndexStmt.getGramLength(), metadataProvider,
+ physicalOptimizationConfig);
+ secondaryIndexHelper.setExternalFiles(files);
+ return secondaryIndexHelper.buildLoadingJobSpec();
+ }
public static JobSpecification buildDropSecondaryIndexJobSpec(CompiledIndexDropStatement indexDropStmt,
AqlMetadataProvider metadataProvider, Dataset dataset) throws AlgebricksException, MetadataException {
@@ -76,7 +89,7 @@
JobSpecification spec = JobSpecificationUtils.createJobSpecification();
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadataProvider
- .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverseName, datasetName, indexName);
+ .splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, indexName);
AsterixStorageProperties storageProperties = AsterixAppContextInfo.getInstance().getStorageProperties();
Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils.getMergePolicyFactory(dataset,
metadataProvider.getMetadataTxnContext());
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 47b2567..597e835 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
@@ -16,12 +16,17 @@
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.IAsterixPropertiesProvider;
import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeWithBuddyIOOperationCallbackFactory;
+import edu.uci.ics.asterix.metadata.feeds.ExternalDataScanOperatorDescriptor;
+import edu.uci.ics.asterix.metadata.utils.ExternalDatasetsRegistry;
import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
import edu.uci.ics.asterix.transaction.management.resource.LSMBTreeLocalResourceMetadata;
+import edu.uci.ics.asterix.transaction.management.resource.ExternalBTreeWithBuddyLocalResourceMetadata;
import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
@@ -35,9 +40,12 @@
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;
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.ExternalBTreeWithBuddyDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.lsm.common.dataflow.LSMTreeIndexCompactOperatorDescriptor;
import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactoryProvider;
@@ -55,21 +63,39 @@
JobSpecification spec = JobSpecificationUtils.createJobSpecification();
AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
- //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
- ILocalResourceMetadata localResourceMetadata = new LSMBTreeLocalResourceMetadata(
- secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields, true,
- dataset.getDatasetId(), mergePolicyFactory, mergePolicyFactoryProperties);
- ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
- localResourceMetadata, LocalResource.LSMBTreeResource);
-
+ ILocalResourceFactoryProvider localResourceFactoryProvider;
+ IIndexDataflowHelperFactory indexDataflowHelperFactory;
+ if (dataset.getDatasetType() == DatasetType.INTERNAL) {
+ //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
+ ILocalResourceMetadata localResourceMetadata = new LSMBTreeLocalResourceMetadata(
+ secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields,
+ true, dataset.getDatasetId(), mergePolicyFactory, mergePolicyFactoryProperties);
+ localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(localResourceMetadata,
+ LocalResource.LSMBTreeResource);
+ indexDataflowHelperFactory = new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
+ dataset.getDatasetId()), mergePolicyFactory, mergePolicyFactoryProperties,
+ new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
+ storageProperties.getBloomFilterFalsePositiveRate(), false);
+ } else {
+ // External dataset local resource and dataflow helper
+ int[] buddyBreeFields = new int[] { numSecondaryKeys };
+ ILocalResourceMetadata localResourceMetadata = new ExternalBTreeWithBuddyLocalResourceMetadata(
+ dataset.getDatasetId(), secondaryComparatorFactories, secondaryRecDesc.getTypeTraits(),
+ mergePolicyFactory, mergePolicyFactoryProperties, buddyBreeFields);
+ localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(localResourceMetadata,
+ LocalResource.ExternalBTreeWithBuddyResource);
+ indexDataflowHelperFactory = new ExternalBTreeWithBuddyDataflowHelperFactory(mergePolicyFactory,
+ mergePolicyFactoryProperties, new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMBTreeWithBuddyIOOperationCallbackFactory.INSTANCE,
+ storageProperties.getBloomFilterFalsePositiveRate(), buddyBreeFields,
+ ExternalDatasetsRegistry.INSTANCE.getDatasetVersion(dataset));
+ }
TreeIndexCreateOperatorDescriptor secondaryIndexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
- secondaryBloomFilterKeyFields, new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
- dataset.getDatasetId()), mergePolicyFactory, mergePolicyFactoryProperties,
- new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
- storageProperties.getBloomFilterFalsePositiveRate(), false), localResourceFactoryProvider,
+ secondaryBloomFilterKeyFields, indexDataflowHelperFactory, localResourceFactoryProvider,
NoOpOperationCallbackFactory.INSTANCE);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,
secondaryPartitionConstraint);
@@ -81,49 +107,100 @@
@Override
public JobSpecification buildLoadingJobSpec() throws AsterixException, AlgebricksException {
JobSpecification spec = JobSpecificationUtils.createJobSpecification();
+ if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+ /*
+ * In case of external data, this method is used to build loading jobs for both initial load on index creation
+ * and transaction load on dataset referesh
+ */
+ // Create external indexing scan operator
+ ExternalDataScanOperatorDescriptor primaryScanOp = createExternalIndexingOp(spec);
+ // Assign op.
+ AlgebricksMetaOperatorDescriptor asterixAssignOp = createExternalAssignOp(spec, numSecondaryKeys);
- // Create dummy key provider for feeding the primary index scan.
- AbstractOperatorDescriptor keyProviderOp = createDummyKeyProviderOp(spec);
+ // If any of the secondary fields are nullable, then add a select op that filters nulls.
+ AlgebricksMetaOperatorDescriptor selectOp = null;
+ if (anySecondaryKeyIsNullable) {
+ selectOp = createFilterNullsSelectOp(spec, numSecondaryKeys);
+ }
- // Create primary index scan op.
- BTreeSearchOperatorDescriptor primaryScanOp = createPrimaryIndexScanOp(spec);
+ // Sort by secondary keys.
+ ExternalSortOperatorDescriptor sortOp = createSortOp(spec, secondaryComparatorFactories, secondaryRecDesc);
- // Assign op.
- AlgebricksMetaOperatorDescriptor asterixAssignOp = createAssignOp(spec, primaryScanOp, numSecondaryKeys);
-
- // If any of the secondary fields are nullable, then add a select op that filters nulls.
- AlgebricksMetaOperatorDescriptor selectOp = null;
- if (anySecondaryKeyIsNullable) {
- selectOp = createFilterNullsSelectOp(spec, numSecondaryKeys);
- }
-
- // Sort by secondary keys.
- ExternalSortOperatorDescriptor sortOp = createSortOp(spec, secondaryComparatorFactories, secondaryRecDesc);
-
- AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
- // Create secondary BTree bulk load op.
- TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(
- spec,
- numSecondaryKeys,
- new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
- mergePolicyFactory, mergePolicyFactoryProperties, new SecondaryIndexOperationTrackerProvider(
- dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
- LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties
- .getBloomFilterFalsePositiveRate(), false), BTree.DEFAULT_FILL_FACTOR);
-
- // Connect the operators.
- spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
- spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, asterixAssignOp, 0);
- if (anySecondaryKeyIsNullable) {
- spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, selectOp, 0);
- spec.connect(new OneToOneConnectorDescriptor(spec), selectOp, 0, sortOp, 0);
+ AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
+ // Create secondary BTree bulk load op.
+ AbstractTreeIndexOperatorDescriptor secondaryBulkLoadOp;
+ ExternalBTreeWithBuddyDataflowHelperFactory dataflowHelperFactory = new ExternalBTreeWithBuddyDataflowHelperFactory(
+ mergePolicyFactory, mergePolicyFactoryProperties, new SecondaryIndexOperationTrackerProvider(
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMBTreeWithBuddyIOOperationCallbackFactory.INSTANCE,
+ storageProperties.getBloomFilterFalsePositiveRate(), new int[] { numSecondaryKeys },
+ ExternalDatasetsRegistry.INSTANCE.getDatasetVersion(dataset));
+ if (externalFiles != null) {
+ // Transaction load
+ secondaryBulkLoadOp = createExternalIndexBulkModifyOp(spec, numSecondaryKeys, dataflowHelperFactory,
+ BTree.DEFAULT_FILL_FACTOR);
+ } else {
+ // Initial load
+ secondaryBulkLoadOp = createTreeIndexBulkLoadOp(spec, numSecondaryKeys, dataflowHelperFactory,
+ BTree.DEFAULT_FILL_FACTOR);
+ }
+ // Connect the operators.
+ spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, asterixAssignOp, 0);
+ if (anySecondaryKeyIsNullable) {
+ spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, selectOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), selectOp, 0, sortOp, 0);
+ } else {
+ spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, sortOp, 0);
+ }
+ spec.connect(new OneToOneConnectorDescriptor(spec), sortOp, 0, secondaryBulkLoadOp, 0);
+ spec.addRoot(secondaryBulkLoadOp);
+ spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+ return spec;
} else {
- spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, sortOp, 0);
+ // Create dummy key provider for feeding the primary index scan.
+ AbstractOperatorDescriptor keyProviderOp = createDummyKeyProviderOp(spec);
+
+ // Create primary index scan op.
+ BTreeSearchOperatorDescriptor primaryScanOp = createPrimaryIndexScanOp(spec);
+
+ // Assign op.
+ AlgebricksMetaOperatorDescriptor asterixAssignOp = createAssignOp(spec, primaryScanOp, numSecondaryKeys);
+
+ // If any of the secondary fields are nullable, then add a select op that filters nulls.
+ AlgebricksMetaOperatorDescriptor selectOp = null;
+ if (anySecondaryKeyIsNullable) {
+ selectOp = createFilterNullsSelectOp(spec, numSecondaryKeys);
+ }
+
+ // Sort by secondary keys.
+ ExternalSortOperatorDescriptor sortOp = createSortOp(spec, secondaryComparatorFactories, secondaryRecDesc);
+
+ AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
+ // Create secondary BTree bulk load op.
+ TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(
+ spec,
+ numSecondaryKeys,
+ new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
+ mergePolicyFactory, mergePolicyFactoryProperties,
+ new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties
+ .getBloomFilterFalsePositiveRate(), false), BTree.DEFAULT_FILL_FACTOR);
+
+ // Connect the operators.
+ spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, asterixAssignOp, 0);
+ if (anySecondaryKeyIsNullable) {
+ spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, selectOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), selectOp, 0, sortOp, 0);
+ } else {
+ spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, sortOp, 0);
+ }
+ spec.connect(new OneToOneConnectorDescriptor(spec), sortOp, 0, secondaryBulkLoadOp, 0);
+ spec.addRoot(secondaryBulkLoadOp);
+ spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+ return spec;
}
- spec.connect(new OneToOneConnectorDescriptor(spec), sortOp, 0, secondaryBulkLoadOp, 0);
- spec.addRoot(secondaryBulkLoadOp);
- spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
- return spec;
}
@Override
@@ -131,15 +208,32 @@
JobSpecification spec = JobSpecificationUtils.createJobSpecification();
AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
- LSMTreeIndexCompactOperatorDescriptor compactOp = new LSMTreeIndexCompactOperatorDescriptor(spec,
- AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
- secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
- secondaryBloomFilterKeyFields, new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
- dataset.getDatasetId()), mergePolicyFactory, mergePolicyFactoryProperties,
- new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
- storageProperties.getBloomFilterFalsePositiveRate(), false),
- NoOpOperationCallbackFactory.INSTANCE);
+ LSMTreeIndexCompactOperatorDescriptor compactOp;
+ if (dataset.getDatasetType() == DatasetType.INTERNAL) {
+ compactOp = new LSMTreeIndexCompactOperatorDescriptor(spec,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider,
+ secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields,
+ new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
+ mergePolicyFactory, mergePolicyFactoryProperties,
+ new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties
+ .getBloomFilterFalsePositiveRate(), false), NoOpOperationCallbackFactory.INSTANCE);
+ } else {
+ // External dataset
+ compactOp = new LSMTreeIndexCompactOperatorDescriptor(spec,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider,
+ secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields,
+ new ExternalBTreeWithBuddyDataflowHelperFactory(mergePolicyFactory, mergePolicyFactoryProperties,
+ new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMBTreeWithBuddyIOOperationCallbackFactory.INSTANCE, storageProperties
+ .getBloomFilterFalsePositiveRate(), new int[] { numSecondaryKeys },
+ ExternalDatasetsRegistry.INSTANCE.getDatasetVersion(dataset)),
+ NoOpOperationCallbackFactory.INSTANCE);
+ }
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, compactOp,
secondaryPartitionConstraint);
spec.addRoot(compactOp);
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 ca049a6..851de4a 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
@@ -22,6 +22,7 @@
import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
+import edu.uci.ics.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
@@ -31,6 +32,7 @@
import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
import edu.uci.ics.asterix.common.transactions.IRecoveryManager.ResourceType;
import edu.uci.ics.asterix.common.transactions.JobId;
+import edu.uci.ics.asterix.external.indexing.operators.ExternalIndexBulkModifyOperatorDescriptor;
import edu.uci.ics.asterix.formats.nontagged.AqlBinaryBooleanInspectorImpl;
import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
@@ -38,7 +40,10 @@
import edu.uci.ics.asterix.metadata.MetadataException;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.ExternalFile;
import edu.uci.ics.asterix.metadata.entities.Index;
+import edu.uci.ics.asterix.metadata.external.IndexingConstants;
+import edu.uci.ics.asterix.metadata.feeds.ExternalDataScanOperatorDescriptor;
import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.IAType;
@@ -123,6 +128,8 @@
protected ILSMMergePolicyFactory mergePolicyFactory;
protected Map<String, String> mergePolicyFactoryProperties;
+ protected List<ExternalFile> externalFiles;
+
// Prevent public construction. Should be created via createIndexCreator().
protected SecondaryIndexOperationsHelper(PhysicalOptimizationConfig physOptConf,
IAsterixPropertiesProvider propertiesProvider) {
@@ -178,24 +185,25 @@
if (dataset == null) {
throw new AsterixException("Unknown dataset " + datasetName);
}
- if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
- throw new AsterixException("Cannot index an external dataset (" + datasetName + ").");
- }
+
itemType = (ARecordType) metadataProvider.findType(dataset.getDataverseName(), dataset.getItemTypeName());
payloadSerde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType);
- numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
numSecondaryKeys = secondaryKeyFields.size();
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> primarySplitsAndConstraint = metadataProvider
- .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverseName, datasetName, datasetName);
- primaryFileSplitProvider = primarySplitsAndConstraint.first;
- primaryPartitionConstraint = primarySplitsAndConstraint.second;
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadataProvider
- .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverseName, datasetName,
- secondaryIndexName);
+ .splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, secondaryIndexName);
secondaryFileSplitProvider = secondarySplitsAndConstraint.first;
secondaryPartitionConstraint = secondarySplitsAndConstraint.second;
- // Must be called in this order.
- setPrimaryRecDescAndComparators();
+
+ if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+ numPrimaryKeys = ExternalIndexingOperations.getRIDSize(dataset);
+ } else {
+ numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> primarySplitsAndConstraint = metadataProvider
+ .splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, datasetName);
+ primaryFileSplitProvider = primarySplitsAndConstraint.first;
+ primaryPartitionConstraint = primarySplitsAndConstraint.second;
+ setPrimaryRecDescAndComparators();
+ }
setSecondaryRecDescAndComparators(indexType, secondaryKeyFields, gramLength, metadataProvider);
numElementsHint = metadataProvider.getCardinalityPerPartitionHint(dataset);
Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils.getMergePolicyFactory(dataset,
@@ -245,9 +253,11 @@
ITypeTraitProvider typeTraitProvider = metadataProvider.getFormat().getTypeTraitProvider();
IBinaryComparatorFactoryProvider comparatorFactoryProvider = metadataProvider.getFormat()
.getBinaryComparatorFactoryProvider();
+ // Record column is 0 for external datasets, numPrimaryKeys for internal ones
+ int recordColumn = dataset.getDatasetType() == DatasetType.INTERNAL ? numPrimaryKeys : 0;
for (int i = 0; i < numSecondaryKeys; i++) {
secondaryFieldAccessEvalFactories[i] = metadataProvider.getFormat().getFieldAccessEvaluatorFactory(
- itemType, secondaryKeyFields.get(i), numPrimaryKeys);
+ itemType, secondaryKeyFields.get(i), recordColumn);
Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(secondaryKeyFields.get(i), itemType);
IAType keyType = keyTypePair.first;
anySecondaryKeyIsNullable = anySecondaryKeyIsNullable || keyTypePair.second;
@@ -257,12 +267,23 @@
secondaryTypeTraits[i] = typeTraitProvider.getTypeTrait(keyType);
secondaryBloomFilterKeyFields[i] = i;
}
- // Add serializers and comparators for primary index fields.
- for (int i = 0; i < numPrimaryKeys; i++) {
- secondaryRecFields[numSecondaryKeys + i] = primaryRecDesc.getFields()[i];
- secondaryTypeTraits[numSecondaryKeys + i] = primaryRecDesc.getTypeTraits()[i];
- if (indexType != IndexType.RTREE) {
- secondaryComparatorFactories[numSecondaryKeys + i] = primaryComparatorFactories[i];
+ if (dataset.getDatasetType() == DatasetType.INTERNAL) {
+ // Add serializers and comparators for primary index fields.
+ for (int i = 0; i < numPrimaryKeys; i++) {
+ secondaryRecFields[numSecondaryKeys + i] = primaryRecDesc.getFields()[i];
+ secondaryTypeTraits[numSecondaryKeys + i] = primaryRecDesc.getTypeTraits()[i];
+ if (indexType != IndexType.RTREE) {
+ secondaryComparatorFactories[numSecondaryKeys + i] = primaryComparatorFactories[i];
+ }
+ }
+ } else {
+ // Add serializers and comparators for RID fields.
+ for (int i = 0; i < numPrimaryKeys; i++) {
+ secondaryRecFields[numSecondaryKeys + i] = IndexingConstants.getSerializerDeserializer(i);
+ secondaryTypeTraits[numSecondaryKeys + i] = IndexingConstants.getTypeTraits(i);
+ if (indexType != IndexType.RTREE) {
+ secondaryComparatorFactories[numSecondaryKeys + i] = IndexingConstants.getComparatorFactory(i);
+ }
}
}
secondaryRecDesc = new RecordDescriptor(secondaryRecFields, secondaryTypeTraits);
@@ -314,8 +335,9 @@
new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
mergePolicyFactory, mergePolicyFactoryProperties, new PrimaryIndexOperationTrackerProvider(
dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
- LSMBTreeIOOperationCallbackFactory.INSTANCE,
- storageProperties.getBloomFilterFalsePositiveRate(), true), false, false, null, searchCallbackFactory);
+ LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties
+ .getBloomFilterFalsePositiveRate(), true), false, false, null,
+ searchCallbackFactory);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, primarySearchOp,
primaryPartitionConstraint);
return primarySearchOp;
@@ -409,4 +431,100 @@
primaryPartitionConstraint);
return asterixSelectOp;
}
+
+ // This method creates a source indexing operator for external data
+ protected ExternalDataScanOperatorDescriptor createExternalIndexingOp(JobSpecification spec)
+ throws AlgebricksException, AsterixException {
+ // A record + primary keys
+ ISerializerDeserializer[] serdes = new ISerializerDeserializer[1 + numPrimaryKeys];
+ ITypeTraits[] typeTraits = new ITypeTraits[1 + numPrimaryKeys];
+ // payload serde and type traits for the record slot
+ serdes[0] = payloadSerde;
+ typeTraits[0] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(itemType);
+ // serdes and type traits for rid fields
+ for (int i = 1; i < serdes.length; i++) {
+ serdes[i] = IndexingConstants.getSerializerDeserializer(i - 1);
+ typeTraits[i] = IndexingConstants.getTypeTraits(i - 1);
+ }
+ // output record desc
+ RecordDescriptor indexerDesc = new RecordDescriptor(serdes, typeTraits);
+
+ // Create the operator and its partition constraits
+ Pair<ExternalDataScanOperatorDescriptor, AlgebricksPartitionConstraint> indexingOpAndConstraints;
+ try {
+ indexingOpAndConstraints = ExternalIndexingOperations.createExternalIndexingOp(spec, metadataProvider,
+ dataset, itemType, indexerDesc, externalFiles);
+ } catch (Exception e) {
+ throw new AlgebricksException(e);
+ }
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, indexingOpAndConstraints.first,
+ indexingOpAndConstraints.second);
+
+ // Set the primary partition constraints to this partition constraints
+ primaryPartitionConstraint = indexingOpAndConstraints.second;
+ return indexingOpAndConstraints.first;
+ }
+
+ 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++) {
+ outColumns[i] = i + numPrimaryKeys + 1;
+ projectionList[i] = i + numPrimaryKeys + 1;
+ }
+
+ IScalarEvaluatorFactory[] sefs = new IScalarEvaluatorFactory[secondaryFieldAccessEvalFactories.length];
+ for (int i = 0; i < secondaryFieldAccessEvalFactories.length; ++i) {
+ sefs[i] = new LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.ScalarEvaluatorFactoryAdapter(
+ secondaryFieldAccessEvalFactories[i]);
+ }
+ //add External RIDs to the projection list
+ for (int i = 0; i < numPrimaryKeys; i++) {
+ projectionList[numSecondaryKeys + i] = i + 1;
+ }
+
+ AssignRuntimeFactory assign = new AssignRuntimeFactory(outColumns, sefs, projectionList);
+ AlgebricksMetaOperatorDescriptor asterixAssignOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
+ new IPushRuntimeFactory[] { assign }, new RecordDescriptor[] { secondaryRecDesc });
+ return asterixAssignOp;
+ }
+
+ protected ExternalIndexBulkModifyOperatorDescriptor createExternalIndexBulkModifyOp(JobSpecification spec,
+ int numSecondaryKeyFields, IIndexDataflowHelperFactory dataflowHelperFactory, float fillFactor)
+ throws MetadataException, AlgebricksException {
+ int[] fieldPermutation = new int[numSecondaryKeyFields + numPrimaryKeys];
+ for (int i = 0; i < numSecondaryKeyFields + numPrimaryKeys; i++) {
+ fieldPermutation[i] = i;
+ }
+ // create a list of file ids
+ int numOfDeletedFiles = 0;
+ for (ExternalFile file : externalFiles) {
+ if (file.getPendingOp() == ExternalFilePendingOp.PENDING_DROP_OP)
+ numOfDeletedFiles++;
+ }
+ int[] deletedFiles = new int[numOfDeletedFiles];
+ int i = 0;
+ for (ExternalFile file : externalFiles) {
+ if (file.getPendingOp() == ExternalFilePendingOp.PENDING_DROP_OP) {
+ deletedFiles[i] = file.getFileNumber();
+ }
+ }
+ ExternalIndexBulkModifyOperatorDescriptor treeIndexBulkLoadOp = new ExternalIndexBulkModifyOperatorDescriptor(
+ spec, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider,
+ secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields,
+ dataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE, deletedFiles, fieldPermutation,
+ fillFactor, numElementsHint);
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, treeIndexBulkLoadOp,
+ secondaryPartitionConstraint);
+ return treeIndexBulkLoadOp;
+ }
+
+ public List<ExternalFile> getExternalFiles() {
+ return externalFiles;
+ }
+
+ public void setExternalFiles(List<ExternalFile> externalFiles) {
+ this.externalFiles = externalFiles;
+ }
}
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 225ee53..37aa427 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
@@ -18,6 +18,7 @@
import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
+import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
@@ -29,11 +30,15 @@
import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
import edu.uci.ics.asterix.metadata.entities.Index;
+import edu.uci.ics.asterix.metadata.external.IndexingConstants;
+import edu.uci.ics.asterix.metadata.feeds.ExternalDataScanOperatorDescriptor;
+import edu.uci.ics.asterix.metadata.utils.ExternalDatasetsRegistry;
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.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
import edu.uci.ics.asterix.transaction.management.resource.LSMRTreeLocalResourceMetadata;
+import edu.uci.ics.asterix.transaction.management.resource.ExternalRTreeLocalResourceMetadata;
import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
@@ -53,10 +58,13 @@
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;
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.common.dataflow.LSMTreeIndexCompactOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.dataflow.ExternalRTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.lsm.rtree.dataflow.LSMRTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactoryProvider;
@@ -80,26 +88,50 @@
JobSpecification spec = JobSpecificationUtils.createJobSpecification();
AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
- //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
- ILocalResourceMetadata localResourceMetadata = new LSMRTreeLocalResourceMetadata(
- secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, primaryComparatorFactories,
- valueProviderFactories, RTreePolicyType.RTREE, AqlMetadataProvider.proposeLinearizer(keyType,
- secondaryComparatorFactories.length), dataset.getDatasetId(), mergePolicyFactory,
- mergePolicyFactoryProperties, primaryKeyFields);
- ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
- localResourceMetadata, LocalResource.LSMRTreeResource);
+ IIndexDataflowHelperFactory indexDataflowHelperFactory;
+ ILocalResourceFactoryProvider localResourceFactoryProvider;
+ if (dataset.getDatasetType() == DatasetType.INTERNAL) {
+ //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
+ ILocalResourceMetadata localResourceMetadata = new LSMRTreeLocalResourceMetadata(
+ secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, primaryComparatorFactories,
+ valueProviderFactories, RTreePolicyType.RTREE, AqlMetadataProvider.proposeLinearizer(keyType,
+ secondaryComparatorFactories.length), dataset.getDatasetId(), mergePolicyFactory,
+ mergePolicyFactoryProperties, primaryKeyFields);
+ localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(localResourceMetadata,
+ LocalResource.LSMRTreeResource);
+ indexDataflowHelperFactory = new LSMRTreeDataflowHelperFactory(valueProviderFactories,
+ RTreePolicyType.RTREE, primaryComparatorFactories, new AsterixVirtualBufferCacheProvider(
+ dataset.getDatasetId()), mergePolicyFactory, mergePolicyFactoryProperties,
+ new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMRTreeIOOperationCallbackFactory.INSTANCE,
+ AqlMetadataProvider.proposeLinearizer(keyType, secondaryComparatorFactories.length),
+ storageProperties.getBloomFilterFalsePositiveRate(), primaryKeyFields);
+ } else {
+ // External dataset
+ // Prepare a LocalResourceMetadata which will be stored in NC's local resource repository
+ ILocalResourceMetadata localResourceMetadata = new ExternalRTreeLocalResourceMetadata(
+ secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
+ ExternalIndexingOperations.getBuddyBtreeComparatorFactories(), valueProviderFactories,
+ RTreePolicyType.RTREE, AqlMetadataProvider.proposeLinearizer(keyType,
+ secondaryComparatorFactories.length), dataset.getDatasetId(), mergePolicyFactory,
+ mergePolicyFactoryProperties, primaryKeyFields);
+ localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(localResourceMetadata,
+ LocalResource.ExternalRTreeResource);
+
+ indexDataflowHelperFactory = new ExternalRTreeDataflowHelperFactory(valueProviderFactories,
+ RTreePolicyType.RTREE, ExternalIndexingOperations.getBuddyBtreeComparatorFactories(),
+ mergePolicyFactory, mergePolicyFactoryProperties, new SecondaryIndexOperationTrackerProvider(
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMRTreeIOOperationCallbackFactory.INSTANCE, AqlMetadataProvider.proposeLinearizer(keyType,
+ secondaryComparatorFactories.length), storageProperties.getBloomFilterFalsePositiveRate(),
+ new int[] { numNestedSecondaryKeyFields },
+ ExternalDatasetsRegistry.INSTANCE.getDatasetVersion(dataset));
+ }
TreeIndexCreateOperatorDescriptor secondaryIndexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, null,
- new LSMRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
- primaryComparatorFactories, new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
- mergePolicyFactory, mergePolicyFactoryProperties, new SecondaryIndexOperationTrackerProvider(
- dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
- LSMRTreeIOOperationCallbackFactory.INSTANCE, AqlMetadataProvider.proposeLinearizer(keyType,
- secondaryComparatorFactories.length), storageProperties
- .getBloomFilterFalsePositiveRate(), primaryKeyFields),
- localResourceFactoryProvider, NoOpOperationCallbackFactory.INSTANCE);
+ indexDataflowHelperFactory, localResourceFactoryProvider, NoOpOperationCallbackFactory.INSTANCE);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,
secondaryPartitionConstraint);
spec.addRoot(secondaryIndexCreateOp);
@@ -125,8 +157,9 @@
}
int numDimensions = NonTaggedFormatUtil.getNumDimensions(spatialType.getTypeTag());
numNestedSecondaryKeyFields = numDimensions * 2;
+ int recordColumn = dataset.getDatasetType() == DatasetType.INTERNAL ? numPrimaryKeys : 0;
secondaryFieldAccessEvalFactories = metadata.getFormat().createMBRFactory(itemType, secondaryKeyFields.get(0),
- numPrimaryKeys, numDimensions);
+ recordColumn, numDimensions);
secondaryComparatorFactories = new IBinaryComparatorFactory[numNestedSecondaryKeyFields];
valueProviderFactories = new IPrimitiveValueProviderFactory[numNestedSecondaryKeyFields];
ISerializerDeserializer[] secondaryRecFields = new ISerializerDeserializer[numPrimaryKeys
@@ -144,9 +177,16 @@
valueProviderFactories[i] = AqlPrimitiveValueProviderFactory.INSTANCE;
}
// Add serializers and comparators for primary index fields.
- for (int i = 0; i < numPrimaryKeys; i++) {
- secondaryRecFields[numNestedSecondaryKeyFields + i] = primaryRecDesc.getFields()[i];
- secondaryTypeTraits[numNestedSecondaryKeyFields + i] = primaryRecDesc.getTypeTraits()[i];
+ if (dataset.getDatasetType() == DatasetType.INTERNAL) {
+ for (int i = 0; i < numPrimaryKeys; i++) {
+ secondaryRecFields[numNestedSecondaryKeyFields + i] = primaryRecDesc.getFields()[i];
+ secondaryTypeTraits[numNestedSecondaryKeyFields + i] = primaryRecDesc.getTypeTraits()[i];
+ }
+ } else {
+ for (int i = 0; i < numPrimaryKeys; i++) {
+ secondaryRecFields[numNestedSecondaryKeyFields + i] = IndexingConstants.getSerializerDeserializer(i);
+ secondaryTypeTraits[numNestedSecondaryKeyFields + i] = IndexingConstants.getTypeTraits(i);
+ }
}
secondaryRecDesc = new RecordDescriptor(secondaryRecFields, secondaryTypeTraits);
primaryKeyFields = new int[numPrimaryKeys];
@@ -158,51 +198,107 @@
@Override
public JobSpecification buildLoadingJobSpec() throws AsterixException, AlgebricksException {
JobSpecification spec = JobSpecificationUtils.createJobSpecification();
+ if (dataset.getDatasetType() == DatasetType.INTERNAL) {
+ // Create dummy key provider for feeding the primary index scan.
+ AbstractOperatorDescriptor keyProviderOp = createDummyKeyProviderOp(spec);
- // Create dummy key provider for feeding the primary index scan.
- AbstractOperatorDescriptor keyProviderOp = createDummyKeyProviderOp(spec);
+ // Create primary index scan op.
+ BTreeSearchOperatorDescriptor primaryScanOp = createPrimaryIndexScanOp(spec);
- // Create primary index scan op.
- BTreeSearchOperatorDescriptor primaryScanOp = createPrimaryIndexScanOp(spec);
+ // Assign op.
+ AlgebricksMetaOperatorDescriptor asterixAssignOp = createAssignOp(spec, primaryScanOp,
+ numNestedSecondaryKeyFields);
- // Assign op.
- AlgebricksMetaOperatorDescriptor asterixAssignOp = createAssignOp(spec, primaryScanOp,
- numNestedSecondaryKeyFields);
+ // If any of the secondary fields are nullable, then add a select op that filters nulls.
+ AlgebricksMetaOperatorDescriptor selectOp = null;
+ if (anySecondaryKeyIsNullable) {
+ selectOp = createFilterNullsSelectOp(spec, numNestedSecondaryKeyFields);
+ }
- // If any of the secondary fields are nullable, then add a select op that filters nulls.
- AlgebricksMetaOperatorDescriptor selectOp = null;
- if (anySecondaryKeyIsNullable) {
- selectOp = createFilterNullsSelectOp(spec, numNestedSecondaryKeyFields);
- }
+ // Sort by secondary keys.
+ ExternalSortOperatorDescriptor sortOp = createSortOp(spec,
+ new IBinaryComparatorFactory[] { AqlMetadataProvider.proposeLinearizer(keyType,
+ secondaryComparatorFactories.length) }, secondaryRecDesc);
- // Sort by secondary keys.
- ExternalSortOperatorDescriptor sortOp = createSortOp(spec,
- new IBinaryComparatorFactory[] { AqlMetadataProvider.proposeLinearizer(keyType,
- secondaryComparatorFactories.length) }, secondaryRecDesc);
-
- AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
- // Create secondary RTree bulk load op.
- TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(spec,
- numNestedSecondaryKeyFields, new LSMRTreeDataflowHelperFactory(valueProviderFactories,
- RTreePolicyType.RTREE, primaryComparatorFactories, new AsterixVirtualBufferCacheProvider(
- dataset.getDatasetId()), mergePolicyFactory, mergePolicyFactoryProperties,
- new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMRTreeIOOperationCallbackFactory.INSTANCE,
- AqlMetadataProvider.proposeLinearizer(keyType, secondaryComparatorFactories.length),
- storageProperties.getBloomFilterFalsePositiveRate(), primaryKeyFields),
- BTree.DEFAULT_FILL_FACTOR);
- // Connect the operators.
- spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
- spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, asterixAssignOp, 0);
- if (anySecondaryKeyIsNullable) {
- spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, selectOp, 0);
- spec.connect(new OneToOneConnectorDescriptor(spec), selectOp, 0, sortOp, 0);
+ AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
+ // Create secondary RTree bulk load op.
+ TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(
+ spec,
+ numNestedSecondaryKeyFields,
+ new LSMRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
+ primaryComparatorFactories, new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
+ mergePolicyFactory, mergePolicyFactoryProperties,
+ new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMRTreeIOOperationCallbackFactory.INSTANCE, AqlMetadataProvider.proposeLinearizer(keyType,
+ secondaryComparatorFactories.length), storageProperties
+ .getBloomFilterFalsePositiveRate(), primaryKeyFields), BTree.DEFAULT_FILL_FACTOR);
+ // Connect the operators.
+ spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, asterixAssignOp, 0);
+ if (anySecondaryKeyIsNullable) {
+ spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, selectOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), selectOp, 0, sortOp, 0);
+ } else {
+ spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, sortOp, 0);
+ }
+ spec.connect(new OneToOneConnectorDescriptor(spec), sortOp, 0, secondaryBulkLoadOp, 0);
+ spec.addRoot(secondaryBulkLoadOp);
+ spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
} else {
- spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, sortOp, 0);
+ // External dataset
+ /*
+ * 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
+ */
+ // Create external indexing scan operator
+ ExternalDataScanOperatorDescriptor primaryScanOp = createExternalIndexingOp(spec);
+ // Assign op.
+ AlgebricksMetaOperatorDescriptor asterixAssignOp = createExternalAssignOp(spec, numNestedSecondaryKeyFields);
+
+ // If any of the secondary fields are nullable, then add a select op that filters nulls.
+ AlgebricksMetaOperatorDescriptor selectOp = null;
+ if (anySecondaryKeyIsNullable) {
+ selectOp = createFilterNullsSelectOp(spec, numSecondaryKeys);
+ }
+
+ // Sort by secondary keys.
+ ExternalSortOperatorDescriptor sortOp = createSortOp(spec,
+ new IBinaryComparatorFactory[] { AqlMetadataProvider.proposeLinearizer(keyType,
+ secondaryComparatorFactories.length) }, secondaryRecDesc);
+ AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
+
+ // Create the dataflow helper factory
+ ExternalRTreeDataflowHelperFactory dataflowHelperFactory = new ExternalRTreeDataflowHelperFactory(
+ valueProviderFactories, RTreePolicyType.RTREE, primaryComparatorFactories, mergePolicyFactory,
+ mergePolicyFactoryProperties, new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMRTreeIOOperationCallbackFactory.INSTANCE,
+ AqlMetadataProvider.proposeLinearizer(keyType, secondaryComparatorFactories.length),
+ storageProperties.getBloomFilterFalsePositiveRate(), new int[] { numNestedSecondaryKeyFields },
+ ExternalDatasetsRegistry.INSTANCE.getDatasetVersion(dataset));
+ // Create secondary RTree bulk load op.
+ AbstractTreeIndexOperatorDescriptor secondaryBulkLoadOp;
+ if (externalFiles != null) {
+ // Transaction load
+ secondaryBulkLoadOp = createExternalIndexBulkModifyOp(spec, numNestedSecondaryKeyFields,
+ dataflowHelperFactory, BTree.DEFAULT_FILL_FACTOR);
+ } else {
+ // Initial load
+ secondaryBulkLoadOp = createTreeIndexBulkLoadOp(spec, numNestedSecondaryKeyFields,
+ dataflowHelperFactory, BTree.DEFAULT_FILL_FACTOR);
+ }
+ // Connect the operators.
+ spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, asterixAssignOp, 0);
+ if (anySecondaryKeyIsNullable) {
+ spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, selectOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), selectOp, 0, sortOp, 0);
+ } else {
+ spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, sortOp, 0);
+ }
+ spec.connect(new OneToOneConnectorDescriptor(spec), sortOp, 0, secondaryBulkLoadOp, 0);
+ spec.addRoot(secondaryBulkLoadOp);
+ spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
}
- spec.connect(new OneToOneConnectorDescriptor(spec), sortOp, 0, secondaryBulkLoadOp, 0);
- spec.addRoot(secondaryBulkLoadOp);
- spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
return spec;
}
@@ -211,17 +307,37 @@
JobSpecification spec = JobSpecificationUtils.createJobSpecification();
AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
- LSMTreeIndexCompactOperatorDescriptor compactOp = new LSMTreeIndexCompactOperatorDescriptor(spec,
- AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
- secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
- secondaryBloomFilterKeyFields, new LSMRTreeDataflowHelperFactory(valueProviderFactories,
- RTreePolicyType.RTREE, primaryComparatorFactories, new AsterixVirtualBufferCacheProvider(
- dataset.getDatasetId()), mergePolicyFactory, mergePolicyFactoryProperties,
- new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMRTreeIOOperationCallbackFactory.INSTANCE,
- AqlMetadataProvider.proposeLinearizer(keyType, secondaryComparatorFactories.length),
- storageProperties.getBloomFilterFalsePositiveRate(), primaryKeyFields),
- NoOpOperationCallbackFactory.INSTANCE);
+ LSMTreeIndexCompactOperatorDescriptor compactOp;
+ if (dataset.getDatasetType() == DatasetType.INTERNAL) {
+ compactOp = new LSMTreeIndexCompactOperatorDescriptor(spec,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider,
+ secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields,
+ new LSMRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
+ primaryComparatorFactories, new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
+ mergePolicyFactory, mergePolicyFactoryProperties,
+ new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMRTreeIOOperationCallbackFactory.INSTANCE, AqlMetadataProvider.proposeLinearizer(keyType,
+ secondaryComparatorFactories.length), storageProperties
+ .getBloomFilterFalsePositiveRate(), primaryKeyFields),
+ NoOpOperationCallbackFactory.INSTANCE);
+ } else {
+ // External dataset
+ compactOp = new LSMTreeIndexCompactOperatorDescriptor(spec,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider,
+ secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields,
+ new ExternalRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
+ primaryComparatorFactories, mergePolicyFactory, mergePolicyFactoryProperties,
+ new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMRTreeIOOperationCallbackFactory.INSTANCE, AqlMetadataProvider.proposeLinearizer(keyType,
+ secondaryComparatorFactories.length), storageProperties
+ .getBloomFilterFalsePositiveRate(), new int[] { numNestedSecondaryKeyFields },
+ ExternalDatasetsRegistry.INSTANCE.getDatasetVersion(dataset)),
+ NoOpOperationCallbackFactory.INSTANCE);
+ }
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, compactOp,
secondaryPartitionConstraint);
spec.addRoot(compactOp);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/AsterixGlobalRecoveryManager.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/AsterixGlobalRecoveryManager.java
new file mode 100644
index 0000000..0af6a7e
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/AsterixGlobalRecoveryManager.java
@@ -0,0 +1,212 @@
+/*
+ * 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.hyracks.bootstrap;
+
+import java.util.List;
+import java.util.Set;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+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.DatasetConfig.ExternalFilePendingOp;
+import edu.uci.ics.asterix.file.ExternalIndexingOperations;
+import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
+import edu.uci.ics.asterix.metadata.api.IClusterEventsSubscriber;
+import edu.uci.ics.asterix.metadata.api.IClusterManagementWork;
+import edu.uci.ics.asterix.metadata.bootstrap.MetadataConstants;
+import edu.uci.ics.asterix.metadata.cluster.IClusterManagementWorkResponse;
+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.entities.ExternalFile;
+import edu.uci.ics.asterix.metadata.entities.Index;
+import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
+import edu.uci.ics.asterix.om.util.AsterixClusterProperties.State;
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class AsterixGlobalRecoveryManager implements IClusterEventsSubscriber {
+
+ private static State state;
+ private static final Logger LOGGER = Logger.getLogger(AsterixGlobalRecoveryManager.class.getName());
+ private HyracksConnection hcc;
+ public static AsterixGlobalRecoveryManager INSTANCE;
+
+ public AsterixGlobalRecoveryManager(HyracksConnection hcc) throws Exception {
+ state = AsterixClusterProperties.INSTANCE.getState();
+ this.hcc = hcc;
+ }
+
+ @Override
+ public Set<IClusterManagementWork> notifyNodeFailure(Set<String> deadNodeIds) {
+ state = AsterixClusterProperties.INSTANCE.getState();
+ AsterixClusterProperties.INSTANCE.setGlobalRecoveryCompleted(false);
+ return null;
+ }
+
+ @Override
+ public Set<IClusterManagementWork> notifyNodeJoin(String joinedNodeId) {
+ // perform global recovery if state changed to active
+ final State newState = AsterixClusterProperties.INSTANCE.getState();
+ boolean needToRecover = !newState.equals(state) && (newState == State.ACTIVE);
+ if (needToRecover) {
+ Thread recoveryThread = new Thread(new Runnable() {
+ @Override
+ public void run() {
+ LOGGER.info("Starting AsterixDB's Global Recovery");
+ MetadataTransactionContext mdTxnCtx = null;
+ try {
+ Thread.sleep(4000);
+ MetadataManager.INSTANCE.init();
+ // Loop over datasets
+ mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ List<Dataverse> dataverses = MetadataManager.INSTANCE.getDataverses(mdTxnCtx);
+ for (Dataverse dataverse : dataverses) {
+ if (!dataverse.getDataverseName().equals(MetadataConstants.METADATA_DATAVERSE_NAME)) {
+ AqlMetadataProvider metadataProvider = new AqlMetadataProvider(dataverse);
+ List<Dataset> datasets = MetadataManager.INSTANCE.getDataverseDatasets(mdTxnCtx,
+ dataverse.getDataverseName());
+ for (Dataset dataset : datasets) {
+ if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+ // External dataset
+ // Get indexes
+ List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx,
+ dataset.getDataverseName(), dataset.getDatasetName());
+ if (indexes.size() > 0) {
+ // Get the state of the dataset
+ ExternalDatasetDetails dsd = (ExternalDatasetDetails) dataset
+ .getDatasetDetails();
+ ExternalDatasetTransactionState datasetState = dsd.getState();
+ if (datasetState == ExternalDatasetTransactionState.BEGIN) {
+ List<ExternalFile> files = MetadataManager.INSTANCE
+ .getDatasetExternalFiles(mdTxnCtx, dataset);
+ // if persumed abort, roll backward
+ // 1. delete all pending files
+ for (ExternalFile file : files) {
+ if (file.getPendingOp() != ExternalFilePendingOp.PENDING_NO_OP) {
+ MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, file);
+ }
+ }
+ // 2. clean artifacts in NCs
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+ JobSpecification jobSpec = ExternalIndexingOperations.buildAbortOp(
+ dataset, indexes, metadataProvider);
+ executeHyracksJob(jobSpec);
+ // 3. correct the dataset state
+ ((ExternalDatasetDetails) dataset.getDatasetDetails())
+ .setState(ExternalDatasetTransactionState.COMMIT);
+ MetadataManager.INSTANCE.updateDataset(mdTxnCtx, dataset);
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ } else if (datasetState == ExternalDatasetTransactionState.READY_TO_COMMIT) {
+ List<ExternalFile> files = MetadataManager.INSTANCE
+ .getDatasetExternalFiles(mdTxnCtx, dataset);
+ // if ready to commit, roll forward
+ // 1. commit indexes in NCs
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+ JobSpecification jobSpec = ExternalIndexingOperations.buildRecoverOp(
+ dataset, indexes, metadataProvider);
+ executeHyracksJob(jobSpec);
+ // 2. add pending files in metadata
+ for (ExternalFile file : files) {
+ if (file.getPendingOp() == ExternalFilePendingOp.PENDING_ADD_OP) {
+ MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, file);
+ file.setPendingOp(ExternalFilePendingOp.PENDING_NO_OP);
+ MetadataManager.INSTANCE.addExternalFile(mdTxnCtx, file);
+ } else if (file.getPendingOp() == ExternalFilePendingOp.PENDING_DROP_OP) {
+ // find original file
+ for (ExternalFile originalFile : files) {
+ if (originalFile.getFileName().equals(file.getFileName())) {
+ MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx,
+ file);
+ MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx,
+ originalFile);
+ break;
+ }
+ }
+ } else if (file.getPendingOp() == ExternalFilePendingOp.PENDING_APPEND_OP) {
+ // find original file
+ for (ExternalFile originalFile : files) {
+ if (originalFile.getFileName().equals(file.getFileName())) {
+ MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx,
+ file);
+ MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx,
+ originalFile);
+ originalFile.setSize(file.getSize());
+ MetadataManager.INSTANCE.addExternalFile(mdTxnCtx,
+ originalFile);
+ }
+ }
+ }
+ }
+ // 3. correct the dataset state
+ ((ExternalDatasetDetails) dataset.getDatasetDetails())
+ .setState(ExternalDatasetTransactionState.COMMIT);
+ MetadataManager.INSTANCE.updateDataset(mdTxnCtx, dataset);
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ }
+ }
+ }
+ }
+ }
+ }
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ } catch (Exception e) {
+ // This needs to be fixed <-- Needs to shutdown the system -->
+ /*
+ * Note: Throwing this illegal state exception will terminate this thread
+ * and feeds listeners will not be notified.
+ */
+ LOGGER.severe("Global recovery was not completed successfully" + e);
+ try {
+ MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+ } catch (Exception e1) {
+ if (LOGGER.isLoggable(Level.SEVERE)) {
+ LOGGER.severe("Exception in aborting" + e.getMessage());
+ }
+ throw new IllegalStateException(e1);
+ }
+ }
+ AsterixClusterProperties.INSTANCE.setGlobalRecoveryCompleted(true);
+ LOGGER.info("Global Recovery Completed");
+ }
+ });
+ state = newState;
+ recoveryThread.start();
+ }
+ return null;
+ }
+
+ private void executeHyracksJob(JobSpecification spec) throws Exception {
+ spec.setMaxReattempts(0);
+ JobId jobId = hcc.startJob(spec);
+ hcc.waitForCompletion(jobId);
+ }
+
+ @Override
+ public void notifyRequestCompletion(IClusterManagementWorkResponse response) {
+ // Do nothing
+ }
+
+ @Override
+ public void notifyStateChange(State previousState, State newState) {
+ // Do nothing?
+ }
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
index 6314d54..ce528bf 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
@@ -37,6 +37,7 @@
import edu.uci.ics.asterix.metadata.MetadataManager;
import edu.uci.ics.asterix.metadata.api.IAsterixStateProxy;
import edu.uci.ics.asterix.metadata.bootstrap.AsterixStateProxy;
+import edu.uci.ics.asterix.metadata.cluster.ClusterManager;
import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
import edu.uci.ics.hyracks.api.application.ICCApplicationEntryPoint;
@@ -94,6 +95,9 @@
setupFeedServer(externalProperties);
feedServer.start();
+ AsterixGlobalRecoveryManager.INSTANCE = new AsterixGlobalRecoveryManager((HyracksConnection) getNewHyracksClientConnection());
+ ClusterManager.INSTANCE.registerSubscriber(AsterixGlobalRecoveryManager.INSTANCE);
+
ccAppCtx.addClusterLifecycleListener(ClusterLifecycleListener.INSTANCE);
}
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm
index b9d7491..0750a58 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm
@@ -1,13 +1,14 @@
-{ "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "DataTypeName": "CompactionPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "CompactionPolicy" ], "PrimaryKey": [ "DataverseName", "CompactionPolicy" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013", "DatasetId": 13, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013", "DatasetId": 2, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013", "DatasetId": 8, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013", "DatasetId": 3, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013", "DatasetId": 1, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Feed", "DataTypeName": "FeedRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName" ], "PrimaryKey": [ "DataverseName", "FeedName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013", "DatasetId": 10, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "DataTypeName": "FeedActivityRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "PrimaryKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013", "DatasetId": 11, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "DataTypeName": "FeedPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "PolicyName" ], "PrimaryKey": [ "DataverseName", "PolicyName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013", "DatasetId": 12, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013", "DatasetId": 7, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013", "DatasetId": 4, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Library", "DataTypeName": "LibraryRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013", "DatasetId": 9, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013", "DatasetId": 5, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013", "DatasetId": 6, "PendingOp": 0 }
\ No newline at end of file
+{ "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "DataTypeName": "CompactionPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "CompactionPolicy" ], "PrimaryKey": [ "DataverseName", "CompactionPolicy" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 13, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 2, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 8, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 3, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 1, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "ExternalFile", "DataTypeName": "ExternalFileRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "FileNumber" ], "PrimaryKey": [ "DataverseName", "DatasetName", "FileNumber" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 14, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Feed", "DataTypeName": "FeedRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName" ], "PrimaryKey": [ "DataverseName", "FeedName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 10, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "DataTypeName": "FeedActivityRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "PrimaryKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 11, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "DataTypeName": "FeedPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "PolicyName" ], "PrimaryKey": [ "DataverseName", "PolicyName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 12, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 7, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 4, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Library", "DataTypeName": "LibraryRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 9, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 5, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 6, "PendingOp": 0 }
\ No newline at end of file
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 66a5a68..4580e90 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,70 +1,73 @@
-{ "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": "Mon Sep 23 00:04:06 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_FeedRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "uuid", "Derived": null, "Timestamp": "Thu Oct 24 01:40:50 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
\ 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": "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
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta19/meta19.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta19/meta19.1.adm
index a8dfc95..c14e3f2 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta19/meta19.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta19/meta19.1.adm
@@ -1,16 +1,17 @@
-{ "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "IndexName": "CompactionPolicy", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "CompactionPolicy" ], "IsPrimary": true, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Feed", "IndexName": "Feed", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "FeedName" ], "IsPrimary": true, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "IndexName": "FeedActivity", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "IsPrimary": true, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "IndexName": "FeedPolicy", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "PolicyName" ], "IsPrimary": true, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Library", "IndexName": "Library", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "IndexName": "CompactionPolicy", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "CompactionPolicy" ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "ExternalFile", "IndexName": "ExternalFile", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "FileNumber" ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Feed", "IndexName": "Feed", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "FeedName" ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "IndexName": "FeedActivity", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "IndexName": "FeedPolicy", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "PolicyName" ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Library", "IndexName": "Library", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm b/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm
index b9d7491..0750a58 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm
@@ -1,13 +1,14 @@
-{ "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "DataTypeName": "CompactionPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "CompactionPolicy" ], "PrimaryKey": [ "DataverseName", "CompactionPolicy" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013", "DatasetId": 13, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013", "DatasetId": 2, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013", "DatasetId": 8, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013", "DatasetId": 3, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013", "DatasetId": 1, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Feed", "DataTypeName": "FeedRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName" ], "PrimaryKey": [ "DataverseName", "FeedName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013", "DatasetId": 10, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "DataTypeName": "FeedActivityRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "PrimaryKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013", "DatasetId": 11, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "DataTypeName": "FeedPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "PolicyName" ], "PrimaryKey": [ "DataverseName", "PolicyName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013", "DatasetId": 12, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013", "DatasetId": 7, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013", "DatasetId": 4, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Library", "DataTypeName": "LibraryRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013", "DatasetId": 9, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013", "DatasetId": 5, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013", "DatasetId": 6, "PendingOp": 0 }
\ No newline at end of file
+{ "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "DataTypeName": "CompactionPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "CompactionPolicy" ], "PrimaryKey": [ "DataverseName", "CompactionPolicy" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 13, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 2, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 8, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 3, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 1, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "ExternalFile", "DataTypeName": "ExternalFileRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "FileNumber" ], "PrimaryKey": [ "DataverseName", "DatasetName", "FileNumber" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 14, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Feed", "DataTypeName": "FeedRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName" ], "PrimaryKey": [ "DataverseName", "FeedName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 10, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "DataTypeName": "FeedActivityRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "PrimaryKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 11, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "DataTypeName": "FeedPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "PolicyName" ], "PrimaryKey": [ "DataverseName", "PolicyName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 12, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 7, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 4, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Library", "DataTypeName": "LibraryRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 9, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 5, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 6, "PendingOp": 0 }
\ No newline at end of file
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 4081076..4580e90 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,70 +1,73 @@
-{ "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": "Mon Sep 23 00:25:26 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_FeedRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "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": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "uuid", "Derived": null, "Timestamp": "Thu Oct 24 01:40:50 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Thu Sep 26 03:23:51 PDT 2013" }
\ 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": "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
diff --git a/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm b/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm
index a8dfc95..c14e3f2 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm
@@ -1,16 +1,17 @@
-{ "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "IndexName": "CompactionPolicy", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "CompactionPolicy" ], "IsPrimary": true, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Feed", "IndexName": "Feed", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "FeedName" ], "IsPrimary": true, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "IndexName": "FeedActivity", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "IsPrimary": true, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "IndexName": "FeedPolicy", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "PolicyName" ], "IsPrimary": true, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Library", "IndexName": "Library", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Thu Sep 26 02:39:06 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "IndexName": "CompactionPolicy", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "CompactionPolicy" ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "ExternalFile", "IndexName": "ExternalFile", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "FileNumber" ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Feed", "IndexName": "Feed", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "FeedName" ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "IndexName": "FeedActivity", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "IndexName": "FeedPolicy", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "PolicyName" ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Library", "IndexName": "Library", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.1.ddl.aql
new file mode 100644
index 0000000..55bda77f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.1.ddl.aql
@@ -0,0 +1,38 @@
+/*
+ * Description : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue : 730, 741
+ * Expected Res : Success
+ * Date : 8th May 2014
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type TwitterUserType as closed {
+ screen-name: string,
+ lang: string,
+ friends-count: int32,
+ statuses-count: int32,
+ name: string,
+ followers-count: int32
+}
+
+create type TweetMessageType as closed {
+ tweetid: int64,
+ user: TwitterUserType,
+ sender-location: point,
+ send-time: datetime,
+ referred-topics: {{ string }},
+ message-text: string,
+ countA: int32,
+ countB: int32
+}
+
+create external dataset TweetMessages(TweetMessageType) using hdfs(("hdfs"="hdfs://127.0.0.1:31888"),("path"="/asterix/tw_for_indexleftouterjoin.adm"),("input-format"="text-input-format"),("format"="adm"));
+
+create index IdIx on TweetMessages(tweetid) type btree;
+create index msgCountAIx on TweetMessages(countA) type btree;
+create index msgCountBIx on TweetMessages(countB) type btree;
+create index twmSndLocIx on TweetMessages(sender-location) type rtree;
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.2.update.aql
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.2.update.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.3.query.aql
new file mode 100644
index 0000000..50e72e3
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.3.query.aql
@@ -0,0 +1,21 @@
+/*
+ * Description : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue : 730, 741
+ * Expected Res : Success
+ * Date : 8th May 2014
+ */
+
+use dataverse test;
+
+for $t1 in dataset('TweetMessages')
+let $n := create-circle($t1.sender-location, 0.5)
+where $t1.tweetid < int64("10")
+order by $t1.tweetid
+return {
+"tweetid1": $t1.tweetid,
+"loc1":$t1.sender-location,
+"nearby-message": for $t2 in dataset('TweetMessages')
+ where spatial-intersect($t2.sender-location, $n)
+ order by $t2.tweetid
+ return {"tweetid2":$t2.tweetid, "loc2":$t2.sender-location}
+};
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/external-indexing/leftouterjoin/leftouterjoin.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/external-indexing/leftouterjoin/leftouterjoin.1.ddl.aql
new file mode 100644
index 0000000..ed033eb
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/external-indexing/leftouterjoin/leftouterjoin.1.ddl.aql
@@ -0,0 +1,36 @@
+/*
+ * Description : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
+ * Issue : 730, 741
+ * Expected Res : Success
+ * Date : 8th May 2014
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type TwitterUserType as closed {
+ screen-name: string,
+ lang: string,
+ friends-count: int32,
+ statuses-count: int32,
+ name: string,
+ followers-count: int32
+}
+
+create type TweetMessageType as closed {
+ tweetid: int64,
+ user: TwitterUserType,
+ sender-location: point,
+ send-time: datetime,
+ referred-topics: {{ string }},
+ message-text: string,
+ countA: int32,
+ countB: int32
+}
+
+create external dataset TweetMessages(TweetMessageType) using hdfs(("hdfs"="hdfs://127.0.0.1:31888"),("path"="/asterix/tw_for_indexleftouterjoin.adm"),("input-format"="text-input-format"),("format"="adm"));
+
+create index IdIx on TweetMessages(tweetid) type btree;
+create index msgCountAIx on TweetMessages(countA) type btree;
+create index msgCountBIx on TweetMessages(countB) type btree;
diff --git a/asterix-app/src/test/resources/runtimets/queries/external-indexing/leftouterjoin/leftouterjoin.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/external-indexing/leftouterjoin/leftouterjoin.2.update.aql
new file mode 100644
index 0000000..16cbac3
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/external-indexing/leftouterjoin/leftouterjoin.2.update.aql
@@ -0,0 +1,6 @@
+/*
+ * Description : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
+ * Issue : 730, 741
+ * Expected Res : Success
+ * Date : 8th May 2014
+ */
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/external-indexing/leftouterjoin/leftouterjoin.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/external-indexing/leftouterjoin/leftouterjoin.3.query.aql
new file mode 100644
index 0000000..408a2e1
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/external-indexing/leftouterjoin/leftouterjoin.3.query.aql
@@ -0,0 +1,14 @@
+use dataverse test;
+
+for $t1 in dataset('TweetMessages')
+where $t1.tweetid < int64("10")
+order by $t1.tweetid
+return {
+"tweetid1": $t1.tweetid,
+"count1":$t1.countA,
+"t2info": for $t2 in dataset('TweetMessages')
+ where $t1.countA /* +indexnl */= $t2.countB
+ order by $t2.tweetid
+ return {"tweetid2": $t2.tweetid,
+ "count2":$t2.countB}
+};
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/external-indexing/rc-format/rc-format.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/external-indexing/rc-format/rc-format.1.ddl.aql
new file mode 100644
index 0000000..4f95913
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/external-indexing/rc-format/rc-format.1.ddl.aql
@@ -0,0 +1,23 @@
+/*
+* Description : Create an external dataset that contains records stored with rc hdfs file format.
+ Build an index over the external dataset age attribute
+ Perform a query over the dataset using the index.
+* Expected Res : Success
+* Date : 3rd Jan 2014
+*/
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type EmployeeType as closed {
+ id: int32,
+ name: string,
+ age: int32
+};
+
+create external dataset EmployeeDataset(EmployeeType)
+using hdfs
+(("hdfs"="hdfs://127.0.0.1:31888"),("path"="/asterix/external-indexing-test.rc"),("input-format"="rc-input-format"),("format"="binary"),("parser"="hive-parser"),("hive-serde"="org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe"));
+
+create index EmployeeAgeIdx on EmployeeDataset(age);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/external-indexing/rc-format/rc-format.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/external-indexing/rc-format/rc-format.2.update.aql
new file mode 100644
index 0000000..569c3e6
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/external-indexing/rc-format/rc-format.2.update.aql
@@ -0,0 +1,7 @@
+/*
+* Description : Create an external dataset that contains records stored with rc hdfs file format.
+ Build an index over the external dataset age attribute
+ Perform a query over the dataset using the index.
+* Expected Res : Success
+* Date : 3rd Jan 2014
+*/
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/external-indexing/rc-format/rc-format.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/external-indexing/rc-format/rc-format.3.query.aql
new file mode 100644
index 0000000..d46031c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/external-indexing/rc-format/rc-format.3.query.aql
@@ -0,0 +1,12 @@
+/*
+* Description : Create an external dataset that contains records stored with rc hdfs file format.
+ Build an index over the external dataset age attribute
+ Perform a query over the dataset using the index.
+* Expected Res : Success
+* Date : 3rd Jan 2014
+*/
+use dataverse test;
+
+for $emp in dataset EmployeeDataset
+where $emp.age = 22
+return $emp;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/external-indexing/rtree-index/rtree-index.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/external-indexing/rtree-index/rtree-index.1.ddl.aql
new file mode 100644
index 0000000..8bd00c5
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/external-indexing/rtree-index/rtree-index.1.ddl.aql
@@ -0,0 +1,19 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type MyRecord as closed {
+ id: int32,
+ point: point,
+ kwds: string,
+ line1: line,
+ line2: line,
+ poly1: polygon,
+ poly2: polygon,
+ rec: rectangle,
+ circle: circle
+}
+
+create external dataset MyData(MyRecord) using hdfs(("hdfs"="hdfs://127.0.0.1:31888"),("path"="/asterix/spatialData.json"),("input-format"="text-input-format"),("format"="adm"));
+
+create index rtree_index_point on MyData(point) type rtree;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/external-indexing/rtree-index/rtree-index.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/external-indexing/rtree-index/rtree-index.2.update.aql
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/external-indexing/rtree-index/rtree-index.2.update.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/external-indexing/rtree-index/rtree-index.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/external-indexing/rtree-index/rtree-index.3.query.aql
new file mode 100644
index 0000000..9986216
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/external-indexing/rtree-index/rtree-index.3.query.aql
@@ -0,0 +1,6 @@
+use dataverse test;
+
+for $o in dataset('MyData')
+where spatial-intersect($o.point, create-polygon([4.0,1.0,4.0,4.0,12.0,4.0,12.0,1.0]))
+order by $o.id
+return {"id":$o.id}
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/external-indexing/sequence-format/sequence-format.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/external-indexing/sequence-format/sequence-format.1.ddl.aql
new file mode 100644
index 0000000..f149f35
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/external-indexing/sequence-format/sequence-format.1.ddl.aql
@@ -0,0 +1,23 @@
+/*
+* Description : Create an external dataset that contains records stored with sequence hdfs file format.
+ Build an index over the external dataset age attribute
+ Perform a query over the dataset using the index.
+* Expected Res : Success
+* Date : 3rd Jan 2014
+*/
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type EmployeeType as closed {
+ id: int32,
+ name: string,
+ age: int32
+};
+
+create external dataset EmployeeDataset(EmployeeType)
+using hdfs
+(("hdfs"="hdfs://127.0.0.1:31888"),("path"="/asterix/external-indexing-test.seq"),("input-format"="sequence-input-format"),("format"="delimited-text"),("delimiter"="|"));
+
+create index EmployeeAgeIdx on EmployeeDataset(age);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/external-indexing/sequence-format/sequence-format.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/external-indexing/sequence-format/sequence-format.2.update.aql
new file mode 100644
index 0000000..593a13b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/external-indexing/sequence-format/sequence-format.2.update.aql
@@ -0,0 +1,7 @@
+/*
+* Description : Create an external dataset that contains records stored with sequence hdfs file format.
+ Build an index over the external dataset age attribute
+ Perform a query over the dataset using the index.
+* Expected Res : Success
+* Date : 3rd Jan 2014
+*/
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/external-indexing/sequence-format/sequence-format.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/external-indexing/sequence-format/sequence-format.3.query.aql
new file mode 100644
index 0000000..b714671
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/external-indexing/sequence-format/sequence-format.3.query.aql
@@ -0,0 +1,12 @@
+/*
+* Description : Create an external dataset that contains records stored with sequence hdfs file format.
+ Build an index over the external dataset age attribute
+ Perform a query over the dataset using the index.
+* Expected Res : Success
+* Date : 3rd Jan 2014
+*/
+use dataverse test;
+
+for $emp in dataset EmployeeDataset
+where $emp.age = 22
+return $emp;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/external-indexing/text-format/text-format.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/external-indexing/text-format/text-format.1.ddl.aql
new file mode 100644
index 0000000..8109531
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/external-indexing/text-format/text-format.1.ddl.aql
@@ -0,0 +1,23 @@
+/*
+* Description : Create an external dataset that contains records stored with text hdfs file format.
+ Build an index over the external dataset age attribute
+ Perform a query over the dataset using the index.
+* Expected Res : Success
+* Date : 3rd Jan 2014
+*/
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type EmployeeType as closed {
+ id: int32,
+ name: string,
+ age: int32
+};
+
+create external dataset EmployeeDataset(EmployeeType)
+using hdfs
+(("hdfs"="hdfs://127.0.0.1:31888"),("path"="/asterix/external-indexing-test.txt"),("input-format"="text-input-format"),("format"="delimited-text"),("delimiter"="|"));
+
+create index EmployeeAgeIdx on EmployeeDataset(age);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/external-indexing/text-format/text-format.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/external-indexing/text-format/text-format.2.update.aql
new file mode 100644
index 0000000..4fb3db0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/external-indexing/text-format/text-format.2.update.aql
@@ -0,0 +1,7 @@
+/*
+* Description : Create an external dataset that contains records stored with text hdfs file format.
+ Build an index over the external dataset age attribute
+ Perform a query over the dataset using the index.
+* Expected Res : Success
+* Date : 3rd Jan 2014
+*/
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/external-indexing/text-format/text-format.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/external-indexing/text-format/text-format.3.query.aql
new file mode 100644
index 0000000..954e877
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/external-indexing/text-format/text-format.3.query.aql
@@ -0,0 +1,12 @@
+/*
+* Description : Create an external dataset that contains records stored with text hdfs file format.
+ Build an index over the external dataset age attribute
+ Perform a query over the dataset using the index.
+* Expected Res : Success
+* Date : 3rd Jan 2014
+*/
+use dataverse test;
+
+for $emp in dataset EmployeeDataset
+where $emp.age = 22
+return $emp;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv19/cross-dv19.1.adm b/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv19/cross-dv19.1.adm
index 62d8a6c..660e2a8 100644
--- a/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv19/cross-dv19.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv19/cross-dv19.1.adm
@@ -1,7 +1,7 @@
-{ "DataverseName": "test1", "DatasetName": "TwitterData", "DataTypeName": "Tweet", "DatasetType": "EXTERNAL", "InternalDetails": null, "ExternalDetails": { "DatasourceAdapter": "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter", "Properties": [ { "Name": "path", "Value": "nc1://data/twitter/extrasmalltweets.txt" }, { "Name": "format", "Value": "adm" } ] }, "Hints": {{ }}, "Timestamp": "Tue Sep 24 22:25:02 PDT 2013", "DatasetId": 119, "PendingOp": 0 }
-{ "DataverseName": "test1", "DatasetName": "t1", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Sep 24 22:25:02 PDT 2013", "DatasetId": 113, "PendingOp": 0 }
-{ "DataverseName": "test1", "DatasetName": "t2", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Sep 24 22:25:02 PDT 2013", "DatasetId": 116, "PendingOp": 0 }
-{ "DataverseName": "test1", "DatasetName": "t3", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Sep 24 22:25:02 PDT 2013", "DatasetId": 117, "PendingOp": 0 }
-{ "DataverseName": "test2", "DatasetName": "t2", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Sep 24 22:25:02 PDT 2013", "DatasetId": 114, "PendingOp": 0 }
-{ "DataverseName": "test2", "DatasetName": "t3", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Sep 24 22:25:02 PDT 2013", "DatasetId": 115, "PendingOp": 0 }
-{ "DataverseName": "test2", "DatasetName": "t4", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Sep 24 22:25:02 PDT 2013", "DatasetId": 118, "PendingOp": 0 }
\ No newline at end of file
+{ "DataverseName": "test1", "DatasetName": "TwitterData", "DataTypeName": "Tweet", "DatasetType": "EXTERNAL", "InternalDetails": null, "ExternalDetails": { "DatasourceAdapter": "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter", "Properties": [ { "Name": "path", "Value": "nc1://data/twitter/extrasmalltweets.txt" }, { "Name": "format", "Value": "adm" } ], "GroupName": "DEFAULT_NG_ALL_NODES", "LastRefreshTime": datetime("2014-06-08T20:30:43.724Z"), "TransactionState": 0, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:30:43 PDT 2014", "DatasetId": 107, "PendingOp": 0 }
+{ "DataverseName": "test1", "DatasetName": "t1", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:30:42 PDT 2014", "DatasetId": 101, "PendingOp": 0 }
+{ "DataverseName": "test1", "DatasetName": "t2", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:30:43 PDT 2014", "DatasetId": 104, "PendingOp": 0 }
+{ "DataverseName": "test1", "DatasetName": "t3", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:30:43 PDT 2014", "DatasetId": 105, "PendingOp": 0 }
+{ "DataverseName": "test2", "DatasetName": "t2", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:30:42 PDT 2014", "DatasetId": 102, "PendingOp": 0 }
+{ "DataverseName": "test2", "DatasetName": "t3", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:30:42 PDT 2014", "DatasetId": 103, "PendingOp": 0 }
+{ "DataverseName": "test2", "DatasetName": "t4", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:30:43 PDT 2014", "DatasetId": 106, "PendingOp": 0 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.1.adm b/asterix-app/src/test/resources/runtimets/results/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.1.adm
new file mode 100644
index 0000000..b4337b3
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/external-indexing/leftouterjoin-rtree/leftouterjoin-rtree.1.adm
@@ -0,0 +1,9 @@
+{ "tweetid1": 1i64, "loc1": point("42.83,72.44"), "nearby-message": [ { "tweetid2": 1i64, "loc2": point("42.83,72.44") }, { "tweetid2": 55i64, "loc2": point("42.77,72.16") }, { "tweetid2": 114i64, "loc2": point("42.87,72.38") } ] }
+{ "tweetid1": 2i64, "loc1": point("34.81,72.44"), "nearby-message": [ { "tweetid2": 2i64, "loc2": point("34.81,72.44") } ] }
+{ "tweetid1": 3i64, "loc1": point("24.54,82.66"), "nearby-message": [ { "tweetid2": 3i64, "loc2": point("24.54,82.66") } ] }
+{ "tweetid1": 4i64, "loc1": point("38.14,68.1"), "nearby-message": [ { "tweetid2": 4i64, "loc2": point("38.14,68.1") } ] }
+{ "tweetid1": 5i64, "loc1": point("35.4,68.89"), "nearby-message": [ { "tweetid2": 5i64, "loc2": point("35.4,68.89") } ] }
+{ "tweetid1": 6i64, "loc1": point("42.75,78.5"), "nearby-message": [ { "tweetid2": 6i64, "loc2": point("42.75,78.5") } ] }
+{ "tweetid1": 7i64, "loc1": point("48.16,71.59"), "nearby-message": [ { "tweetid2": 7i64, "loc2": point("48.16,71.59") }, { "tweetid2": 42i64, "loc2": point("47.86,71.93") }, { "tweetid2": 192i64, "loc2": point("48.12,72.0") } ] }
+{ "tweetid1": 8i64, "loc1": point("36.17,72.56"), "nearby-message": [ { "tweetid2": 8i64, "loc2": point("36.17,72.56") } ] }
+{ "tweetid1": 9i64, "loc1": point("38.02,70.38"), "nearby-message": [ { "tweetid2": 9i64, "loc2": point("38.02,70.38") }, { "tweetid2": 51i64, "loc2": point("37.65,70.54") } ] }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/external-indexing/leftouterjoin/leftouterjoin.1.adm b/asterix-app/src/test/resources/runtimets/results/external-indexing/leftouterjoin/leftouterjoin.1.adm
new file mode 100644
index 0000000..1907bca
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/external-indexing/leftouterjoin/leftouterjoin.1.adm
@@ -0,0 +1,9 @@
+{ "tweetid1": 1i64, "count1": 1, "t2info": [ ] }
+{ "tweetid1": 2i64, "count1": 2, "t2info": [ { "tweetid2": 60i64, "count2": 2 } ] }
+{ "tweetid1": 3i64, "count1": 3, "t2info": [ { "tweetid2": 105i64, "count2": 3 }, { "tweetid2": 206i64, "count2": 3 } ] }
+{ "tweetid1": 4i64, "count1": 4, "t2info": [ ] }
+{ "tweetid1": 5i64, "count1": 5, "t2info": [ { "tweetid2": 138i64, "count2": 5 }, { "tweetid2": 175i64, "count2": 5 } ] }
+{ "tweetid1": 6i64, "count1": 6, "t2info": [ { "tweetid2": 148i64, "count2": 6 } ] }
+{ "tweetid1": 7i64, "count1": 7, "t2info": [ { "tweetid2": 125i64, "count2": 7 } ] }
+{ "tweetid1": 8i64, "count1": 8, "t2info": [ ] }
+{ "tweetid1": 9i64, "count1": 9, "t2info": [ { "tweetid2": 141i64, "count2": 9 } ] }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/external-indexing/rc-format/rc-format.1.adm b/asterix-app/src/test/resources/runtimets/results/external-indexing/rc-format/rc-format.1.adm
new file mode 100644
index 0000000..3ee4a1e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/external-indexing/rc-format/rc-format.1.adm
@@ -0,0 +1,2 @@
+{ "id": 3, "name": "Samuel", "age": 22 }
+{ "id": 10, "name": "David", "age": 22 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/external-indexing/rtree-index/rtree-index.1.adm b/asterix-app/src/test/resources/runtimets/results/external-indexing/rtree-index/rtree-index.1.adm
new file mode 100644
index 0000000..d22217a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/external-indexing/rtree-index/rtree-index.1.adm
@@ -0,0 +1,2 @@
+{ "id": 12 }
+{ "id": 20 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/external-indexing/sequence-format/sequence-format.1.adm b/asterix-app/src/test/resources/runtimets/results/external-indexing/sequence-format/sequence-format.1.adm
new file mode 100644
index 0000000..3ee4a1e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/external-indexing/sequence-format/sequence-format.1.adm
@@ -0,0 +1,2 @@
+{ "id": 3, "name": "Samuel", "age": 22 }
+{ "id": 10, "name": "David", "age": 22 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/external-indexing/text-format/text-format.1.adm b/asterix-app/src/test/resources/runtimets/results/external-indexing/text-format/text-format.1.adm
new file mode 100644
index 0000000..3ee4a1e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/external-indexing/text-format/text-format.1.adm
@@ -0,0 +1,2 @@
+{ "id": 3, "name": "Samuel", "age": 22 }
+{ "id": 10, "name": "David", "age": 22 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf23/udf23.1.adm b/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf23/udf23.1.adm
index f3636a4..1b4a1f6 100644
--- a/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf23/udf23.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf23/udf23.1.adm
@@ -1,6 +1,6 @@
-{ "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "DataTypeName": "CompactionPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "CompactionPolicy" ], "PrimaryKey": [ "DataverseName", "CompactionPolicy" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Oct 24 01:49:04 PDT 2013", "DatasetId": 13, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Oct 24 01:49:04 PDT 2013", "DatasetId": 2, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Oct 24 01:49:04 PDT 2013", "DatasetId": 8, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Oct 24 01:49:04 PDT 2013", "DatasetId": 3, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Oct 24 01:49:04 PDT 2013", "DatasetId": 1, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Feed", "DataTypeName": "FeedRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName" ], "PrimaryKey": [ "DataverseName", "FeedName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Thu Oct 24 01:49:04 PDT 2013", "DatasetId": 10, "PendingOp": 0 }
\ No newline at end of file
+{ "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "DataTypeName": "CompactionPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "CompactionPolicy" ], "PrimaryKey": [ "DataverseName", "CompactionPolicy" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 13, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 2, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 8, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 3, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 1, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "ExternalFile", "DataTypeName": "ExternalFileRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "FileNumber" ], "PrimaryKey": [ "DataverseName", "DatasetName", "FileNumber" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 14, "PendingOp": 0 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterix-app/src/test/resources/runtimets/testsuite.xml
index 8cf4cd9..6344bc1 100644
--- a/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -4896,6 +4896,38 @@
</compilation-unit>
</test-case>
</test-group>
+ <test-group name="external-indexing">
+ <test-case FilePath="external-indexing">
+ <compilation-unit name="text-format">
+ <output-dir compare="Text">text-format</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-indexing">
+ <compilation-unit name="sequence-format">
+ <output-dir compare="Text">sequence-format</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-indexing">
+ <compilation-unit name="rc-format">
+ <output-dir compare="Text">rc-format</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-indexing">
+ <compilation-unit name="rtree-index">
+ <output-dir compare="Text">rtree-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-indexing">
+ <compilation-unit name="leftouterjoin">
+ <output-dir compare="Text">leftouterjoin</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-indexing">
+ <compilation-unit name="leftouterjoin-rtree">
+ <output-dir compare="Text">leftouterjoin-rtree</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
<test-group name="temporal">
<test-case FilePath="temporal">
<compilation-unit name="parse_02">
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/base/Statement.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/base/Statement.java
index 2948f51..6bdca80 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/base/Statement.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/base/Statement.java
@@ -43,7 +43,8 @@
DISCONNECT_FEED,
CREATE_FUNCTION,
FUNCTION_DROP,
- COMPACT
+ COMPACT,
+ EXTERNAL_DATASET_REFRESH
}
public abstract Kind getKind();
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/ExternalDetailsDecl.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/ExternalDetailsDecl.java
index 4233225..5f58a38 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/ExternalDetailsDecl.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/ExternalDetailsDecl.java
@@ -17,22 +17,49 @@
import java.util.Map;
public class ExternalDetailsDecl implements IDatasetDetailsDecl {
- private Map<String, String> properties;
- private String adapter;
+ private Map<String, String> properties;
+ private String adapter;
+ private Identifier nodegroupName;
+ private String compactionPolicy;
+ private Map<String, String> compactionPolicyProperties;
- public void setAdapter(String adapter) {
- this.adapter = adapter;
+ public void setAdapter(String adapter) {
+ this.adapter = adapter;
+ }
+
+ public void setProperties(Map<String, String> properties) {
+ this.properties = properties;
+ }
+
+ public String getAdapter() {
+ return adapter;
+ }
+
+ public Map<String, String> getProperties() {
+ return properties;
+ }
+
+ public Identifier getNodegroupName() {
+ return nodegroupName;
+ }
+
+ public void setNodegroupName(Identifier nodegroupName) {
+ this.nodegroupName = nodegroupName;
+ }
+
+ public String getCompactionPolicy() {
+ return compactionPolicy;
}
- public void setProperties(Map<String, String> properties) {
- this.properties = properties;
+ public void setCompactionPolicy(String compactionPolicy) {
+ this.compactionPolicy = compactionPolicy;
}
- public String getAdapter() {
- return adapter;
+ public Map<String, String> getCompactionPolicyProperties() {
+ return compactionPolicyProperties;
}
- public Map<String, String> getProperties() {
- return properties;
+ public void setCompactionPolicyProperties(Map<String, String> compactionPolicyProperties) {
+ this.compactionPolicyProperties = compactionPolicyProperties;
}
}
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/RefreshExternalDatasetStatement.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/RefreshExternalDatasetStatement.java
new file mode 100644
index 0000000..69098df
--- /dev/null
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/RefreshExternalDatasetStatement.java
@@ -0,0 +1,59 @@
+/*
+ * 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.aql.expression;
+
+import edu.uci.ics.asterix.aql.base.Statement;
+import edu.uci.ics.asterix.aql.expression.visitor.IAqlExpressionVisitor;
+import edu.uci.ics.asterix.aql.expression.visitor.IAqlVisitorWithVoidReturn;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+
+public class RefreshExternalDatasetStatement implements Statement{
+
+ private Identifier dataverseName;
+ private Identifier datasetName;
+
+ public Identifier getDatasetName() {
+ return datasetName;
+ }
+
+ public void setDatasetName(Identifier datasetName) {
+ this.datasetName = datasetName;
+ }
+
+ public Identifier getDataverseName() {
+ return dataverseName;
+ }
+
+ public void setDataverseName(Identifier dataverseName) {
+ this.dataverseName = dataverseName;
+ }
+
+ @Override
+ public <R, T> R accept(IAqlExpressionVisitor<R, T> visitor, T arg)
+ throws AsterixException {
+ return null;
+ }
+
+ @Override
+ public <T> void accept(IAqlVisitorWithVoidReturn<T> visitor, T arg)
+ throws AsterixException {
+ }
+
+ @Override
+ public Kind getKind() {
+ return Kind.EXTERNAL_DATASET_REFRESH;
+ }
+
+}
diff --git a/asterix-aql/src/main/javacc/AQL.jj b/asterix-aql/src/main/javacc/AQL.jj
index e00bb2c..ad86862 100644
--- a/asterix-aql/src/main/javacc/AQL.jj
+++ b/asterix-aql/src/main/javacc/AQL.jj
@@ -202,6 +202,7 @@
| stmt = FeedStatement()
| stmt = CompactStatement()
| stmt = Query()
+ | stmt = RefreshExternalDatasetStatement()
)
{
return stmt;
@@ -320,11 +321,16 @@
<LEFTPAREN> typeName = Identifier() <RIGHTPAREN>
ifNotExists = IfNotExists()
"using" adapterName = AdapterName() properties = Configuration()
+ ("on" nodeGroupName = Identifier() )?
( "hints" hints = Properties() )?
+ ( "using" "compaction" "policy" compactionPolicy = CompactionPolicy() compactionPolicyProperties = Configuration() )?
{
ExternalDetailsDecl edd = new ExternalDetailsDecl();
edd.setAdapter(adapterName);
edd.setProperties(properties);
+ edd.setNodegroupName(nodeGroupName != null? new Identifier(nodeGroupName): null);
+ edd.setCompactionPolicy(compactionPolicy);
+ edd.setCompactionPolicyProperties(compactionPolicyProperties);
dsetDecl = new DatasetDecl(nameComponents.first,
nameComponents.second,
new Identifier(typeName),
@@ -364,6 +370,21 @@
}
}
+RefreshExternalDatasetStatement RefreshExternalDatasetStatement() throws ParseException:
+{
+ RefreshExternalDatasetStatement redss = new RefreshExternalDatasetStatement();
+ Pair<Identifier,Identifier> nameComponents = null;
+ String datasetName = null;
+}
+{
+ "refresh external" <DATASET> nameComponents = QualifiedName()
+ {
+ redss.setDataverseName(nameComponents.first);
+ redss.setDatasetName(nameComponents.second);
+ return redss;
+ }
+}
+
CreateIndexStatement IndexSpecification() throws ParseException:
{
CreateIndexStatement cis = new CreateIndexStatement();
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/DatasetConfig.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/DatasetConfig.java
index 8481817..ec11c5d 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/DatasetConfig.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/DatasetConfig.java
@@ -38,4 +38,16 @@
LENGTH_PARTITIONED_NGRAM_INVIX
}
+ public enum ExternalDatasetTransactionState {
+ COMMIT, // The committed state <- nothing is required->
+ BEGIN, // The state after starting the refresh transaction <- will either abort moving to committed state or move to ready to commit->
+ READY_TO_COMMIT // The transaction is ready to commit <- can only move forward to committed state->
+ };
+
+ public enum ExternalFilePendingOp {
+ PENDING_NO_OP, // the stored file is part of a committed transaction nothing is required
+ PENDING_ADD_OP, // the stored file is part of an ongoing transaction (will be added if transaction succeed)
+ PENDING_DROP_OP, // the stored file is part of an ongoing transaction (will be dropped if transaction succeed)
+ PENDING_APPEND_OP // the stored file is part of an ongoing transaction (will be updated if transaction succeed)
+ };
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java
index fcb0786..63f4c3c 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java
@@ -23,7 +23,6 @@
import java.util.List;
import java.util.Map;
import java.util.Set;
-
import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -81,7 +80,7 @@
int did = getDIDfromRID(resourceID);
DatasetInfo dsInfo = datasetInfos.get(did);
if (dsInfo == null) {
- dsInfo = new DatasetInfo(did);
+ dsInfo = new DatasetInfo(did,!index.hasMemoryComponents());
} else if (dsInfo.indexes.containsKey(resourceID)) {
throw new HyracksDataException("Index with resource ID " + resourceID + " already exists.");
}
@@ -132,7 +131,7 @@
}
dsInfo.indexes.remove(resourceID);
- if (dsInfo.referenceCount == 0 && dsInfo.isOpen && dsInfo.indexes.isEmpty()) {
+ if (dsInfo.referenceCount == 0 && dsInfo.isOpen && dsInfo.indexes.isEmpty() && !dsInfo.isExternal) {
List<IVirtualBufferCache> vbcs = getVirtualBufferCaches(did);
assert vbcs != null;
for (IVirtualBufferCache vbc : vbcs) {
@@ -142,7 +141,6 @@
datasetVirtualBufferCaches.remove(did);
datasetOpTrackers.remove(did);
}
-
}
public synchronized void declareActiveIOOperation(int datasetID) throws HyracksDataException {
@@ -177,7 +175,8 @@
+ " since it does not exist.");
}
- if (!dsInfo.isOpen) {
+ // This is not needed for external datasets' indexes since they never use the virtual buffer cache.
+ if (!dsInfo.isOpen && !dsInfo.isExternal) {
List<IVirtualBufferCache> vbcs = getVirtualBufferCaches(did);
assert vbcs != null;
long additionalSize = 0;
@@ -213,10 +212,8 @@
.get(dsInfo.datasetID);
if (opTracker != null && opTracker.getNumActiveOperations() == 0 && dsInfo.referenceCount == 0
&& dsInfo.isOpen) {
-
closeDataset(dsInfo);
return true;
-
}
}
return false;
@@ -292,7 +289,6 @@
opTracker = new PrimaryIndexOperationTracker(this, datasetID);
datasetOpTrackers.put(datasetID, opTracker);
}
-
return opTracker;
}
}
@@ -342,11 +338,13 @@
private final int datasetID;
private long lastAccess;
private int numActiveIOOps;
+ private final boolean isExternal;
- public DatasetInfo(int datasetID) {
+ public DatasetInfo(int datasetID, boolean isExternal) {
this.indexes = new HashMap<Long, IndexInfo>();
this.lastAccess = -1;
this.datasetID = datasetID;
+ this.isExternal = isExternal;
}
public void touch() {
@@ -442,7 +440,6 @@
for (IVirtualBufferCache vbc : vbcs) {
used -= vbc.getNumPages() * vbc.getPageSize();
}
-
}
@Override
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallback.java
new file mode 100644
index 0000000..57f8658
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallback.java
@@ -0,0 +1,55 @@
+/*
+ * 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.common.ioopcallbacks;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTreeWithBuddyDiskComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMOperationType;
+
+public class LSMBTreeWithBuddyIOOperationCallback extends AbstractLSMIOOperationCallback {
+
+ @Override
+ public void afterOperation(LSMOperationType opType, List<ILSMComponent> oldComponents, ILSMComponent newComponent)
+ throws HyracksDataException {
+ if (newComponent != null) {
+ LSMBTreeWithBuddyDiskComponent btreeComponent = (LSMBTreeWithBuddyDiskComponent) newComponent;
+ putLSNIntoMetadata(btreeComponent.getBTree(), oldComponents);
+ putLSNIntoMetadata(btreeComponent.getBuddyBTree(), oldComponents);
+ }
+ }
+
+ @Override
+ public long getComponentLSN(List<ILSMComponent> diskComponents) throws HyracksDataException {
+ if (diskComponents == null) {
+ // Implies a flush IO operation.
+ synchronized (this) {
+ long lsn = immutableLastLSNs[readIndex];
+ readIndex = (readIndex + 1) % immutableLastLSNs.length;
+ return lsn;
+ }
+ }
+ // Get max LSN from the diskComponents. Implies a merge IO operation or Recovery operation.
+ long maxLSN = -1;
+ for (Object o : diskComponents) {
+ LSMBTreeWithBuddyDiskComponent btreeComponent = (LSMBTreeWithBuddyDiskComponent) o;
+ maxLSN = Math.max(getTreeIndexLSN(btreeComponent.getBTree()), maxLSN);
+ }
+ return maxLSN;
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallbackFactory.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallbackFactory.java
new file mode 100644
index 0000000..1144b6d
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallbackFactory.java
@@ -0,0 +1,33 @@
+/*
+ * 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.common.ioopcallbacks;
+
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+
+public class LSMBTreeWithBuddyIOOperationCallbackFactory implements ILSMIOOperationCallbackFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static LSMBTreeWithBuddyIOOperationCallbackFactory INSTANCE = new LSMBTreeWithBuddyIOOperationCallbackFactory();
+
+ private LSMBTreeWithBuddyIOOperationCallbackFactory() {
+ }
+
+ @Override
+ public ILSMIOOperationCallback createIOOperationCallback() {
+ return new LSMBTreeWithBuddyIOOperationCallback();
+ }
+}
diff --git a/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java b/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java
index 6f2e0a0..0bbbca3 100644
--- a/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java
+++ b/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java
@@ -94,7 +94,7 @@
+ "\n> ");
}
- if (!equalStrings(lineExpected.split("Timestamp")[0], lineActual.split("Timestamp")[0])) {
+ if (!equalStrings(lineExpected.split("Time")[0], lineActual.split("Time")[0])) {
fail("Result for " + scriptFile + " changed at line " + num + ":\n< " + lineExpected + "\n> "
+ lineActual);
}
diff --git a/asterix-external-data/pom.xml b/asterix-external-data/pom.xml
index 2284e68..96ff24f 100644
--- a/asterix-external-data/pom.xml
+++ b/asterix-external-data/pom.xml
@@ -172,5 +172,15 @@
<artifactId>microsoft-windowsazure-api</artifactId>
<version>0.4.4</version>
</dependency>
+ <dependency>
+ <groupId>org.apache.hive</groupId>
+ <artifactId>hive-exec</artifactId>
+ <version>0.11.0</version>
+ </dependency>
+ <dependency>
+ <groupId>javax.jdo</groupId>
+ <artifactId>jdo2-api</artifactId>
+ <version>2.3-20090302111651</version>
+ </dependency>
</dependencies>
</project>
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 b10ebcf..cdab917 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
@@ -14,15 +14,26 @@
*/
package edu.uci.ics.asterix.external.adapter.factory;
+import java.io.FileNotFoundException;
+import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
+import java.util.List;
import java.util.Map;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileSplit;
import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.JobConf;
+import edu.uci.ics.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
import edu.uci.ics.asterix.external.dataset.adapter.HDFSAdapter;
+import edu.uci.ics.asterix.external.indexing.dataflow.HDFSObjectTupleParserFactory;
+import edu.uci.ics.asterix.metadata.entities.ExternalFile;
import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
import edu.uci.ics.asterix.metadata.feeds.IGenericAdapterFactory;
import edu.uci.ics.asterix.om.types.ARecordType;
@@ -33,6 +44,7 @@
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import edu.uci.ics.hyracks.api.context.ICCContext;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.hdfs.dataflow.ConfFactory;
import edu.uci.ics.hyracks.hdfs.dataflow.InputSplitsFactory;
@@ -54,6 +66,11 @@
public static final String KEY_INPUT_FORMAT = "input-format";
public static final String INPUT_FORMAT_TEXT = "text-input-format";
public static final String INPUT_FORMAT_SEQUENCE = "sequence-input-format";
+ // New
+ public static final String KEY_PARSER = "parser";
+ public static final String PARSER_HIVE = "hive-parser";
+ public static final String INPUT_FORMAT_RC = "rc-input-format";
+ public static final String FORMAT_BINARY = "binary";
private transient AlgebricksPartitionConstraint clusterLocations;
private String[] readSchedule;
@@ -77,22 +94,27 @@
return scheduler;
}
- private static final Map<String, String> formatClassNames = initInputFormatMap();
+ protected static final Map<String, String> formatClassNames = initInputFormatMap();
- private static Map<String, String> initInputFormatMap() {
+ protected static Map<String, String> initInputFormatMap() {
Map<String, String> formatClassNames = new HashMap<String, String>();
formatClassNames.put(INPUT_FORMAT_TEXT, "org.apache.hadoop.mapred.TextInputFormat");
formatClassNames.put(INPUT_FORMAT_SEQUENCE, "org.apache.hadoop.mapred.SequenceFileInputFormat");
+ formatClassNames.put(INPUT_FORMAT_RC, "org.apache.hadoop.hive.ql.io.RCFileInputFormat");
return formatClassNames;
}
+ public JobConf getJobConf() throws HyracksDataException {
+ return confFactory.getConf();
+ }
+
@Override
public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
JobConf conf = confFactory.getConf();
InputSplit[] inputSplits = inputSplitsFactory.getSplits();
String nodeName = ctx.getJobletContext().getApplicationContext().getNodeId();
HDFSAdapter hdfsAdapter = new HDFSAdapter(atype, readSchedule, executed, inputSplits, conf, nodeName,
- parserFactory, ctx);
+ parserFactory, ctx, configuration, files);
return hdfsAdapter;
}
@@ -101,14 +123,17 @@
return HDFS_ADAPTER_NAME;
}
- private JobConf configureJobConf(Map<String, String> configuration) throws Exception {
+ public static JobConf configureJobConf(Map<String, String> configuration) throws Exception {
JobConf conf = new JobConf();
+ String formatClassName = (String) formatClassNames.get(((String) configuration.get(KEY_INPUT_FORMAT)).trim());
+ if (formatClassName == null) {
+ formatClassName = ((String) configuration.get(KEY_INPUT_FORMAT)).trim();
+ }
conf.set("fs.default.name", ((String) configuration.get(KEY_HDFS_URL)).trim());
conf.set("fs.hdfs.impl", "org.apache.hadoop.hdfs.DistributedFileSystem");
conf.setClassLoader(HDFSAdapter.class.getClassLoader());
conf.set("mapred.input.dir", ((String) configuration.get(KEY_PATH)).trim());
- conf.set("mapred.input.format.class",
- (String) formatClassNames.get(((String) configuration.get(KEY_INPUT_FORMAT)).trim()));
+ conf.set("mapred.input.format.class", formatClassName);
return conf;
}
@@ -138,7 +163,13 @@
clusterLocations = getClusterLocations();
int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
- InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
+ // if files list was set, we restrict the splits to the list since this dataset is indexed
+ InputSplit[] inputSplits;
+ if (files == null) {
+ inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
+ } else {
+ inputSplits = getSplits(conf);
+ }
inputSplitsFactory = new InputSplitsFactory(inputSplits);
readSchedule = hdfsScheduler.getLocationConstraints(inputSplits);
@@ -155,7 +186,7 @@
return SupportedOperation.READ;
}
- private static AlgebricksPartitionConstraint getClusterLocations() {
+ public static AlgebricksPartitionConstraint getClusterLocations() {
ArrayList<String> locs = new ArrayList<String>();
Map<String, String[]> stores = AsterixAppContextInfo.getInstance().getMetadataProperties().getStores();
for (String i : stores.keySet()) {
@@ -164,6 +195,7 @@
for (int j = 0; j < nodeStores.length; j++) {
for (int k = 0; k < numIODevices; k++) {
locs.add(i);
+ locs.add(i);
}
}
}
@@ -172,4 +204,105 @@
return new AlgebricksAbsolutePartitionConstraint(cluster);
}
+ /*
+ * This method is overridden to do the following:
+ * 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 {
+ String specifiedFormat = (String) configuration.get(KEY_FORMAT);
+ if (specifiedFormat == null) {
+ throw new IllegalArgumentException(" Unspecified data format");
+ } else if (FORMAT_DELIMITED_TEXT.equalsIgnoreCase(specifiedFormat)) {
+ parserFactory = getDelimitedDataTupleParserFactory((ARecordType) sourceDatatype, false);
+ } else if (FORMAT_ADM.equalsIgnoreCase((String) configuration.get(KEY_FORMAT))) {
+ parserFactory = getADMDataTupleParserFactory((ARecordType) sourceDatatype, false);
+ } else if (FORMAT_BINARY.equalsIgnoreCase((String) configuration.get(KEY_FORMAT))) {
+ parserFactory = new HDFSObjectTupleParserFactory((ARecordType) atype, this, configuration);
+ } else {
+ throw new IllegalArgumentException(" format " + configuration.get(KEY_FORMAT) + " not supported");
+ }
+ }
+
+ /**
+ * Instead of creating the split using the input format, we do it manually
+ * This function returns fileSplits (1 per hdfs file block) irrespective of the number of partitions
+ * and the produced splits only cover intersection between current files in hdfs and files stored internally
+ * in AsterixDB
+ * 1. NoOp means appended file
+ * 2. AddOp means new file
+ * 3. UpdateOp means the delta of a file
+ *
+ * @return
+ * @throws IOException
+ */
+ protected InputSplit[] getSplits(JobConf conf) throws IOException {
+ // Create file system object
+ FileSystem fs = FileSystem.get(conf);
+ ArrayList<FileSplit> fileSplits = new ArrayList<FileSplit>();
+ ArrayList<ExternalFile> orderedExternalFiles = new ArrayList<ExternalFile>();
+ // Create files splits
+ for (ExternalFile file : files) {
+ Path filePath = new Path(file.getFileName());
+ FileStatus fileStatus;
+ try {
+ fileStatus = fs.getFileStatus(filePath);
+ } catch (FileNotFoundException e) {
+ // file was deleted at some point, skip to next file
+ continue;
+ }
+ if (file.getPendingOp() == ExternalFilePendingOp.PENDING_ADD_OP
+ && fileStatus.getModificationTime() == file.getLastModefiedTime().getTime()) {
+ // Get its information from HDFS name node
+ BlockLocation[] fileBlocks = fs.getFileBlockLocations(fileStatus, 0, file.getSize());
+ // Create a split per block
+ for (BlockLocation block : fileBlocks) {
+ if (block.getOffset() < file.getSize()) {
+ fileSplits.add(new FileSplit(filePath, block.getOffset(), (block.getLength() + block
+ .getOffset()) < file.getSize() ? block.getLength() : (file.getSize() - block
+ .getOffset()), block.getHosts()));
+ orderedExternalFiles.add(file);
+ }
+ }
+ } else if (file.getPendingOp() == ExternalFilePendingOp.PENDING_NO_OP
+ && fileStatus.getModificationTime() == file.getLastModefiedTime().getTime()) {
+ long oldSize = 0L;
+ long newSize = file.getSize();
+ for (int i = 0; i < files.size(); i++) {
+ if (files.get(i).getFileName() == file.getFileName() && files.get(i).getSize() != file.getSize()) {
+ newSize = files.get(i).getSize();
+ oldSize = file.getSize();
+ break;
+ }
+ }
+
+ // Get its information from HDFS name node
+ BlockLocation[] fileBlocks = fs.getFileBlockLocations(fileStatus, 0, newSize);
+ // Create a split per block
+ for (BlockLocation block : fileBlocks) {
+ if (block.getOffset() + block.getLength() > oldSize) {
+ if (block.getOffset() < newSize) {
+ // Block interact with delta -> Create a split
+ long startCut = (block.getOffset() > oldSize) ? 0L : oldSize - block.getOffset();
+ long endCut = (block.getOffset() + block.getLength() < newSize) ? 0L : block.getOffset()
+ + block.getLength() - newSize;
+ long splitLength = block.getLength() - startCut - endCut;
+ fileSplits.add(new FileSplit(filePath, block.getOffset() + startCut, splitLength, block
+ .getHosts()));
+ orderedExternalFiles.add(file);
+ }
+ }
+ }
+ }
+ }
+ fs.close();
+ files = orderedExternalFiles;
+ return fileSplits.toArray(new FileSplit[fileSplits.size()]);
+ }
+
+ // Used to tell the factory to restrict the splits to the intersection between this list and the actual files on hdfs side
+ public void setFiles(List<ExternalFile> files) {
+ this.files = files;
+ }
+
}
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
new file mode 100644
index 0000000..00104e0
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSIndexingAdapterFactory.java
@@ -0,0 +1,190 @@
+/*
+ * 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.external.adapter.factory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+
+import edu.uci.ics.asterix.external.dataset.adapter.HDFSIndexingAdapter;
+import edu.uci.ics.asterix.external.indexing.dataflow.HDFSIndexingParserFactory;
+import edu.uci.ics.asterix.external.indexing.dataflow.IndexingScheduler;
+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.AUnionType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
+import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
+import edu.uci.ics.asterix.runtime.operators.file.DelimitedDataParser;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.api.context.ICCContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+import edu.uci.ics.hyracks.hdfs.dataflow.ConfFactory;
+import edu.uci.ics.hyracks.hdfs.dataflow.InputSplitsFactory;
+
+@SuppressWarnings("deprecation")
+public class HDFSIndexingAdapterFactory extends HDFSAdapterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private transient AlgebricksPartitionConstraint clusterLocations;
+ private String[] readSchedule;
+ private boolean executed[];
+ private InputSplitsFactory inputSplitsFactory;
+ private ConfFactory confFactory;
+ private IAType atype;
+ private boolean configured = false;
+ public static IndexingScheduler hdfsScheduler;
+ private static boolean initialized = false;
+ private Map<String, String> configuration;
+
+ public static final String HDFS_INDEXING_ADAPTER = "hdfs-indexing-adapter";
+
+ private static IndexingScheduler initializeHDFSScheduler() {
+ ICCContext ccContext = AsterixAppContextInfo.getInstance().getCCApplicationContext().getCCContext();
+ IndexingScheduler scheduler = null;
+ try {
+ scheduler = new IndexingScheduler(ccContext.getClusterControllerInfo().getClientNetAddress(), ccContext
+ .getClusterControllerInfo().getClientNetPort());
+ } catch (HyracksException e) {
+ throw new IllegalStateException("Cannot obtain hdfs scheduler");
+ }
+ return scheduler;
+ }
+
+ @Override
+ public SupportedOperation getSupportedOperations() {
+ return SupportedOperation.READ;
+ }
+
+ @Override
+ public String getName() {
+ return HDFS_INDEXING_ADAPTER;
+ }
+
+ @Override
+ public AdapterType getAdapterType() {
+ return AdapterType.GENERIC;
+ }
+
+ @Override
+ public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+ if (!configured) {
+ throw new IllegalStateException("Adapter factory has not been configured yet");
+ }
+ return (AlgebricksPartitionConstraint) clusterLocations;
+ }
+
+ @Override
+ public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
+ JobConf conf = confFactory.getConf();
+ InputSplit[] inputSplits = inputSplitsFactory.getSplits();
+ String nodeName = ctx.getJobletContext().getApplicationContext().getNodeId();
+ ((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),
+ (String) configuration.get(KEY_FORMAT));
+ return hdfsIndexingAdapter;
+ }
+
+ @Override
+ public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
+ if (!initialized) {
+ hdfsScheduler = initializeHDFSScheduler();
+ initialized = true;
+ }
+ this.configuration = configuration;
+ JobConf conf = HDFSAdapterFactory.configureJobConf(configuration);
+ confFactory = new ConfFactory(conf);
+ clusterLocations = getClusterLocations();
+ InputSplit[] inputSplits = getSplits(conf);
+ inputSplitsFactory = new InputSplitsFactory(inputSplits);
+ readSchedule = hdfsScheduler.getLocationConstraints(inputSplits);
+ executed = new boolean[readSchedule.length];
+ Arrays.fill(executed, false);
+ configured = true;
+ atype = (IAType) outputType;
+ // 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_PARSER));
+ }
+
+ /**
+ * A static function that creates and return delimited text data parser
+ * @param recordType (the record type to be parsed)
+ * @param delimiter (the dilimiter value)
+ * @return
+ */
+ public static DelimitedDataParser getDilimitedDataParser(ARecordType recordType, Character delimiter){
+ int n = recordType.getFieldTypes().length;
+ IValueParserFactory[] fieldParserFactories = new IValueParserFactory[n];
+ for (int i = 0; i < n; i++) {
+ ATypeTag tag = null;
+ if (recordType.getFieldTypes()[i].getTypeTag() == ATypeTag.UNION) {
+ List<IAType> unionTypes = ((AUnionType) recordType.getFieldTypes()[i]).getUnionList();
+ if (unionTypes.size() != 2 && unionTypes.get(0).getTypeTag() != ATypeTag.NULL) {
+ throw new NotImplementedException("Non-optional UNION type is not supported.");
+ }
+ tag = unionTypes.get(1).getTypeTag();
+ } else {
+ tag = recordType.getFieldTypes()[i].getTypeTag();
+ }
+ if (tag == null) {
+ throw new NotImplementedException("Failed to get the type information for field " + i + ".");
+ }
+ IValueParserFactory vpf = typeToValueParserFactMap.get(tag);
+ if (vpf == null) {
+ throw new NotImplementedException("No value parser factory for delimited fields of type " + tag);
+ }
+ fieldParserFactories[i] = vpf;
+ }
+ return new DelimitedDataParser(recordType, fieldParserFactories, delimiter);
+ }
+
+ public static AlgebricksPartitionConstraint getClusterLocations() {
+ ArrayList<String> locs = new ArrayList<String>();
+ Map<String, String[]> stores = AsterixAppContextInfo.getInstance().getMetadataProperties().getStores();
+ for (String i : stores.keySet()) {
+ String[] nodeStores = stores.get(i);
+ int numIODevices = AsterixClusterProperties.INSTANCE.getNumberOfIODevices(i);
+ for (int j = 0; j < nodeStores.length; j++) {
+ for (int k = 0; k < numIODevices; k++) {
+ locs.add(i);
+ }
+ }
+ }
+ String[] cluster = new String[locs.size()];
+ cluster = locs.toArray(cluster);
+ return new AlgebricksAbsolutePartitionConstraint(cluster);
+ }
+}
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 669dc61..ab59241 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
@@ -14,10 +14,12 @@
*/
package edu.uci.ics.asterix.external.adapter.factory;
+import java.util.List;
import java.util.Map;
import edu.uci.ics.asterix.external.dataset.adapter.HDFSAdapter;
import edu.uci.ics.asterix.external.dataset.adapter.HiveAdapter;
+import edu.uci.ics.asterix.metadata.entities.ExternalFile;
import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
import edu.uci.ics.asterix.metadata.feeds.IGenericAdapterFactory;
import edu.uci.ics.asterix.om.types.ARecordType;
@@ -77,7 +79,7 @@
}
}
- private void populateConfiguration(Map<String, String> configuration) throws Exception {
+ public static void populateConfiguration(Map<String, String> configuration) throws Exception {
/** configure hive */
String database = (String) configuration.get(HIVE_DATABASE);
String tablePath = null;
@@ -104,4 +106,9 @@
return hdfsAdapterFactory.getPartitionConstraint();
}
+ @Override
+ public void setFiles(List<ExternalFile> files) {
+ this.files = files;
+ }
+
}
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 1a8c0df..8c37841 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
@@ -15,6 +15,7 @@
package edu.uci.ics.asterix.external.adapter.factory;
import java.io.File;
+import java.util.List;
import java.util.Map;
import java.util.logging.Level;
@@ -23,12 +24,14 @@
import edu.uci.ics.asterix.external.util.DNSResolverFactory;
import edu.uci.ics.asterix.external.util.INodeResolver;
import edu.uci.ics.asterix.external.util.INodeResolverFactory;
+import edu.uci.ics.asterix.metadata.entities.ExternalFile;
import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
import edu.uci.ics.asterix.metadata.feeds.IGenericAdapterFactory;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
@@ -142,4 +145,9 @@
return nodeResolver;
}
+ @Override
+ public void setFiles(List<ExternalFile> files) throws AlgebricksException{
+ throw new AlgebricksException("can't set files for this Adapter");
+ }
+
}
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 f69eb2d..1dcd351 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
@@ -21,6 +21,7 @@
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.external.util.INodeResolver;
+import edu.uci.ics.asterix.metadata.entities.ExternalFile;
import edu.uci.ics.asterix.metadata.feeds.ConditionalPushTupleParserFactory;
import edu.uci.ics.asterix.metadata.feeds.IAdapterFactory;
import edu.uci.ics.asterix.om.types.ARecordType;
@@ -60,6 +61,8 @@
protected ITupleParserFactory parserFactory;
protected ITupleParser parser;
+
+ protected List<ExternalFile> files;
protected static final HashMap<ATypeTag, IValueParserFactory> typeToValueParserFactMap = new HashMap<ATypeTag, IValueParserFactory>();
static {
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 1a046a5..7e7c6c0 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
@@ -16,18 +16,20 @@
import java.io.IOException;
import java.io.InputStream;
+import java.util.List;
+import java.util.Map;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.Counters.Counter;
import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.mapred.SequenceFileInputFormat;
import org.apache.hadoop.mapred.TextInputFormat;
-
+import edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory;
+import edu.uci.ics.asterix.external.indexing.input.GenericFileAwareRecordReader;
+import edu.uci.ics.asterix.external.indexing.input.GenericRecordReader;
+import edu.uci.ics.asterix.external.indexing.input.TextualDataReader;
+import edu.uci.ics.asterix.external.indexing.input.TextualFullScanDataReader;
+import edu.uci.ics.asterix.metadata.entities.ExternalFile;
import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
@@ -35,7 +37,7 @@
/**
* Provides functionality for fetching external data stored in an HDFS instance.
*/
-@SuppressWarnings({ "deprecation", "rawtypes" })
+@SuppressWarnings({ "deprecation" })
public class HDFSAdapter extends FileSystemBasedAdapter {
private static final long serialVersionUID = 1L;
@@ -45,168 +47,45 @@
private transient InputSplit[] inputSplits;
private transient JobConf conf;
private transient String nodeName;
+ private transient List<ExternalFile> files;
+ private transient Map<String, String> configuration;
public HDFSAdapter(IAType atype, String[] readSchedule, boolean[] executed, InputSplit[] inputSplits, JobConf conf,
- String nodeName, ITupleParserFactory parserFactory, IHyracksTaskContext ctx) throws HyracksDataException {
+ String nodeName, ITupleParserFactory parserFactory, IHyracksTaskContext ctx,
+ Map<String, String> configuration, List<ExternalFile> files) throws HyracksDataException {
super(parserFactory, atype, ctx);
this.readSchedule = readSchedule;
this.executed = executed;
this.inputSplits = inputSplits;
this.conf = conf;
this.nodeName = nodeName;
+ this.files = files;
+ this.configuration = configuration;
}
- private Reporter getReporter() {
- Reporter reporter = new Reporter() {
-
- @Override
- public Counter getCounter(Enum<?> arg0) {
- return null;
- }
-
- @Override
- public Counter getCounter(String arg0, String arg1) {
- return null;
- }
-
- @Override
- public InputSplit getInputSplit() throws UnsupportedOperationException {
- return null;
- }
-
- @Override
- public void incrCounter(Enum<?> arg0, long arg1) {
- }
-
- @Override
- public void incrCounter(String arg0, String arg1, long arg2) {
- }
-
- @Override
- public void setStatus(String arg0) {
- }
-
- @Override
- public void progress() {
- }
- };
-
- return reporter;
- }
-
+ /*
+ * The method below was modified to take care of the following
+ * 1. when target files are not null, it generates a file aware input stream that validate against the files
+ * 2. if the data is binary, it returns a generic reader
+ */
@Override
public InputStream getInputStream(int partition) throws IOException {
-
- return new InputStream() {
-
- private RecordReader<Object, Text> reader;
- private Object key;
- private Text value;
- private boolean hasMore = false;
- private int EOL = "\n".getBytes()[0];
- private Text pendingValue = null;
- private int currentSplitIndex = 0;
-
- @SuppressWarnings("unchecked")
- private boolean moveToNext() throws IOException {
- for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
- /**
- * read all the partitions scheduled to the current node
- */
- if (readSchedule[currentSplitIndex].equals(nodeName)) {
- /**
- * pick an unread split to read
- * synchronize among simultaneous partitions in the same machine
- */
- synchronized (executed) {
- if (executed[currentSplitIndex] == false) {
- executed[currentSplitIndex] = true;
- } else {
- continue;
- }
- }
-
- /**
- * read the split
- */
- reader = getRecordReader(currentSplitIndex);
- key = reader.createKey();
- value = (Text) reader.createValue();
- return true;
- }
- }
- return false;
+ if ((conf.getInputFormat() instanceof TextInputFormat || conf.getInputFormat() instanceof SequenceFileInputFormat)
+ && (HDFSAdapterFactory.FORMAT_ADM.equalsIgnoreCase((String) configuration
+ .get(HDFSAdapterFactory.KEY_FORMAT)) || HDFSAdapterFactory.FORMAT_DELIMITED_TEXT
+ .equalsIgnoreCase((String) configuration.get(HDFSAdapterFactory.KEY_FORMAT)))) {
+ if (files != null) {
+ return new TextualDataReader(inputSplits, readSchedule, nodeName, conf, executed, files);
+ } else {
+ return new TextualFullScanDataReader(executed, inputSplits, readSchedule, nodeName, conf);
}
-
- @Override
- public int read(byte[] buffer, int offset, int len) throws IOException {
- if (reader == null) {
- if (!moveToNext()) {
- //nothing to read
- return -1;
- }
- }
-
- int numBytes = 0;
- if (pendingValue != null) {
- System.arraycopy(pendingValue.getBytes(), 0, buffer, offset + numBytes, pendingValue.getLength());
- buffer[offset + numBytes + pendingValue.getLength()] = (byte) EOL;
- numBytes += pendingValue.getLength() + 1;
- pendingValue = null;
- }
-
- while (numBytes < len) {
- hasMore = reader.next(key, value);
- if (!hasMore) {
- while (moveToNext()) {
- hasMore = reader.next(key, value);
- if (hasMore) {
- //move to the next non-empty split
- break;
- }
- }
- }
- if (!hasMore) {
- return (numBytes == 0) ? -1 : numBytes;
- }
- int sizeOfNextTuple = value.getLength() + 1;
- if (numBytes + sizeOfNextTuple > len) {
- // cannot add tuple to current buffer
- // but the reader has moved pass the fetched tuple
- // we need to store this for a subsequent read call.
- // and return this then.
- pendingValue = value;
- break;
- } else {
- System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.getLength());
- buffer[offset + numBytes + value.getLength()] = (byte) EOL;
- numBytes += sizeOfNextTuple;
- }
- }
- return numBytes;
+ } else {
+ if (files != null) {
+ return new GenericFileAwareRecordReader(inputSplits, readSchedule, nodeName, conf, executed, files);
+ } else {
+ return new GenericRecordReader(inputSplits, readSchedule, nodeName, conf, executed);
}
-
- @Override
- public int read() throws IOException {
- throw new NotImplementedException("Use read(byte[], int, int");
- }
-
- private RecordReader getRecordReader(int slitIndex) throws IOException {
- if (conf.getInputFormat() instanceof SequenceFileInputFormat) {
- SequenceFileInputFormat format = (SequenceFileInputFormat) conf.getInputFormat();
- RecordReader reader = format.getRecordReader(
- (org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
- return reader;
- } else {
- TextInputFormat format = (TextInputFormat) conf.getInputFormat();
- RecordReader reader = format.getRecordReader(
- (org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
- return reader;
- }
- }
-
- };
-
+ }
}
}
\ No newline at end of file
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
new file mode 100644
index 0000000..0c2d3e5
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSIndexingAdapter.java
@@ -0,0 +1,74 @@
+/*
+ * 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.external.dataset.adapter;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.List;
+
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+
+import edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory;
+import edu.uci.ics.asterix.external.indexing.input.GenericFileAwareRecordReader;
+import edu.uci.ics.asterix.external.indexing.input.RCFileDataReader;
+import edu.uci.ics.asterix.external.indexing.input.TextualDataReader;
+import edu.uci.ics.asterix.metadata.entities.ExternalFile;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
+
+@SuppressWarnings("deprecation")
+public class HDFSIndexingAdapter extends FileSystemBasedAdapter{
+
+ private static final long serialVersionUID = 1L;
+ private transient String[] readSchedule;
+ private transient boolean executed[];
+ private transient InputSplit[] inputSplits;
+ private transient JobConf conf;
+ private final List<ExternalFile> files;
+ private transient String nodeName;
+ // file input-format <text, seq, rc>
+ 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);
+ this.nodeName = nodeName;
+ this.readSchedule = readSchedule;
+ this.executed = executed;
+ this.inputSplits = inputSplits;
+ this.conf = conf;
+ this.files = files;
+ this.inputFormat = inputFormat;
+ this.format = format;
+ }
+
+ @Override
+ public InputStream getInputStream(int partition) throws IOException {
+ if(inputFormat.equals(HDFSAdapterFactory.INPUT_FORMAT_RC)){
+ return new RCFileDataReader(inputSplits, readSchedule, nodeName, conf, executed, files);
+ } else if(format.equals(HDFSAdapterFactory.FORMAT_ADM) || format.equals(HDFSAdapterFactory.FORMAT_DELIMITED_TEXT)){
+ return new TextualDataReader(inputSplits,readSchedule,nodeName,conf,executed,files);
+ } else{
+ return new GenericFileAwareRecordReader(inputSplits, readSchedule, nodeName, conf, executed, files);
+ }
+ }
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/AbstractIndexingTupleParser.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/AbstractIndexingTupleParser.java
new file mode 100644
index 0000000..8fe4c70
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/AbstractIndexingTupleParser.java
@@ -0,0 +1,99 @@
+/*
+ * 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.external.indexing.dataflow;
+
+import java.io.DataOutput;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.asterix.external.indexing.input.AbstractHDFSReader;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.AMutableInt64;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
+
+public abstract class AbstractIndexingTupleParser implements ITupleParser{
+
+ protected ArrayTupleBuilder tb;
+ protected DataOutput dos;
+ protected final FrameTupleAppender appender;
+ protected final ByteBuffer frame;
+ protected final ARecordType recType;
+ protected final IHyracksTaskContext ctx;
+ protected final IAsterixHDFSRecordParser deserializer;
+ protected final AMutableInt32 aMutableInt = new AMutableInt32(0);
+ protected final AMutableInt64 aMutableLong = new AMutableInt64(0);
+
+ @SuppressWarnings("rawtypes")
+ protected final ISerializerDeserializer intSerde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
+ @SuppressWarnings("rawtypes")
+ protected final ISerializerDeserializer longSerde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
+
+ public AbstractIndexingTupleParser(IHyracksTaskContext ctx, ARecordType recType, IAsterixHDFSRecordParser deserializer) throws HyracksDataException {
+ appender = new FrameTupleAppender(ctx.getFrameSize());
+ frame = ctx.allocateFrame();
+ this.recType = recType;
+ this.ctx = ctx;
+ this.deserializer = deserializer;
+ }
+
+ @Override
+ public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
+ AbstractHDFSReader inReader = (AbstractHDFSReader) in;
+ appender.reset(frame, true);
+ Object record;
+ try {
+ inReader.initialize();
+ record = inReader.readNext();
+ while (record != null) {
+ tb.reset();
+ deserializer.parse(record, tb.getDataOutput());
+ tb.addFieldEndOffset();
+ //append indexing fields
+ appendIndexingData(tb, inReader);
+ addTupleToFrame(writer);
+ record = inReader.readNext();
+ }
+ if (appender.getTupleCount() > 0) {
+ FrameUtils.flushFrame(frame, writer);
+ }
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ protected abstract void appendIndexingData(ArrayTupleBuilder tb,
+ AbstractHDFSReader inReader) throws Exception;
+
+ protected void addTupleToFrame(IFrameWriter writer) throws HyracksDataException {
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ FrameUtils.flushFrame(frame, writer);
+ appender.reset(frame, true);
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ throw new IllegalStateException("Record is too big to fit in a frame");
+ }
+ }
+ }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/AdmOrDelimitedControlledTupleParser.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/AdmOrDelimitedControlledTupleParser.java
new file mode 100644
index 0000000..78baa4a
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/AdmOrDelimitedControlledTupleParser.java
@@ -0,0 +1,244 @@
+/*
+ * 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.external.indexing.dataflow;
+
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.asterix.external.indexing.input.AbstractHDFSLookupInputStream;
+import edu.uci.ics.asterix.metadata.external.IndexingConstants;
+import edu.uci.ics.asterix.om.base.AInt32;
+import edu.uci.ics.asterix.om.base.AInt64;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.runtime.operators.file.IDataParser;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+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;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+
+/**
+ * class implementation for IControlledTupleParser. It provides common
+ * functionality involved in parsing data in an external text format (adm or delimited text) in a pipelined manner and packing
+ * frames with formed tuples.
+ */
+public class AdmOrDelimitedControlledTupleParser implements IControlledTupleParser {
+
+ private ArrayTupleBuilder tb;
+ private transient DataOutput dos;
+ private final FrameTupleAppender appender;
+ private final ByteBuffer frame;
+ protected final ARecordType recType;
+ private IDataParser parser;
+ private boolean propagateInput;
+ private int[] propagatedFields;
+ private int[] ridFields;
+ private RecordDescriptor inRecDesc;
+ private FrameTupleAccessor tupleAccessor;
+ private FrameTupleReference frameTuple;
+ private ByteBufferInputStream bbis;
+ private DataInputStream dis;
+ private AbstractHDFSLookupInputStream in;
+ private boolean parserInitialized = false;
+ private boolean retainNull;
+ protected byte nullByte;
+ protected ArrayTupleBuilder nullTupleBuild;
+
+ public AdmOrDelimitedControlledTupleParser(IHyracksTaskContext ctx, ARecordType recType,
+ AbstractHDFSLookupInputStream in, boolean propagateInput, RecordDescriptor inRecDesc, IDataParser parser,
+ int[] propagatedFields, int[] ridFields, boolean retainNull, INullWriterFactory iNullWriterFactory)
+ throws HyracksDataException {
+ this.recType = recType;
+ this.in = in;
+ this.propagateInput = propagateInput;
+ this.retainNull = retainNull;
+ this.inRecDesc = inRecDesc;
+ this.propagatedFields = propagatedFields;
+ this.ridFields = ridFields;
+ this.parser = parser;
+ this.tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(), inRecDesc);
+ appender = new FrameTupleAppender(ctx.getFrameSize());
+ frame = ctx.allocateFrame();
+ if (propagateInput) {
+ tb = new ArrayTupleBuilder(propagatedFields.length + 1);
+ } else {
+ tb = new ArrayTupleBuilder(1);
+ }
+ frameTuple = new FrameTupleReference();
+ dos = tb.getDataOutput();
+ appender.reset(frame, true);
+ bbis = new ByteBufferInputStream();
+ dis = new DataInputStream(bbis);
+ nullByte = ATypeTag.NULL.serialize();
+ if (retainNull) {
+ INullWriter nullWriter = iNullWriterFactory.createNullWriter();
+ nullTupleBuild = new ArrayTupleBuilder(1);
+ DataOutput out = nullTupleBuild.getDataOutput();
+ try {
+ nullWriter.writeNull(out);
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ } else {
+ nullTupleBuild = null;
+ }
+ }
+
+ @Override
+ public void close(IFrameWriter writer) throws Exception {
+ try {
+ in.close();
+ if (appender.getTupleCount() > 0) {
+ FrameUtils.flushFrame(frame, writer);
+ }
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ @Override
+ public void parseNext(IFrameWriter writer, ByteBuffer frameBuffer) throws HyracksDataException {
+ try {
+ int tupleCount = 0;
+ int tupleIndex = 0;
+ tupleAccessor.reset(frameBuffer);
+ tupleCount = tupleAccessor.getTupleCount();
+ int fieldSlotsLength = tupleAccessor.getFieldSlotsLength();
+ // Loop over tuples
+ while (tupleIndex < tupleCount) {
+ boolean found = false;
+ int tupleStartOffset = tupleAccessor.getTupleStartOffset(tupleIndex) + fieldSlotsLength;
+ int fileNumberStartOffset = tupleAccessor.getFieldStartOffset(tupleIndex,
+ ridFields[IndexingConstants.FILE_NUMBER_FIELD_INDEX]);
+ // Check if null <- for outer join ->
+ if (frameBuffer.get(tupleStartOffset + fileNumberStartOffset) == nullByte) {
+ } else {
+ // Get file number
+ bbis.setByteBuffer(frameBuffer, tupleStartOffset + fileNumberStartOffset);
+ int fileNumber = ((AInt32) inRecDesc.getFields()[ridFields[IndexingConstants.FILE_NUMBER_FIELD_INDEX]]
+ .deserialize(dis)).getIntegerValue();
+ // Get record offset
+ bbis.setByteBuffer(
+ frameBuffer,
+ tupleStartOffset
+ + tupleAccessor.getFieldStartOffset(tupleIndex,
+ ridFields[IndexingConstants.RECORD_OFFSET_FIELD_INDEX]));
+ long recordOffset = ((AInt64) inRecDesc.getFields()[ridFields[IndexingConstants.RECORD_OFFSET_FIELD_INDEX]]
+ .deserialize(dis)).getLongValue();
+ found = in.fetchRecord(fileNumber, recordOffset);
+ }
+ if (found) {
+ // Since we now know the inputStream is ready, we can safely initialize the parser
+ // We can't do that earlier since the parser will start pulling from the stream and if it is not ready,
+ // The parser will automatically release its resources
+ if (!parserInitialized) {
+ parser.initialize(in, recType, true);
+ parserInitialized = true;
+ }
+ tb.reset();
+ if (propagateInput) {
+ frameTuple.reset(tupleAccessor, tupleIndex);
+ for (int i = 0; i < propagatedFields.length; i++) {
+ dos.write(frameTuple.getFieldData(propagatedFields[i]),
+ frameTuple.getFieldStart(propagatedFields[i]),
+ frameTuple.getFieldLength(propagatedFields[i]));
+ tb.addFieldEndOffset();
+ }
+ }
+ parser.parse(tb.getDataOutput());
+ tb.addFieldEndOffset();
+ addTupleToFrame(writer);
+ } else if (propagateInput && retainNull) {
+ tb.reset();
+ frameTuple.reset(tupleAccessor, tupleIndex);
+ for (int i = 0; i < propagatedFields.length; i++) {
+ dos.write(frameTuple.getFieldData(propagatedFields[i]),
+ frameTuple.getFieldStart(propagatedFields[i]),
+ frameTuple.getFieldLength(propagatedFields[i]));
+ tb.addFieldEndOffset();
+ }
+ dos.write(nullTupleBuild.getByteArray());
+ tb.addFieldEndOffset();
+ addTupleToFrame(writer);
+ }
+ tupleIndex++;
+ }
+ } catch (Exception e) {
+ // un expected error, we try to close the inputstream and throw an exception
+ try {
+ in.close();
+ } catch (IOException e1) {
+ e1.printStackTrace();
+ }
+ throw new HyracksDataException(e);
+ }
+ }
+
+ // For debugging
+ public void prettyPrint(FrameTupleAccessor tupleAccessor, RecordDescriptor recDesc) {
+ ByteBufferInputStream bbis = new ByteBufferInputStream();
+ DataInputStream dis = new DataInputStream(bbis);
+ int tc = tupleAccessor.getTupleCount();
+ System.err.println("TC: " + tc);
+ for (int i = 0; i < tc; ++i) {
+ System.err.print(i + ":(" + tupleAccessor.getTupleStartOffset(i) + ", "
+ + tupleAccessor.getTupleEndOffset(i) + ")[");
+ for (int j = 0; j < tupleAccessor.getFieldCount(); ++j) {
+ System.err.print(j + ":(" + tupleAccessor.getFieldStartOffset(i, j) + ", "
+ + tupleAccessor.getFieldEndOffset(i, j) + ") ");
+ System.err.print("{");
+ bbis.setByteBuffer(
+ tupleAccessor.getBuffer(),
+ tupleAccessor.getTupleStartOffset(i) + tupleAccessor.getFieldSlotsLength()
+ + tupleAccessor.getFieldStartOffset(i, j));
+ try {
+ byte tag = dis.readByte();
+ if (tag == nullByte) {
+ System.err.print("NULL");
+ } else {
+ bbis.setByteBuffer(tupleAccessor.getBuffer(), tupleAccessor.getTupleStartOffset(i)
+ + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(i, j));
+ System.err.print(recDesc.getFields()[j].deserialize(dis));
+ }
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ System.err.print("}");
+ }
+ System.err.println("]");
+ }
+ }
+
+ protected void addTupleToFrame(IFrameWriter writer) throws HyracksDataException {
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ FrameUtils.flushFrame(frame, writer);
+ appender.reset(frame, true);
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ throw new IllegalStateException();
+ }
+ }
+
+ }
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/AdmOrDelimitedIndexingTupleParser.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/AdmOrDelimitedIndexingTupleParser.java
new file mode 100644
index 0000000..74009e0
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/AdmOrDelimitedIndexingTupleParser.java
@@ -0,0 +1,108 @@
+/*
+ * 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.external.indexing.dataflow;
+
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.external.indexing.input.AbstractHDFSReader;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.AMutableInt64;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.runtime.operators.file.IDataParser;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
+
+public class AdmOrDelimitedIndexingTupleParser implements ITupleParser {
+
+ private ArrayTupleBuilder tb;
+ private final FrameTupleAppender appender;
+ private final ByteBuffer frame;
+ private final ARecordType recType;
+ private final IDataParser parser;
+ private final AMutableInt32 aMutableInt = new AMutableInt32(0);
+ private final AMutableInt64 aMutableLong = new AMutableInt64(0);
+
+ @SuppressWarnings("rawtypes")
+ private ISerializerDeserializer intSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT32);
+ @SuppressWarnings("rawtypes")
+ private ISerializerDeserializer longSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT64);
+
+ public AdmOrDelimitedIndexingTupleParser(IHyracksTaskContext ctx, ARecordType recType, IDataParser parser)
+ throws HyracksDataException {
+ this.parser = parser;
+ this.recType = recType;
+ appender = new FrameTupleAppender(ctx.getFrameSize());
+ tb = new ArrayTupleBuilder(3);
+ frame = ctx.allocateFrame();
+ }
+
+ @Override
+ public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
+ // Cast the input stream to a record reader
+ AbstractHDFSReader inReader = (AbstractHDFSReader) in;
+ appender.reset(frame, true);
+ try {
+ parser.initialize(in, recType, true);
+ while (true) {
+ tb.reset();
+ if (!parser.parse(tb.getDataOutput())) {
+ break;
+ }
+ tb.addFieldEndOffset();
+ appendIndexingData(tb, inReader);
+ addTupleToFrame(writer);
+ }
+ if (appender.getTupleCount() > 0) {
+ FrameUtils.flushFrame(frame, writer);
+ }
+ } catch (AsterixException ae) {
+ throw new HyracksDataException(ae);
+ } catch (Exception ioe) {
+ throw new HyracksDataException(ioe);
+ }
+ }
+
+ // This function is used to append RID to Hyracks tuple
+ @SuppressWarnings("unchecked")
+ private void appendIndexingData(ArrayTupleBuilder tb, AbstractHDFSReader inReader) throws Exception {
+ aMutableInt.setValue(inReader.getFileNumber());
+ aMutableLong.setValue(inReader.getReaderPosition());
+ tb.addField(intSerde, aMutableInt);
+ tb.addField(longSerde, aMutableLong);
+ }
+
+ private void addTupleToFrame(IFrameWriter writer) throws HyracksDataException {
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ FrameUtils.flushFrame(frame, writer);
+ appender.reset(frame, true);
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ throw new IllegalStateException("Record is too big to fit in a frame");
+ }
+ }
+ }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/FileIndexTupleTranslator.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/FileIndexTupleTranslator.java
new file mode 100644
index 0000000..bf048f7
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/FileIndexTupleTranslator.java
@@ -0,0 +1,85 @@
+/*
+ * 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.external.indexing.dataflow;
+
+import java.io.IOException;
+
+import edu.uci.ics.asterix.builders.RecordBuilder;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.metadata.entities.ExternalFile;
+import edu.uci.ics.asterix.metadata.external.FilesIndexDescription;
+import edu.uci.ics.asterix.om.base.ADateTime;
+import edu.uci.ics.asterix.om.base.AInt64;
+import edu.uci.ics.asterix.om.base.AMutableDateTime;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.AMutableInt64;
+import edu.uci.ics.asterix.om.base.AMutableString;
+import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+@SuppressWarnings("unchecked")
+public class FileIndexTupleTranslator {
+ private ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(FilesIndexDescription.FILE_INDEX_RECORD_DESCRIPTOR.getFieldCount());
+ private RecordBuilder recordBuilder = new RecordBuilder();
+ private ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
+ private AMutableInt32 aInt32 = new AMutableInt32(0);
+ private AMutableInt64 aInt64 = new AMutableInt64(0);
+ private AMutableString aString = new AMutableString(null);
+ private AMutableDateTime aDateTime = new AMutableDateTime(0);
+ private ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING);
+ private ISerializerDeserializer<ADateTime> dateTimeSerde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATETIME);
+ private ISerializerDeserializer<AInt64> longSerde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
+ private ArrayTupleReference tuple = new ArrayTupleReference();
+
+ public ITupleReference getTupleFromFile(ExternalFile file) throws IOException, AsterixException{
+ tupleBuilder.reset();
+ //File Number
+ aInt32.setValue(file.getFileNumber());
+ FilesIndexDescription.FILE_INDEX_RECORD_DESCRIPTOR.getFields()[0].serialize(aInt32, tupleBuilder.getDataOutput());
+ tupleBuilder.addFieldEndOffset();
+
+ //File Record
+ recordBuilder.reset(FilesIndexDescription.EXTERNAL_FILE_RECORD_TYPE);
+ // write field 0 (File Name)
+ fieldValue.reset();
+ aString.setValue(file.getFileName());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ recordBuilder.addField(0, fieldValue);
+
+ //write field 1 (File Size)
+ fieldValue.reset();
+ aInt64.setValue(file.getSize());
+ longSerde.serialize(aInt64, fieldValue.getDataOutput());
+ recordBuilder.addField(1, fieldValue);
+
+ //write field 2 (File Mod Date)
+ fieldValue.reset();
+ aDateTime.setValue(file.getLastModefiedTime().getTime());
+ dateTimeSerde.serialize(aDateTime, fieldValue.getDataOutput());
+ recordBuilder.addField(2, fieldValue);
+
+ //write the record
+ recordBuilder.write(tupleBuilder.getDataOutput(), true);
+ tupleBuilder.addFieldEndOffset();
+ tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+ return tuple;
+ }
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSIndexingParserFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSIndexingParserFactory.java
new file mode 100644
index 0000000..da37399
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSIndexingParserFactory.java
@@ -0,0 +1,134 @@
+/*
+ * 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.external.indexing.dataflow;
+
+import java.util.Map;
+
+import org.apache.hadoop.mapred.JobConf;
+
+import edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory;
+import edu.uci.ics.asterix.external.adapter.factory.HDFSIndexingAdapterFactory;
+import edu.uci.ics.asterix.external.adapter.factory.StreamBasedAdapterFactory;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.runtime.operators.file.ADMDataParser;
+import edu.uci.ics.asterix.runtime.operators.file.DelimitedDataParser;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
+
+/**
+ * This is the parser factory for parsers used to do indexing
+ */
+@SuppressWarnings("deprecation")
+public class HDFSIndexingParserFactory implements ITupleParserFactory {
+
+ private static final long serialVersionUID = 1L;
+ // file input-format <text, seq, rc>
+ private String inputFormat;
+ // content format <adm, delimited-text, binary>
+ private String format;
+ // delimiter in case of delimited text
+ private String delimiter;
+ // parser class name in case of binary format
+ private String parserClassName;
+ // the expected data type
+ private ARecordType atype;
+ // the hadoop job conf
+ private transient JobConf jobConf;
+ // adapter arguments
+ private Map<String,String> arguments;
+
+ public HDFSIndexingParserFactory(ARecordType atype, String inputFormat, String format, String delimiter,
+ String parserClassName) {
+ this.inputFormat = inputFormat;
+ this.format = format;
+ this.parserClassName = parserClassName;
+ this.delimiter = delimiter;
+ this.atype = atype;
+ }
+
+ @Override
+ public ITupleParser createTupleParser(IHyracksTaskContext ctx) throws HyracksDataException {
+ if (format == null) {
+ throw new IllegalArgumentException("Unspecified data format");
+ }
+ if (inputFormat == null) {
+ throw new IllegalArgumentException("Unspecified data format");
+ }
+ if (!inputFormat.equalsIgnoreCase(HDFSAdapterFactory.INPUT_FORMAT_RC) && !inputFormat.equalsIgnoreCase(HDFSAdapterFactory.INPUT_FORMAT_TEXT)
+ && !inputFormat.equalsIgnoreCase(HDFSAdapterFactory.INPUT_FORMAT_SEQUENCE)) {
+ throw new IllegalArgumentException("External Indexing not supportd for format " + inputFormat);
+ }
+ // Do some real work here
+ /*
+ * Choices are:
+ * 1. TxtOrSeq (Object) indexing tuple parser
+ * 2. RC indexing tuple parser
+ * 3. textual data tuple parser
+ */
+ if (format.equalsIgnoreCase(StreamBasedAdapterFactory.FORMAT_ADM)) {
+ // choice 3 with adm data parser
+ ADMDataParser dataParser = new ADMDataParser();
+ return new AdmOrDelimitedIndexingTupleParser(ctx, atype, dataParser);
+ } else if (format.equalsIgnoreCase(StreamBasedAdapterFactory.FORMAT_DELIMITED_TEXT)) {
+ // choice 3 with delimited data parser
+ DelimitedDataParser dataParser = HDFSIndexingAdapterFactory.getDilimitedDataParser(atype, delimiter.charAt(0));
+ return new AdmOrDelimitedIndexingTupleParser(ctx, atype, dataParser);
+ }
+
+ // binary data with a special parser --> create the parser
+ IAsterixHDFSRecordParser objectParser;
+ if (parserClassName.equalsIgnoreCase(HDFSAdapterFactory.PARSER_HIVE)) {
+ objectParser = new HiveObjectParser();
+ } else {
+ try {
+ objectParser = (IAsterixHDFSRecordParser) Class.forName(parserClassName).newInstance();
+ } catch (Exception e) {
+ throw new HyracksDataException("Unable to create object parser", e);
+ }
+ }
+ try {
+ objectParser.initialize(atype, arguments, jobConf);
+ } catch (Exception e) {
+ throw new HyracksDataException("Unable to initialize object parser", e);
+ }
+
+ if(inputFormat.equalsIgnoreCase(HDFSAdapterFactory.INPUT_FORMAT_RC)){
+ // Case 2
+ return new RCFileIndexingTupleParser(ctx, atype, objectParser);
+ } else{
+ // Case 1
+ return new TextOrSeqIndexingTupleParser(ctx, atype, objectParser);
+ }
+ }
+
+ public JobConf getJobConf() {
+ return jobConf;
+ }
+
+ public void setJobConf(JobConf jobConf) {
+ this.jobConf = jobConf;
+ }
+
+ public Map<String,String> getArguments() {
+ return arguments;
+ }
+
+ public void setArguments(Map<String,String> arguments) {
+ this.arguments = arguments;
+ }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSLookupAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSLookupAdapter.java
new file mode 100644
index 0000000..90bde5a
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSLookupAdapter.java
@@ -0,0 +1,178 @@
+/*
+ * 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.external.indexing.dataflow;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.JobConf;
+
+import edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory;
+import edu.uci.ics.asterix.external.adapter.factory.HDFSIndexingAdapterFactory;
+import edu.uci.ics.asterix.external.indexing.input.RCFileLookupReader;
+import edu.uci.ics.asterix.external.indexing.input.SequenceFileLookupInputStream;
+import edu.uci.ics.asterix.external.indexing.input.SequenceFileLookupReader;
+import edu.uci.ics.asterix.external.indexing.input.TextFileLookupInputStream;
+import edu.uci.ics.asterix.external.indexing.input.TextFileLookupReader;
+import edu.uci.ics.asterix.metadata.external.ExternalFileIndexAccessor;
+import edu.uci.ics.asterix.metadata.external.IControlledAdapter;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.runtime.operators.file.ADMDataParser;
+import edu.uci.ics.asterix.runtime.operators.file.DelimitedDataParser;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+@SuppressWarnings("deprecation")
+public class HDFSLookupAdapter implements IControlledAdapter, Serializable {
+
+ private static final long serialVersionUID = 1L;
+
+ private RecordDescriptor inRecDesc;
+ private boolean propagateInput;
+ private int[] ridFields;
+ private int[] propagatedFields;
+ private IAType atype;
+ private Map<String, String> configuration;
+ private IHyracksTaskContext ctx;
+ private IControlledTupleParser parser;
+ private ExternalFileIndexAccessor fileIndexAccessor;
+ private boolean retainNull;
+
+ public HDFSLookupAdapter(IAType atype, RecordDescriptor inRecDesc, Map<String, String> adapterConfiguration,
+ boolean propagateInput, int[] ridFields, int[] propagatedFields, IHyracksTaskContext ctx,
+ ExternalFileIndexAccessor fileIndexAccessor, boolean retainNull) {
+ this.configuration = adapterConfiguration;
+ this.atype = atype;
+ this.ctx = ctx;
+ this.inRecDesc = inRecDesc;
+ this.propagatedFields = propagatedFields;
+ this.propagateInput = propagateInput;
+ this.propagatedFields = propagatedFields;
+ this.fileIndexAccessor = fileIndexAccessor;
+ this.ridFields = ridFields;
+ this.retainNull = retainNull;
+ }
+
+ /*
+ * This function is not easy to read and could be refactored into a better structure but for now it works
+ */
+ @Override
+ public void initialize(IHyracksTaskContext ctx, INullWriterFactory iNullWriterFactory) throws Exception {
+ JobConf jobConf = HDFSAdapterFactory.configureJobConf(configuration);
+ // Create the lookup reader and the controlled parser
+ if (configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT).equals(HDFSAdapterFactory.INPUT_FORMAT_RC)) {
+ configureRCFile(jobConf, iNullWriterFactory);
+ } else if (configuration.get(HDFSAdapterFactory.KEY_FORMAT).equals(HDFSAdapterFactory.FORMAT_ADM)) {
+ // create an adm parser
+ ADMDataParser dataParser = new ADMDataParser();
+ if (configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT).equals(HDFSAdapterFactory.INPUT_FORMAT_TEXT)) {
+ // Text input format
+ TextFileLookupInputStream in = new TextFileLookupInputStream(fileIndexAccessor, jobConf);
+ parser = new AdmOrDelimitedControlledTupleParser(ctx, (ARecordType) atype, in, propagateInput,
+ inRecDesc, dataParser, propagatedFields, ridFields, retainNull, iNullWriterFactory);
+ } else {
+ // Sequence input format
+ SequenceFileLookupInputStream in = new SequenceFileLookupInputStream(fileIndexAccessor, jobConf);
+ parser = new AdmOrDelimitedControlledTupleParser(ctx, (ARecordType) atype, in, propagateInput,
+ inRecDesc, dataParser, propagatedFields, ridFields, retainNull, iNullWriterFactory);
+ }
+ } else if (configuration.get(HDFSAdapterFactory.KEY_FORMAT).equals(HDFSAdapterFactory.FORMAT_DELIMITED_TEXT)) {
+ // create a delimited text parser
+ DelimitedDataParser dataParser = HDFSIndexingAdapterFactory.getDilimitedDataParser((ARecordType) atype,
+ (configuration.get(HDFSAdapterFactory.KEY_DELIMITER)).charAt(0));
+ if (configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT).equals(HDFSAdapterFactory.INPUT_FORMAT_TEXT)) {
+ // Text input format
+ TextFileLookupInputStream in = new TextFileLookupInputStream(fileIndexAccessor, jobConf);
+ parser = new AdmOrDelimitedControlledTupleParser(ctx, (ARecordType) atype, in, propagateInput,
+ inRecDesc, dataParser, propagatedFields, ridFields, retainNull, iNullWriterFactory);
+ } else {
+ // Sequence input format
+ SequenceFileLookupInputStream in = new SequenceFileLookupInputStream(fileIndexAccessor, jobConf);
+ parser = new AdmOrDelimitedControlledTupleParser(ctx, (ARecordType) atype, in, propagateInput,
+ inRecDesc, dataParser, propagatedFields, ridFields, retainNull, iNullWriterFactory);
+ }
+ } else {
+ configureGenericSeqOrText(jobConf, iNullWriterFactory);
+ }
+ }
+
+ private void configureGenericSeqOrText(JobConf jobConf, INullWriterFactory iNullWriterFactory) throws IOException {
+ if (configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT).equals(HDFSAdapterFactory.INPUT_FORMAT_TEXT)) {
+ // Text input format
+ TextFileLookupReader reader = new TextFileLookupReader(fileIndexAccessor, jobConf);
+ parser = new SeqOrTxtControlledTupleParser(ctx, createRecordParser(jobConf), reader, propagateInput,
+ propagatedFields, inRecDesc, ridFields, retainNull, iNullWriterFactory);
+ } else {
+ // Sequence input format
+ SequenceFileLookupReader reader = new SequenceFileLookupReader(fileIndexAccessor, jobConf);
+ parser = new SeqOrTxtControlledTupleParser(ctx, createRecordParser(jobConf), reader, propagateInput,
+ propagatedFields, inRecDesc, ridFields, retainNull, iNullWriterFactory);
+ }
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer, IFrameWriter writer) throws Exception {
+ parser.parseNext(writer, buffer);
+ }
+
+ @Override
+ public void close(IFrameWriter writer) throws Exception {
+ parser.close(writer);
+ }
+
+ @Override
+ public void fail() throws Exception {
+ // Do nothing
+ }
+
+ private void configureRCFile(Configuration jobConf, INullWriterFactory iNullWriterFactory) throws IOException, Exception {
+ // RCFileLookupReader
+ RCFileLookupReader reader = new RCFileLookupReader(fileIndexAccessor,
+ HDFSAdapterFactory.configureJobConf(configuration));
+ parser = new RCFileControlledTupleParser(ctx, createRecordParser(jobConf), reader, propagateInput,
+ propagatedFields, inRecDesc, ridFields, retainNull, iNullWriterFactory);
+ }
+
+ private IAsterixHDFSRecordParser createRecordParser(Configuration jobConf) throws HyracksDataException {
+ // Create the record parser
+ // binary data with a special parser --> create the parser
+ IAsterixHDFSRecordParser objectParser;
+ if (configuration.get(HDFSAdapterFactory.KEY_PARSER).equals(HDFSAdapterFactory.PARSER_HIVE)) {
+ objectParser = new HiveObjectParser();
+ } else {
+ try {
+ objectParser = (IAsterixHDFSRecordParser) Class.forName(
+ configuration.get(HDFSAdapterFactory.KEY_PARSER)).newInstance();
+ } catch (Exception e) {
+ throw new HyracksDataException("Unable to create object parser", e);
+ }
+ }
+ // initialize the parser
+ try {
+ objectParser.initialize((ARecordType) atype, configuration, jobConf);
+ } catch (Exception e) {
+ throw new HyracksDataException("Unable to initialize object parser", e);
+ }
+
+ return objectParser;
+ }
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSLookupAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSLookupAdapterFactory.java
new file mode 100644
index 0000000..a6dda87
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSLookupAdapterFactory.java
@@ -0,0 +1,173 @@
+/*
+ * 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.external.indexing.dataflow;
+
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.asterix.common.dataflow.IAsterixApplicationContextInfo;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
+import edu.uci.ics.asterix.metadata.MetadataException;
+import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
+import edu.uci.ics.asterix.metadata.entities.Index;
+import edu.uci.ics.asterix.metadata.external.ExternalFileIndexAccessor;
+import edu.uci.ics.asterix.metadata.external.ExternalLoopkupOperatorDiscriptor;
+import edu.uci.ics.asterix.metadata.external.IControlledAdapter;
+import edu.uci.ics.asterix.metadata.external.IControlledAdapterFactory;
+import edu.uci.ics.asterix.metadata.external.IndexingConstants;
+import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
+import edu.uci.ics.asterix.metadata.utils.ExternalDatasetsRegistry;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexSearchOperationCallbackFactory;
+import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+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.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
+
+// This class takes care of creating the adapter based on the formats and input format
+public class HDFSLookupAdapterFactory implements IControlledAdapterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private Map<String, String> adapterConfiguration;
+ private IAType atype;
+ private boolean propagateInput;
+ private int[] ridFields;
+ private int[] propagatedFields;
+ private boolean retainNull;
+
+ @Override
+ public void configure(IAType atype, boolean propagateInput, int[] ridFields,
+ Map<String, String> adapterConfiguration, boolean retainNull) {
+ this.adapterConfiguration = adapterConfiguration;
+ this.atype = atype;
+ this.propagateInput = propagateInput;
+ this.ridFields = ridFields;
+ this.retainNull = retainNull;
+ }
+
+ @Override
+ public IControlledAdapter createAdapter(IHyracksTaskContext ctx, ExternalFileIndexAccessor fileIndexAccessor,
+ RecordDescriptor inRecDesc) {
+ if (propagateInput) {
+ configurePropagatedFields(inRecDesc);
+ }
+ return new HDFSLookupAdapter(atype, inRecDesc, adapterConfiguration, propagateInput, ridFields,
+ propagatedFields, ctx, fileIndexAccessor, retainNull);
+ }
+
+ private void configurePropagatedFields(RecordDescriptor inRecDesc) {
+ int ptr = 0;
+ boolean skip = false;
+ propagatedFields = new int[inRecDesc.getFieldCount() - ridFields.length];
+ for (int i = 0; i < inRecDesc.getFieldCount(); i++) {
+ if (ptr < ridFields.length) {
+ skip = false;
+ for (int j = 0; j < ridFields.length; j++) {
+ if (ridFields[j] == i) {
+ ptr++;
+ skip = true;
+ break;
+ }
+ }
+ if (!skip)
+ propagatedFields[i - ptr] = i;
+ } else {
+ propagatedFields[i - ptr] = i;
+ }
+ }
+ }
+
+ /*
+ * This function creates an operator that uses the built indexes in asterix to perform record lookup over external data
+ */
+ public static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildExternalDataLookupRuntime(
+ JobSpecification jobSpec, Dataset dataset, Index secondaryIndex, int[] ridIndexes, boolean retainInput,
+ IVariableTypeEnvironment typeEnv, List<LogicalVariable> outputVars, IOperatorSchema opSchema,
+ JobGenContext context, AqlMetadataProvider metadataProvider, boolean retainNull) throws AlgebricksException {
+
+ // Get data type
+ IAType itemType = null;
+ try {
+ itemType = MetadataManager.INSTANCE.getDatatype(metadataProvider.getMetadataTxnContext(),
+ dataset.getDataverseName(), dataset.getItemTypeName()).getDatatype();
+ } catch (MetadataException e) {
+ e.printStackTrace();
+ throw new AlgebricksException("Unable to get item type from metadata " + e);
+ }
+ if (itemType.getTypeTag() != ATypeTag.RECORD) {
+ throw new AlgebricksException("Can only scan datasets of records.");
+ }
+
+ // Create the adapter factory <- right now there is only one. if there are more in the future, we can create a map->
+ ExternalDatasetDetails datasetDetails = (ExternalDatasetDetails) dataset.getDatasetDetails();
+ HDFSLookupAdapterFactory adapterFactory = new HDFSLookupAdapterFactory();
+ adapterFactory.configure(itemType, retainInput, ridIndexes, datasetDetails.getProperties(), retainNull);
+
+ Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo;
+ try {
+ compactionInfo = DatasetUtils.getMergePolicyFactory(dataset, metadataProvider.getMetadataTxnContext());
+ } catch (MetadataException e) {
+ throw new AlgebricksException(" Unabel to create merge policy factory for external dataset", e);
+ }
+
+ // Create the file index data flow helper
+ ExternalBTreeDataflowHelperFactory indexDataflowHelperFactory = new ExternalBTreeDataflowHelperFactory(
+ compactionInfo.first, compactionInfo.second, new SecondaryIndexOperationTrackerProvider(
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMBTreeIOOperationCallbackFactory.INSTANCE, metadataProvider.getStorageProperties()
+ .getBloomFilterFalsePositiveRate(), ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(
+ dataset, metadataProvider));
+
+ // Create the out record descriptor, appContext and fileSplitProvider for the files index
+ RecordDescriptor outRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
+ IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc;
+ try {
+ spPc = metadataProvider.splitProviderAndPartitionConstraintsForFilesIndex(dataset.getDataverseName(),
+ dataset.getDatasetName(),
+ dataset.getDatasetName().concat(IndexingConstants.EXTERNAL_FILE_INDEX_NAME_SUFFIX), false);
+ } catch (Exception e) {
+ throw new AlgebricksException(e);
+ }
+
+ // Create the operator
+ ExternalLoopkupOperatorDiscriptor op = new ExternalLoopkupOperatorDiscriptor(jobSpec, adapterFactory,
+ outRecDesc, indexDataflowHelperFactory, retainInput, appContext.getIndexLifecycleManagerProvider(),
+ appContext.getStorageManagerInterface(), spPc.first, dataset.getDatasetId(), metadataProvider
+ .getStorageProperties().getBloomFilterFalsePositiveRate(),
+ new SecondaryIndexSearchOperationCallbackFactory(),retainNull,context.getNullWriterFactory());
+
+ // Return value
+ return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(op, spPc.second);
+ }
+}
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
new file mode 100644
index 0000000..a95cabc
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSObjectTupleParser.java
@@ -0,0 +1,83 @@
+/*
+ * 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.external.indexing.dataflow;
+
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.external.indexing.input.AbstractHDFSReader;
+import edu.uci.ics.asterix.om.types.ARecordType;
+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.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
+
+/*
+ * 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();
+ this.deserializer = deserializer;
+ tb = new ArrayTupleBuilder(1);
+ }
+
+ @Override
+ public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
+ AbstractHDFSReader reader = (AbstractHDFSReader) in;
+ appender.reset(frame, true);
+ Object object;
+ try {
+ reader.initialize();
+ object = reader.readNext();
+ while (object!= null) {
+ tb.reset();
+ deserializer.parse(object, tb.getDataOutput());
+ tb.addFieldEndOffset();
+ addTupleToFrame(writer);
+ object = reader.readNext();
+ }
+ if (appender.getTupleCount() > 0) {
+ FrameUtils.flushFrame(frame, writer);
+ }
+ } catch (AsterixException ae) {
+ throw new HyracksDataException(ae);
+ } catch (Exception e) {
+ 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);
+ appender.reset(frame, true);
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ throw new IllegalStateException();
+ }
+ }
+ }
+
+}
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
new file mode 100644
index 0000000..af78902
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSObjectTupleParserFactory.java
@@ -0,0 +1,66 @@
+/*
+ * 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.external.indexing.dataflow;
+
+import java.util.Map;
+
+import edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
+
+public class HDFSObjectTupleParserFactory implements ITupleParserFactory{
+ private static final long serialVersionUID = 1L;
+ // parser class name in case of binary format
+ private String parserClassName;
+ // the expected data type
+ private ARecordType atype;
+ // the hadoop job conf
+ 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;
+ if (parserClassName.equals(HDFSAdapterFactory.PARSER_HIVE)) {
+ objectParser = new HiveObjectParser();
+ } else {
+ try {
+ objectParser = (IAsterixHDFSRecordParser) Class.forName(parserClassName).newInstance();
+ } catch (Exception e) {
+ throw new HyracksDataException("Unable to create object parser", e);
+ }
+ }
+ try {
+ objectParser.initialize(atype, arguments, adapterFactory.getJobConf());
+ } 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/HiveObjectParser.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HiveObjectParser.java
new file mode 100644
index 0000000..3651bb9
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HiveObjectParser.java
@@ -0,0 +1,413 @@
+/*
+ * 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.external.indexing.dataflow;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.serde.Constants;
+import org.apache.hadoop.hive.serde2.SerDe;
+import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.BooleanObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.ByteObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.DoubleObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.FloatObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.ShortObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampObjectInspector;
+import org.apache.hadoop.io.Writable;
+
+import edu.uci.ics.asterix.builders.IARecordBuilder;
+import edu.uci.ics.asterix.builders.OrderedListBuilder;
+import edu.uci.ics.asterix.builders.RecordBuilder;
+import edu.uci.ics.asterix.builders.UnorderedListBuilder;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.om.base.temporal.GregorianCalendarSystem;
+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.IAType;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
+
+@SuppressWarnings("deprecation")
+public class HiveObjectParser implements IAsterixHDFSRecordParser{
+
+ private static final String KEY_HIVE_SERDE = "hive-serde";
+ private ARecordType aRecord;
+ private SerDe hiveSerde;
+ private StructObjectInspector oi;
+ private IARecordBuilder recBuilder;
+ private ArrayBackedValueStorage fieldValueBuffer;
+ private ArrayBackedValueStorage listItemBuffer;
+ private byte[] fieldTypeTags;
+ private IAType[] fieldTypes;
+ private OrderedListBuilder orderedListBuilder;
+ private UnorderedListBuilder unorderedListBuilder;
+ private boolean initialized = false;
+ private List<StructField> fieldRefs;
+
+ @SuppressWarnings({ "unchecked" })
+ @Override
+ public void initialize(ARecordType record, Map<String, String> arguments, Configuration hadoopConfig) throws Exception {
+ if(!initialized){
+ this.aRecord = record;
+ int n = record.getFieldNames().length;
+ fieldTypes = record.getFieldTypes();
+
+ //create the hive table schema.
+ Properties tbl = new Properties();
+ tbl.put(Constants.LIST_COLUMNS, getCommaDelimitedColNames(record));
+ tbl.put(Constants.LIST_COLUMN_TYPES, getColTypes(record));
+ String hiveSerdeClassName = (String) arguments.get(KEY_HIVE_SERDE);
+ if(hiveSerdeClassName == null){
+ throw new IllegalArgumentException("no hive serde provided for hive deserialized records");
+ }
+ hiveSerde = (SerDe) Class.forName(hiveSerdeClassName).newInstance();
+ hiveSerde.initialize(hadoopConfig, tbl);
+ oi = (StructObjectInspector) hiveSerde.getObjectInspector();
+
+ fieldValueBuffer = new ArrayBackedValueStorage();
+ recBuilder = new RecordBuilder();
+ recBuilder.reset(record);
+ recBuilder.init();
+ fieldTypeTags = new byte[n];
+ for (int i = 0; i < n; i++) {
+ ATypeTag tag = record.getFieldTypes()[i].getTypeTag();
+ fieldTypeTags[i] = tag.serialize();
+ }
+ fieldRefs = (List<StructField>) oi.getAllStructFieldRefs();
+ initialized = true;
+ }
+ }
+
+ private Object getColTypes(ARecordType record) throws Exception{
+ int n = record.getFieldTypes().length;
+ if(n < 1){
+ throw new HyracksDataException("Failed to get columns of record");
+ }
+ ATypeTag tag = null;
+
+ //First Column
+ if (record.getFieldTypes()[0].getTypeTag() == ATypeTag.UNION) {
+ List<IAType> unionTypes = ((AUnionType) record.getFieldTypes()[0]).getUnionList();
+ if (unionTypes.size() != 2 && unionTypes.get(0).getTypeTag() != ATypeTag.NULL) {
+ throw new NotImplementedException("Non-optional UNION type is not supported.");
+ }
+ tag = unionTypes.get(1).getTypeTag();
+ } else {
+ tag = record.getFieldTypes()[0].getTypeTag();
+ }
+ if (tag == null) {
+ throw new NotImplementedException("Failed to get the type information for field " + 0 + ".");
+ }
+ String cols = getHiveTypeString(tag);
+
+ for (int i = 1; i < n; i++) {
+ tag = null;
+ if (record.getFieldTypes()[i].getTypeTag() == ATypeTag.UNION) {
+ List<IAType> unionTypes = ((AUnionType) record.getFieldTypes()[i]).getUnionList();
+ if (unionTypes.size() != 2 && unionTypes.get(0).getTypeTag() != ATypeTag.NULL) {
+ throw new NotImplementedException("Non-optional UNION type is not supported.");
+ }
+ tag = unionTypes.get(1).getTypeTag();
+ } else {
+ tag = record.getFieldTypes()[i].getTypeTag();
+ }
+ if (tag == null) {
+ throw new NotImplementedException("Failed to get the type information for field " + i + ".");
+ }
+ cols = cols + "," + getHiveTypeString(tag);
+ }
+ return cols;
+ }
+
+ private String getCommaDelimitedColNames(ARecordType record) throws Exception {
+ if(record.getFieldNames().length < 1){
+ throw new HyracksDataException("Can't deserialize hive records with no closed columns");
+ }
+
+ String cols = record.getFieldNames()[0];
+ for(int i=1; i<record.getFieldNames().length; i++){
+ cols = cols + "," + record.getFieldNames()[i];
+ }
+ return cols;
+ }
+
+ private String getHiveTypeString(ATypeTag tag) throws Exception{
+ switch(tag){
+ case BOOLEAN:
+ return Constants.BOOLEAN_TYPE_NAME;
+ case DATE:
+ return Constants.DATE_TYPE_NAME;
+ case DATETIME:
+ return Constants.DATETIME_TYPE_NAME;
+ case DOUBLE:
+ return Constants.DOUBLE_TYPE_NAME;
+ case FLOAT:
+ return Constants.FLOAT_TYPE_NAME;
+ case INT16:
+ return Constants.SMALLINT_TYPE_NAME;
+ case INT32:
+ return Constants.INT_TYPE_NAME;
+ case INT64:
+ return Constants.BIGINT_TYPE_NAME;
+ case INT8:
+ return Constants.TINYINT_TYPE_NAME;
+ case ORDEREDLIST:
+ return Constants.LIST_TYPE_NAME;
+ case STRING:
+ return Constants.STRING_TYPE_NAME;
+ case TIME:
+ return Constants.DATETIME_TYPE_NAME;
+ case UNORDEREDLIST:
+ return Constants.LIST_TYPE_NAME;
+ default:
+ throw new HyracksDataException("Can't get hive type for field of type " + tag);
+ }
+ }
+ @Override
+ public void parse(Object object, DataOutput output) throws Exception {
+ if(object == null){
+ throw new HyracksDataException("Hive parser can't parse null objects");
+ }
+ Object hiveObject = hiveSerde.deserialize((Writable) object);
+ int n = aRecord.getFieldNames().length;
+ List<Object> attributesValues = oi.getStructFieldsDataAsList(hiveObject);
+ recBuilder.reset(aRecord);
+ recBuilder.init();
+ for(int i=0;i<n;i++){
+ fieldValueBuffer.reset();
+ fieldValueBuffer.getDataOutput().writeByte(fieldTypeTags[i]);
+ ObjectInspector foi = fieldRefs.get(i).getFieldObjectInspector();
+ //get field type
+ switch(fieldTypes[i].getTypeTag()){
+ case BOOLEAN:
+ parseBoolean(attributesValues.get(i), (BooleanObjectInspector)foi, fieldValueBuffer.getDataOutput());
+ break;
+ case TIME:
+ parseTime(attributesValues.get(i), (TimestampObjectInspector)foi, fieldValueBuffer.getDataOutput());
+ break;
+ case DATE:
+ parseDate(attributesValues.get(i), (TimestampObjectInspector)foi, fieldValueBuffer.getDataOutput());
+ break;
+ case DATETIME:
+ parseDateTime(attributesValues.get(i), (TimestampObjectInspector)foi, fieldValueBuffer.getDataOutput());
+ break;
+ case DOUBLE:
+ parseDouble(attributesValues.get(i), (DoubleObjectInspector)foi, fieldValueBuffer.getDataOutput());
+ break;
+ case FLOAT:
+ parseFloat(attributesValues.get(i), (FloatObjectInspector)foi, fieldValueBuffer.getDataOutput());
+ break;
+ case INT8:
+ parseInt8(attributesValues.get(i), (ByteObjectInspector)foi, fieldValueBuffer.getDataOutput());
+ break;
+ case INT16:
+ parseInt16(attributesValues.get(i), (ShortObjectInspector)foi, fieldValueBuffer.getDataOutput());
+ break;
+ case INT32:
+ parseInt32(attributesValues.get(i), (IntObjectInspector)foi, fieldValueBuffer.getDataOutput());
+ break;
+ case INT64:
+ parseInt64(attributesValues.get(i), (LongObjectInspector)foi, fieldValueBuffer.getDataOutput());
+ break;
+ case STRING:
+ parseString(attributesValues.get(i), (StringObjectInspector)foi, fieldValueBuffer.getDataOutput());
+ break;
+ case ORDEREDLIST:
+ parseOrderedList((AOrderedListType)fieldTypes[i], attributesValues.get(i), (ListObjectInspector)foi);
+ break;
+ case UNORDEREDLIST:
+ parseUnorderedList((AUnorderedListType)fieldTypes[i], attributesValues.get(i), (ListObjectInspector)foi);
+ break;
+ default:
+ throw new HyracksDataException("Can't get hive type for field of type " + fieldTypes[i].getTypeTag());
+ }
+ recBuilder.addField(i, fieldValueBuffer);
+ }
+ recBuilder.write(output, true);
+ }
+
+ private void parseInt64(Object obj, LongObjectInspector foi, DataOutput dataOutput) throws IOException {
+ dataOutput.writeLong(foi.get(obj));
+ }
+
+ private void parseInt32(Object obj, IntObjectInspector foi, DataOutput dataOutput) throws IOException {
+ if(obj == null){
+ throw new HyracksDataException("can't parse null field");
+ }
+ dataOutput.writeInt(foi.get(obj));
+ }
+
+ private void parseInt16(Object obj, ShortObjectInspector foi, DataOutput dataOutput) throws IOException {
+ dataOutput.writeShort(foi.get(obj));
+ }
+
+ private void parseFloat(Object obj, FloatObjectInspector foi, DataOutput dataOutput) throws IOException {
+ dataOutput.writeFloat(foi.get(obj));
+ }
+
+ private void parseDouble(Object obj, DoubleObjectInspector foi, DataOutput dataOutput) throws IOException {
+ dataOutput.writeDouble(foi.get(obj));
+ }
+
+ private void parseDateTime(Object obj, TimestampObjectInspector foi, DataOutput dataOutput) throws IOException {
+ dataOutput.writeLong(foi.getPrimitiveJavaObject(obj).getTime()) ;
+ }
+
+ private void parseDate(Object obj, TimestampObjectInspector foi, DataOutput dataOutput) throws IOException {
+ long chrononTimeInMs = foi.getPrimitiveJavaObject(obj).getTime();
+ short temp = 0;
+ if (chrononTimeInMs < 0 && chrononTimeInMs % GregorianCalendarSystem.CHRONON_OF_DAY != 0) {
+ temp = 1;
+ }
+ dataOutput.writeInt((int) (chrononTimeInMs / GregorianCalendarSystem.CHRONON_OF_DAY) - temp);
+ }
+
+ private void parseBoolean(Object obj, BooleanObjectInspector foi, DataOutput dataOutput) throws IOException {
+ dataOutput.writeBoolean(foi.get(obj));
+ }
+
+ private void parseInt8(Object obj, ByteObjectInspector foi, DataOutput dataOutput) throws IOException {
+ dataOutput.writeByte(foi.get(obj));
+ }
+
+ private void parseString(Object obj, StringObjectInspector foi, DataOutput dataOutput) throws IOException {
+ dataOutput.writeUTF(foi.getPrimitiveJavaObject(obj));
+ }
+
+ private void parseTime(Object obj,
+ TimestampObjectInspector foi, DataOutput dataOutput) throws IOException {
+ dataOutput.writeInt((int)(foi.getPrimitiveJavaObject(obj).getTime() % 86400000));
+ }
+
+ private void parseOrderedList(AOrderedListType aOrderedListType, Object obj, ListObjectInspector foi) throws IOException {
+ OrderedListBuilder orderedListBuilder = getOrderedListBuilder();
+ IAType itemType = null;
+ if (aOrderedListType != null)
+ itemType = aOrderedListType.getItemType();
+ orderedListBuilder.reset(aOrderedListType);
+
+ int n = foi.getListLength(obj);
+ for(int i=0; i<n;i++){
+ Object element = foi.getListElement(obj, i);
+ ObjectInspector eoi = foi.getListElementObjectInspector();
+ if(element == null){
+ throw new HyracksDataException("can't parse hive list with null values");
+ }
+
+ parseHiveListItem(element, eoi, listItemBuffer, itemType);
+ orderedListBuilder.addItem(listItemBuffer);
+ }
+ orderedListBuilder.write(fieldValueBuffer.getDataOutput(), true);
+ }
+
+ private void parseUnorderedList(AUnorderedListType uoltype, Object obj, ListObjectInspector oi) throws IOException, AsterixException {
+ UnorderedListBuilder unorderedListBuilder = getUnorderedListBuilder();
+ IAType itemType = null;
+ if (uoltype != null)
+ itemType = uoltype.getItemType();
+ byte tagByte = itemType.getTypeTag().serialize();
+ unorderedListBuilder.reset(uoltype);
+
+ int n = oi.getListLength(obj);
+ for(int i=0; i<n;i++){
+ Object element = oi.getListElement(obj, i);
+ ObjectInspector eoi = oi.getListElementObjectInspector();
+ if(element == null){
+ throw new HyracksDataException("can't parse hive list with null values");
+ }
+ listItemBuffer.reset();
+ listItemBuffer.getDataOutput().writeByte(tagByte);
+ parseHiveListItem(element, eoi, listItemBuffer, itemType);
+ unorderedListBuilder.addItem(listItemBuffer);
+ }
+ unorderedListBuilder.write(fieldValueBuffer.getDataOutput(), true);
+ }
+
+ private void parseHiveListItem(Object obj, ObjectInspector eoi,
+ ArrayBackedValueStorage fieldValueBuffer, IAType itemType) throws IOException {
+ //get field type
+ switch(itemType.getTypeTag()){
+ case BOOLEAN:
+ parseBoolean(obj, (BooleanObjectInspector)eoi, fieldValueBuffer.getDataOutput());
+ break;
+ case TIME:
+ parseTime(obj, (TimestampObjectInspector)eoi, fieldValueBuffer.getDataOutput());
+ break;
+ case DATE:
+ parseDate(obj, (TimestampObjectInspector)eoi, fieldValueBuffer.getDataOutput());
+ break;
+ case DATETIME:
+ parseDateTime(obj, (TimestampObjectInspector)eoi, fieldValueBuffer.getDataOutput());
+ break;
+ case DOUBLE:
+ parseDouble(obj, (DoubleObjectInspector)eoi, fieldValueBuffer.getDataOutput());
+ break;
+ case FLOAT:
+ parseFloat(obj, (FloatObjectInspector)eoi, fieldValueBuffer.getDataOutput());
+ break;
+ case INT8:
+ parseInt8(obj, (ByteObjectInspector)eoi, fieldValueBuffer.getDataOutput());
+ break;
+ case INT16:
+ parseInt16(obj, (ShortObjectInspector)eoi, fieldValueBuffer.getDataOutput());
+ break;
+ case INT32:
+ parseInt32(obj, (IntObjectInspector)eoi, fieldValueBuffer.getDataOutput());
+ break;
+ case INT64:
+ parseInt64(obj, (LongObjectInspector)eoi, fieldValueBuffer.getDataOutput());
+ break;
+ case STRING:
+ parseString(obj, (StringObjectInspector)eoi, fieldValueBuffer.getDataOutput());
+ break;
+ default:
+ throw new HyracksDataException("doesn't support hive data with list of non-primitive types");
+ }
+ }
+
+ private OrderedListBuilder getOrderedListBuilder() {
+ if (orderedListBuilder != null)
+ return orderedListBuilder;
+ else{
+ orderedListBuilder = new OrderedListBuilder();
+ return orderedListBuilder;
+ }
+ }
+
+ private UnorderedListBuilder getUnorderedListBuilder() {
+ if (unorderedListBuilder != null)
+ return unorderedListBuilder;
+ else{
+ unorderedListBuilder = new UnorderedListBuilder();
+ return unorderedListBuilder;
+ }
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/IAsterixHDFSRecordParser.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/IAsterixHDFSRecordParser.java
new file mode 100644
index 0000000..70f8c41
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/IAsterixHDFSRecordParser.java
@@ -0,0 +1,51 @@
+/*
+ * 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.external.indexing.dataflow;
+
+import java.io.DataOutput;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+
+import edu.uci.ics.asterix.om.types.ARecordType;
+
+/**
+ * This interface is provided for users to implements in order to support their own
+ * it should be included sometimes in the future in the external library
+ * input parsing
+ * @author alamouda
+ *
+ */
+public interface IAsterixHDFSRecordParser {
+
+ /**
+ * This method is called once upon creating the serde before starting to parse objects
+ * @param record
+ * The description of the expected dataset record.
+ * @param arguments
+ * The arguments passed when creating the external dataset
+ */
+ public void initialize(ARecordType record, Map<String, String> arguments, Configuration hadoopConfig) throws Exception;
+
+ /**
+ * This function takes an object, parse it and then serialize it into an adm record in the output buffer
+ * @param object
+ * the serialized I/O object
+ * @param output
+ * output buffer where deserialized object need to be serialized
+ */
+ public void parse(Object object, DataOutput output) throws Exception;
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/IControlledTupleParser.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/IControlledTupleParser.java
new file mode 100644
index 0000000..92dde7a
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/IControlledTupleParser.java
@@ -0,0 +1,40 @@
+/*
+ * 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.external.indexing.dataflow;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * This interface is to be implemented by parsers used in a pipelined hyracks job where input is not ready all at once
+ */
+public interface IControlledTupleParser {
+ /**
+ * This function should flush the tuples setting in the frame writer buffer
+ * and free all resources
+ */
+ public void close(IFrameWriter writer) throws Exception;
+
+ /**
+ * This function is called when there are more data ready for parsing in the input stream
+ * @param writer
+ * a frame writer that is used to push outgoig frames
+ * @param frameBuffer
+ * a frame buffer containing the incoming tuples, used for propagating fields.
+ */
+ public void parseNext(IFrameWriter writer, ByteBuffer frameBuffer) throws HyracksDataException;
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/IControlledTupleParserFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/IControlledTupleParserFactory.java
new file mode 100644
index 0000000..2aca761
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/IControlledTupleParserFactory.java
@@ -0,0 +1,19 @@
+/*
+ * 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.external.indexing.dataflow;
+
+public interface IControlledTupleParserFactory {
+ public IControlledTupleParser createTupleParser();
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/IndexingScheduler.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/IndexingScheduler.java
new file mode 100644
index 0000000..b207eb9
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/IndexingScheduler.java
@@ -0,0 +1,347 @@
+/*
+ * 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.external.indexing.dataflow;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.Random;
+import java.util.logging.Logger;
+
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.mapred.InputSplit;
+
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.hdfs.scheduler.Scheduler;
+
+@SuppressWarnings("deprecation")
+public class IndexingScheduler {
+ private static final Logger LOGGER = Logger.getLogger(Scheduler.class.getName());
+
+ /** a list of NCs */
+ private String[] NCs;
+
+ /** a map from ip to NCs */
+ private Map<String, List<String>> ipToNcMapping = new HashMap<String, List<String>>();
+
+ /** a map from the NC name to the index */
+ private Map<String, Integer> ncNameToIndex = new HashMap<String, Integer>();
+
+ /** a map from NC name to the NodeControllerInfo */
+ private Map<String, NodeControllerInfo> ncNameToNcInfos;
+
+ /**
+ * The constructor of the scheduler.
+ *
+ * @param ncNameToNcInfos
+ * @throws HyracksException
+ */
+ public IndexingScheduler(String ipAddress, int port) throws HyracksException {
+ try {
+ IHyracksClientConnection hcc = new HyracksConnection(ipAddress, port);
+ this.ncNameToNcInfos = hcc.getNodeControllerInfos();
+ loadIPAddressToNCMap(ncNameToNcInfos);
+ } catch (Exception e) {
+ throw new HyracksException(e);
+ }
+ }
+
+ /**
+ * Set location constraints for a file scan operator with a list of file
+ * splits. It tries to assign splits to their local machines fairly
+ * Locality is more important than fairness
+ *
+ * @throws HyracksDataException
+ */
+ public String[] getLocationConstraints(InputSplit[] splits) throws HyracksException {
+ if (splits == null) {
+ /** deal the case when the splits array is null */
+ return new String[] {};
+ }
+ int[] workloads = new int[NCs.length];
+ Arrays.fill(workloads, 0);
+ String[] locations = new String[splits.length];
+ Map<String, IntWritable> locationToNumOfSplits = new HashMap<String, IntWritable>();
+ /**
+ * upper bound is number of splits
+ */
+ int upperBoundSlots = splits.length;
+
+ try {
+ Random random = new Random(System.currentTimeMillis());
+ boolean scheduled[] = new boolean[splits.length];
+ Arrays.fill(scheduled, false);
+ /**
+ * scan the splits and build the popularity map
+ * give the machines with less local splits more scheduling priority
+ */
+ buildPopularityMap(splits, locationToNumOfSplits);
+ HashMap<String, Integer> locationToNumOfAssignement = new HashMap<String, Integer>();
+ for (String location : locationToNumOfSplits.keySet()) {
+ locationToNumOfAssignement.put(location, 0);
+ }
+ /**
+ * push data-local upper-bounds slots to each machine
+ */
+ scheduleLocalSlots(splits, workloads, locations, upperBoundSlots, random, scheduled, locationToNumOfSplits,
+ locationToNumOfAssignement);
+
+ int dataLocalCount = 0;
+ for (int i = 0; i < scheduled.length; i++) {
+ if (scheduled[i] == true) {
+ dataLocalCount++;
+ }
+ }
+ LOGGER.info("Data local rate: "
+ + (scheduled.length == 0 ? 0.0 : ((float) dataLocalCount / (float) (scheduled.length))));
+ /**
+ * push non-data-local upper-bounds slots to each machine
+ */
+ locationToNumOfAssignement.clear();
+ for(String nc: NCs){
+ locationToNumOfAssignement.put(nc, 0);
+ }
+ for(int i=0; i< scheduled.length;i++){
+ if(scheduled[i])
+ {
+ locationToNumOfAssignement.put(locations[i], locationToNumOfAssignement.get(locations[i])+1);
+ }
+ }
+
+ scheduleNonLocalSlots(splits, workloads, locations, upperBoundSlots, scheduled,locationToNumOfAssignement);
+ return locations;
+ } catch (IOException e) {
+ throw new HyracksException(e);
+ }
+ }
+
+ /**
+ * Schedule non-local slots to each machine
+ *
+ * @param splits
+ * The HDFS file splits.
+ * @param workloads
+ * The current capacity of each machine.
+ * @param locations
+ * The result schedule.
+ * @param slotLimit
+ * The maximum slots of each machine.
+ * @param scheduled
+ * Indicate which slot is scheduled.
+ * @param locationToNumOfAssignement
+ */
+ private void scheduleNonLocalSlots(InputSplit[] splits, final int[] workloads, String[] locations, int slotLimit,
+ boolean[] scheduled, final HashMap<String,Integer> locationToNumOfAssignement) throws IOException, UnknownHostException {
+
+ PriorityQueue<String> scheduleCadndiates = new PriorityQueue<String>(NCs.length, new Comparator<String>() {
+ @Override
+ public int compare(String s1, String s2) {
+ return locationToNumOfAssignement.get(s1).compareTo(locationToNumOfAssignement.get(s2));
+ }
+
+ });
+
+
+ for(String nc:NCs){
+ scheduleCadndiates.add(nc);
+ }
+ /**
+ * schedule no-local file reads
+ */
+ for (int i = 0; i < splits.length; i++) {
+ /** if there is no data-local NC choice, choose a random one */
+ if (!scheduled[i]) {
+ String selectedNcName = scheduleCadndiates.remove();
+ if (selectedNcName != null) {
+ int ncIndex = ncNameToIndex.get(selectedNcName);
+ workloads[ncIndex]++;
+ scheduled[i] = true;
+ locations[i] = selectedNcName;
+ locationToNumOfAssignement.put(selectedNcName, workloads[ncIndex]);
+ scheduleCadndiates.add(selectedNcName);
+ }
+ }
+ }
+ }
+
+ /**
+ * Schedule data-local slots to each machine.
+ *
+ * @param splits
+ * The HDFS file splits.
+ * @param workloads
+ * The current capacity of each machine.
+ * @param locations
+ * The result schedule.
+ * @param slots
+ * The maximum slots of each machine.
+ * @param random
+ * The random generator.
+ * @param scheduled
+ * Indicate which slot is scheduled.
+ * @throws IOException
+ * @throws UnknownHostException
+ */
+ private void scheduleLocalSlots(InputSplit[] splits, int[] workloads, String[] locations, int slots, Random random,
+ boolean[] scheduled, final Map<String, IntWritable> locationToNumSplits,
+ final HashMap<String, Integer> locationToNumOfAssignement) throws IOException, UnknownHostException {
+ /** scheduling candidates will be ordered inversely according to their popularity */
+ PriorityQueue<String> scheduleCadndiates = new PriorityQueue<String>(3, new Comparator<String>() {
+ @Override
+ public int compare(String s1, String s2) {
+ int assignmentDifference = locationToNumOfAssignement.get(s1).compareTo(
+ locationToNumOfAssignement.get(s2));
+ if (assignmentDifference != 0) {
+ return assignmentDifference;
+ }
+ return locationToNumSplits.get(s1).compareTo(locationToNumSplits.get(s2));
+ }
+
+ });
+
+ for (int i = 0; i < splits.length; i++) {
+ if (scheduled[i]) {
+ continue;
+ }
+ /**
+ * get the location of all the splits
+ */
+ String[] locs = splits[i].getLocations();
+ if (locs.length > 0) {
+ scheduleCadndiates.clear();
+ for (int j = 0; j < locs.length; j++) {
+ scheduleCadndiates.add(locs[j]);
+ }
+
+ for (String candidate : scheduleCadndiates) {
+ /**
+ * get all the IP addresses from the name
+ */
+ InetAddress[] allIps = InetAddress.getAllByName(candidate);
+ /**
+ * iterate overa all ips
+ */
+ for (InetAddress ip : allIps) {
+ /**
+ * if the node controller exists
+ */
+ if (ipToNcMapping.get(ip.getHostAddress()) != null) {
+ /**
+ * set the ncs
+ */
+ List<String> dataLocations = ipToNcMapping.get(ip.getHostAddress());
+ int arrayPos = random.nextInt(dataLocations.size());
+ String nc = dataLocations.get(arrayPos);
+ int pos = ncNameToIndex.get(nc);
+ /**
+ * check if the node is already full
+ */
+ if (workloads[pos] < slots) {
+ locations[i] = nc;
+ workloads[pos]++;
+ scheduled[i] = true;
+ locationToNumOfAssignement
+ .put(candidate, locationToNumOfAssignement.get(candidate) + 1);
+ break;
+ }
+ }
+ }
+ /**
+ * break the loop for data-locations if the schedule has
+ * already been found
+ */
+ if (scheduled[i] == true) {
+ break;
+ }
+ }
+ }
+ }
+ }
+
+ /**
+ * Scan the splits once and build a popularity map
+ *
+ * @param splits
+ * the split array
+ * @param locationToNumOfSplits
+ * the map to be built
+ * @throws IOException
+ */
+ private void buildPopularityMap(InputSplit[] splits, Map<String, IntWritable> locationToNumOfSplits)
+ throws IOException {
+ for (InputSplit split : splits) {
+ String[] locations = split.getLocations();
+ for (String loc : locations) {
+ IntWritable locCount = locationToNumOfSplits.get(loc);
+ if (locCount == null) {
+ locCount = new IntWritable(0);
+ locationToNumOfSplits.put(loc, locCount);
+ }
+ locCount.set(locCount.get() + 1);
+ }
+ }
+ }
+
+ /**
+ * Load the IP-address-to-NC map from the NCNameToNCInfoMap
+ *
+ * @param ncNameToNcInfos
+ * @throws HyracksException
+ */
+ private void loadIPAddressToNCMap(Map<String, NodeControllerInfo> ncNameToNcInfos) throws HyracksException {
+ try {
+ NCs = new String[ncNameToNcInfos.size()];
+ ipToNcMapping.clear();
+ ncNameToIndex.clear();
+ int i = 0;
+
+ /**
+ * build the IP address to NC map
+ */
+ for (Map.Entry<String, NodeControllerInfo> entry : ncNameToNcInfos.entrySet()) {
+ String ipAddr = InetAddress.getByAddress(entry.getValue().getNetworkAddress().getIpAddress())
+ .getHostAddress();
+ List<String> matchedNCs = ipToNcMapping.get(ipAddr);
+ if (matchedNCs == null) {
+ matchedNCs = new ArrayList<String>();
+ ipToNcMapping.put(ipAddr, matchedNCs);
+ }
+ matchedNCs.add(entry.getKey());
+ NCs[i] = entry.getKey();
+ i++;
+ }
+
+ /**
+ * set up the NC name to index mapping
+ */
+ for (i = 0; i < NCs.length; i++) {
+ ncNameToIndex.put(NCs[i], i);
+ }
+ } catch (Exception e) {
+ throw new HyracksException(e);
+ }
+ }
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/RCFileControlledTupleParser.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/RCFileControlledTupleParser.java
new file mode 100644
index 0000000..d845ce3
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/RCFileControlledTupleParser.java
@@ -0,0 +1,204 @@
+/*
+ * 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.external.indexing.dataflow;
+
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.asterix.external.indexing.input.RCFileLookupReader;
+import edu.uci.ics.asterix.metadata.external.IndexingConstants;
+import edu.uci.ics.asterix.om.base.AInt32;
+import edu.uci.ics.asterix.om.base.AInt64;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+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;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+
+public class RCFileControlledTupleParser implements IControlledTupleParser {
+
+ private ArrayTupleBuilder tb;
+ private transient DataOutput dos;
+ private final FrameTupleAppender appender;
+ private final ByteBuffer frame;
+ private boolean propagateInput;
+ private int[] propagatedFields;
+ private FrameTupleReference frameTuple;
+ private IAsterixHDFSRecordParser parser;
+ private RCFileLookupReader reader;
+ private int[] ridFields;
+ private RecordDescriptor inRecDesc;
+ private FrameTupleAccessor tupleAccessor;
+ private ByteBufferInputStream bbis;
+ private DataInputStream dis;
+ private boolean retainNull;
+ protected byte nullByte;
+ protected ArrayTupleBuilder nullTupleBuild;
+
+ public RCFileControlledTupleParser(IHyracksTaskContext ctx, IAsterixHDFSRecordParser parser,
+ RCFileLookupReader reader, boolean propagateInput, int[] propagatedFields, RecordDescriptor inRecDesc,
+ int[] ridFields, boolean retainNull, INullWriterFactory iNullWriterFactory) throws HyracksDataException {
+ appender = new FrameTupleAppender(ctx.getFrameSize());
+ frame = ctx.allocateFrame();
+ this.parser = parser;
+ this.reader = reader;
+ this.propagateInput = propagateInput;
+ this.propagatedFields = propagatedFields;
+ this.retainNull = retainNull;
+ this.inRecDesc = inRecDesc;
+ this.ridFields = ridFields;
+ this.tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(), inRecDesc);
+ if (propagateInput) {
+ tb = new ArrayTupleBuilder(propagatedFields.length + 1);
+ } else {
+ tb = new ArrayTupleBuilder(1);
+ }
+ frameTuple = new FrameTupleReference();
+ dos = tb.getDataOutput();
+ appender.reset(frame, true);
+ bbis = new ByteBufferInputStream();
+ dis = new DataInputStream(bbis);
+ appender.reset(frame, true);
+ nullByte = ATypeTag.NULL.serialize();
+ if (retainNull) {
+ INullWriter nullWriter = iNullWriterFactory.createNullWriter();
+ nullTupleBuild = new ArrayTupleBuilder(1);
+ DataOutput out = nullTupleBuild.getDataOutput();
+ try {
+ nullWriter.writeNull(out);
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ } else {
+ nullTupleBuild = null;
+ }
+ }
+
+ @Override
+ public void close(IFrameWriter writer) throws Exception {
+ try {
+ reader.close();
+ if (appender.getTupleCount() > 0) {
+ FrameUtils.flushFrame(frame, writer);
+ }
+ } catch (IOException ioe) {
+ throw new HyracksDataException(ioe);
+ }
+ }
+
+ @Override
+ public void parseNext(IFrameWriter writer, ByteBuffer frameBuffer) throws HyracksDataException {
+ try {
+ int tupleCount = 0;
+ int tupleIndex = 0;
+ Object object;
+ tupleAccessor.reset(frameBuffer);
+ tupleCount = tupleAccessor.getTupleCount();
+ int fieldSlotsLength = tupleAccessor.getFieldSlotsLength();
+ // Loop over tuples
+ while (tupleIndex < tupleCount) {
+ int tupleStartOffset = tupleAccessor.getTupleStartOffset(tupleIndex) + fieldSlotsLength;
+ int fileNumberStartOffset = tupleAccessor.getFieldStartOffset(tupleIndex,
+ ridFields[IndexingConstants.FILE_NUMBER_FIELD_INDEX]);
+ // Check if null <- for outer join ->
+ if (frameBuffer.get(tupleStartOffset + fileNumberStartOffset) == nullByte) {
+ object = null;
+ } else {
+ // Get file number
+ bbis.setByteBuffer(frameBuffer, tupleStartOffset + fileNumberStartOffset);
+ int fileNumber = ((AInt32) inRecDesc.getFields()[ridFields[IndexingConstants.FILE_NUMBER_FIELD_INDEX]]
+ .deserialize(dis)).getIntegerValue();
+ // Get record group offset
+ bbis.setByteBuffer(
+ frameBuffer,
+ tupleStartOffset
+ + tupleAccessor.getFieldStartOffset(tupleIndex,
+ ridFields[IndexingConstants.RECORD_OFFSET_FIELD_INDEX]));
+ long recordOffset = ((AInt64) inRecDesc.getFields()[ridFields[IndexingConstants.RECORD_OFFSET_FIELD_INDEX]]
+ .deserialize(dis)).getLongValue();
+ // Get row number
+ bbis.setByteBuffer(
+ frameBuffer,
+ tupleStartOffset
+ + tupleAccessor.getFieldStartOffset(tupleIndex,
+ ridFields[IndexingConstants.ROW_NUMBER_FIELD_INDEX]));
+ int rowNumber = ((AInt32) inRecDesc.getFields()[ridFields[IndexingConstants.ROW_NUMBER_FIELD_INDEX]]
+ .deserialize(dis)).getIntegerValue();
+
+ // Read record from external source
+ object = reader.read(fileNumber, recordOffset, rowNumber);
+ }
+ if (object != null) {
+ tb.reset();
+ if (propagateInput) {
+ frameTuple.reset(tupleAccessor, tupleIndex);
+ for (int i = 0; i < propagatedFields.length; i++) {
+ dos.write(frameTuple.getFieldData(propagatedFields[i]),
+ frameTuple.getFieldStart(propagatedFields[i]),
+ frameTuple.getFieldLength(propagatedFields[i]));
+ tb.addFieldEndOffset();
+ }
+ }
+ // parse record
+ parser.parse(object, tb.getDataOutput());
+ tb.addFieldEndOffset();
+ addTupleToFrame(writer);
+ } else if (propagateInput && retainNull) {
+ tb.reset();
+ frameTuple.reset(tupleAccessor, tupleIndex);
+ for (int i = 0; i < propagatedFields.length; i++) {
+ dos.write(frameTuple.getFieldData(propagatedFields[i]),
+ frameTuple.getFieldStart(propagatedFields[i]),
+ frameTuple.getFieldLength(propagatedFields[i]));
+ tb.addFieldEndOffset();
+ }
+ dos.write(nullTupleBuild.getByteArray());
+ tb.addFieldEndOffset();
+ addTupleToFrame(writer);
+ }
+ tupleIndex++;
+ }
+ } catch (Exception e) {
+ // Something went wrong, try to close the reader and then throw an exception <-this should never happen->
+ try {
+ reader.close();
+ } catch (Exception e1) {
+ e.addSuppressed(e1);
+ }
+ 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);
+ appender.reset(frame, true);
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ throw new IllegalStateException();
+ }
+ }
+ }
+
+}
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
new file mode 100644
index 0000000..92e07f4
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/RCFileIndexingTupleParser.java
@@ -0,0 +1,57 @@
+/*
+ * 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.external.indexing.dataflow;
+
+import edu.uci.ics.asterix.external.indexing.input.AbstractHDFSReader;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+
+public class RCFileIndexingTupleParser extends AbstractIndexingTupleParser{
+
+ 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);
+ tb = new ArrayTupleBuilder(4);
+ dos = tb.getDataOutput();
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ protected void appendIndexingData(ArrayTupleBuilder tb,
+ AbstractHDFSReader inReader) throws Exception {
+ aMutableInt.setValue(inReader.getFileNumber());
+ aMutableLong.setValue(inReader.getReaderPosition());
+ //add file number
+ tb.addField(intSerde, aMutableInt);
+ //add record offset
+ tb.addField(longSerde, aMutableLong);
+ //add row number
+ if(aMutableInt.getIntegerValue().equals(lastFileNumber) && aMutableLong.getLongValue() == lastByteLocation){
+ rowNumber++;
+ }else{
+ lastFileNumber = aMutableInt.getIntegerValue();
+ lastByteLocation = aMutableLong.getLongValue();
+ rowNumber = 0;
+ }
+ aMutableInt.setValue(rowNumber);
+ tb.addField(intSerde, aMutableInt);
+ }
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/SeqOrTxtControlledTupleParser.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/SeqOrTxtControlledTupleParser.java
new file mode 100644
index 0000000..1359d2d
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/SeqOrTxtControlledTupleParser.java
@@ -0,0 +1,193 @@
+/*
+ * 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.external.indexing.dataflow;
+
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.asterix.external.indexing.input.ILookupReader;
+import edu.uci.ics.asterix.metadata.external.IndexingConstants;
+import edu.uci.ics.asterix.om.base.AInt32;
+import edu.uci.ics.asterix.om.base.AInt64;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+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;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+
+public class SeqOrTxtControlledTupleParser implements IControlledTupleParser {
+
+ private ArrayTupleBuilder tb;
+ private transient DataOutput dos;
+ private final FrameTupleAppender appender;
+ private final ByteBuffer frame;
+ private boolean propagateInput;
+ private int[] propagatedFields;
+ private FrameTupleReference frameTuple;
+ private IAsterixHDFSRecordParser parser;
+ private ILookupReader reader;
+ private int[] ridFields;
+ private RecordDescriptor inRecDesc;
+ private FrameTupleAccessor tupleAccessor;
+ private ByteBufferInputStream bbis;
+ private DataInputStream dis;
+ private boolean retainNull;
+ protected byte nullByte;
+ protected ArrayTupleBuilder nullTupleBuild;
+
+ public SeqOrTxtControlledTupleParser(IHyracksTaskContext ctx, IAsterixHDFSRecordParser parser,
+ ILookupReader reader, boolean propagateInput, int[] propagatedFields, RecordDescriptor inRecDesc,
+ int[] ridFields, boolean retainNull, INullWriterFactory iNullWriterFactory) throws HyracksDataException {
+ appender = new FrameTupleAppender(ctx.getFrameSize());
+ frame = ctx.allocateFrame();
+ this.parser = parser;
+ this.reader = reader;
+ this.propagateInput = propagateInput;
+ this.ridFields = ridFields;
+ this.retainNull = retainNull;
+ if (propagateInput) {
+ tb = new ArrayTupleBuilder(propagatedFields.length + 1);
+ frameTuple = new FrameTupleReference();
+ this.propagatedFields = propagatedFields;
+ } else {
+ tb = new ArrayTupleBuilder(1);
+ }
+ dos = tb.getDataOutput();
+ appender.reset(frame, true);
+ this.tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(), inRecDesc);
+ bbis = new ByteBufferInputStream();
+ dis = new DataInputStream(bbis);
+ nullByte = ATypeTag.NULL.serialize();
+ if (retainNull) {
+ INullWriter nullWriter = iNullWriterFactory.createNullWriter();
+ nullTupleBuild = new ArrayTupleBuilder(1);
+ DataOutput out = nullTupleBuild.getDataOutput();
+ try {
+ nullWriter.writeNull(out);
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ } else {
+ nullTupleBuild = null;
+ }
+ }
+
+ @Override
+ public void close(IFrameWriter writer) throws Exception {
+ try {
+ reader.close();
+ if (appender.getTupleCount() > 0) {
+ FrameUtils.flushFrame(frame, writer);
+ }
+ } catch (IOException ioe) {
+ throw new HyracksDataException(ioe);
+ }
+ }
+
+ @Override
+ public void parseNext(IFrameWriter writer, ByteBuffer frameBuffer) throws HyracksDataException {
+ try {
+ int tupleCount = 0;
+ int tupleIndex = 0;
+ Object record;
+ tupleAccessor.reset(frameBuffer);
+ tupleCount = tupleAccessor.getTupleCount();
+ int fieldSlotsLength = tupleAccessor.getFieldSlotsLength();
+ // Loop over incoming tuples
+ while (tupleIndex < tupleCount) {
+ int tupleStartOffset = tupleAccessor.getTupleStartOffset(tupleIndex) + fieldSlotsLength;
+ int fileNumberStartOffset = tupleAccessor.getFieldStartOffset(tupleIndex,
+ ridFields[IndexingConstants.FILE_NUMBER_FIELD_INDEX]);
+ // Check if null <- for outer join ->
+ if (frameBuffer.get(tupleStartOffset + fileNumberStartOffset) == nullByte) {
+ record = null;
+ } else {
+ // Get file number
+ bbis.setByteBuffer(frameBuffer, tupleStartOffset + fileNumberStartOffset);
+ int fileNumber = ((AInt32) inRecDesc.getFields()[ridFields[IndexingConstants.FILE_NUMBER_FIELD_INDEX]]
+ .deserialize(dis)).getIntegerValue();
+ // Get record offset
+ bbis.setByteBuffer(
+ frameBuffer,
+ tupleStartOffset
+ + tupleAccessor.getFieldStartOffset(tupleIndex,
+ ridFields[IndexingConstants.RECORD_OFFSET_FIELD_INDEX]));
+ long recordOffset = ((AInt64) inRecDesc.getFields()[ridFields[IndexingConstants.RECORD_OFFSET_FIELD_INDEX]]
+ .deserialize(dis)).getLongValue();
+ // Read the record
+ record = reader.read(fileNumber, recordOffset);
+ }
+ if (record != null) {
+ tb.reset();
+ if (propagateInput) {
+ frameTuple.reset(tupleAccessor, tupleIndex);
+ for (int i = 0; i < propagatedFields.length; i++) {
+ dos.write(frameTuple.getFieldData(propagatedFields[i]),
+ frameTuple.getFieldStart(propagatedFields[i]),
+ frameTuple.getFieldLength(propagatedFields[i]));
+ tb.addFieldEndOffset();
+ }
+ }
+ // parse it
+ parser.parse(record, tb.getDataOutput());
+ tb.addFieldEndOffset();
+ addTupleToFrame(writer);
+ } else if (propagateInput && retainNull) {
+ tb.reset();
+ frameTuple.reset(tupleAccessor, tupleIndex);
+ for (int i = 0; i < propagatedFields.length; i++) {
+ dos.write(frameTuple.getFieldData(propagatedFields[i]),
+ frameTuple.getFieldStart(propagatedFields[i]),
+ frameTuple.getFieldLength(propagatedFields[i]));
+ tb.addFieldEndOffset();
+ }
+ dos.write(nullTupleBuild.getByteArray());
+ tb.addFieldEndOffset();
+ addTupleToFrame(writer);
+ }
+ tupleIndex++;
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ try {
+ reader.close();
+ } catch (Exception e2) {
+ e.addSuppressed(e2);
+ }
+ throw new HyracksDataException(e);
+ }
+ }
+
+ private void addTupleToFrame(IFrameWriter writer) throws HyracksDataException {
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ FrameUtils.flushFrame(frame, writer);
+ appender.reset(frame, true);
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ throw new IllegalStateException();
+ }
+ }
+ }
+
+}
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
new file mode 100644
index 0000000..3127a68
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/TextOrSeqIndexingTupleParser.java
@@ -0,0 +1,42 @@
+/*
+ * 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.external.indexing.dataflow;
+
+import edu.uci.ics.asterix.external.indexing.input.AbstractHDFSReader;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+
+public class TextOrSeqIndexingTupleParser extends AbstractIndexingTupleParser{
+ public TextOrSeqIndexingTupleParser(IHyracksTaskContext ctx,
+ ARecordType recType, IAsterixHDFSRecordParser deserializer)
+ throws HyracksDataException {
+ super(ctx, recType, deserializer);
+ tb = new ArrayTupleBuilder(3);
+ dos = tb.getDataOutput();
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ protected void appendIndexingData(ArrayTupleBuilder tb,
+ 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/main/java/edu/uci/ics/asterix/external/indexing/input/AbstractHDFSLookupInputStream.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/AbstractHDFSLookupInputStream.java
new file mode 100644
index 0000000..5bbf5be
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/AbstractHDFSLookupInputStream.java
@@ -0,0 +1,114 @@
+/*
+ * 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.external.indexing.input;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+
+import edu.uci.ics.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
+import edu.uci.ics.asterix.metadata.entities.ExternalFile;
+import edu.uci.ics.asterix.metadata.external.ExternalFileIndexAccessor;
+
+/*
+ * This class is used for seek and read of external data of format adm or delimited text in sequence of text input format
+ */
+@SuppressWarnings("deprecation")
+public abstract class AbstractHDFSLookupInputStream extends InputStream {
+
+ protected String pendingValue = null;
+ protected FileSystem fs;
+ protected int fileNumber = -1;
+ protected int EOL = "\n".getBytes()[0];
+ protected boolean skipFile = false;
+ protected ExternalFile file = new ExternalFile(null, null, 0, null, null, 0, ExternalFilePendingOp.PENDING_NO_OP);
+ protected ExternalFileIndexAccessor filesIndexAccessor;
+
+ public AbstractHDFSLookupInputStream(ExternalFileIndexAccessor filesIndexAccessor, JobConf conf) throws IOException {
+ this.filesIndexAccessor = filesIndexAccessor;
+ fs = FileSystem.get(conf);
+ }
+
+ @Override
+ public int read(byte[] buffer, int offset, int len) throws IOException {
+ if (pendingValue != null) {
+ int size = pendingValue.length() + 1;
+ if (size > len) {
+ return 0;
+ }
+ System.arraycopy(pendingValue.getBytes(), 0, buffer, offset, pendingValue.length());
+ buffer[offset + pendingValue.length()] = (byte) EOL;
+ pendingValue = null;
+ return size;
+ }
+ return -1;
+ }
+
+ public boolean fetchRecord(int fileNumber, long recordOffset) throws Exception {
+ if (fileNumber != this.fileNumber) {
+ // New file number
+ this.fileNumber = fileNumber;
+ filesIndexAccessor.searchForFile(fileNumber, file);
+
+ try {
+ FileStatus fileStatus = fs.getFileStatus(new Path(file.getFileName()));
+ if (fileStatus.getModificationTime() != file.getLastModefiedTime().getTime()) {
+ this.fileNumber = fileNumber;
+ skipFile = true;
+ return false;
+ } else {
+ this.fileNumber = fileNumber;
+ skipFile = false;
+ openFile(file.getFileName());
+ }
+ } catch (FileNotFoundException e) {
+ // We ignore File not found exceptions <- it means file was deleted and so we don't care about it anymore ->
+ this.fileNumber = fileNumber;
+ skipFile = true;
+ return false;
+ }
+ } else if (skipFile) {
+ return false;
+ }
+ return read(recordOffset);
+ }
+
+ @Override
+ public int read() throws IOException {
+ return 0;
+ }
+
+ protected abstract boolean read(long byteLocation);
+
+ protected abstract void openFile(String fileName) throws IOException;
+
+ @Override
+ public void close() throws IOException {
+ super.close();
+ }
+
+ public ExternalFileIndexAccessor getExternalFileIndexAccessor() {
+ return filesIndexAccessor;
+ }
+
+ public void setExternalFileIndexAccessor(ExternalFileIndexAccessor filesIndexAccessor) {
+ this.filesIndexAccessor = filesIndexAccessor;
+ }
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/AbstractHDFSReader.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/AbstractHDFSReader.java
new file mode 100644
index 0000000..0d340a3
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/AbstractHDFSReader.java
@@ -0,0 +1,98 @@
+/*
+ * 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.external.indexing.input;
+
+import java.io.InputStream;
+
+import org.apache.hadoop.mapred.Counters.Counter;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.Reporter;
+
+/***
+ * an abstract class to be used for reading hdfs based datasets one record at a time <- used for indexing->
+ */
+@SuppressWarnings("deprecation")
+public abstract class AbstractHDFSReader extends InputStream{
+
+ /***
+ * This function should be called once to do initial setup before starting to read records
+ * @return true if ready for reading
+ */
+ abstract public boolean initialize() throws Exception;
+
+ /***
+ * @return the next object read or null if reached end of stream
+ */
+ abstract public Object readNext() throws Exception;
+
+ /**
+ * @return the file name of the current filesplit being read
+ * @throws Exception in case of end of records is reached
+ */
+ abstract public String getFileName() throws Exception;
+
+ /**
+ * @return return the reader position of last record read
+ * @throws Exception in case of end of records is reached
+ */
+ abstract public long getReaderPosition() throws Exception;
+
+ /**
+ *
+ * @return the file number of the file being read
+ * @throws Exception
+ */
+ abstract public int getFileNumber() throws Exception;
+
+ protected Reporter getReporter() {
+ Reporter reporter = new Reporter() {
+
+ @Override
+ public Counter getCounter(Enum<?> arg0) {
+ return null;
+ }
+
+ @Override
+ public Counter getCounter(String arg0, String arg1) {
+ return null;
+ }
+
+ @Override
+ public InputSplit getInputSplit()
+ throws UnsupportedOperationException {
+ return null;
+ }
+
+ @Override
+ public void incrCounter(Enum<?> arg0, long arg1) {
+ }
+
+ @Override
+ public void incrCounter(String arg0, String arg1, long arg2) {
+ }
+
+ @Override
+ public void setStatus(String arg0) {
+ }
+
+ @Override
+ public void progress() {
+ }
+ };
+
+ return reporter;
+ }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/GenericFileAwareRecordReader.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/GenericFileAwareRecordReader.java
new file mode 100644
index 0000000..f6a877b
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/GenericFileAwareRecordReader.java
@@ -0,0 +1,125 @@
+/*
+ * 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.external.indexing.input;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+
+import edu.uci.ics.asterix.metadata.entities.ExternalFile;
+/**
+ * This is a generic reader used for indexing external dataset or for performing full scan for external dataset with
+ * a stored snapshot
+ * @author alamouda
+ *
+ */
+
+@SuppressWarnings("deprecation")
+public class GenericFileAwareRecordReader extends GenericRecordReader{
+
+ private List<ExternalFile> files;
+ private FileSystem hadoopFS;
+ private long recordOffset = 0L;
+
+ public GenericFileAwareRecordReader(InputSplit[] inputSplits, String[] readSchedule, String nodeName, JobConf conf,
+ boolean[] executed, List<ExternalFile> files) throws IOException {
+ super(inputSplits, readSchedule, nodeName, conf, executed);
+ this.files = files;
+ hadoopFS = FileSystem.get(conf);
+ }
+
+ private boolean moveToNext() throws IOException {
+ for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
+ /**
+ * read all the partitions scheduled to the current node
+ */
+ if (readSchedule[currentSplitIndex].equals(nodeName)) {
+ /**
+ * pick an unread split to read synchronize among
+ * simultaneous partitions in the same machine
+ */
+ synchronized (executed) {
+ if (executed[currentSplitIndex] == false) {
+ executed[currentSplitIndex] = true;
+ } else {
+ continue;
+ }
+ }
+
+ /**
+ * read the split
+ */
+ try{
+ String fileName = ((FileSplit) (inputSplits[currentSplitIndex])).getPath()
+ .toUri().getPath();
+ FileStatus fileStatus = hadoopFS.getFileStatus(new Path(fileName));
+ //skip if not the same file stored in the files snapshot
+ if(fileStatus.getModificationTime() != files.get(currentSplitIndex).getLastModefiedTime().getTime())
+ continue;
+ reader = getRecordReader(currentSplitIndex);
+ }
+ catch(Exception e){
+ continue;
+ }
+ key = reader.createKey();
+ value = reader.createValue();
+ return true;
+ }
+ }
+ return false;
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public Object readNext() throws IOException {
+
+ if (reader == null) {
+ return null;
+ }
+ recordOffset = reader.getPos();
+ if (reader.next(key, value)) {
+ return value;
+ }
+ while (moveToNext()) {
+ recordOffset = reader.getPos();
+ if (reader.next(key, value)) {
+ return value;
+ }
+ }
+ return null;
+ }
+
+ @Override
+ public String getFileName() throws Exception {
+ return files.get(currentSplitIndex).getFileName();
+ }
+
+ @Override
+ public long getReaderPosition() throws Exception {
+ return recordOffset;
+ }
+
+ @Override
+ public int getFileNumber() throws Exception {
+ return files.get(currentSplitIndex).getFileNumber();
+ }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/GenericRecordReader.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/GenericRecordReader.java
new file mode 100644
index 0000000..f6938e7
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/GenericRecordReader.java
@@ -0,0 +1,131 @@
+/*
+ * 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.external.indexing.input;
+
+import java.io.IOException;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+
+/**
+ * This class can be used by any input format to perform full scan operations
+ */
+
+@SuppressWarnings({ "rawtypes", "unchecked", "deprecation" })
+public class GenericRecordReader extends AbstractHDFSReader {
+
+ protected RecordReader reader;
+ protected Object key;
+ protected Object value;
+ protected int currentSplitIndex = 0;
+ protected boolean executed[];
+ protected InputSplit[] inputSplits;
+ protected String[] readSchedule;
+ protected String nodeName;
+ protected JobConf conf;
+
+ public GenericRecordReader(InputSplit[] inputSplits, String[] readSchedule, String nodeName, JobConf conf,
+ boolean executed[]) {
+ this.inputSplits = inputSplits;
+ this.readSchedule = readSchedule;
+ this.nodeName = nodeName;
+ this.conf = conf;
+ this.executed = executed;
+ }
+
+ @Override
+ public boolean initialize() throws IOException {
+ return moveToNext();
+ }
+
+ private boolean moveToNext() throws IOException {
+ for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
+ /**
+ * read all the partitions scheduled to the current node
+ */
+ if (readSchedule[currentSplitIndex].equals(nodeName)) {
+ /**
+ * pick an unread split to read synchronize among
+ * simultaneous partitions in the same machine
+ */
+ synchronized (executed) {
+ if (executed[currentSplitIndex] == false) {
+ executed[currentSplitIndex] = true;
+ } else {
+ continue;
+ }
+ }
+
+ /**
+ * read the split
+ */
+ reader = getRecordReader(currentSplitIndex);
+ key = reader.createKey();
+ value = reader.createValue();
+ return true;
+ }
+ }
+ return false;
+ }
+
+ protected RecordReader getRecordReader(int slitIndex) throws IOException {
+ RecordReader reader = conf.getInputFormat().getRecordReader(
+ (org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
+ return reader;
+ }
+
+ @Override
+ public Object readNext() throws IOException {
+ if (reader == null) {
+ return null;
+ }
+ if (reader.next(key, value)) {
+ return value;
+ }
+ while (moveToNext()) {
+ if (reader.next(key, value)) {
+ return value;
+ }
+ }
+ return null;
+ }
+
+ @Override
+ public String getFileName() throws Exception {
+ return null;
+ }
+
+ @Override
+ public long getReaderPosition() throws Exception {
+ return reader.getPos();
+ }
+
+ @Override
+ public int getFileNumber() throws Exception {
+ throw new NotImplementedException("This reader doesn't support this function");
+ }
+
+ @Override
+ public int read(byte[] buffer, int offset, int len) throws IOException {
+ throw new NotImplementedException("Use readNext()");
+ }
+
+ @Override
+ public int read() throws IOException {
+ throw new NotImplementedException("Use readNext()");
+ }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/ILookupReader.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/ILookupReader.java
new file mode 100644
index 0000000..32b9a02
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/ILookupReader.java
@@ -0,0 +1,20 @@
+/*
+ * 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.external.indexing.input;
+
+public interface ILookupReader {
+ public Object read(int fileNumber, long recordOffset) throws Exception;
+ public void close();
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/RCFileDataReader.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/RCFileDataReader.java
new file mode 100644
index 0000000..a0b9a6e
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/RCFileDataReader.java
@@ -0,0 +1,176 @@
+/*
+ * 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.external.indexing.input;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+
+import edu.uci.ics.asterix.metadata.entities.ExternalFile;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+
+//Used in two cases:
+//1. building an index over a dataset
+//2. performing full scan over a dataset that has built index (to provide consistent view) with RCFile format
+
+@SuppressWarnings({ "rawtypes", "deprecation" })
+public class RCFileDataReader extends AbstractHDFSReader {
+
+ private RecordReader reader;
+ private Object key;
+ private Object value;
+ private int currentSplitIndex = 0;
+ private String fileName;
+ private long recordGroupOffset;
+ private long nextRecordGroupOffset;
+ private boolean executed[];
+ private InputSplit[] inputSplits;
+ private String[] readSchedule;
+ private String nodeName;
+ private JobConf conf;
+ private List<ExternalFile> files;
+ private FileSystem hadoopFS;
+
+ public RCFileDataReader(InputSplit[] inputSplits, String[] readSchedule, String nodeName, JobConf conf,
+ boolean executed[], List<ExternalFile> files) throws IOException {
+ this.executed = executed;
+ this.inputSplits = inputSplits;
+ this.readSchedule = readSchedule;
+ this.nodeName = nodeName;
+ this.conf = conf;
+ this.files = files;
+ hadoopFS = FileSystem.get(conf);
+ }
+
+ private boolean moveToNext() throws IOException {
+ for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
+ /**
+ * read all the partitions scheduled to the current node
+ */
+ if (readSchedule[currentSplitIndex].equals(nodeName)) {
+ /**
+ * pick an unread split to read synchronize among
+ * simultaneous partitions in the same machine
+ */
+ synchronized (executed) {
+ if (executed[currentSplitIndex] == false) {
+ executed[currentSplitIndex] = true;
+ } else {
+ continue;
+ }
+ }
+
+ /**
+ * read the split
+ */
+ try {
+ if (files != null) {
+ fileName = ((FileSplit) (inputSplits[currentSplitIndex])).getPath().toUri().getPath();
+ FileStatus fileStatus = hadoopFS.getFileStatus(new Path(fileName));
+ //skip if not the same file stored in the files snapshot
+ if (fileStatus.getModificationTime() != files.get(currentSplitIndex).getLastModefiedTime()
+ .getTime())
+ continue;
+ }
+ reader = getRecordReader(currentSplitIndex);
+ recordGroupOffset = -1;
+ nextRecordGroupOffset = reader.getPos();
+ } catch (Exception e) {
+ continue;
+ }
+ key = reader.createKey();
+ value = reader.createValue();
+ return true;
+ }
+ }
+ return false;
+ }
+
+ @Override
+ public int read(byte[] buffer, int offset, int len) throws IOException {
+ throw new NotImplementedException("Use readNext()");
+ }
+
+ @Override
+ public int read() throws IOException {
+ throw new NotImplementedException("Use readNext()");
+ }
+
+ private RecordReader getRecordReader(int slitIndex) throws IOException {
+ RecordReader reader;
+ try{
+ reader = conf.getInputFormat().getRecordReader(
+ (org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
+ } catch(Exception e){
+ e.printStackTrace();
+ throw e;
+ }
+ return reader;
+ }
+
+ @Override
+ public boolean initialize() throws IOException {
+ return moveToNext();
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public Object readNext() throws IOException {
+
+ if (reader == null) {
+ return null;
+ }
+ if (reader.next(key, value)) {
+ if (reader.getPos() != nextRecordGroupOffset) {
+ recordGroupOffset = nextRecordGroupOffset;
+ nextRecordGroupOffset = reader.getPos();
+ }
+ return value;
+ }
+ while (moveToNext()) {
+ if (reader.next(key, value)) {
+ if (reader.getPos() != nextRecordGroupOffset) {
+ recordGroupOffset = nextRecordGroupOffset;
+ nextRecordGroupOffset = reader.getPos();
+ }
+ return value;
+ }
+ }
+ return null;
+ }
+
+ @Override
+ public String getFileName() throws Exception {
+ return files.get(currentSplitIndex).getFileName();
+ }
+
+ @Override
+ public long getReaderPosition() throws Exception {
+ return recordGroupOffset;
+ }
+
+ @Override
+ public int getFileNumber() throws Exception {
+ return files.get(currentSplitIndex).getFileNumber();
+ }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/RCFileLookupReader.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/RCFileLookupReader.java
new file mode 100644
index 0000000..88422e2
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/RCFileLookupReader.java
@@ -0,0 +1,103 @@
+/*
+ * 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.external.indexing.input;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.RCFile.Reader;
+import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Writable;
+
+import edu.uci.ics.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
+import edu.uci.ics.asterix.metadata.entities.ExternalFile;
+import edu.uci.ics.asterix.metadata.external.ExternalFileIndexAccessor;
+
+public class RCFileLookupReader {
+ private FileSystem fs;
+ private Configuration conf;
+ private int fileNumber = -1;
+ private int rowNumber;
+ private long recordGroupOffset;
+ private Reader reader;
+ boolean skipFile = false;
+ private LongWritable rcKey = new LongWritable();
+ private BytesRefArrayWritable rcValue = new BytesRefArrayWritable();
+ private ExternalFile currentFile = new ExternalFile(null, null, 0, null, null, 0L,
+ ExternalFilePendingOp.PENDING_NO_OP);
+ private ExternalFileIndexAccessor filesIndexAccessor;
+
+ public RCFileLookupReader(ExternalFileIndexAccessor filesIndexAccessor, Configuration conf) throws IOException {
+ fs = FileSystem.get(conf);
+ this.conf = conf;
+ this.filesIndexAccessor = filesIndexAccessor;
+ }
+
+ public Writable read(int fileNumber, long recordGroupOffset, int rowNumber) throws Exception {
+ if (fileNumber != this.fileNumber) {
+ filesIndexAccessor.searchForFile(fileNumber, currentFile);
+ try {
+ FileStatus fileStatus = fs.getFileStatus(new Path(currentFile.getFileName()));
+ if (fileStatus.getModificationTime() != currentFile.getLastModefiedTime().getTime()) {
+ this.fileNumber = fileNumber;
+ skipFile = true;
+ return null;
+ } else {
+ this.fileNumber = fileNumber;
+ skipFile = false;
+ }
+ } catch (FileNotFoundException e) {
+ // Couldn't find file, skip it
+ this.fileNumber = fileNumber;
+ skipFile = true;
+ return null;
+ }
+ // Close old file and open new one
+ if (reader != null)
+ reader.close();
+ reader = new Reader(fs, new Path(currentFile.getFileName()), conf);
+ this.recordGroupOffset = -1;
+ this.rowNumber = -1;
+ } else if (skipFile) {
+ return null;
+ }
+ // Seek to the record group if needed
+ if (recordGroupOffset != this.recordGroupOffset) {
+ this.recordGroupOffset = recordGroupOffset;
+ if(reader.getPosition() != recordGroupOffset)
+ reader.seek(recordGroupOffset);
+ reader.resetBuffer();
+ this.rowNumber = -1;
+ }
+
+ // skip rows to the record row
+ while (this.rowNumber < rowNumber) {
+ reader.next(rcKey);
+ reader.getCurrentRow(rcValue);
+ this.rowNumber++;
+ }
+ return rcValue;
+ }
+
+ public void close() throws Exception {
+ if (reader != null)
+ reader.close();
+ }
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/SequenceFileLookupInputStream.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/SequenceFileLookupInputStream.java
new file mode 100644
index 0000000..a5ee2e8
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/SequenceFileLookupInputStream.java
@@ -0,0 +1,73 @@
+/*
+ * 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.external.indexing.input;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import edu.uci.ics.asterix.metadata.external.ExternalFileIndexAccessor;
+
+
+@SuppressWarnings("deprecation")
+public class SequenceFileLookupInputStream extends AbstractHDFSLookupInputStream{
+
+ private SequenceFile.Reader reader;
+ private Writable seqKey;
+ private Text seqValue = new Text();
+ private Configuration conf;
+
+ public SequenceFileLookupInputStream(ExternalFileIndexAccessor fileIndexAccessor, JobConf conf) throws IOException{
+ super(fileIndexAccessor, conf);
+ this.conf = conf;
+ }
+
+ @Override
+ protected void openFile(String fileName) throws IOException {
+ if (reader != null) {
+ reader.close();
+ }
+ reader = new SequenceFile.Reader(fs, new Path(fileName), conf);
+ seqKey = (Writable) ReflectionUtils.newInstance(reader.getKeyClass(), conf);
+ }
+
+ @Override
+ public void close() throws IOException {
+ if (reader != null) {
+ reader.close();
+ }
+ super.close();
+ }
+
+ @Override
+ protected boolean read(long recordOffset) {
+ try {
+ reader.seek(recordOffset);
+ reader.next(seqKey, seqValue);
+ pendingValue = seqValue.toString();
+ return true;
+ } catch (IOException e) {
+ // Same Question: seek and read failed afer openning file succeede, should we do something about it?
+ e.printStackTrace();
+ return false;
+ }
+ }
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/SequenceFileLookupReader.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/SequenceFileLookupReader.java
new file mode 100644
index 0000000..4c8816e
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/SequenceFileLookupReader.java
@@ -0,0 +1,106 @@
+/*
+ * 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.external.indexing.input;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.SequenceFile.Reader;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import edu.uci.ics.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
+import edu.uci.ics.asterix.metadata.entities.ExternalFile;
+import edu.uci.ics.asterix.metadata.external.ExternalFileIndexAccessor;
+
+public class SequenceFileLookupReader implements ILookupReader {
+
+ private Reader reader;
+ private Writable key;
+ private Writable value;
+ private FileSystem fs;
+ private int fileNumber = -1;
+ private boolean skipFile = false;
+ private ExternalFile file = new ExternalFile(null, null, 0, null, null, 0L, ExternalFilePendingOp.PENDING_NO_OP);
+ private ExternalFileIndexAccessor filesIndexAccessor;
+ private Configuration conf;
+
+ public SequenceFileLookupReader(ExternalFileIndexAccessor filesIndexAccessor, Configuration conf)
+ throws IOException {
+ fs = FileSystem.get(conf);
+ this.filesIndexAccessor = filesIndexAccessor;
+ this.conf = conf;
+ }
+
+ @Override
+ public Writable read(int fileNumber, long recordOffset) throws Exception {
+ if (fileNumber != this.fileNumber) {
+ //get file name
+ this.fileNumber = fileNumber;
+ filesIndexAccessor.searchForFile(fileNumber, file);
+ try {
+ FileStatus fileStatus = fs.getFileStatus(new Path(file.getFileName()));
+ if (fileStatus.getModificationTime() != file.getLastModefiedTime().getTime()) {
+ this.fileNumber = fileNumber;
+ skipFile = true;
+ return null;
+ } else {
+ this.fileNumber = fileNumber;
+ skipFile = false;
+ openFile(file.getFileName());
+ }
+ } catch (FileNotFoundException e) {
+ // file was not found, do nothing and skip its tuples
+ this.fileNumber = fileNumber;
+ skipFile = true;
+ return null;
+ }
+ } else if (skipFile) {
+ return null;
+ }
+ reader.seek(recordOffset);
+ reader.next(key, value);
+ return value;
+ }
+
+ @SuppressWarnings("deprecation")
+ private void openFile(String FileName) throws IOException {
+ if (reader != null)
+ try {
+ reader.close();
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ reader = new SequenceFile.Reader(fs, new Path(FileName), conf);
+ key = (Writable) ReflectionUtils.newInstance(reader.getKeyClass(), conf);
+ value = (Writable) ReflectionUtils.newInstance(reader.getValueClass(), conf);
+ }
+
+ @Override
+ public void close() {
+ if (reader != null)
+ try {
+ reader.close();
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/TextFileLookupInputStream.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/TextFileLookupInputStream.java
new file mode 100644
index 0000000..0102b28
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/TextFileLookupInputStream.java
@@ -0,0 +1,61 @@
+/*
+ * 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.external.indexing.input;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+
+import edu.uci.ics.asterix.metadata.external.ExternalFileIndexAccessor;
+
+
+@SuppressWarnings("deprecation")
+public class TextFileLookupInputStream extends AbstractHDFSLookupInputStream{
+
+ private FSDataInputStream reader;
+ public TextFileLookupInputStream(ExternalFileIndexAccessor filesIndexAccessor, JobConf conf) throws IOException{
+ super(filesIndexAccessor, conf);
+ }
+ @Override
+ protected void openFile(String fileName) throws IOException {
+ if (reader != null) {
+ reader.close();
+ }
+ reader = fs.open(new Path(fileName));
+ }
+
+ @Override
+ public void close() throws IOException {
+ if (reader != null) {
+ reader.close();
+ }
+ super.close();
+ }
+
+ @Override
+ protected boolean read(long recordOffset) {
+ try {
+ reader.seek(recordOffset);
+ pendingValue = reader.readLine();
+ return true;
+ } catch (IOException e) {
+ // file was opened and then when trying to seek and read, an error occurred <- should we throw an exception ???->
+ e.printStackTrace();
+ return false;
+ }
+ }
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/TextFileLookupReader.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/TextFileLookupReader.java
new file mode 100644
index 0000000..3f84503
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/TextFileLookupReader.java
@@ -0,0 +1,90 @@
+/*
+ * 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.external.indexing.input;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import edu.uci.ics.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
+import edu.uci.ics.asterix.metadata.entities.ExternalFile;
+import edu.uci.ics.asterix.metadata.external.ExternalFileIndexAccessor;
+
+public class TextFileLookupReader implements ILookupReader {
+ private FileSystem fs;
+ private int fileNumber = -1;
+ private boolean skipFile = false;
+ private ExternalFile file = new ExternalFile(null, null, 0, null, null, 0L, ExternalFilePendingOp.PENDING_NO_OP);
+ private ExternalFileIndexAccessor filesIndexAccessor;
+ private FSDataInputStream reader;
+
+ public TextFileLookupReader(ExternalFileIndexAccessor filesIndexAccessor, Configuration conf) throws IOException {
+ fs = FileSystem.get(conf);
+ this.filesIndexAccessor = filesIndexAccessor;
+ }
+
+ @SuppressWarnings("deprecation")
+ @Override
+ public String read(int fileNumber, long recordOffset) throws Exception {
+ if (fileNumber != this.fileNumber) {
+ this.fileNumber = fileNumber;
+ filesIndexAccessor.searchForFile(fileNumber, file);
+
+ try {
+ FileStatus fileStatus = fs.getFileStatus(new Path(file.getFileName()));
+ if (fileStatus.getModificationTime() != file.getLastModefiedTime().getTime()) {
+ this.fileNumber = fileNumber;
+ skipFile = true;
+ return null;
+ } else {
+ this.fileNumber = fileNumber;
+ skipFile = false;
+ openFile(file.getFileName());
+ }
+ } catch (FileNotFoundException e) {
+ // File is not there, skip it and do nothing
+ this.fileNumber = fileNumber;
+ skipFile = true;
+ return null;
+ }
+ } else if (skipFile) {
+ return null;
+ }
+ reader.seek(recordOffset);
+ return reader.readLine();
+ }
+
+ private void openFile(String FileName) throws IOException {
+ if (reader != null) {
+ reader.close();
+ }
+ reader = fs.open(new Path(FileName));
+ }
+
+ public void close() {
+ if (reader != null)
+ try {
+ reader.close();
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/TextualDataReader.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/TextualDataReader.java
new file mode 100644
index 0000000..550fdc6
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/TextualDataReader.java
@@ -0,0 +1,234 @@
+/*
+ * 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.external.indexing.input;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.SequenceFileInputFormat;
+import org.apache.hadoop.mapred.TextInputFormat;
+
+import edu.uci.ics.asterix.metadata.entities.ExternalFile;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+
+// Used in two cases:
+// 1. building an index over a dataset
+// 2. performing full scan over a dataset that has built index (to provide consistent view)
+
+@SuppressWarnings({ "rawtypes", "deprecation" })
+public class TextualDataReader extends AbstractHDFSReader {
+
+ private RecordReader<Object, Text> reader;
+ private Object key;
+ private Text value;
+ private boolean hasMore = false;
+ private int EOL = "\n".getBytes()[0];
+ private Text pendingValue = null;
+ private int currentSplitIndex = 0;
+ private String fileName;
+ private long recordOffset;
+ private boolean executed[];
+ private InputSplit[] inputSplits;
+ private String[] readSchedule;
+ private String nodeName;
+ private JobConf conf;
+ private List<ExternalFile> files;
+ private FileSystem hadoopFS;
+
+ public TextualDataReader(InputSplit[] inputSplits, String[] readSchedule, String nodeName, JobConf conf,
+ boolean executed[], List<ExternalFile> files) throws IOException {
+ this.executed = executed;
+ this.inputSplits = inputSplits;
+ this.readSchedule = readSchedule;
+ this.nodeName = nodeName;
+ this.conf = conf;
+ this.files = files;
+ hadoopFS = FileSystem.get(conf);
+ }
+
+ @Override
+ public boolean initialize() throws Exception {
+ return moveToNext();
+ }
+
+ @Override
+ public Object readNext() throws Exception {
+ if (reader == null) {
+ return null;
+ }
+ recordOffset = reader.getPos();
+ if (reader.next(key, value)) {
+ return value;
+ }
+ while (moveToNext()) {
+ recordOffset = reader.getPos();
+ if (reader.next(key, value)) {
+ return value;
+ }
+ }
+ return null;
+ }
+
+ @Override
+ public int getFileNumber() throws Exception {
+ return files.get(currentSplitIndex).getFileNumber();
+ }
+
+ @Override
+ public String getFileName() throws Exception {
+ return files.get(currentSplitIndex).getFileName();
+ }
+
+ @Override
+ public long getReaderPosition() throws Exception {
+ return recordOffset;
+ }
+
+ @Override
+ public int read() throws IOException {
+ throw new NotImplementedException("Use read(byte[], int, int");
+ }
+
+ @SuppressWarnings("unchecked")
+ private boolean moveToNext() throws IOException {
+ for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
+ /**
+ * read all the partitions scheduled to the current node
+ */
+ if (readSchedule[currentSplitIndex].equals(nodeName)) {
+ /**
+ * pick an unread split to read synchronize among
+ * simultaneous partitions in the same machine
+ */
+ synchronized (executed) {
+ if (executed[currentSplitIndex] == false) {
+ executed[currentSplitIndex] = true;
+ } else {
+ continue;
+ }
+ }
+
+ /**
+ * read the split
+ */
+ try {
+ if (files != null) {
+ fileName = ((FileSplit) (inputSplits[currentSplitIndex])).getPath().toUri().getPath();
+ FileStatus fileStatus = hadoopFS.getFileStatus(new Path(fileName));
+ // Skip if not the same file stored in the files snapshot
+ if (fileStatus.getModificationTime() != files.get(currentSplitIndex).getLastModefiedTime()
+ .getTime())
+ continue;
+ }
+ // It is the same file
+ reader = getRecordReader(currentSplitIndex);
+ } catch (Exception e) {
+ // ignore exceptions <-- This might change later -->
+ continue;
+ }
+ key = reader.createKey();
+ value = (Text) reader.createValue();
+ return true;
+ }
+ }
+ return false;
+ }
+
+
+ private RecordReader getRecordReader(int splitIndex) throws IOException {
+ RecordReader reader;
+ if (conf.getInputFormat() instanceof SequenceFileInputFormat) {
+ SequenceFileInputFormat format = (SequenceFileInputFormat) conf.getInputFormat();
+ reader = format.getRecordReader((org.apache.hadoop.mapred.FileSplit) inputSplits[splitIndex], conf,
+ getReporter());
+ } else {
+ TextInputFormat format = (TextInputFormat) conf.getInputFormat();
+ reader = format.getRecordReader((org.apache.hadoop.mapred.FileSplit) inputSplits[splitIndex], conf,
+ getReporter());
+ }
+ return reader;
+ }
+
+ // Return one record at a time <to preserve the indexing information>
+ @Override
+ public int read(byte[] buffer, int offset, int len) throws IOException {
+ if (reader == null) {
+ if (!moveToNext()) {
+ // nothing to read
+ return -1;
+ }
+ }
+
+ int numBytes = 0;
+ if (pendingValue != null) {
+ int sizeOfNextTuple = pendingValue.getLength() + 1;
+ if (sizeOfNextTuple > len) {
+ return 0;
+ }
+ System.arraycopy(pendingValue.getBytes(), 0, buffer, offset + numBytes, pendingValue.getLength());
+ buffer[offset + numBytes + pendingValue.getLength()] = (byte) EOL;
+ numBytes += pendingValue.getLength() + 1;
+ pendingValue = null;
+ return numBytes;
+ }
+ if (numBytes < len) {
+ //store the byte location
+ recordOffset = reader.getPos();
+ hasMore = reader.next(key, value);
+ if (!hasMore) {
+ while (moveToNext()) {
+ //store the byte location
+ recordOffset = reader.getPos();
+ hasMore = reader.next(key, value);
+ if (hasMore) {
+ //return the value read
+ int sizeOfNextTuple = value.getLength() + 1;
+ if (numBytes + sizeOfNextTuple > len) {
+ pendingValue = value;
+ return 0;
+ } else {
+ System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.getLength());
+ buffer[offset + numBytes + value.getLength()] = (byte) EOL;
+ numBytes += sizeOfNextTuple;
+ return numBytes;
+ }
+ }
+ }
+ return -1;
+ } else {
+ //return the value read
+ int sizeOfNextTuple = value.getLength() + 1;
+ if (numBytes + sizeOfNextTuple > len) {
+ pendingValue = value;
+ return 0;
+ } else {
+ System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.getLength());
+ buffer[offset + numBytes + value.getLength()] = (byte) EOL;
+ numBytes += sizeOfNextTuple;
+ return numBytes;
+ }
+ }
+ }
+ return numBytes;
+ }
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/TextualFullScanDataReader.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/TextualFullScanDataReader.java
new file mode 100644
index 0000000..e80adb0
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/input/TextualFullScanDataReader.java
@@ -0,0 +1,211 @@
+/*
+ * 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.external.indexing.input;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.SequenceFileInputFormat;
+import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.hadoop.mapred.Counters.Counter;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+
+@SuppressWarnings("deprecation")
+public class TextualFullScanDataReader extends InputStream{
+
+ private RecordReader<Object, Text> reader;
+ private Object key;
+ private Text value;
+ private boolean hasMore = false;
+ private int EOL = "\n".getBytes()[0];
+ private Text pendingValue = null;
+ private int currentSplitIndex = 0;
+ private boolean executed[];
+ private InputSplit[] inputSplits;
+ private String[] readSchedule;
+ private String nodeName;
+ private JobConf conf;
+
+ public TextualFullScanDataReader(boolean executed[], InputSplit[] inputSplits, String[] readSchedule, String nodeName, JobConf conf){
+ this.executed = executed;
+ this.inputSplits = inputSplits;
+ this.readSchedule = readSchedule;
+ this.nodeName = nodeName;
+ this.conf = conf;
+ }
+
+ @Override
+ public int available() {
+ return 1;
+ }
+
+ @SuppressWarnings("unchecked")
+ private boolean moveToNext() throws IOException {
+ for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
+ /**
+ * read all the partitions scheduled to the current node
+ */
+ if (readSchedule[currentSplitIndex].equals(nodeName)) {
+ /**
+ * pick an unread split to read synchronize among
+ * simultaneous partitions in the same machine
+ */
+ synchronized (executed) {
+ if (executed[currentSplitIndex] == false) {
+ executed[currentSplitIndex] = true;
+ } else {
+ continue;
+ }
+ }
+
+ /**
+ * read the split
+ */
+ reader = getRecordReader(currentSplitIndex);
+ key = reader.createKey();
+ value = (Text) reader.createValue();
+ return true;
+ }
+ }
+ return false;
+ }
+
+ @Override
+ public int read(byte[] buffer, int offset, int len)
+ throws IOException {
+ if (reader == null) {
+ if (!moveToNext()) {
+ // nothing to read
+ return -1;
+ }
+ }
+
+ int numBytes = 0;
+ if (pendingValue != null) {
+ int sizeOfNextTuple = pendingValue.getLength() + 1;
+ if (sizeOfNextTuple > len) {
+ return 0;
+ }
+ System.arraycopy(pendingValue.getBytes(), 0, buffer,
+ offset + numBytes, pendingValue.getLength());
+ buffer[offset + numBytes + pendingValue.getLength()] = (byte) EOL;
+ numBytes += pendingValue.getLength() + 1;
+ pendingValue = null;
+ }
+
+ while (numBytes < len) {
+ hasMore = reader.next(key, value);
+ if (!hasMore) {
+ while (moveToNext()) {
+ hasMore = reader.next(key, value);
+ if (hasMore) {
+ // move to the next non-empty split
+ break;
+ }
+ }
+ }
+ if (!hasMore) {
+ return (numBytes == 0) ? -1 : numBytes;
+ }
+ int sizeOfNextTuple = value.getLength() + 1;
+ if (numBytes + sizeOfNextTuple > len) {
+ // cannot add tuple to current buffer
+ // but the reader has moved pass the fetched tuple
+ // we need to store this for a subsequent read call.
+ // and return this then.
+ pendingValue = value;
+ break;
+ } else {
+ System.arraycopy(value.getBytes(), 0, buffer,
+ offset + numBytes, value.getLength());
+ buffer[offset + numBytes + value.getLength()] = (byte) EOL;
+ numBytes += sizeOfNextTuple;
+ }
+ }
+ return numBytes;
+ }
+
+ @Override
+ public int read() throws IOException {
+ throw new NotImplementedException(
+ "Use read(byte[], int, int");
+ }
+
+ @SuppressWarnings("rawtypes")
+ private RecordReader getRecordReader(int splitIndex)
+ throws IOException {
+ if (conf.getInputFormat() instanceof SequenceFileInputFormat) {
+ SequenceFileInputFormat format = (SequenceFileInputFormat) conf
+ .getInputFormat();
+ RecordReader reader = format
+ .getRecordReader(
+ (org.apache.hadoop.mapred.FileSplit) inputSplits[splitIndex],
+ conf, getReporter());
+ return reader;
+ } else {
+ TextInputFormat format = (TextInputFormat) conf
+ .getInputFormat();
+ RecordReader reader = format
+ .getRecordReader(
+ (org.apache.hadoop.mapred.FileSplit) inputSplits[splitIndex],
+ conf, getReporter());
+ return reader;
+ }
+ }
+
+ private Reporter getReporter() {
+ Reporter reporter = new Reporter() {
+
+ @Override
+ public Counter getCounter(Enum<?> arg0) {
+ return null;
+ }
+
+ @Override
+ public Counter getCounter(String arg0, String arg1) {
+ return null;
+ }
+
+ @Override
+ public InputSplit getInputSplit() throws UnsupportedOperationException {
+ return null;
+ }
+
+ @Override
+ public void incrCounter(Enum<?> arg0, long arg1) {
+ }
+
+ @Override
+ public void incrCounter(String arg0, String arg1, long arg2) {
+ }
+
+ @Override
+ public void setStatus(String arg0) {
+ }
+
+ @Override
+ public void progress() {
+ }
+ };
+
+ return reporter;
+ }
+}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/operators/AbstractExternalDatasetIndexesOperatorDescriptor.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/operators/AbstractExternalDatasetIndexesOperatorDescriptor.java
new file mode 100644
index 0000000..8646d4b
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/operators/AbstractExternalDatasetIndexesOperatorDescriptor.java
@@ -0,0 +1,118 @@
+/*
+ * 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.external.indexing.operators;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+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;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorNodePushable;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeWithBuddyDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.dataflow.ExternalRTreeDataflowHelperFactory;
+
+// This is an operator that takes a single file index and an array of secondary indexes
+// it is intended to be used for
+// 1. commit transaction operation
+// 2. abort transaction operation
+// 3. recover transaction operation
+public abstract class AbstractExternalDatasetIndexesOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ private ExternalBTreeDataflowHelperFactory filesIndexDataflowHelperFactory;
+ private IndexInfoOperatorDescriptor fileIndexInfo;
+ private List<ExternalBTreeWithBuddyDataflowHelperFactory> bTreeIndexesDataflowHelperFactories;
+ private List<IndexInfoOperatorDescriptor> bTreeIndexesInfos;
+ private List<ExternalRTreeDataflowHelperFactory> rTreeIndexesDataflowHelperFactories;
+ private List<IndexInfoOperatorDescriptor> rTreeIndexesInfos;
+
+ public AbstractExternalDatasetIndexesOperatorDescriptor(IOperatorDescriptorRegistry spec,
+ ExternalBTreeDataflowHelperFactory filesIndexDataflowHelperFactory,
+ IndexInfoOperatorDescriptor fileIndexesInfo,
+ List<ExternalBTreeWithBuddyDataflowHelperFactory> bTreeIndexesDataflowHelperFactories,
+ List<IndexInfoOperatorDescriptor> bTreeIndexesInfos,
+ List<ExternalRTreeDataflowHelperFactory> rTreeIndexesDataflowHelperFactories,
+ List<IndexInfoOperatorDescriptor> rTreeIndexesInfos) {
+ super(spec, 0, 0);
+ this.filesIndexDataflowHelperFactory = filesIndexDataflowHelperFactory;
+ this.fileIndexInfo = fileIndexesInfo;
+ this.bTreeIndexesDataflowHelperFactories = bTreeIndexesDataflowHelperFactories;
+ this.bTreeIndexesInfos = bTreeIndexesInfos;
+ this.rTreeIndexesDataflowHelperFactories = rTreeIndexesDataflowHelperFactories;
+ this.rTreeIndexesInfos = rTreeIndexesInfos;
+ }
+
+ // opening and closing the index is done inside these methods since we don't always need open indexes
+ protected abstract void performOpOnIndex(
+ IIndexDataflowHelperFactory indexDataflowHelperFactory, IHyracksTaskContext ctx,
+ IndexInfoOperatorDescriptor fileIndexInfo, int partition) throws Exception;
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+ return new AbstractOperatorNodePushable() {
+
+ @Override
+ public void initialize() throws HyracksDataException {
+ try {
+ // only in partition of device id = 0, we perform the operation on the files index
+ if(fileIndexInfo.getFileSplitProvider().getFileSplits()[partition].getIODeviceId() == 0){
+ performOpOnIndex(filesIndexDataflowHelperFactory, ctx, fileIndexInfo, partition);
+ }
+ // perform operation on btrees
+ for (int i = 0; i < bTreeIndexesDataflowHelperFactories.size(); i++) {
+ performOpOnIndex(bTreeIndexesDataflowHelperFactories.get(i), ctx,
+ bTreeIndexesInfos.get(i), partition);
+ }
+ // perform operation on rtrees
+ for (int i = 0; i < rTreeIndexesDataflowHelperFactories.size(); i++) {
+ performOpOnIndex(rTreeIndexesDataflowHelperFactories.get(i), ctx,
+ rTreeIndexesInfos.get(i), partition);
+ }
+ } catch (Exception e) {
+ // This should never happen <unless there is a hardware failure or something serious>
+ throw new HyracksDataException(e);
+ }
+ }
+
+ @Override
+ public void deinitialize() throws HyracksDataException {
+ }
+
+ @Override
+ public int getInputArity() {
+ return 0;
+ }
+
+ @Override
+ public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc)
+ throws HyracksDataException {
+ }
+
+ @Override
+ public IFrameWriter getInputFrameWriter(int index) {
+ return null;
+ }
+
+ };
+ }
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/operators/ExternalDatasetIndexesAbortOperatorDescriptor.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/operators/ExternalDatasetIndexesAbortOperatorDescriptor.java
new file mode 100644
index 0000000..d90b7ed
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/operators/ExternalDatasetIndexesAbortOperatorDescriptor.java
@@ -0,0 +1,55 @@
+/*
+ * 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.external.indexing.operators;
+
+import java.io.File;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.util.IndexFileNameUtil;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeWithBuddyDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbortRecoverLSMIndexFileManager;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.dataflow.ExternalRTreeDataflowHelperFactory;
+
+public class ExternalDatasetIndexesAbortOperatorDescriptor extends AbstractExternalDatasetIndexesOperatorDescriptor {
+
+ private static final long serialVersionUID = 1L;
+
+ public ExternalDatasetIndexesAbortOperatorDescriptor(IOperatorDescriptorRegistry spec,
+ ExternalBTreeDataflowHelperFactory filesIndexDataflowHelperFactory,
+ IndexInfoOperatorDescriptor fileIndexesInfo,
+ List<ExternalBTreeWithBuddyDataflowHelperFactory> bTreeIndexesDataflowHelperFactories,
+ List<IndexInfoOperatorDescriptor> bTreeIndexesInfos,
+ List<ExternalRTreeDataflowHelperFactory> rTreeIndexesDataflowHelperFactories,
+ List<IndexInfoOperatorDescriptor> rTreeIndexesInfos) {
+ super(spec, filesIndexDataflowHelperFactory, fileIndexesInfo, bTreeIndexesDataflowHelperFactories,
+ bTreeIndexesInfos, rTreeIndexesDataflowHelperFactories, rTreeIndexesInfos);
+ }
+
+ @Override
+ protected void performOpOnIndex(IIndexDataflowHelperFactory indexDataflowHelperFactory, IHyracksTaskContext ctx,
+ IndexInfoOperatorDescriptor fileIndexInfo, int partition) throws Exception {
+ FileReference file = new FileReference(new File(IndexFileNameUtil.prepareFileName(fileIndexInfo
+ .getFileSplitProvider().getFileSplits()[partition].getLocalFile().getFile().getPath(), fileIndexInfo
+ .getFileSplitProvider().getFileSplits()[partition].getIODeviceId())));
+ AbortRecoverLSMIndexFileManager fileManager = new AbortRecoverLSMIndexFileManager(file);
+ fileManager.deleteTransactionFiles();
+ }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/operators/ExternalDatasetIndexesCommitOperatorDescriptor.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/operators/ExternalDatasetIndexesCommitOperatorDescriptor.java
new file mode 100644
index 0000000..11f19f1
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/operators/ExternalDatasetIndexesCommitOperatorDescriptor.java
@@ -0,0 +1,63 @@
+/*
+ * 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.external.indexing.operators;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.util.IndexFileNameUtil;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeWithBuddyDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ITwoPCIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.dataflow.ExternalRTreeDataflowHelperFactory;
+
+public class ExternalDatasetIndexesCommitOperatorDescriptor extends AbstractExternalDatasetIndexesOperatorDescriptor {
+
+ public ExternalDatasetIndexesCommitOperatorDescriptor(IOperatorDescriptorRegistry spec,
+ ExternalBTreeDataflowHelperFactory filesIndexDataflowHelperFactory,
+ IndexInfoOperatorDescriptor fileIndexesInfo,
+ List<ExternalBTreeWithBuddyDataflowHelperFactory> bTreeIndexesDataflowHelperFactories,
+ List<IndexInfoOperatorDescriptor> bTreeIndexesInfos,
+ List<ExternalRTreeDataflowHelperFactory> rTreeIndexesDataflowHelperFactories,
+ List<IndexInfoOperatorDescriptor> rTreeIndexesInfos) {
+ super(spec, filesIndexDataflowHelperFactory, fileIndexesInfo, bTreeIndexesDataflowHelperFactories,
+ bTreeIndexesInfos, rTreeIndexesDataflowHelperFactories, rTreeIndexesInfos);
+ }
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ protected void performOpOnIndex(IIndexDataflowHelperFactory indexDataflowHelperFactory, IHyracksTaskContext ctx,
+ IndexInfoOperatorDescriptor fileIndexInfo, int partition) throws Exception {
+ System.err.println("performing the operation on "+ IndexFileNameUtil.prepareFileName(fileIndexInfo.getFileSplitProvider()
+ .getFileSplits()[partition].getLocalFile().getFile().getPath(), fileIndexInfo.getFileSplitProvider()
+ .getFileSplits()[partition].getIODeviceId()));
+ // Get DataflowHelper
+ IIndexDataflowHelper indexHelper = indexDataflowHelperFactory.createIndexDataflowHelper(fileIndexInfo, ctx, partition);
+ // Get index
+ IIndex index = indexHelper.getIndexInstance();
+ // commit transaction
+ ((ITwoPCIndex) index).commitTransaction();
+ System.err.println("operation on "+ IndexFileNameUtil.prepareFileName(fileIndexInfo.getFileSplitProvider()
+ .getFileSplits()[partition].getLocalFile().getFile().getPath(), fileIndexInfo.getFileSplitProvider()
+ .getFileSplits()[partition].getIODeviceId()) + " Succeded");
+
+ }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/operators/ExternalDatasetIndexesRecoverOperatorDescriptor.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/operators/ExternalDatasetIndexesRecoverOperatorDescriptor.java
new file mode 100644
index 0000000..b290eee
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/operators/ExternalDatasetIndexesRecoverOperatorDescriptor.java
@@ -0,0 +1,55 @@
+/*
+ * 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.external.indexing.operators;
+
+import java.io.File;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.util.IndexFileNameUtil;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeWithBuddyDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbortRecoverLSMIndexFileManager;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.dataflow.ExternalRTreeDataflowHelperFactory;
+
+public class ExternalDatasetIndexesRecoverOperatorDescriptor extends AbstractExternalDatasetIndexesOperatorDescriptor {
+
+ private static final long serialVersionUID = 1L;
+
+ public ExternalDatasetIndexesRecoverOperatorDescriptor(IOperatorDescriptorRegistry spec,
+ ExternalBTreeDataflowHelperFactory filesIndexDataflowHelperFactory,
+ IndexInfoOperatorDescriptor fileIndexesInfo,
+ List<ExternalBTreeWithBuddyDataflowHelperFactory> bTreeIndexesDataflowHelperFactories,
+ List<IndexInfoOperatorDescriptor> bTreeIndexesInfos,
+ List<ExternalRTreeDataflowHelperFactory> rTreeIndexesDataflowHelperFactories,
+ List<IndexInfoOperatorDescriptor> rTreeIndexesInfos) {
+ super(spec, filesIndexDataflowHelperFactory, fileIndexesInfo, bTreeIndexesDataflowHelperFactories,
+ bTreeIndexesInfos, rTreeIndexesDataflowHelperFactories, rTreeIndexesInfos);
+ }
+
+ @Override
+ protected void performOpOnIndex(IIndexDataflowHelperFactory indexDataflowHelperFactory, IHyracksTaskContext ctx,
+ IndexInfoOperatorDescriptor fileIndexInfo, int partition) throws Exception {
+ FileReference file = new FileReference(new File(IndexFileNameUtil.prepareFileName(fileIndexInfo
+ .getFileSplitProvider().getFileSplits()[partition].getLocalFile().getFile().getPath(), fileIndexInfo
+ .getFileSplitProvider().getFileSplits()[partition].getIODeviceId())));
+ AbortRecoverLSMIndexFileManager fileManager = new AbortRecoverLSMIndexFileManager(file);
+ fileManager.recoverTransaction();
+ }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/operators/ExternalIndexBulkModifyOperatorDescriptor.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/operators/ExternalIndexBulkModifyOperatorDescriptor.java
new file mode 100644
index 0000000..7369fd7
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/operators/ExternalIndexBulkModifyOperatorDescriptor.java
@@ -0,0 +1,65 @@
+/*
+ * 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.external.indexing.operators;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
+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.impls.NoOpOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.common.file.NoOpLocalResourceFactoryProvider;
+
+public class ExternalIndexBulkModifyOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ private final int[] deletedFiles;
+ private final int[] fieldPermutation;
+ private final float fillFactor;
+ private final long numElementsHint;
+
+ public ExternalIndexBulkModifyOperatorDescriptor(IOperatorDescriptorRegistry spec,
+ IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
+ IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
+ IBinaryComparatorFactory[] comparatorFactories, int[] bloomFilterKeyFields,
+ IIndexDataflowHelperFactory dataflowHelperFactory,
+ IModificationOperationCallbackFactory modificationOpCallbackFactory, int[] deletedFiles,
+ int[] fieldPermutation, float fillFactor, long numElementsHint) {
+ super(spec, 1, 0, null, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
+ comparatorFactories, bloomFilterKeyFields, dataflowHelperFactory, null, false, false, null,
+ NoOpLocalResourceFactoryProvider.INSTANCE, NoOpOperationCallbackFactory.INSTANCE,
+ modificationOpCallbackFactory);
+ this.deletedFiles = deletedFiles;
+ this.fieldPermutation = fieldPermutation;
+ this.fillFactor = fillFactor;
+ this.numElementsHint = numElementsHint;
+ }
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
+ return new ExternalIndexBulkModifyOperatorNodePushable(this, ctx, partition, fieldPermutation, fillFactor,
+ numElementsHint, recordDescProvider, deletedFiles);
+ }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/operators/ExternalIndexBulkModifyOperatorNodePushable.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/operators/ExternalIndexBulkModifyOperatorNodePushable.java
new file mode 100644
index 0000000..1ff2e56
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/operators/ExternalIndexBulkModifyOperatorNodePushable.java
@@ -0,0 +1,104 @@
+/*
+ * 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.external.indexing.operators;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.asterix.metadata.external.FilesIndexDescription;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+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;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITwoPCIndexBulkLoader;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexBulkLoadOperatorNodePushable;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ITwoPCIndex;
+
+public class ExternalIndexBulkModifyOperatorNodePushable extends IndexBulkLoadOperatorNodePushable {
+
+ private final int[] deletedFiles;
+ private ArrayTupleBuilder buddyBTreeTupleBuilder = new ArrayTupleBuilder(
+ FilesIndexDescription.FILE_BUDDY_BTREE_RECORD_DESCRIPTOR.getFieldCount());
+ private AMutableInt32 fileNumber = new AMutableInt32(0);
+ private ArrayTupleReference deleteTuple = new ArrayTupleReference();
+
+ public ExternalIndexBulkModifyOperatorNodePushable(ExternalIndexBulkModifyOperatorDescriptor opDesc,
+ IHyracksTaskContext ctx, int partition, int[] fieldPermutation, float fillFactor, long numElementsHint,
+ IRecordDescriptorProvider recordDescProvider, int[] deletedFiles) {
+ super(opDesc, ctx, partition, fieldPermutation, fillFactor, false, numElementsHint, false, recordDescProvider);
+ this.deletedFiles = deletedFiles;
+ }
+
+ // We override this method to do two things
+ // when creating the bulkLoader, it creates a transaction bulk loader
+ // It uses the bulkLoader to insert delete tuples for the deleted files
+ @Override
+ public void open() throws HyracksDataException {
+ RecordDescriptor recDesc = recDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
+ accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
+ indexHelper.open();
+ index = indexHelper.getIndexInstance();
+ try {
+ // Transactional BulkLoader
+ bulkLoader = ((ITwoPCIndex) index).createTransactionBulkLoader(fillFactor, verifyInput, deletedFiles.length,
+ checkIfEmptyIndex);
+ // Delete files
+ for (int i = 0; i < deletedFiles.length; i++) {
+ fileNumber.setValue(deletedFiles[i]);
+ FilesIndexDescription.getBuddyBTreeTupleFromFileNumber(deleteTuple, buddyBTreeTupleBuilder, fileNumber);
+ ((ITwoPCIndexBulkLoader) bulkLoader).delete(deleteTuple);
+ }
+ } catch (Exception e) {
+ ((ITwoPCIndexBulkLoader) bulkLoader).abort();
+ indexHelper.close();
+ throw new HyracksDataException(e);
+ }
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ accessor.reset(buffer);
+ int tupleCount = accessor.getTupleCount();
+ for (int i = 0; i < tupleCount; i++) {
+ tuple.reset(accessor, i);
+ try {
+ bulkLoader.add(tuple);
+ } catch (IndexException e) {
+ ((ITwoPCIndexBulkLoader) bulkLoader).abort();
+ throw new HyracksDataException(e);
+ }
+ }
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ try {
+ bulkLoader.end();
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ } finally {
+ indexHelper.close();
+ }
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ ((ITwoPCIndexBulkLoader) bulkLoader).abort();
+ }
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/operators/IndexInfoOperatorDescriptor.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/operators/IndexInfoOperatorDescriptor.java
new file mode 100644
index 0000000..98055c4
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/operators/IndexInfoOperatorDescriptor.java
@@ -0,0 +1,122 @@
+/*
+ * 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.external.indexing.operators;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITupleFilterFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactoryProvider;
+
+/*
+ * This is a hack used to optain multiple index instances in a single operator and it is not actually used as an operator
+ */
+public class IndexInfoOperatorDescriptor implements IIndexOperatorDescriptor{
+
+ private static final long serialVersionUID = 1L;
+ private final IFileSplitProvider fileSplitProvider;
+ private final IStorageManagerInterface storageManager;
+ private final IIndexLifecycleManagerProvider lifecycleManagerProvider;
+ public IndexInfoOperatorDescriptor(IFileSplitProvider fileSplitProvider,IStorageManagerInterface storageManager,
+ IIndexLifecycleManagerProvider lifecycleManagerProvider){
+ this.fileSplitProvider = fileSplitProvider;
+ this.lifecycleManagerProvider = lifecycleManagerProvider;
+ this.storageManager = storageManager;
+
+ }
+
+ @Override
+ public ActivityId getActivityId() {
+ return null;
+ }
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
+ return null;
+ }
+
+ @Override
+ public IFileSplitProvider getFileSplitProvider() {
+ return fileSplitProvider;
+ }
+
+ @Override
+ public IStorageManagerInterface getStorageManager() {
+ return storageManager;
+ }
+
+ @Override
+ public IIndexLifecycleManagerProvider getLifecycleManagerProvider() {
+ return lifecycleManagerProvider;
+ }
+
+ @Override
+ public RecordDescriptor getRecordDescriptor() {
+ return null;
+ }
+
+ @Override
+ public IIndexDataflowHelperFactory getIndexDataflowHelperFactory() {
+ return null;
+ }
+
+ @Override
+ public boolean getRetainInput() {
+ return false;
+ }
+
+ @Override
+ public ISearchOperationCallbackFactory getSearchOpCallbackFactory() {
+ return null;
+ }
+
+ @Override
+ public IModificationOperationCallbackFactory getModificationOpCallbackFactory() {
+ return null;
+ }
+
+ @Override
+ public ITupleFilterFactory getTupleFilterFactory() {
+ return null;
+ }
+
+ @Override
+ public ILocalResourceFactoryProvider getLocalResourceFactoryProvider() {
+ return null;
+ }
+
+ @Override
+ public boolean getRetainNull() {
+ return false;
+ }
+
+ @Override
+ public INullWriterFactory getNullWriterFactory() {
+ return null;
+ }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/IDatasetDetails.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/IDatasetDetails.java
index 7cabb80..3ed94de 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/IDatasetDetails.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/IDatasetDetails.java
@@ -16,6 +16,7 @@
import java.io.DataOutput;
import java.io.Serializable;
+import java.util.Map;
import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -23,6 +24,8 @@
public interface IDatasetDetails extends Serializable {
public DatasetType getDatasetType();
-
+ public String getNodeGroupName();
public void writeDatasetDetailsRecordType(DataOutput out) throws HyracksDataException;
+ public String getCompactionPolicy();
+ public Map<String, String> getCompactionPolicyProperties();
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java
index 3bdf73a..07cb8ab 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java
@@ -34,6 +34,7 @@
import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
import edu.uci.ics.asterix.metadata.entities.Datatype;
import edu.uci.ics.asterix.metadata.entities.Dataverse;
+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.FeedActivity.FeedActivityType;
@@ -835,4 +836,68 @@
return dataverseAdapters;
}
+ @Override
+ public List<ExternalFile> getDatasetExternalFiles(MetadataTransactionContext mdTxnCtx, Dataset dataset)
+ throws MetadataException {
+ List<ExternalFile> externalFiles;
+ try {
+ externalFiles = metadataNode.getExternalFiles(mdTxnCtx.getJobId(), dataset);
+ } catch (RemoteException e) {
+ throw new MetadataException(e);
+ }
+ return externalFiles;
+ }
+
+ @Override
+ public void addExternalFile(MetadataTransactionContext ctx, ExternalFile externalFile) throws MetadataException {
+ try {
+ metadataNode.addExternalFile(ctx.getJobId(), externalFile);
+ } catch (RemoteException e) {
+ throw new MetadataException(e);
+ }
+ }
+
+ @Override
+ public void dropExternalFile(MetadataTransactionContext ctx, ExternalFile externalFile) throws MetadataException {
+ try {
+ metadataNode.dropExternalFile(ctx.getJobId(), externalFile.getDataverseName(), externalFile.getDatasetName(),
+ externalFile.getFileNumber());
+ } catch (RemoteException e) {
+ throw new MetadataException(e);
+ }
+ }
+
+ @Override
+ public ExternalFile getExternalFile(MetadataTransactionContext ctx, String dataverseName, String datasetName,
+ Integer fileNumber) throws MetadataException {
+ ExternalFile file;
+ try {
+ file = metadataNode.getExternalFile(ctx.getJobId(), dataverseName, datasetName, fileNumber);
+ } catch (RemoteException e) {
+ throw new MetadataException(e);
+ }
+ return file;
+ }
+
+ //TODO: Optimize <-- use keys instead of object -->
+ @Override
+ public void dropDatasetExternalFiles(MetadataTransactionContext mdTxnCtx, Dataset dataset) throws MetadataException {
+ try {
+ metadataNode.dropExternalFiles(mdTxnCtx.getJobId(), dataset);
+ } catch (RemoteException e) {
+ throw new MetadataException(e);
+ }
+ }
+
+ @Override
+ public void updateDataset(MetadataTransactionContext ctx, Dataset dataset) throws MetadataException {
+ try {
+ metadataNode.updateDataset(ctx.getJobId(), dataset);
+ } catch (RemoteException e) {
+ throw new MetadataException(e);
+ }
+ // reflect the dataset into the cache
+ ctx.dropDataset(dataset);
+ ctx.addDataset(dataset);
+ }
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
index 24bc208..5a8d8e6 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
@@ -48,6 +48,8 @@
import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
import edu.uci.ics.asterix.metadata.entities.Datatype;
import edu.uci.ics.asterix.metadata.entities.Dataverse;
+import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
+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.FeedActivity.FeedActivityType;
@@ -63,6 +65,7 @@
import edu.uci.ics.asterix.metadata.entitytupletranslators.DatasourceAdapterTupleTranslator;
import edu.uci.ics.asterix.metadata.entitytupletranslators.DatatypeTupleTranslator;
import edu.uci.ics.asterix.metadata.entitytupletranslators.DataverseTupleTranslator;
+import edu.uci.ics.asterix.metadata.entitytupletranslators.ExternalFileTupleTranslator;
import edu.uci.ics.asterix.metadata.entitytupletranslators.FeedActivityTupleTranslator;
import edu.uci.ics.asterix.metadata.entitytupletranslators.FeedPolicyTupleTranslator;
import edu.uci.ics.asterix.metadata.entitytupletranslators.FeedTupleTranslator;
@@ -77,6 +80,7 @@
import edu.uci.ics.asterix.metadata.valueextractors.MetadataEntityValueExtractor;
import edu.uci.ics.asterix.metadata.valueextractors.NestedDatatypeNameValueExtractor;
import edu.uci.ics.asterix.metadata.valueextractors.TupleCopyValueExtractor;
+import edu.uci.ics.asterix.om.base.AInt32;
import edu.uci.ics.asterix.om.base.AMutableString;
import edu.uci.ics.asterix.om.base.AString;
import edu.uci.ics.asterix.om.types.BuiltinType;
@@ -189,6 +193,13 @@
dataset.getDatasetName(), IndexType.BTREE, id.getPrimaryKey(), true, dataset.getPendingOp());
addIndex(jobId, primaryIndex);
+ // Add an entry for the node group
+ ITupleReference nodeGroupTuple = createTuple(id.getNodeGroupName(), dataset.getDataverseName(),
+ dataset.getDatasetName());
+ insertTupleIntoIndex(jobId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, nodeGroupTuple);
+ } else if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+ //added for external data
+ ExternalDatasetDetails id = (ExternalDatasetDetails) dataset.getDatasetDetails();
ITupleReference nodeGroupTuple = createTuple(id.getNodeGroupName(), dataset.getDataverseName(),
dataset.getDatasetName());
insertTupleIntoIndex(jobId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, nodeGroupTuple);
@@ -417,20 +428,19 @@
}
// Delete entry from secondary index 'group'.
- if (dataset.getDatasetType() == DatasetType.INTERNAL) {
- InternalDatasetDetails id = (InternalDatasetDetails) dataset.getDatasetDetails();
- ITupleReference groupNameSearchKey = createTuple(id.getNodeGroupName(), dataverseName, datasetName);
- // Searches the index for the tuple to be deleted. Acquires an S
- // lock on the GROUPNAME_ON_DATASET_INDEX index.
- try {
- ITupleReference groupNameTuple = getTupleToBeDeleted(jobId,
- MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, groupNameSearchKey);
- deleteTupleFromIndex(jobId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, groupNameTuple);
- } catch (TreeIndexException tie) {
- // ignore this exception and continue deleting all relevant
- // artifacts.
- }
+ ITupleReference groupNameSearchKey = createTuple(dataset.getDatasetDetails().getNodeGroupName(),
+ dataverseName, datasetName);
+ // Searches the index for the tuple to be deleted. Acquires an S
+ // lock on the GROUPNAME_ON_DATASET_INDEX index.
+ try {
+ ITupleReference groupNameTuple = getTupleToBeDeleted(jobId,
+ MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, groupNameSearchKey);
+ deleteTupleFromIndex(jobId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, groupNameTuple);
+ } catch (TreeIndexException tie) {
+ // ignore this exception and continue deleting all relevant
+ // artifacts.
}
+
// Delete entry from secondary index 'type'.
ITupleReference dataTypeSearchKey = createTuple(dataverseName, dataset.getItemTypeName(), datasetName);
// Searches the index for the tuple to be deleted. Acquires an S
@@ -445,11 +455,22 @@
}
// Delete entry(s) from the 'indexes' dataset.
- if (dataset.getDatasetType() == DatasetType.INTERNAL) {
- List<Index> datasetIndexes = getDatasetIndexes(jobId, dataverseName, datasetName);
- if (datasetIndexes != null) {
- for (Index index : datasetIndexes) {
- dropIndex(jobId, dataverseName, datasetName, index.getIndexName());
+ List<Index> datasetIndexes = getDatasetIndexes(jobId, dataverseName, datasetName);
+ if (datasetIndexes != null) {
+ for (Index index : datasetIndexes) {
+ dropIndex(jobId, dataverseName, datasetName, index.getIndexName());
+ }
+ }
+
+ if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+ // Delete External Files
+ // As a side effect, acquires an S lock on the 'ExternalFile' dataset
+ // on behalf of txnId.
+ List<ExternalFile> datasetFiles = getExternalFiles(jobId, dataset);
+ if (datasetFiles != null && datasetFiles.size() > 0) {
+ // Drop all external files in this dataset.
+ for (ExternalFile file : datasetFiles) {
+ dropExternalFile(jobId, dataverseName, file.getDatasetName(), file.getFileNumber());
}
}
}
@@ -1557,4 +1578,142 @@
}
}
+ @Override
+ public void addExternalFile(JobId jobId, ExternalFile externalFile) throws MetadataException, RemoteException {
+ try {
+ // Insert into the 'externalFiles' dataset.
+ ExternalFileTupleTranslator tupleReaderWriter = new ExternalFileTupleTranslator(true);
+ ITupleReference externalFileTuple = tupleReaderWriter.getTupleFromMetadataEntity(externalFile);
+ insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, externalFileTuple);
+ } catch (TreeIndexDuplicateKeyException e) {
+ throw new MetadataException("An externalFile with this number " + externalFile.getFileNumber()
+ + " already exists in dataset '" + externalFile.getDatasetName() + "' in dataverse '"
+ + externalFile.getDataverseName() + "'.", e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
+
+ @Override
+ public List<ExternalFile> getExternalFiles(JobId jobId, Dataset dataset) throws MetadataException, RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataset.getDataverseName(), dataset.getDatasetName());
+ ExternalFileTupleTranslator tupleReaderWriter = new ExternalFileTupleTranslator(false);
+ IValueExtractor<ExternalFile> valueExtractor = new MetadataEntityValueExtractor<ExternalFile>(
+ tupleReaderWriter);
+ List<ExternalFile> results = new ArrayList<ExternalFile>();
+ searchIndex(jobId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, searchKey, valueExtractor, results);
+ return results;
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
+
+ @Override
+ public void dropExternalFile(JobId jobId, String dataverseName, String datasetName, int fileNumber)
+ throws MetadataException, RemoteException {
+ try {
+ // Delete entry from the 'ExternalFile' dataset.
+ ITupleReference searchKey = createExternalFileSearchTuple(dataverseName, datasetName, fileNumber);
+ // Searches the index for the tuple to be deleted. Acquires an S
+ // lock on the 'ExternalFile' dataset.
+ ITupleReference datasetTuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET,
+ searchKey);
+ deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, datasetTuple);
+ } catch (TreeIndexException e) {
+ throw new MetadataException("Couldn't drop externalFile.", e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
+
+ @Override
+ public void dropExternalFiles(JobId jobId, Dataset dataset) throws MetadataException, RemoteException {
+ List<ExternalFile> files;
+ try {
+ files = getExternalFiles(jobId, dataset);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ try {
+ //loop through files and delete them
+ for (int i = 0; i < files.size(); i++) {
+ dropExternalFile(jobId, files.get(i).getDataverseName(), files.get(i).getDatasetName(), files.get(i)
+ .getFileNumber());
+ }
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
+
+ // This method is used to create a search tuple for external data file since the search tuple has an int value
+ @SuppressWarnings("unchecked")
+ public ITupleReference createExternalFileSearchTuple(String dataverseName, String datasetName, int fileNumber)
+ throws HyracksDataException {
+ ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ASTRING);
+ ISerializerDeserializer<AInt32> intSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT32);
+
+ AMutableString aString = new AMutableString("");
+ ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(3);
+
+ //dataverse field
+ aString.setValue(dataverseName);
+ stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+ tupleBuilder.addFieldEndOffset();
+
+ //dataset field
+ aString.setValue(datasetName);
+ stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+ tupleBuilder.addFieldEndOffset();
+
+ //file number field
+ intSerde.serialize(new AInt32(fileNumber), tupleBuilder.getDataOutput());
+ tupleBuilder.addFieldEndOffset();
+
+ ArrayTupleReference tuple = new ArrayTupleReference();
+ tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+ return tuple;
+ }
+
+ @Override
+ public ExternalFile getExternalFile(JobId jobId, String dataverseName, String datasetName, Integer fileNumber)
+ throws MetadataException, RemoteException {
+ try {
+ ITupleReference searchKey = createExternalFileSearchTuple(dataverseName, datasetName, fileNumber);
+ ExternalFileTupleTranslator tupleReaderWriter = new ExternalFileTupleTranslator(false);
+ IValueExtractor<ExternalFile> valueExtractor = new MetadataEntityValueExtractor<ExternalFile>(
+ tupleReaderWriter);
+ List<ExternalFile> results = new ArrayList<ExternalFile>();
+ searchIndex(jobId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, searchKey, valueExtractor, results);
+ if (results.isEmpty()) {
+ return null;
+ }
+ return results.get(0);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
+
+ @Override
+ public void updateDataset(JobId jobId, Dataset dataset) throws MetadataException, RemoteException {
+ try {
+ // This method will delete previous entry of the dataset and insert the new one
+ // Delete entry from the 'datasets' dataset.
+ ITupleReference searchKey;
+ searchKey = createTuple(dataset.getDataverseName(), dataset.getDatasetName());
+ // Searches the index for the tuple to be deleted. Acquires an S
+ // lock on the 'dataset' dataset.
+ ITupleReference datasetTuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey);
+ deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
+ // Previous tuple was deleted
+ // Insert into the 'dataset' dataset.
+ DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(true);
+ datasetTuple = tupleReaderWriter.getTupleFromMetadataEntity(dataset);
+ insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataManager.java
index a605b02..59e255e 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataManager.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataManager.java
@@ -28,6 +28,7 @@
import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
import edu.uci.ics.asterix.metadata.entities.Datatype;
import edu.uci.ics.asterix.metadata.entities.Dataverse;
+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.FeedActivity.FeedActivityType;
@@ -607,4 +608,69 @@
public List<Library> getDataverseLibraries(MetadataTransactionContext ctx, String dataverseName)
throws MetadataException;
+ /**
+ * @param mdTxnCtx
+ * MetadataTransactionContext of an active metadata transaction.
+ * @param externalFile
+ * An instance of type ExternalFile that represents the external file being
+ * added
+ * @throws MetadataException
+ */
+ public void addExternalFile(MetadataTransactionContext mdTxnCtx, ExternalFile externalFile) throws MetadataException;
+
+ /**
+ * @param mdTxnCtx
+ * MetadataTransactionContext of an active metadata transaction.
+ * @param dataset
+ * An instance of type Dataset that represents the "external" dataset
+ * @return A list of external files belonging to the dataset
+ * @throws MetadataException
+ */
+ public List<ExternalFile> getDatasetExternalFiles(MetadataTransactionContext mdTxnCtx, Dataset dataset)
+ throws MetadataException;
+
+ /**
+ * @param mdTxnCtx
+ * MetadataTransactionContext of an active metadata transaction.
+ * @param externalFile
+ * An instance of type ExternalFile that represents the external file being
+ * dropped
+ * @throws MetadataException
+ */
+ public void dropExternalFile(MetadataTransactionContext mdTxnCtx, ExternalFile externalFile) throws MetadataException;
+
+ /**
+ * @param mdTxnCtx
+ * MetadataTransactionContext of an active metadata transaction.
+ * @param dataset
+ * An instance of type dataset that owns the external files being
+ * dropped
+ * @throws MetadataException
+ */
+ public void dropDatasetExternalFiles(MetadataTransactionContext mdTxnCtx, Dataset dataset) throws MetadataException;
+
+ /**
+ * Get en external file
+ * @param mdTxnCtx
+ * @param dataverseName
+ * @param datasetName
+ * @param fileNumber
+ * @return
+ * @throws MetadataException
+ */
+ public ExternalFile getExternalFile(MetadataTransactionContext mdTxnCtx, String dataverseName, String datasetName, Integer fileNumber)
+ throws MetadataException;
+
+ /**
+ * update an existing dataset in metadata.
+ *
+ * @param ctx
+ * MetadataTransactionContext of an active metadata transaction.
+ * @param dataset
+ * Existing Dataset.
+ * @throws MetadataException
+ * For example, if the dataset already exists.
+ */
+ public void updateDataset(MetadataTransactionContext ctx, Dataset dataset) throws MetadataException;
+
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataNode.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataNode.java
index 45a350b..56fbcb5 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataNode.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataNode.java
@@ -30,6 +30,7 @@
import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
import edu.uci.ics.asterix.metadata.entities.Datatype;
import edu.uci.ics.asterix.metadata.entities.Dataverse;
+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.FeedActivity.FeedActivityType;
@@ -675,5 +676,90 @@
*/
public List<FeedActivity> getDatasetsServedByFeed(JobId jobId, String dataverseName, String deedName)
throws MetadataException, RemoteException;
+
+ /**
+ * @param jobId
+ * A globally unique id for an active metadata transaction.
+ * @param externalFile
+ * An object representing the external file entity
+ * @throws MetadataException
+ * for example, if the file already exists.
+ * @throws RemoteException
+ */
+ public void addExternalFile(JobId jobId, ExternalFile externalFile) throws MetadataException, RemoteException;
+
+ /**
+ * @param jobId
+ * A globally unique id for an active metadata transaction.
+ * @param dataset
+ * A dataset the files belongs to.
+ * @throws MetadataException
+ * @throws RemoteException
+ */
+ public List<ExternalFile> getExternalFiles(JobId jobId, Dataset dataset) throws MetadataException, RemoteException;
+
+ /**
+ * Deletes an externalFile , acquiring local locks on behalf of the given
+ * transaction id.
+ *
+ * @param jobId
+ * A globally unique id for an active metadata transaction.
+ * @param dataverseName
+ * dataverse asociated with the external dataset that owns the file to be deleted.
+ * @param datasetName
+ * Name of dataset owning the file to be deleted.
+ * @param fileNumber
+ * the id number for the file to be deleted
+ * @throws RemoteException
+ */
+ public void dropExternalFile(JobId jobId, String dataverseName, String datasetName, int fileNumber)
+ throws MetadataException, RemoteException;
+
+ /**
+ * Deletes all external files belonging to a dataset, acquiring local locks on behalf of the given
+ * transaction id.
+ *
+ * @param jobId
+ * A globally unique id for an active metadata transaction.
+ * @param dataset
+ * An external dataset the files belong to.
+ * @throws RemoteException
+ */
+ public void dropExternalFiles(JobId jobId, Dataset dataset) throws MetadataException, RemoteException;
+
+ /**
+ * Retrieves the file with given number, in given dataverse and dataset,
+ * acquiring local locks on behalf of the given transaction id.
+ *
+ * @param jobId
+ * A globally unique id for an active metadata transaction.
+ * @param dataverseName
+ * Name of the datavers holding the given dataset.
+ * @param datasetName
+ * Name of the dataset holding the index.
+ * @param fileNumber
+ * Number of the file
+ * @return An ExternalFile instance.
+ * @throws MetadataException
+ * For example, if the index does not exist.
+ * @throws RemoteException
+ */
+ public ExternalFile getExternalFile(JobId jobId, String dataverseName, String datasetName, Integer fileNumber)
+ throws MetadataException, RemoteException;
+
+
+ /**
+ * update an existing dataset in the metadata, acquiring local locks on behalf
+ * of the given transaction id.
+ *
+ * @param jobId
+ * A globally unique id for an active metadata transaction.
+ * @param dataset
+ * updated Dataset instance.
+ * @throws MetadataException
+ * For example, if the dataset already exists.
+ * @throws RemoteException
+ */
+ public void updateDataset(JobId jobId, Dataset dataset) throws MetadataException, RemoteException;
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
index 478e548..a299d0ae 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -49,6 +49,7 @@
import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
import edu.uci.ics.asterix.metadata.entities.Datatype;
import edu.uci.ics.asterix.metadata.entities.Dataverse;
+import edu.uci.ics.asterix.metadata.entities.ExternalFile;
import edu.uci.ics.asterix.metadata.entities.FeedPolicy;
import edu.uci.ics.asterix.metadata.entities.Index;
import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
@@ -123,7 +124,8 @@
MetadataPrimaryIndexes.NODEGROUP_DATASET, MetadataPrimaryIndexes.FUNCTION_DATASET,
MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, MetadataPrimaryIndexes.FEED_DATASET,
MetadataPrimaryIndexes.FEED_ACTIVITY_DATASET, MetadataPrimaryIndexes.FEED_POLICY_DATASET,
- MetadataPrimaryIndexes.LIBRARY_DATASET, MetadataPrimaryIndexes.COMPACTION_POLICY_DATASET };
+ MetadataPrimaryIndexes.LIBRARY_DATASET, MetadataPrimaryIndexes.COMPACTION_POLICY_DATASET,
+ MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET };
secondaryIndexes = new IMetadataIndex[] { MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX,
MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX,
@@ -487,9 +489,27 @@
}
}
}
+ if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+ // if the dataset has no indexes, delete all its files
+ List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName,
+ datasetName);
+ if (indexes.size() == 0) {
+ List<ExternalFile> files = MetadataManager.INSTANCE.getDatasetExternalFiles(mdTxnCtx,
+ dataset);
+ for (ExternalFile file : files) {
+ MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, file);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Dropped an external file: " + dataverseName + "." + datasetName
+ + "." + file.getFileNumber());
+ }
+ }
+ }
+ }
}
}
}
+ // the commit wasn't there before. yet, everything was working correctly!!!!!!!!!!!
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Completed DDL recovery.");
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
index 632b6fd..ce17321 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
@@ -38,6 +38,7 @@
public static IMetadataIndex FEED_ACTIVITY_DATASET;
public static IMetadataIndex FEED_POLICY_DATASET;
public static IMetadataIndex COMPACTION_POLICY_DATASET;
+ public static IMetadataIndex EXTERNAL_FILE_DATASET;
public static final int METADATA_DATASET_ID = 0;
public static final int DATAVERSE_DATASET_ID = 1;
@@ -54,6 +55,7 @@
public static final int FEED_ACTIVITY_DATASET_ID = 11;
public static final int FEED_POLICY_DATASET_ID = 12;
public static final int COMPACTION_POLICY_DATASET_ID = 13;
+ public static final int EXTERNAL_FILE_DATASET_ID = 14;
public static final int FIRST_AVAILABLE_USER_DATASET_ID = 100;
@@ -126,5 +128,10 @@
BuiltinType.ASTRING }, new String[] { "DataverseName", "CompactionPolicy" }, 0,
MetadataRecordTypes.COMPACTION_POLICY_RECORDTYPE, COMPACTION_POLICY_DATASET_ID, true,
new int[] { 0, 1 });
+
+ EXTERNAL_FILE_DATASET = new MetadataIndex("ExternalFile", null, 4, new IAType[] { BuiltinType.ASTRING,
+ BuiltinType.ASTRING, BuiltinType.AINT32 },
+ new String[] { "DataverseName", "DatasetName", "FileNumber" }, 0,
+ MetadataRecordTypes.EXTERNAL_FILE_RECORDTYPE, EXTERNAL_FILE_DATASET_ID, true, new int[] { 0, 1, 2 });
}
}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataRecordTypes.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataRecordTypes.java
index c400daa..11c5370 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataRecordTypes.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataRecordTypes.java
@@ -56,6 +56,7 @@
public static ARecordType FEED_ACTIVITY_DETAILS_RECORDTYPE;
public static ARecordType LIBRARY_RECORDTYPE;
public static ARecordType COMPACTION_POLICY_RECORDTYPE;
+ public static ARecordType EXTERNAL_FILE_RECORDTYPE;
/**
* Create all metadata record types.
@@ -97,6 +98,8 @@
COMPACTION_POLICY_RECORDTYPE = createCompactionPolicyRecordType();
+ EXTERNAL_FILE_RECORDTYPE = createExternalFileRecordType();
+
} catch (AsterixException e) {
throw new MetadataException(e);
}
@@ -165,12 +168,21 @@
// external details.
public static final int EXTERNAL_DETAILS_ARECORD_DATASOURCE_ADAPTER_FIELD_INDEX = 0;
public static final int EXTERNAL_DETAILS_ARECORD_PROPERTIES_FIELD_INDEX = 1;
+ public static final int EXTERNAL_DETAILS_ARECORD_GROUPNAME_FIELD_INDEX = 2;
+ public static final int EXTERNAL_DETAILS_ARECORD_LAST_REFRESH_TIME_FIELD_INDEX = 3;
+ public static final int EXTERNAL_DETAILS_ARECORD_TRANSACTION_STATE_FIELD_INDEX = 4;
+ public static final int EXTERNAL_DETAILS_ARECORD_COMPACTION_POLICY_FIELD_INDEX = 5;
+ public static final int EXTERNAL_DETAILS_ARECORD_COMPACTION_POLICY_PROPERTIES_FIELD_INDEX = 6;
private static final ARecordType createExternalDetailsRecordType() throws AsterixException {
AOrderedListType orderedPropertyListType = new AOrderedListType(DATASOURCE_ADAPTER_PROPERTIES_RECORDTYPE, null);
- String[] fieldNames = { "DatasourceAdapter", "Properties" };
- IAType[] fieldTypes = { BuiltinType.ASTRING, orderedPropertyListType };
+ AOrderedListType compactionPolicyPropertyListType = new AOrderedListType(
+ COMPACTION_POLICY_PROPERTIES_RECORDTYPE, null);
+ String[] fieldNames = { "DatasourceAdapter", "Properties", "GroupName", "LastRefreshTime", "TransactionState",
+ "CompactionPolicy", "CompactionPolicyProperties" };
+ IAType[] fieldTypes = { BuiltinType.ASTRING, orderedPropertyListType, BuiltinType.ASTRING,
+ BuiltinType.ADATETIME, BuiltinType.AINT32, BuiltinType.ASTRING, compactionPolicyPropertyListType };
return new ARecordType(null, fieldNames, fieldTypes, true);
}
@@ -460,4 +472,20 @@
IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING };
return new ARecordType("LibraryRecordType", fieldNames, fieldTypes, true);
}
+
+ public static final int EXTERNAL_FILE_ARECORD_DATAVERSENAME_FIELD_INDEX = 0;
+ public static final int EXTERNAL_FILE_ARECORD_DATASET_NAME_FIELD_INDEX = 1;
+ public static final int EXTERNAL_FILE_ARECORD_FILE_NUMBER_FIELD_INDEX = 2;
+ public static final int EXTERNAL_FILE_ARECORD_FILE_NAME_FIELD_INDEX = 3;
+ public static final int EXTERNAL_FILE_ARECORD_FILE_SIZE_FIELD_INDEX = 4;
+ public static final int EXTERNAL_FILE_ARECORD_FILE_MOD_DATE_FIELD_INDEX = 5;
+ public static final int EXTERNAL_FILE_ARECORD_FILE_PENDING_OP_FIELD_INDEX = 6;
+
+ private static ARecordType createExternalFileRecordType() throws AsterixException {
+ String[] fieldNames = { "DataverseName", "DatasetName", "FileNumber", "FileName", "FileSize", "FileModTime",
+ "PendingOp" };
+ IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.AINT32, BuiltinType.ASTRING,
+ BuiltinType.AINT64, BuiltinType.ADATETIME, BuiltinType.AINT32 };
+ return new ARecordType("ExternalFileRecordType", fieldNames, fieldTypes, true);
+ }
}
\ No newline at end of file
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 fc73854..50d9ab2 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
@@ -19,6 +19,7 @@
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
+import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.logging.Level;
@@ -26,6 +27,7 @@
import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
+import edu.uci.ics.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
import edu.uci.ics.asterix.common.config.GlobalConfig;
import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
@@ -36,6 +38,7 @@
import edu.uci.ics.asterix.common.dataflow.IAsterixApplicationContextInfo;
import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeWithBuddyIOOperationCallbackFactory;
import edu.uci.ics.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
import edu.uci.ics.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
import edu.uci.ics.asterix.common.parse.IParseFileSplitsDecl;
@@ -45,6 +48,7 @@
import edu.uci.ics.asterix.formats.base.IDataFormat;
import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
+import edu.uci.ics.asterix.metadata.IDatasetDetails;
import edu.uci.ics.asterix.metadata.MetadataException;
import edu.uci.ics.asterix.metadata.MetadataManager;
import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
@@ -57,12 +61,13 @@
import edu.uci.ics.asterix.metadata.entities.Datatype;
import edu.uci.ics.asterix.metadata.entities.Dataverse;
import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
+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.FeedActivity.FeedActivityDetails;
import edu.uci.ics.asterix.metadata.entities.FeedPolicy;
import edu.uci.ics.asterix.metadata.entities.Index;
-import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
+import edu.uci.ics.asterix.metadata.external.IndexingConstants;
import edu.uci.ics.asterix.metadata.feeds.BuiltinFeedPolicies;
import edu.uci.ics.asterix.metadata.feeds.EndFeedMessage;
import edu.uci.ics.asterix.metadata.feeds.ExternalDataScanOperatorDescriptor;
@@ -75,6 +80,7 @@
import edu.uci.ics.asterix.metadata.feeds.IGenericAdapterFactory;
import edu.uci.ics.asterix.metadata.feeds.ITypedAdapterFactory;
import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
+import edu.uci.ics.asterix.metadata.utils.ExternalDatasetsRegistry;
import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.ATypeTag;
@@ -83,6 +89,8 @@
import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
import edu.uci.ics.asterix.runtime.base.AsterixTupleFilterFactory;
+import edu.uci.ics.asterix.runtime.external.ExternalBTreeSearchOperatorDescriptor;
+import edu.uci.ics.asterix.runtime.external.ExternalRTreeSearchOperatorDescriptor;
import edu.uci.ics.asterix.runtime.formats.FormatUtils;
import edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat;
import edu.uci.ics.asterix.runtime.job.listener.JobEventListenerFactory;
@@ -146,11 +154,13 @@
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.LSMInvertedIndexDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.PartitionedLSMInvertedIndexDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.dataflow.ExternalRTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.lsm.rtree.dataflow.LSMRTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.rtree.dataflow.RTreeSearchOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
@@ -172,6 +182,7 @@
private final Dataverse defaultDataverse;
private JobId jobId;
+ private Map<String, Integer> locks;
private final AsterixStorageProperties storageProperties;
@@ -323,14 +334,14 @@
IAType itemType = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataset.getDataverseName(), itemTypeName)
.getDatatype();
return buildExternalDataScannerRuntime(jobSpec, itemType, (ExternalDatasetDetails) dataset.getDatasetDetails(),
- NonTaggedDataFormat.INSTANCE);
+ NonTaggedDataFormat.INSTANCE, dataset);
}
@SuppressWarnings("rawtypes")
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildExternalDataScannerRuntime(
- JobSpecification jobSpec, IAType itemType, ExternalDatasetDetails datasetDetails, IDataFormat format)
- throws AlgebricksException {
+ 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.");
}
@@ -356,6 +367,23 @@
Map<String, String> configuration = datasetDetails.getProperties();
+ // check to see if dataset is indexed
+ Index filesIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
+ dataset.getDatasetName(),
+ dataset.getDatasetName().concat(IndexingConstants.EXTERNAL_FILE_INDEX_NAME_SUFFIX));
+
+ if (filesIndex != null && filesIndex.getPendingOp() == 0) {
+ // get files
+ List<ExternalFile> files = MetadataManager.INSTANCE.getDatasetExternalFiles(mdTxnCtx, dataset);
+ Iterator<ExternalFile> iterator = files.iterator();
+ while (iterator.hasNext()) {
+ if (iterator.next().getPendingOp() != ExternalFilePendingOp.PENDING_NO_OP) {
+ iterator.remove();
+ }
+ }
+ ((IGenericAdapterFactory) adapterFactory).setFiles(files);
+ }
+
switch (adapterFactory.getAdapterType()) {
case GENERIC:
((IGenericAdapterFactory) adapterFactory).configure(configuration, (ARecordType) itemType);
@@ -493,10 +521,11 @@
int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
Object implConfig) throws AlgebricksException {
boolean isSecondary = true;
+ int numSecondaryKeys = 0;
try {
Index primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
dataset.getDatasetName(), dataset.getDatasetName());
- if (primaryIndex != null) {
+ if (primaryIndex != null && dataset.getDatasetType() != DatasetType.EXTERNAL) {
isSecondary = !indexName.equals(primaryIndex.getIndexName());
}
int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
@@ -507,7 +536,7 @@
if (isSecondary) {
Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
dataset.getDatasetName(), indexName);
- int numSecondaryKeys = secondaryIndex.getKeyFieldNames().size();
+ numSecondaryKeys = secondaryIndex.getKeyFieldNames().size();
bloomFilterKeyFields = new int[numSecondaryKeys];
for (int i = 0; i < numSecondaryKeys; i++) {
bloomFilterKeyFields[i] = i;
@@ -532,7 +561,7 @@
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc;
try {
- spPc = splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataset.getDataverseName(),
+ spPc = splitProviderAndPartitionConstraintsForDataset(dataset.getDataverseName(),
dataset.getDatasetName(), indexName);
} catch (Exception e) {
throw new AlgebricksException(e);
@@ -562,16 +591,34 @@
Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils.getMergePolicyFactory(
dataset, mdTxnCtx);
AsterixRuntimeComponentsProvider rtcProvider = AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
- BTreeSearchOperatorDescriptor btreeSearchOp = new BTreeSearchOperatorDescriptor(jobSpec, outputRecDesc,
- appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(), spPc.first,
- typeTraits, comparatorFactories, bloomFilterKeyFields, lowKeyFields, highKeyFields,
- lowKeyInclusive, highKeyInclusive, new LSMBTreeDataflowHelperFactory(
- new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), compactionInfo.first,
- compactionInfo.second, isSecondary ? new SecondaryIndexOperationTrackerProvider(
- dataset.getDatasetId()) : new PrimaryIndexOperationTrackerProvider(
- dataset.getDatasetId()), rtcProvider, LSMBTreeIOOperationCallbackFactory.INSTANCE,
- storageProperties.getBloomFilterFalsePositiveRate(), !isSecondary), retainInput,
- retainNull, context.getNullWriterFactory(), searchCallbackFactory);
+ BTreeSearchOperatorDescriptor btreeSearchOp;
+ if (dataset.getDatasetType() == DatasetType.INTERNAL) {
+ btreeSearchOp = new BTreeSearchOperatorDescriptor(jobSpec, outputRecDesc,
+ appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(),
+ spPc.first, typeTraits, comparatorFactories, bloomFilterKeyFields, lowKeyFields, highKeyFields,
+ lowKeyInclusive, highKeyInclusive, new LSMBTreeDataflowHelperFactory(
+ new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), compactionInfo.first,
+ compactionInfo.second, isSecondary ? new SecondaryIndexOperationTrackerProvider(
+ dataset.getDatasetId()) : new PrimaryIndexOperationTrackerProvider(
+ dataset.getDatasetId()), rtcProvider,
+ LSMBTreeIOOperationCallbackFactory.INSTANCE, getStorageProperties()
+ .getBloomFilterFalsePositiveRate(), !isSecondary), retainInput, retainNull,
+ context.getNullWriterFactory(), searchCallbackFactory);
+ } else {
+ // External dataset <- use the btree with buddy btree->
+ // Be Careful of Key Start Index ?
+ int[] buddyBreeFields = new int[] { numSecondaryKeys };
+ ExternalBTreeWithBuddyDataflowHelperFactory indexDataflowHelperFactory = new ExternalBTreeWithBuddyDataflowHelperFactory(
+ compactionInfo.first, compactionInfo.second, new SecondaryIndexOperationTrackerProvider(
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMBTreeWithBuddyIOOperationCallbackFactory.INSTANCE, getStorageProperties()
+ .getBloomFilterFalsePositiveRate(), buddyBreeFields,
+ ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, this));
+ btreeSearchOp = new ExternalBTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, rtcProvider,
+ rtcProvider, spPc.first, typeTraits, comparatorFactories, bloomFilterKeyFields, lowKeyFields,
+ highKeyFields, lowKeyInclusive, highKeyInclusive, indexDataflowHelperFactory, retainInput,
+ retainNull, context.getNullWriterFactory(), searchCallbackFactory);
+ }
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeSearchOp, spPc.second);
@@ -580,6 +627,24 @@
}
}
+ /* BTreeSearchOperatorDescriptor btreeSearchOp = new BTreeSearchOperatorDescriptor(jobSpec, outputRecDesc,
+ appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(), spPc.first,
+ typeTraits, comparatorFactories, bloomFilterKeyFields, lowKeyFields, highKeyFields,
+ lowKeyInclusive, highKeyInclusive, new LSMBTreeDataflowHelperFactory(
+ new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), compactionInfo.first,
+ compactionInfo.second, isSecondary ? new SecondaryIndexOperationTrackerProvider(
+ dataset.getDatasetId()) : new PrimaryIndexOperationTrackerProvider(
+ dataset.getDatasetId()), rtcProvider, LSMBTreeIOOperationCallbackFactory.INSTANCE,
+ storageProperties.getBloomFilterFalsePositiveRate(), !isSecondary), retainInput,
+ retainNull, context.getNullWriterFactory(), searchCallbackFactory);
+
+ return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeSearchOp, spPc.second);
+
+ } catch (MetadataException me) {
+ throw new AlgebricksException(me);
+ }
+ }*/
+
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildRtreeRuntime(JobSpecification jobSpec,
List<LogicalVariable> outputVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
JobGenContext context, boolean retainInput, boolean retainNull, Dataset dataset, String indexName,
@@ -615,6 +680,7 @@
}
RecordDescriptor outputRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
+ // IS NOT THE VARIABLE BELOW ALWAYS = 0 ??
int keysStartIndex = outputRecDesc.getFieldCount() - numNestedSecondaryKeyFields - numPrimaryKeys;
if (retainInput) {
keysStartIndex -= numNestedSecondaryKeyFields;
@@ -624,7 +690,7 @@
ITypeTraits[] typeTraits = JobGenHelper.variablesToTypeTraits(outputVars, keysStartIndex,
numNestedSecondaryKeyFields + numPrimaryKeys, typeEnv, context);
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc = splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc = splitProviderAndPartitionConstraintsForDataset(
dataset.getDataverseName(), dataset.getDatasetName(), indexName);
IBinaryComparatorFactory[] primaryComparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
@@ -638,17 +704,38 @@
Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils.getMergePolicyFactory(
dataset, mdTxnCtx);
ISearchOperationCallbackFactory searchCallbackFactory = new SecondaryIndexSearchOperationCallbackFactory();
- RTreeSearchOperatorDescriptor rtreeSearchOp = new RTreeSearchOperatorDescriptor(jobSpec, outputRecDesc,
- appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(), spPc.first,
- typeTraits, comparatorFactories, keyFields, 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(),
- btreeFields), retainInput, retainNull, context.getNullWriterFactory(),
- searchCallbackFactory);
+
+ RTreeSearchOperatorDescriptor rtreeSearchOp;
+ if (dataset.getDatasetType() == DatasetType.INTERNAL) {
+ rtreeSearchOp = new RTreeSearchOperatorDescriptor(jobSpec, outputRecDesc,
+ appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(),
+ spPc.first, typeTraits, comparatorFactories, keyFields, 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), getStorageProperties()
+ .getBloomFilterFalsePositiveRate(), btreeFields), retainInput, retainNull,
+ context.getNullWriterFactory(), searchCallbackFactory);
+
+ } else {
+ // External Dataset
+ ExternalRTreeDataflowHelperFactory indexDataflowHelperFactory = new ExternalRTreeDataflowHelperFactory(
+ valueProviderFactories, RTreePolicyType.RTREE,
+ IndexingConstants.getBuddyBtreeComparatorFactories(), compactionInfo.first,
+ compactionInfo.second, new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMRTreeIOOperationCallbackFactory.INSTANCE,
+ proposeLinearizer(nestedKeyType.getTypeTag(), comparatorFactories.length),
+ getStorageProperties().getBloomFilterFalsePositiveRate(),
+ new int[] { numNestedSecondaryKeyFields },
+ ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, this));
+ // Create the operator
+ rtreeSearchOp = new ExternalRTreeSearchOperatorDescriptor(jobSpec, outputRecDesc,
+ appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(),
+ spPc.first, typeTraits, comparatorFactories, keyFields, indexDataflowHelperFactory,
+ retainInput, retainNull, context.getNullWriterFactory(), searchCallbackFactory);
+ }
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(rtreeSearchOp, spPc.second);
} catch (MetadataException me) {
@@ -696,9 +783,6 @@
public IDataSourceIndex<String, AqlSourceId> findDataSourceIndex(String indexId, AqlSourceId dataSourceId)
throws AlgebricksException {
AqlDataSource ads = findDataSource(dataSourceId);
- if (ads.getDatasourceType() == AqlDataSourceType.EXTERNAL_DATASET) {
- throw new AlgebricksException("No index for external dataset " + dataSourceId);
- }
Dataset dataset = ((DatasetDataSource) ads).getDataset();
try {
@@ -785,7 +869,7 @@
IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
itemType, context.getBinaryComparatorFactoryProvider());
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForDataset(
dataSource.getId().getDataverseName(), datasetName, indexName);
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
@@ -852,7 +936,7 @@
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
itemType, context.getBinaryComparatorFactoryProvider());
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForDataset(
dataSource.getId().getDataverseName(), datasetName, indexName);
// prepare callback
@@ -1051,7 +1135,7 @@
}
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForDataset(
dataverseName, datasetName, indexName);
// prepare callback
@@ -1179,7 +1263,7 @@
secondaryKeyType.getTypeTag(), indexType, secondaryIndex.getGramLength());
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForDataset(
dataverseName, datasetName, indexName);
// prepare callback
@@ -1281,7 +1365,7 @@
IBinaryComparatorFactory[] primaryComparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
dataset, recType, context.getBinaryComparatorFactoryProvider());
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForDataset(
dataverseName, datasetName, indexName);
int[] btreeFields = new int[primaryComparatorFactories.length];
for (int k = 0; k < btreeFields.length; k++) {
@@ -1356,11 +1440,9 @@
} else {
numElementsHint = Long.parseLong(numElementsHintString);
}
-
int numPartitions = 0;
- InternalDatasetDetails datasetDetails = (InternalDatasetDetails) dataset.getDatasetDetails();
- List<String> nodeGroup = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, datasetDetails.getNodeGroupName())
- .getNodeNames();
+ List<String> nodeGroup = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx,
+ dataset.getDatasetDetails().getNodeGroupName()).getNodeNames();
for (String nd : nodeGroup) {
numPartitions += AsterixClusterProperties.INSTANCE.getNumberOfIODevices(nd);
}
@@ -1372,9 +1454,9 @@
return AsterixBuiltinFunctions.lookupFunction(fid);
}
- public Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
+ public Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitProviderAndPartitionConstraintsForDataset(
String dataverseName, String datasetName, String targetIdxName) throws AlgebricksException {
- FileSplit[] splits = splitsForInternalOrFeedDataset(mdTxnCtx, dataverseName, datasetName, targetIdxName);
+ FileSplit[] splits = splitsForDataset(mdTxnCtx, dataverseName, datasetName, targetIdxName);
return splitProviderAndPartitionConstraints(splits);
}
@@ -1418,16 +1500,12 @@
return splits.toArray(new FileSplit[] {});
}
- private FileSplit[] splitsForInternalOrFeedDataset(MetadataTransactionContext mdTxnCtx, String dataverseName,
- String datasetName, String targetIdxName) throws AlgebricksException {
-
+ private FileSplit[] splitsForDataset(MetadataTransactionContext mdTxnCtx, String dataverseName, String datasetName,
+ String targetIdxName) throws AlgebricksException {
try {
File relPathFile = new File(getRelativePath(dataverseName, datasetName + "_idx_" + targetIdxName));
Dataset dataset = MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverseName, datasetName);
- if (dataset.getDatasetType() != DatasetType.INTERNAL) {
- throw new AlgebricksException("Not an internal dataset");
- }
- InternalDatasetDetails datasetDetails = (InternalDatasetDetails) dataset.getDatasetDetails();
+ IDatasetDetails datasetDetails = dataset.getDatasetDetails();
List<String> nodeGroup = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, datasetDetails.getNodeGroupName())
.getNodeNames();
if (nodeGroup == null) {
@@ -1597,4 +1675,73 @@
return wrappedProperties;
}
+ public Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitProviderAndPartitionConstraintsForFilesIndex(
+ String dataverseName, String datasetName, String targetIdxName, boolean create) throws AlgebricksException {
+ FileSplit[] splits = splitsForFilesIndex(mdTxnCtx, dataverseName, datasetName, targetIdxName, create);
+ return splitProviderAndPartitionConstraints(splits);
+ }
+
+ private FileSplit[] splitsForFilesIndex(MetadataTransactionContext mdTxnCtx, String dataverseName,
+ String datasetName, String targetIdxName, boolean create) throws AlgebricksException {
+
+ try {
+ File relPathFile = new File(getRelativePath(dataverseName, datasetName + "_idx_" + targetIdxName));
+ Dataset dataset = MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverseName, datasetName);
+ ExternalDatasetDetails datasetDetails = (ExternalDatasetDetails) dataset.getDatasetDetails();
+ List<String> nodeGroup = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, datasetDetails.getNodeGroupName())
+ .getNodeNames();
+ if (nodeGroup == null) {
+ throw new AlgebricksException("Couldn't find node group " + datasetDetails.getNodeGroupName());
+ }
+
+ List<FileSplit> splitArray = new ArrayList<FileSplit>();
+ for (String nd : nodeGroup) {
+ String[] nodeStores = stores.get(nd);
+ if (nodeStores == null) {
+ LOGGER.warning("Node " + nd + " has no stores.");
+ throw new AlgebricksException("Node " + nd + " has no stores.");
+ } else {
+ // Only the first partition when create
+ String[] ioDevices = AsterixClusterProperties.INSTANCE.getIODevices(nd);
+ if (create) {
+ for (int j = 0; j < nodeStores.length; j++) {
+ File f = new File(ioDevices[0] + File.separator + nodeStores[j] + File.separator
+ + relPathFile);
+ splitArray.add(new FileSplit(nd, new FileReference(f), 0));
+ }
+ } else {
+ int numIODevices = AsterixClusterProperties.INSTANCE.getNumberOfIODevices(nd);
+ for (int j = 0; j < nodeStores.length; j++) {
+ for (int k = 0; k < numIODevices; k++) {
+ File f = new File(ioDevices[0] + File.separator + nodeStores[j] + File.separator
+ + relPathFile);
+ splitArray.add(new FileSplit(nd, new FileReference(f), 0));
+ }
+ }
+ }
+ }
+ }
+ FileSplit[] splits = new FileSplit[splitArray.size()];
+ int i = 0;
+ for (FileSplit fs : splitArray) {
+ splits[i++] = fs;
+ }
+ return splits;
+ } catch (MetadataException me) {
+ throw new AlgebricksException(me);
+ }
+ }
+
+ public AsterixStorageProperties getStorageProperties() {
+ return storageProperties;
+ }
+
+ public Map<String, Integer> getLocks() {
+ return locks;
+ }
+
+ public void setLocks(Map<String, Integer> locks) {
+ this.locks = locks;
+ }
+
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/ExternalDatasetDetails.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/ExternalDatasetDetails.java
index 18cef340..3255b34 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/ExternalDatasetDetails.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/ExternalDatasetDetails.java
@@ -17,16 +17,20 @@
import java.io.DataOutput;
import java.io.IOException;
+import java.util.Date;
import java.util.Map;
import edu.uci.ics.asterix.builders.IARecordBuilder;
import edu.uci.ics.asterix.builders.OrderedListBuilder;
import edu.uci.ics.asterix.builders.RecordBuilder;
import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
+import edu.uci.ics.asterix.common.config.DatasetConfig.ExternalDatasetTransactionState;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import edu.uci.ics.asterix.metadata.IDatasetDetails;
import edu.uci.ics.asterix.metadata.bootstrap.MetadataRecordTypes;
+import edu.uci.ics.asterix.om.base.ADateTime;
+import edu.uci.ics.asterix.om.base.AInt32;
import edu.uci.ics.asterix.om.base.AMutableString;
import edu.uci.ics.asterix.om.base.AString;
import edu.uci.ics.asterix.om.types.AOrderedListType;
@@ -41,13 +45,21 @@
private static final long serialVersionUID = 1L;
private final String adapter;
private final Map<String, String> properties;
+ private final String nodeGroupName;
+ private Date lastRefreshTime;
+ private ExternalDatasetTransactionState state;
+ protected String compactionPolicy;
+ protected Map<String, String> compactionPolicyProperties;
- private final static ARecordType externalRecordType = MetadataRecordTypes.EXTERNAL_DETAILS_RECORDTYPE;
- private final static ARecordType propertyRecordType = MetadataRecordTypes.DATASOURCE_ADAPTER_PROPERTIES_RECORDTYPE;
-
- public ExternalDatasetDetails(String adapter, Map<String, String> properties) {
+ public ExternalDatasetDetails(String adapter, Map<String, String> properties, String nodeGroupName, Date lastRefreshTime, ExternalDatasetTransactionState state,
+ String compactionPolicy, Map<String, String> compactionPolicyProperties) {
this.properties = properties;
this.adapter = adapter;
+ this.nodeGroupName = nodeGroupName;
+ this.lastRefreshTime = lastRefreshTime;
+ this.state = state;
+ this.compactionPolicy = compactionPolicy;
+ this.compactionPolicyProperties = compactionPolicyProperties;
}
public String getAdapter() {
@@ -63,16 +75,22 @@
return DatasetType.EXTERNAL;
}
- @Override
+ @SuppressWarnings("unchecked")
+ @Override
public void writeDatasetDetailsRecordType(DataOutput out) throws HyracksDataException {
IARecordBuilder externalRecordBuilder = new RecordBuilder();
OrderedListBuilder listBuilder = new OrderedListBuilder();
ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
ArrayBackedValueStorage itemValue = new ArrayBackedValueStorage();
- externalRecordBuilder.reset(externalRecordType);
+ externalRecordBuilder.reset(MetadataRecordTypes.EXTERNAL_DETAILS_RECORDTYPE);
AMutableString aString = new AMutableString("");
+
ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(BuiltinType.ASTRING);
+ ISerializerDeserializer<ADateTime> dateTimeSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ADATETIME);
+ ISerializerDeserializer<AInt32> intSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT32);
// write field 0
fieldValue.reset();
@@ -82,18 +100,57 @@
fieldValue);
// write field 1
- listBuilder.reset((AOrderedListType) externalRecordType.getFieldTypes()[1]);
+ listBuilder.reset((AOrderedListType) MetadataRecordTypes.EXTERNAL_DETAILS_RECORDTYPE.getFieldTypes()[1]);
for (Map.Entry<String, String> property : this.properties.entrySet()) {
String name = property.getKey();
String value = property.getValue();
itemValue.reset();
- writePropertyTypeRecord(name, value, itemValue.getDataOutput());
+ writePropertyTypeRecord(name, value, itemValue.getDataOutput(), MetadataRecordTypes.DATASOURCE_ADAPTER_PROPERTIES_RECORDTYPE);
listBuilder.addItem(itemValue);
}
fieldValue.reset();
listBuilder.write(fieldValue.getDataOutput(), true);
externalRecordBuilder.addField(MetadataRecordTypes.EXTERNAL_DETAILS_ARECORD_PROPERTIES_FIELD_INDEX, fieldValue);
+ // write field 2
+ fieldValue.reset();
+ aString.setValue(getNodeGroupName());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ externalRecordBuilder.addField(MetadataRecordTypes.EXTERNAL_DETAILS_ARECORD_GROUPNAME_FIELD_INDEX, fieldValue);
+
+
+ // write field 3
+ fieldValue.reset();
+ dateTimeSerde.serialize(new ADateTime(lastRefreshTime.getTime()), fieldValue.getDataOutput());
+ externalRecordBuilder.addField(MetadataRecordTypes.EXTERNAL_DETAILS_ARECORD_LAST_REFRESH_TIME_FIELD_INDEX, fieldValue);
+
+ // write field 4
+ fieldValue.reset();
+ intSerde.serialize(new AInt32(state.ordinal()), fieldValue.getDataOutput());
+ externalRecordBuilder.addField(MetadataRecordTypes.EXTERNAL_DETAILS_ARECORD_TRANSACTION_STATE_FIELD_INDEX, fieldValue);
+
+ // write field 6
+ fieldValue.reset();
+ aString.setValue(getCompactionPolicy().toString());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ externalRecordBuilder.addField(MetadataRecordTypes.EXTERNAL_DETAILS_ARECORD_COMPACTION_POLICY_FIELD_INDEX,
+ fieldValue);
+
+ // write field 7
+ listBuilder
+ .reset((AOrderedListType) MetadataRecordTypes.EXTERNAL_DETAILS_RECORDTYPE.getFieldTypes()[MetadataRecordTypes.EXTERNAL_DETAILS_ARECORD_COMPACTION_POLICY_PROPERTIES_FIELD_INDEX]);
+ for (Map.Entry<String, String> property : compactionPolicyProperties.entrySet()) {
+ String name = property.getKey();
+ String value = property.getValue();
+ itemValue.reset();
+ writePropertyTypeRecord(name, value, itemValue.getDataOutput(),
+ MetadataRecordTypes.COMPACTION_POLICY_PROPERTIES_RECORDTYPE);
+ listBuilder.addItem(itemValue);
+ }
+ fieldValue.reset();
+ listBuilder.write(fieldValue.getDataOutput(), true);
+ externalRecordBuilder.addField(
+ MetadataRecordTypes.EXTERNAL_DETAILS_ARECORD_COMPACTION_POLICY_PROPERTIES_FIELD_INDEX, fieldValue);
try {
externalRecordBuilder.write(out, true);
} catch (IOException | AsterixException e) {
@@ -102,10 +159,12 @@
}
- public void writePropertyTypeRecord(String name, String value, DataOutput out) throws HyracksDataException {
+ @SuppressWarnings("unchecked")
+ protected void writePropertyTypeRecord(String name, String value, DataOutput out, ARecordType recordType)
+ throws HyracksDataException {
IARecordBuilder propertyRecordBuilder = new RecordBuilder();
ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
- propertyRecordBuilder.reset(propertyRecordType);
+ propertyRecordBuilder.reset(recordType);
AMutableString aString = new AMutableString("");
ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(BuiltinType.ASTRING);
@@ -128,4 +187,35 @@
throw new HyracksDataException(e);
}
}
+
+ @Override
+ public String getNodeGroupName() {
+ return nodeGroupName;
+ }
+
+ public Date getTimestamp() {
+ return lastRefreshTime;
+ }
+
+ public void setRefreshTimestamp(Date timestamp) {
+ this.lastRefreshTime = timestamp;
+ }
+
+ public ExternalDatasetTransactionState getState() {
+ return state;
+ }
+
+ public void setState(ExternalDatasetTransactionState state) {
+ this.state = state;
+ }
+
+ @Override
+ public String getCompactionPolicy() {
+ return compactionPolicy;
+ }
+
+ @Override
+ public Map<String, String> getCompactionPolicyProperties() {
+ return compactionPolicyProperties;
+ }
}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/ExternalFile.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/ExternalFile.java
new file mode 100644
index 0000000..8f2fe3a
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/ExternalFile.java
@@ -0,0 +1,140 @@
+
+/*
+ * 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.metadata.entities;
+
+import java.util.Date;
+
+import edu.uci.ics.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
+import edu.uci.ics.asterix.metadata.MetadataCache;
+import edu.uci.ics.asterix.metadata.api.IMetadataEntity;
+
+public class ExternalFile implements IMetadataEntity, Comparable<ExternalFile>{
+
+ /**
+ * A class for metadata entity externalFile
+ * This class represents an external dataset file and is intended for use for saving external data snapshot
+ */
+ private static final long serialVersionUID = 1L;
+
+ private String dataverseName;
+ private String datasetName;
+ private Date lastModefiedTime;
+ private long size;
+ private String fileName;
+ private int fileNumber;
+ private ExternalFilePendingOp pendingOp;
+
+ public ExternalFile(String dataverseName, String datasetName, int fileNumber, String fileName, Date lastModefiedTime, long size, ExternalFilePendingOp pendingOp) {
+ this.dataverseName = dataverseName;
+ this.datasetName = datasetName;
+ this.fileNumber = fileNumber;
+ this.fileName = fileName;
+ this.lastModefiedTime = lastModefiedTime;
+ this.size = size;
+ this.setPendingOp(pendingOp);
+ }
+
+ public String getDataverseName() {
+ return dataverseName;
+ }
+
+ public void setDataverseName(String dataverseName) {
+ this.dataverseName = dataverseName;
+ }
+
+ public String getDatasetName() {
+ return datasetName;
+ }
+
+ public void setDatasetName(String datasetName) {
+ this.datasetName = datasetName;
+ }
+
+ public Date getLastModefiedTime() {
+ return lastModefiedTime;
+ }
+
+ public void setLastModefiedTime(Date lastModefiedTime) {
+ this.lastModefiedTime = lastModefiedTime;
+ }
+
+ public long getSize() {
+ return size;
+ }
+
+ public void setSize(long size) {
+ this.size = size;
+ }
+
+ public String getFileName() {
+ return fileName;
+ }
+
+ public void setFileName(String fileName) {
+ this.fileName = fileName;
+ }
+
+ public int getFileNumber() {
+ return fileNumber;
+ }
+
+ public void setFileNumber(int fileNumber) {
+ this.fileNumber = fileNumber;
+ }
+
+ public ExternalFilePendingOp getPendingOp() {
+ return pendingOp;
+ }
+
+ public void setPendingOp(ExternalFilePendingOp pendingOp) {
+ this.pendingOp = pendingOp;
+ }
+
+ @Override
+ public Object addToCache(MetadataCache cache) {
+ return null;
+ //return cache.addExternalFileIfNotExists(this);
+ }
+
+ @Override
+ public Object dropFromCache(MetadataCache cache) {
+ return null;
+ //return cache.dropExternalFile(this);
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj == null)
+ return false;
+ if (obj == this)
+ return true;
+ if (!(obj instanceof ExternalFile))
+ return false;
+ ExternalFile anotherFile = (ExternalFile) obj;
+ if (fileNumber != anotherFile.fileNumber)
+ return false;
+ if (!dataverseName.equals(anotherFile.dataverseName))
+ return false;
+ if (!datasetName.equals(anotherFile.datasetName))
+ return false;
+ return true;
+ }
+
+ @Override
+ public int compareTo(ExternalFile o) {
+ return this.fileNumber - o.getFileNumber();
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/InternalDatasetDetails.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/InternalDatasetDetails.java
index 212f525..b1ef68d 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/InternalDatasetDetails.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/InternalDatasetDetails.java
@@ -72,6 +72,7 @@
this.compactionPolicyProperties = compactionPolicyProperties;
}
+ @Override
public String getNodeGroupName() {
return nodeGroupName;
}
@@ -96,10 +97,12 @@
return partitioningStrategy;
}
+ @Override
public String getCompactionPolicy() {
return compactionPolicy;
}
+ @Override
public Map<String, String> getCompactionPolicyProperties() {
return compactionPolicyProperties;
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
index 4bc0994..8d8ff25 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
@@ -22,6 +22,7 @@
import java.io.IOException;
import java.util.ArrayList;
import java.util.Calendar;
+import java.util.Date;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
@@ -31,6 +32,7 @@
import edu.uci.ics.asterix.builders.RecordBuilder;
import edu.uci.ics.asterix.builders.UnorderedListBuilder;
import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
+import edu.uci.ics.asterix.common.config.DatasetConfig.ExternalDatasetTransactionState;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import edu.uci.ics.asterix.metadata.IDatasetDetails;
@@ -43,6 +45,7 @@
import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails.FileStructure;
import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails.PartitioningStrategy;
import edu.uci.ics.asterix.om.base.ABoolean;
+import edu.uci.ics.asterix.om.base.ADateTime;
import edu.uci.ics.asterix.om.base.AInt32;
import edu.uci.ics.asterix.om.base.AMutableInt32;
import edu.uci.ics.asterix.om.base.AMutableString;
@@ -176,7 +179,39 @@
.getStringValue();
properties.put(key, value);
}
- datasetDetails = new ExternalDatasetDetails(adapter, properties);
+ String nodeGroupName = ((AString) datasetDetailsRecord
+ .getValueByPos(MetadataRecordTypes.EXTERNAL_DETAILS_ARECORD_GROUPNAME_FIELD_INDEX))
+ .getStringValue();
+
+ // Timestamp
+ Date timestamp = new Date(
+ (((ADateTime) datasetDetailsRecord
+ .getValueByPos(MetadataRecordTypes.EXTERNAL_DETAILS_ARECORD_LAST_REFRESH_TIME_FIELD_INDEX)))
+ .getChrononTime());
+ // State
+ ExternalDatasetTransactionState state = ExternalDatasetTransactionState.values()[((AInt32) datasetDetailsRecord
+ .getValueByPos(MetadataRecordTypes.EXTERNAL_DETAILS_ARECORD_TRANSACTION_STATE_FIELD_INDEX))
+ .getIntegerValue()];
+ // Compaction Policy
+ String compactionPolicy = ((AString) datasetDetailsRecord
+ .getValueByPos(MetadataRecordTypes.EXTERNAL_DETAILS_ARECORD_COMPACTION_POLICY_FIELD_INDEX))
+ .getStringValue();
+ // Compaction Policy Properties
+ cursor = ((AOrderedList) datasetDetailsRecord
+ .getValueByPos(MetadataRecordTypes.EXTERNAL_DETAILS_ARECORD_COMPACTION_POLICY_PROPERTIES_FIELD_INDEX))
+ .getCursor();
+ Map<String, String> compactionPolicyProperties = new LinkedHashMap<String, String>();
+ while (cursor.next()) {
+ ARecord field = (ARecord) cursor.get();
+ key = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_NAME_FIELD_INDEX))
+ .getStringValue();
+ value = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_VALUE_FIELD_INDEX))
+ .getStringValue();
+ compactionPolicyProperties.put(key, value);
+ }
+
+ datasetDetails = new ExternalDatasetDetails(adapter, properties, nodeGroupName, timestamp, state,
+ compactionPolicy, compactionPolicyProperties);
}
Map<String, String> hints = getDatasetHints(datasetRecord);
@@ -305,6 +340,7 @@
return hints;
}
+ @SuppressWarnings("unchecked")
private void writeDatasetHintRecord(String name, String value, DataOutput out) throws HyracksDataException {
IARecordBuilder propertyRecordBuilder = new RecordBuilder();
ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/ExternalFileTupleTranslator.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/ExternalFileTupleTranslator.java
new file mode 100644
index 0000000..3c29457
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/ExternalFileTupleTranslator.java
@@ -0,0 +1,216 @@
+/*
+ * 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.metadata.entitytupletranslators;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.Date;
+
+import edu.uci.ics.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.metadata.MetadataException;
+import edu.uci.ics.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
+import edu.uci.ics.asterix.metadata.bootstrap.MetadataRecordTypes;
+import edu.uci.ics.asterix.metadata.entities.ExternalFile;
+import edu.uci.ics.asterix.om.base.ADateTime;
+import edu.uci.ics.asterix.om.base.AInt32;
+import edu.uci.ics.asterix.om.base.AInt64;
+import edu.uci.ics.asterix.om.base.AMutableDateTime;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.AMutableInt64;
+import edu.uci.ics.asterix.om.base.ARecord;
+import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public class ExternalFileTupleTranslator extends
+ AbstractTupleTranslator<ExternalFile> {
+ // Field indexes of serialized ExternalFile in a tuple.
+ // First key field.
+ public static final int EXTERNAL_FILE_DATAVERSENAME_TUPLE_FIELD_INDEX = 0;
+ // Second key field.
+ public static final int EXTERNAL_FILE_DATASETNAME_TUPLE_FIELD_INDEX = 1;
+ // Third key field
+ public static final int EXTERNAL_FILE_NUMBER_TUPLE_FIELD_INDEX = 2;
+ // Payload field containing serialized ExternalFile.
+ public static final int EXTERNAL_FILE_PAYLOAD_TUPLE_FIELD_INDEX = 3;
+
+ protected AMutableInt32 aInt32 = new AMutableInt32(0);
+ protected AMutableDateTime aDateTime = new AMutableDateTime(0);
+ protected AMutableInt64 aInt64 = new AMutableInt64(0);
+
+ @SuppressWarnings("unchecked")
+ protected ISerializerDeserializer<AInt32> intSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT32);
+ @SuppressWarnings("unchecked")
+ protected ISerializerDeserializer<ADateTime> dateTimeSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ADATETIME);
+ @SuppressWarnings("unchecked")
+ protected ISerializerDeserializer<AInt64> longSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT64);
+ @SuppressWarnings("unchecked")
+ private ISerializerDeserializer<ARecord> recordSerDes = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(MetadataRecordTypes.EXTERNAL_FILE_RECORDTYPE);
+
+ public ExternalFileTupleTranslator(boolean getTuple) {
+ super(getTuple, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET
+ .getFieldCount());
+ }
+
+ @Override
+ public ExternalFile getMetadataEntityFromTuple(ITupleReference tuple)
+ throws MetadataException, IOException {
+ byte[] serRecord = tuple
+ .getFieldData(EXTERNAL_FILE_PAYLOAD_TUPLE_FIELD_INDEX);
+ int recordStartOffset = tuple
+ .getFieldStart(EXTERNAL_FILE_PAYLOAD_TUPLE_FIELD_INDEX);
+ int recordLength = tuple
+ .getFieldLength(EXTERNAL_FILE_PAYLOAD_TUPLE_FIELD_INDEX);
+ ByteArrayInputStream stream = new ByteArrayInputStream(serRecord,
+ recordStartOffset, recordLength);
+ DataInput in = new DataInputStream(stream);
+ ARecord externalFileRecord = (ARecord) recordSerDes.deserialize(in);
+ return createExternalFileFromARecord(externalFileRecord);
+ }
+
+ private ExternalFile createExternalFileFromARecord(
+ ARecord externalFileRecord) {
+ String dataverseName = ((AString) externalFileRecord
+ .getValueByPos(MetadataRecordTypes.EXTERNAL_FILE_ARECORD_DATAVERSENAME_FIELD_INDEX))
+ .getStringValue();
+ String datasetName = ((AString) externalFileRecord
+ .getValueByPos(MetadataRecordTypes.EXTERNAL_FILE_ARECORD_DATASET_NAME_FIELD_INDEX))
+ .getStringValue();
+ int fileNumber = ((AInt32) externalFileRecord
+ .getValueByPos(MetadataRecordTypes.EXTERNAL_FILE_ARECORD_FILE_NUMBER_FIELD_INDEX))
+ .getIntegerValue();
+ String fileName = ((AString) externalFileRecord
+ .getValueByPos(MetadataRecordTypes.EXTERNAL_FILE_ARECORD_FILE_NAME_FIELD_INDEX))
+ .getStringValue();
+ long fileSize = ((AInt64) externalFileRecord
+ .getValueByPos(MetadataRecordTypes.EXTERNAL_FILE_ARECORD_FILE_SIZE_FIELD_INDEX))
+ .getLongValue();
+ Date lastMoDifiedDate = new Date(
+ ((ADateTime) externalFileRecord
+ .getValueByPos(MetadataRecordTypes.EXTERNAL_FILE_ARECORD_FILE_MOD_DATE_FIELD_INDEX))
+ .getChrononTime());
+ ExternalFilePendingOp pendingOp = ExternalFilePendingOp.values()[((AInt32) externalFileRecord
+ .getValueByPos(MetadataRecordTypes.EXTERNAL_FILE_ARECORD_FILE_PENDING_OP_FIELD_INDEX))
+ .getIntegerValue()];
+ return new ExternalFile(dataverseName, datasetName, fileNumber,
+ fileName, lastMoDifiedDate, fileSize, pendingOp);
+ }
+
+ @Override
+ public ITupleReference getTupleFromMetadataEntity(ExternalFile externalFile)
+ throws MetadataException, IOException {
+ // write the key in the first 3 fields of the tuple
+ tupleBuilder.reset();
+ // dataverse name
+ aString.setValue(externalFile.getDataverseName());
+ stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+ tupleBuilder.addFieldEndOffset();
+ // dataset name
+ aString.setValue(externalFile.getDatasetName());
+ stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+ tupleBuilder.addFieldEndOffset();
+ // file number
+ aInt32.setValue(externalFile.getFileNumber());
+ intSerde.serialize(aInt32, tupleBuilder.getDataOutput());
+ tupleBuilder.addFieldEndOffset();
+
+ // write the pay-load in the fourth field of the tuple
+ recordBuilder.reset(MetadataRecordTypes.EXTERNAL_FILE_RECORDTYPE);
+
+ // write field 0
+ fieldValue.reset();
+ aString.setValue(externalFile.getDataverseName());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ recordBuilder
+ .addField(
+ MetadataRecordTypes.EXTERNAL_FILE_ARECORD_DATAVERSENAME_FIELD_INDEX,
+ fieldValue);
+
+ // write field 1
+ fieldValue.reset();
+ aString.setValue(externalFile.getDatasetName());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ recordBuilder
+ .addField(
+ MetadataRecordTypes.EXTERNAL_FILE_ARECORD_DATASET_NAME_FIELD_INDEX,
+ fieldValue);
+
+ // write field 2
+ fieldValue.reset();
+ aInt32.setValue(externalFile.getFileNumber());
+ intSerde.serialize(aInt32, fieldValue.getDataOutput());
+ recordBuilder
+ .addField(
+ MetadataRecordTypes.EXTERNAL_FILE_ARECORD_FILE_NUMBER_FIELD_INDEX,
+ fieldValue);
+
+ // write field 3
+ fieldValue.reset();
+ aString.setValue(externalFile.getFileName());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ recordBuilder
+ .addField(
+ MetadataRecordTypes.EXTERNAL_FILE_ARECORD_FILE_NAME_FIELD_INDEX,
+ fieldValue);
+
+ // write field 4
+ fieldValue.reset();
+ aInt64.setValue(externalFile.getSize());
+ longSerde.serialize(aInt64, fieldValue.getDataOutput());
+ recordBuilder
+ .addField(
+ MetadataRecordTypes.EXTERNAL_FILE_ARECORD_FILE_SIZE_FIELD_INDEX,
+ fieldValue);
+
+ // write field 5
+ fieldValue.reset();
+ aDateTime.setValue(externalFile.getLastModefiedTime().getTime());
+ dateTimeSerde.serialize(aDateTime, fieldValue.getDataOutput());
+ recordBuilder
+ .addField(
+ MetadataRecordTypes.EXTERNAL_FILE_ARECORD_FILE_MOD_DATE_FIELD_INDEX,
+ fieldValue);
+
+ // write field 6
+ fieldValue.reset();
+ aInt32.setValue(externalFile.getPendingOp().ordinal());
+ intSerde.serialize(aInt32, fieldValue.getDataOutput());
+ recordBuilder
+ .addField(
+ MetadataRecordTypes.EXTERNAL_FILE_ARECORD_FILE_PENDING_OP_FIELD_INDEX,
+ fieldValue);
+
+ // write record
+ try {
+ recordBuilder.write(tupleBuilder.getDataOutput(), true);
+ } catch (AsterixException e) {
+ throw new MetadataException(e);
+ }
+ tupleBuilder.addFieldEndOffset();
+
+ tuple.reset(tupleBuilder.getFieldEndOffsets(),
+ tupleBuilder.getByteArray());
+ return tuple;
+ }
+}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/external/ExternalFileIndexAccessor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/external/ExternalFileIndexAccessor.java
new file mode 100644
index 0000000..9f77fdb
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/external/ExternalFileIndexAccessor.java
@@ -0,0 +1,143 @@
+/*
+ * 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.metadata.external;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.Serializable;
+import java.util.Date;
+
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.metadata.entities.ExternalFile;
+import edu.uci.ics.asterix.om.base.ADateTime;
+import edu.uci.ics.asterix.om.base.AInt64;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.ARecord;
+import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
+import edu.uci.ics.hyracks.storage.am.btree.util.BTreeUtils;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.ExternalBTree;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessorInternal;
+
+/*
+ * This class was created specifically to facilitate accessing
+ * external file index when doing external lookup during runtime
+ */
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public class ExternalFileIndexAccessor implements Serializable {
+
+ private static final long serialVersionUID = 1L;
+ private ExternalBTreeDataflowHelper indexDataflowHelper;
+ private ExternalLoopkupOperatorDiscriptor opDesc;
+
+ private IHyracksTaskContext ctx;
+ private ExternalBTree index;
+ private ArrayTupleBuilder searchKeyTupleBuilder;
+ private ArrayTupleReference searchKey;
+ private MultiComparator searchCmp;
+ private AMutableInt32 currentFileNumber = new AMutableInt32(-1);
+ private ISerializerDeserializer intSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT32);
+ private RangePredicate searchPredicate;
+ private ILSMIndexAccessorInternal fileIndexAccessor;
+ private IIndexCursor fileIndexSearchCursor;
+
+ public ExternalFileIndexAccessor(ExternalBTreeDataflowHelper indexDataflowHelper, ExternalLoopkupOperatorDiscriptor opDesc) {
+ this.indexDataflowHelper = indexDataflowHelper;
+ this.opDesc = opDesc;
+ }
+
+ public void openIndex() throws HyracksDataException {
+ // Open the index and get the instance
+ indexDataflowHelper.open();
+ index = (ExternalBTree) indexDataflowHelper.getIndexInstance();
+ try {
+ // Create search key and search predicate objects
+ searchKey = new ArrayTupleReference();
+ searchKeyTupleBuilder = new ArrayTupleBuilder(FilesIndexDescription.FILE_KEY_SIZE);
+ searchKeyTupleBuilder.reset();
+ searchKeyTupleBuilder.addField(intSerde, currentFileNumber);
+ searchKey.reset(searchKeyTupleBuilder.getFieldEndOffsets(), searchKeyTupleBuilder.getByteArray());
+ searchCmp = BTreeUtils.getSearchMultiComparator(index.getComparatorFactories(), searchKey);
+ searchPredicate = new RangePredicate(searchKey, searchKey, true, true, searchCmp, searchCmp);
+
+ // create the accessor and the cursor using the passed version
+ ISearchOperationCallback searchCallback = opDesc.getSearchOpCallbackFactory()
+ .createSearchOperationCallback(indexDataflowHelper.getResourceID(), ctx);
+ fileIndexAccessor = index.createAccessor(searchCallback, indexDataflowHelper.getVersion());
+ fileIndexSearchCursor = fileIndexAccessor.createSearchCursor(false);
+ } catch (Exception e) {
+ indexDataflowHelper.close();
+ throw new HyracksDataException(e);
+ }
+ }
+
+ public void searchForFile(int fileNumber, ExternalFile file) throws Exception {
+ // Set search parameters
+ currentFileNumber.setValue(fileNumber);
+ searchKeyTupleBuilder.reset();
+ searchKeyTupleBuilder.addField(intSerde, currentFileNumber);
+ searchKey.reset(searchKeyTupleBuilder.getFieldEndOffsets(), searchKeyTupleBuilder.getByteArray());
+ fileIndexSearchCursor.reset();
+
+ // Perform search
+ fileIndexAccessor.search(fileIndexSearchCursor, searchPredicate);
+ if (fileIndexSearchCursor.hasNext()) {
+ fileIndexSearchCursor.next();
+ ITupleReference tuple = fileIndexSearchCursor.getTuple();
+ // Deserialize
+ byte[] serRecord = tuple.getFieldData(FilesIndexDescription.FILE_PAYLOAD_INDEX);
+ int recordStartOffset = tuple.getFieldStart(FilesIndexDescription.FILE_PAYLOAD_INDEX);
+ int recordLength = tuple.getFieldLength(FilesIndexDescription.FILE_PAYLOAD_INDEX);
+ ByteArrayInputStream stream = new ByteArrayInputStream(serRecord, recordStartOffset, recordLength);
+ DataInput in = new DataInputStream(stream);
+ ARecord externalFileRecord = (ARecord) FilesIndexDescription.EXTERNAL_FILE_RECORD_SERDE.deserialize(in);
+ setExternalFileFromARecord(externalFileRecord, file);
+ } else {
+ // This should never happen
+ throw new HyracksDataException("Was not able to find a file in the files index");
+ }
+ }
+
+ private void setExternalFileFromARecord(ARecord externalFileRecord, ExternalFile file) {
+ file.setFileName(((AString) externalFileRecord
+ .getValueByPos(FilesIndexDescription.EXTERNAL_FILE_NAME_FIELD_INDEX)).getStringValue());
+ file.setSize(((AInt64) externalFileRecord.getValueByPos(FilesIndexDescription.EXTERNAL_FILE_SIZE_FIELD_INDEX))
+ .getLongValue());
+ file.setLastModefiedTime((new Date(((ADateTime) externalFileRecord
+ .getValueByPos(FilesIndexDescription.EXTERNAL_FILE_MOD_DATE_FIELD_INDEX)).getChrononTime())));
+ }
+
+ public void closeIndex() throws HyracksDataException {
+ try {
+ fileIndexSearchCursor.close();
+ } finally {
+ indexDataflowHelper.close();
+ }
+ }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/external/ExternalLoopkupOperatorDiscriptor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/external/ExternalLoopkupOperatorDiscriptor.java
new file mode 100644
index 0000000..99c747b
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/external/ExternalLoopkupOperatorDiscriptor.java
@@ -0,0 +1,127 @@
+/*
+ * 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.metadata.external;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexSearchOperationCallbackFactory;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+
+/*
+ * This operator is intended for using record ids to access data in external sources
+ */
+public class ExternalLoopkupOperatorDiscriptor extends AbstractTreeIndexOperatorDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ private IControlledAdapterFactory adapterFactory;
+ private INullWriterFactory iNullWriterFactory;
+
+ public ExternalLoopkupOperatorDiscriptor(IOperatorDescriptorRegistry spec,
+ IControlledAdapterFactory adapterFactory, RecordDescriptor outRecDesc,
+ ExternalBTreeDataflowHelperFactory externalFilesIndexDataFlowHelperFactory, boolean propagateInput,
+ IIndexLifecycleManagerProvider lcManagerProvider, IStorageManagerInterface storageManager,
+ IFileSplitProvider fileSplitProvider, int datasetId, double bloomFilterFalsePositiveRate,
+ ISearchOperationCallbackFactory searchOpCallbackFactory, boolean retainNull,
+ INullWriterFactory iNullWriterFactory) {
+ super(spec, 1, 1, outRecDesc, storageManager, lcManagerProvider, fileSplitProvider,
+ FilesIndexDescription.EXTERNAL_FILE_INDEX_TYPE_TRAITS,
+ FilesIndexDescription.FILES_INDEX_COMP_FACTORIES, FilesIndexDescription.BLOOM_FILTER_FIELDS,
+ externalFilesIndexDataFlowHelperFactory, null, propagateInput, retainNull, iNullWriterFactory, null,
+ new SecondaryIndexSearchOperationCallbackFactory(), null);
+ this.adapterFactory = adapterFactory;
+ this.iNullWriterFactory = iNullWriterFactory;
+ }
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+ final IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
+ throws HyracksDataException {
+ // Create a file index accessor to be used for files lookup operations
+ // Note that all file index accessors will use partition 0 since we only have 1 files index per NC
+ final ExternalFileIndexAccessor fileIndexAccessor = new ExternalFileIndexAccessor(
+ (ExternalBTreeDataflowHelper) dataflowHelperFactory.createIndexDataflowHelper(this, ctx, partition),
+ this);
+ return new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
+ // The adapter that uses the file index along with the coming tuples to access files in HDFS
+ private IControlledAdapter adapter = adapterFactory.createAdapter(ctx, fileIndexAccessor,
+ recordDescProvider.getInputRecordDescriptor(getActivityId(), 0));
+
+ @Override
+ public void open() throws HyracksDataException {
+ //Open the file index accessor here
+ fileIndexAccessor.openIndex();
+ try {
+ adapter.initialize(ctx, iNullWriterFactory);
+ } catch (Exception e) {
+ // close the files index
+ fileIndexAccessor.closeIndex();
+ throw new HyracksDataException("error during opening a controlled adapter", e);
+ }
+ writer.open();
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ try {
+ adapter.close(writer);
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new HyracksDataException("controlled adapter failed to close", e);
+ } finally {
+ //close the file index
+ fileIndexAccessor.closeIndex();
+ writer.close();
+ }
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ try {
+ adapter.fail();
+ writer.fail();
+ } catch (Exception e) {
+ throw new HyracksDataException("controlled adapter failed to clean up", e);
+ } finally {
+ // close the open index
+ fileIndexAccessor.closeIndex();
+ }
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ try {
+ adapter.nextFrame(buffer, writer);
+ } catch (Exception e) {
+ throw new HyracksDataException("controlled adapter failed to process a frame", e);
+ }
+ }
+
+ };
+ }
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/external/FilesIndexDescription.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/external/FilesIndexDescription.java
new file mode 100644
index 0000000..463aa73
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/external/FilesIndexDescription.java
@@ -0,0 +1,96 @@
+/*
+ * 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.metadata.external;
+
+import java.io.IOException;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+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;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
+
+@SuppressWarnings("rawtypes")
+public class FilesIndexDescription {
+ public final static int FILE_INDEX_TUPLE_SIZE = 2;
+ public final static int FILE_KEY_INDEX = 0;
+ public final static int FILE_KEY_SIZE = 1;
+ public final static int FILE_PAYLOAD_INDEX = 1;
+ public static RecordDescriptor FILE_INDEX_RECORD_DESCRIPTOR;
+ public static RecordDescriptor FILE_BUDDY_BTREE_RECORD_DESCRIPTOR;
+ public final static String[] payloadFieldNames = { "FileName", "FileSize", "FileModDate" };
+ public final static IAType[] payloadFieldTypes = { BuiltinType.ASTRING, BuiltinType.AINT64, BuiltinType.ADATETIME };
+ public static ARecordType EXTERNAL_FILE_RECORD_TYPE;
+ public static ISerializerDeserializer EXTERNAL_FILE_RECORD_SERDE;
+ public static final ISerializerDeserializer[] EXTERNAL_FILE_BUDDY_BTREE_FIELDS = new ISerializerDeserializer[1];
+ public static final ITypeTraits[] EXTERNAL_FILE_BUDDY_BTREE_TYPE_TRAITS = new ITypeTraits[1];
+ public static final ISerializerDeserializer[] EXTERNAL_FILE_TUPLE_FIELDS = new ISerializerDeserializer[FILE_INDEX_TUPLE_SIZE];
+ public static final ITypeTraits[] EXTERNAL_FILE_INDEX_TYPE_TRAITS = new ITypeTraits[FILE_INDEX_TUPLE_SIZE];
+ public static final IBinaryComparatorFactory[] FILES_INDEX_COMP_FACTORIES = new IBinaryComparatorFactory[] { AqlBinaryComparatorFactoryProvider.INSTANCE
+ .getBinaryComparatorFactory(BuiltinType.AINT32, true) };
+ public static final int[] BLOOM_FILTER_FIELDS = { 0 };
+ public static final int EXTERNAL_FILE_NAME_FIELD_INDEX = 0;
+ public static final int EXTERNAL_FILE_SIZE_FIELD_INDEX = 1;
+ public static final int EXTERNAL_FILE_MOD_DATE_FIELD_INDEX = 2;
+ static {
+ try {
+ EXTERNAL_FILE_RECORD_TYPE = new ARecordType("ExternalFileRecordType", payloadFieldNames, payloadFieldTypes,
+ true);
+ EXTERNAL_FILE_RECORD_SERDE = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(EXTERNAL_FILE_RECORD_TYPE);
+
+ EXTERNAL_FILE_TUPLE_FIELDS[FILE_KEY_INDEX] = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(IndexingConstants.FILE_NUMBER_FIELD_TYPE);
+ EXTERNAL_FILE_TUPLE_FIELDS[FILE_PAYLOAD_INDEX] = EXTERNAL_FILE_RECORD_SERDE;
+ EXTERNAL_FILE_BUDDY_BTREE_FIELDS[FILE_KEY_INDEX] = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(IndexingConstants.FILE_NUMBER_FIELD_TYPE);
+
+ EXTERNAL_FILE_INDEX_TYPE_TRAITS[FILE_KEY_INDEX] = AqlTypeTraitProvider.INSTANCE
+ .getTypeTrait(IndexingConstants.FILE_NUMBER_FIELD_TYPE);
+ EXTERNAL_FILE_INDEX_TYPE_TRAITS[FILE_PAYLOAD_INDEX] = AqlTypeTraitProvider.INSTANCE
+ .getTypeTrait(EXTERNAL_FILE_RECORD_TYPE);
+ EXTERNAL_FILE_BUDDY_BTREE_TYPE_TRAITS[FILE_KEY_INDEX] = AqlTypeTraitProvider.INSTANCE
+ .getTypeTrait(IndexingConstants.FILE_NUMBER_FIELD_TYPE);
+
+ FILE_INDEX_RECORD_DESCRIPTOR = new RecordDescriptor(EXTERNAL_FILE_TUPLE_FIELDS,
+ EXTERNAL_FILE_INDEX_TYPE_TRAITS);
+
+ FILE_BUDDY_BTREE_RECORD_DESCRIPTOR = new RecordDescriptor(EXTERNAL_FILE_BUDDY_BTREE_FIELDS,
+ EXTERNAL_FILE_BUDDY_BTREE_TYPE_TRAITS);
+ } catch (Exception e) {
+ e.printStackTrace();
+ System.exit(1);
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ public static void getBuddyBTreeTupleFromFileNumber(ArrayTupleReference tuple, ArrayTupleBuilder tupleBuilder,
+ AMutableInt32 aInt32) throws IOException, AsterixException {
+ tupleBuilder.reset();
+ FilesIndexDescription.FILE_BUDDY_BTREE_RECORD_DESCRIPTOR.getFields()[0].serialize(aInt32,
+ tupleBuilder.getDataOutput());
+ tupleBuilder.addFieldEndOffset();
+ tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+ }
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/external/IControlledAdapter.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/external/IControlledAdapter.java
new file mode 100644
index 0000000..402d7bb
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/external/IControlledAdapter.java
@@ -0,0 +1,61 @@
+/*
+ * 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.metadata.external;
+
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ *
+ * @author alamouda
+ *
+ */
+public interface IControlledAdapter extends Serializable {
+
+ /**
+ *
+ * @param ctx
+ * @param recordDescriptors
+ * @throws Exception
+ */
+ public void initialize(IHyracksTaskContext ctx, INullWriterFactory iNullWriterFactory) throws Exception;
+
+ /**
+ *
+ * @param buffer
+ * @param writer
+ * @throws HyracksDataException
+ */
+ public void nextFrame(ByteBuffer buffer, IFrameWriter writer) throws Exception;
+
+ /**
+ *
+ * @param writer
+ * @throws HyracksDataException
+ */
+ public void close(IFrameWriter writer) throws Exception;
+
+ /**
+ * Gives the adapter a chance to clean up
+ * @param writer
+ * @throws HyracksDataException
+ */
+ public void fail() throws Exception;
+}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/external/IControlledAdapterFactory.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/external/IControlledAdapterFactory.java
new file mode 100644
index 0000000..035ec6f
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/external/IControlledAdapterFactory.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.metadata.external;
+
+import java.io.Serializable;
+import java.util.Map;
+
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public interface IControlledAdapterFactory extends Serializable {
+ public IControlledAdapter createAdapter(IHyracksTaskContext ctx, ExternalFileIndexAccessor fileIndexAccessor,
+ RecordDescriptor inRecDesc);
+
+ public void configure(IAType atype, boolean propagateInput, int[] ridFields,
+ Map<String, String> adapterConfiguration, boolean retainNull);
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/external/IndexingConstants.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/external/IndexingConstants.java
new file mode 100644
index 0000000..78f7c4b
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/external/IndexingConstants.java
@@ -0,0 +1,202 @@
+/*
+ * 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.metadata.external;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.runtime.evaluators.TupleFieldEvaluatorFactory;
+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;
+
+@SuppressWarnings("rawtypes")
+public class IndexingConstants {
+
+ public static final String EXTERNAL_FILE_INDEX_NAME_SUFFIX = "FilesIndex";
+ public static final String KEY_INPUT_FORMAT = "input-format";
+ public static final String INPUT_FORMAT_RC = "rc-input-format";
+ public static final String INPUT_FORMAT_RC_FULLY_QUALIFIED = "org.apache.hadoop.hive.ql.io.RCFileInputFormat";
+
+ //Field Types
+ public static final IAType FILE_NUMBER_FIELD_TYPE = BuiltinType.AINT32;
+ public static final IAType RECORD_OFFSET_FIELD_TYPE = BuiltinType.AINT64;
+ public static final IAType ROW_NUMBER_FIELD_TYPE = BuiltinType.AINT32;
+
+ //Comparator Factories
+ private static final IBinaryComparatorFactory fileNumberCompFactory = AqlBinaryComparatorFactoryProvider.INSTANCE
+ .getBinaryComparatorFactory(BuiltinType.AINT32, true);
+ private static final IBinaryComparatorFactory recordOffsetCompFactory = AqlBinaryComparatorFactoryProvider.INSTANCE
+ .getBinaryComparatorFactory(BuiltinType.AINT64, true);
+ private static final IBinaryComparatorFactory rowNumberCompFactory = AqlBinaryComparatorFactoryProvider.INSTANCE
+ .getBinaryComparatorFactory(BuiltinType.AINT32, true);
+
+ private static final IBinaryComparatorFactory[] rCFileRIDComparatorFactories = {fileNumberCompFactory, recordOffsetCompFactory, rowNumberCompFactory};
+ private static final IBinaryComparatorFactory[] txtSeqFileRIDComparatorFactories = {fileNumberCompFactory, recordOffsetCompFactory};
+
+ private static final IBinaryComparatorFactory[] buddyBtreeComparatorFactories = {fileNumberCompFactory};
+
+ //Serdes
+ private static ISerializerDeserializer fileNumberSerializerDeserializer;
+ private static ISerializerDeserializer recordOffsetSerializerDeserializer;
+ private static ISerializerDeserializer rowNumberSerializerDeserializer;
+
+ //Type Traits
+ private static ITypeTraits fileNumberTypeTraits;
+ private static ITypeTraits recordOffsetTypeTraits;
+ private static ITypeTraits rowNumberTypeTraits;
+
+ //IScalarEvaluatorFactories
+ private static final IScalarEvaluatorFactory fileNumberEvalFactory;
+ private static final IScalarEvaluatorFactory recordOffsetEvalFactory;
+ private static final IScalarEvaluatorFactory rowNumberEvalFactory;
+ public static final int FILE_NUMBER_FIELD_INDEX = 0;
+ public static final int RECORD_OFFSET_FIELD_INDEX = 1;
+ public static final int ROW_NUMBER_FIELD_INDEX = 2;
+
+ public static final ArrayList<String> RecordIDFields = new ArrayList<String>();
+
+ static {
+
+ fileNumberSerializerDeserializer = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(FILE_NUMBER_FIELD_TYPE);
+ recordOffsetSerializerDeserializer = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(RECORD_OFFSET_FIELD_TYPE);
+ rowNumberSerializerDeserializer = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(ROW_NUMBER_FIELD_TYPE);
+
+ fileNumberTypeTraits = AqlTypeTraitProvider.INSTANCE.getTypeTrait(FILE_NUMBER_FIELD_TYPE);
+ recordOffsetTypeTraits = AqlTypeTraitProvider.INSTANCE.getTypeTrait(RECORD_OFFSET_FIELD_TYPE);
+ rowNumberTypeTraits = AqlTypeTraitProvider.INSTANCE.getTypeTrait(ROW_NUMBER_FIELD_TYPE);
+
+ fileNumberEvalFactory = new TupleFieldEvaluatorFactory(1);
+ recordOffsetEvalFactory = new TupleFieldEvaluatorFactory(2);
+ rowNumberEvalFactory = new TupleFieldEvaluatorFactory(3);
+
+ // Add field names
+ RecordIDFields.add("FileNumber");
+ RecordIDFields.add("RecordOffset");
+ RecordIDFields.add("RowNumber");
+ }
+
+ // This function returns the size of the RID for the passed file input format
+ public static int getRIDSize(String fileInputFormat) {
+ if (fileInputFormat.equals(INPUT_FORMAT_RC) || fileInputFormat.equals(INPUT_FORMAT_RC_FULLY_QUALIFIED))
+ return 3;
+ else
+ return 2;
+ }
+
+ // This function returns the size of the RID for the passed file input format
+ public static IBinaryComparatorFactory[] getComparatorFactories(String fileInputFormat) {
+ if (fileInputFormat.equals(INPUT_FORMAT_RC) || fileInputFormat.equals(INPUT_FORMAT_RC_FULLY_QUALIFIED))
+ return rCFileRIDComparatorFactories;
+ else
+ return txtSeqFileRIDComparatorFactories;
+ }
+
+ public static IAType getFieldType(int fieldNumber) throws AsterixException {
+ switch (fieldNumber) {
+ case 0:
+ return FILE_NUMBER_FIELD_TYPE;
+ case 1:
+ return RECORD_OFFSET_FIELD_TYPE;
+ case 2:
+ return ROW_NUMBER_FIELD_TYPE;
+ default:
+ throw new AsterixException("Unknown external field RID number");
+ }
+ }
+
+ public static IBinaryComparatorFactory getComparatorFactory(int fieldNumber)
+ throws AsterixException {
+ switch (fieldNumber) {
+ case 0:
+ return fileNumberCompFactory;
+ case 1:
+ return recordOffsetCompFactory;
+ case 2:
+ return rowNumberCompFactory;
+ default:
+ throw new AsterixException("Unknown external field RID number");
+ }
+ }
+
+ public static ISerializerDeserializer getSerializerDeserializer(int fieldNumber)
+ throws AsterixException {
+ switch (fieldNumber) {
+ case 0:
+ return fileNumberSerializerDeserializer;
+ case 1:
+ return recordOffsetSerializerDeserializer;
+ case 2:
+ return rowNumberSerializerDeserializer;
+ default:
+ throw new AsterixException("Unknown external field RID number");
+ }
+ }
+
+ public static ITypeTraits getTypeTraits(int fieldNumber)
+ throws AsterixException {
+ switch (fieldNumber) {
+ case 0:
+ return fileNumberTypeTraits;
+ case 1:
+ return recordOffsetTypeTraits;
+ case 2:
+ return rowNumberTypeTraits;
+ default:
+ throw new AsterixException("Unknown external field RID number");
+ }
+ }
+
+ public static IScalarEvaluatorFactory getEvalFactory(int fieldNumber) throws AsterixException{
+ switch (fieldNumber) {
+ case 0:
+ return fileNumberEvalFactory;
+ case 1:
+ return recordOffsetEvalFactory;
+ case 2:
+ return rowNumberEvalFactory;
+ default:
+ throw new AsterixException("Unknown external field RID number");
+ }
+ }
+
+ public static IBinaryComparatorFactory[] getBuddyBtreeComparatorFactories() {
+ return buddyBtreeComparatorFactories;
+ }
+
+ public static int getRIDSize(Dataset dataset) {
+ return getRIDSize(((ExternalDatasetDetails) dataset.getDatasetDetails()).getProperties().get(KEY_INPUT_FORMAT));
+ }
+
+ public static List<String> getRIDKeys(Dataset dataset) {
+ String fileInputFormat = ((ExternalDatasetDetails) dataset.getDatasetDetails()).getProperties().get(KEY_INPUT_FORMAT);
+ if (fileInputFormat.equals(INPUT_FORMAT_RC) || fileInputFormat.equals(INPUT_FORMAT_RC_FULLY_QUALIFIED))
+ return RecordIDFields;
+ else
+ return RecordIDFields.subList(0, ROW_NUMBER_FIELD_INDEX);
+ }
+}
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 48c9a2c..1bcb509 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
@@ -51,7 +51,7 @@
writer.open();
IDatasourceAdapter adapter = null;
try {
- adapter = ((IAdapterFactory) adapterFactory).createAdapter(ctx, partition);
+ adapter = adapterFactory.createAdapter(ctx, partition);
adapter.start(partition, writer);
} catch (Exception e) {
throw new HyracksDataException("exception during reading from external data source", e);
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 16c3c80..47aa8d9 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
@@ -14,9 +14,12 @@
*/
package edu.uci.ics.asterix.metadata.feeds;
+import java.util.List;
import java.util.Map;
+import edu.uci.ics.asterix.metadata.entities.ExternalFile;
import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
public interface IGenericAdapterFactory extends IAdapterFactory {
@@ -24,4 +27,6 @@
public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception;
+ public void setFiles(List<ExternalFile> files) throws AlgebricksException;
+
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/utils/DatasetUtils.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/utils/DatasetUtils.java
index 57f2042..b0de324 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/utils/DatasetUtils.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/utils/DatasetUtils.java
@@ -20,6 +20,8 @@
import java.util.Map;
import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
+import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
import edu.uci.ics.asterix.metadata.MetadataException;
import edu.uci.ics.asterix.metadata.MetadataManager;
@@ -27,7 +29,9 @@
import edu.uci.ics.asterix.metadata.bootstrap.MetadataConstants;
import edu.uci.ics.asterix.metadata.entities.CompactionPolicy;
import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
+import edu.uci.ics.asterix.metadata.external.IndexingConstants;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -43,19 +47,27 @@
public static IBinaryComparatorFactory[] computeKeysBinaryComparatorFactories(Dataset dataset,
ARecordType itemType, IBinaryComparatorFactoryProvider comparatorFactoryProvider)
throws AlgebricksException {
- if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
- throw new AlgebricksException("not implemented");
- }
List<String> partitioningKeys = getPartitioningKeys(dataset);
IBinaryComparatorFactory[] bcfs = new IBinaryComparatorFactory[partitioningKeys.size()];
- for (int i = 0; i < partitioningKeys.size(); i++) {
- IAType keyType;
- try {
- keyType = itemType.getFieldType(partitioningKeys.get(i));
- } catch (IOException e) {
- throw new AlgebricksException(e);
+ if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+ // Get comparators for RID fields.
+ for (int i = 0; i < partitioningKeys.size(); i++) {
+ try {
+ bcfs[i] = IndexingConstants.getComparatorFactory(i);
+ } catch (AsterixException e) {
+ throw new AlgebricksException(e);
+ }
}
- bcfs[i] = comparatorFactoryProvider.getBinaryComparatorFactory(keyType, true);
+ } else {
+ for (int i = 0; i < partitioningKeys.size(); i++) {
+ IAType keyType;
+ try {
+ keyType = itemType.getFieldType(partitioningKeys.get(i));
+ } catch (IOException e) {
+ throw new AlgebricksException(e);
+ }
+ bcfs[i] = comparatorFactoryProvider.getBinaryComparatorFactory(keyType, true);
+ }
}
return bcfs;
}
@@ -113,6 +125,9 @@
}
public static List<String> getPartitioningKeys(Dataset dataset) {
+ if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+ return IndexingConstants.getRIDKeys(dataset);
+ }
return ((InternalDatasetDetails) dataset.getDatasetDetails()).getPartitioningKey();
}
@@ -132,8 +147,7 @@
public static Pair<ILSMMergePolicyFactory, Map<String, String>> getMergePolicyFactory(Dataset dataset,
MetadataTransactionContext mdTxnCtx) throws AlgebricksException, MetadataException {
- InternalDatasetDetails datasetDetails = (InternalDatasetDetails) dataset.getDatasetDetails();
- String policyName = datasetDetails.getCompactionPolicy();
+ String policyName = dataset.getDatasetDetails().getCompactionPolicy();
CompactionPolicy compactionPolicy = MetadataManager.INSTANCE.getCompactionPolicy(mdTxnCtx,
MetadataConstants.METADATA_DATAVERSE_NAME, policyName);
String compactionPolicyFactoryClassName = compactionPolicy.getClassName();
@@ -143,8 +157,7 @@
} catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
throw new AlgebricksException(e);
}
- Map<String, String> properties = ((InternalDatasetDetails) dataset.getDatasetDetails())
- .getCompactionPolicyProperties();
+ Map<String, String> properties = dataset.getDatasetDetails().getCompactionPolicyProperties();
return new Pair<ILSMMergePolicyFactory, Map<String, String>>(mergePolicyFactory, properties);
}
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/utils/ExternalDatasetAccessManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/utils/ExternalDatasetAccessManager.java
new file mode 100644
index 0000000..79d695c
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/utils/ExternalDatasetAccessManager.java
@@ -0,0 +1,96 @@
+package edu.uci.ics.asterix.metadata.utils;
+
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+public class ExternalDatasetAccessManager {
+ // a version to indicate the current version of the dataset
+ private int version;
+ // a lock to allow concurrent build index operation and serialize refresh operations
+ private ReentrantReadWriteLock datasetLock;
+ // a lock per version of the dataset to keep a version alive while queries are still assigned to it
+ private ReentrantReadWriteLock v0Lock;
+ private ReentrantReadWriteLock v1Lock;
+
+ public ExternalDatasetAccessManager() {
+ this.version = 0;
+ this.v0Lock = new ReentrantReadWriteLock(false);
+ this.v1Lock = new ReentrantReadWriteLock(false);
+ this.datasetLock = new ReentrantReadWriteLock(true);
+ }
+
+ public int getVersion() {
+ return version;
+ }
+
+ public void setVersion(int version) {
+ this.version = version;
+ }
+
+ public ReentrantReadWriteLock getV0Lock() {
+ return v0Lock;
+ }
+
+ public void setV0Lock(ReentrantReadWriteLock v0Lock) {
+ this.v0Lock = v0Lock;
+ }
+
+ public ReentrantReadWriteLock getV1Lock() {
+ return v1Lock;
+ }
+
+ public void setV1Lock(ReentrantReadWriteLock v1Lock) {
+ this.v1Lock = v1Lock;
+ }
+
+ public int refreshBegin() {
+ datasetLock.writeLock().lock();
+ if (version == 0) {
+ v1Lock.writeLock().lock();
+ } else {
+ v0Lock.writeLock().lock();
+ }
+ return version;
+ }
+
+ public void refreshEnd(boolean success) {
+ if (version == 0) {
+ v1Lock.writeLock().unlock();
+ if (success) {
+ version = 1;
+ }
+ } else {
+ v0Lock.writeLock().unlock();
+ if (success) {
+ version = 0;
+ }
+ }
+ datasetLock.writeLock().unlock();
+ }
+
+ public int buildIndexBegin() {
+ datasetLock.readLock().lock();
+ return version;
+ }
+
+ public void buildIndexEnd() {
+ datasetLock.readLock().unlock();
+ }
+
+ public int queryBegin() {
+ if (version == 0) {
+ v0Lock.readLock().lock();
+ return 0;
+ } else {
+ v1Lock.readLock().lock();
+ return 1;
+ }
+ }
+
+ public void queryEnd(int version) {
+ if (version == 0) {
+ v0Lock.readLock().unlock();
+ } else {
+ v1Lock.readLock().unlock();
+ }
+ }
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/utils/ExternalDatasetsRegistry.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/utils/ExternalDatasetsRegistry.java
new file mode 100644
index 0000000..84d25d3
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/utils/ExternalDatasetsRegistry.java
@@ -0,0 +1,193 @@
+package edu.uci.ics.asterix.metadata.utils;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.Dataverse;
+
+/**
+ * This is a singelton class used to maintain the version of each external dataset with indexes
+ * It should be consolidated once a better global dataset lock management is introduced.
+ *
+ * @author alamouda
+ */
+public class ExternalDatasetsRegistry {
+ public static ExternalDatasetsRegistry INSTANCE = new ExternalDatasetsRegistry();
+ private HashMap<String, HashMap<String, ExternalDatasetAccessManager>> globalRegister;
+
+ private ExternalDatasetsRegistry() {
+ globalRegister = new HashMap<String, HashMap<String, ExternalDatasetAccessManager>>();
+ }
+
+ /**
+ * Get the current version of the dataset
+ *
+ * @param dataset
+ * @return
+ */
+ public int getDatasetVersion(Dataset dataset) {
+ HashMap<String, ExternalDatasetAccessManager> dataverseRegister;
+ ExternalDatasetAccessManager datasetAccessMgr;
+ synchronized (this) {
+ dataverseRegister = globalRegister.get(dataset.getDataverseName());
+ if (dataverseRegister == null) {
+ // Create a register for the dataverse, and put the dataset their with the initial value of 0
+ dataverseRegister = new HashMap<String, ExternalDatasetAccessManager>();
+ datasetAccessMgr = new ExternalDatasetAccessManager();
+ dataverseRegister.put(dataset.getDatasetName(), datasetAccessMgr);
+ globalRegister.put(dataset.getDataverseName(), dataverseRegister);
+ } else {
+ datasetAccessMgr = dataverseRegister.get(dataset.getDatasetName());
+ if (datasetAccessMgr == null) {
+ datasetAccessMgr = new ExternalDatasetAccessManager();
+ dataverseRegister.put(dataset.getDatasetName(), datasetAccessMgr);
+ }
+ }
+ }
+ return datasetAccessMgr.getVersion();
+ }
+
+ public int getAndLockDatasetVersion(Dataset dataset, AqlMetadataProvider metadataProvider) {
+
+ Map<String, Integer> locks = null;
+ String lockKey = dataset.getDataverseName() + "." + dataset.getDatasetName();
+ // check first if the lock was aquired already
+ locks = metadataProvider.getLocks();
+ if (locks == null) {
+ locks = new HashMap<String, Integer>();
+ metadataProvider.setLocks(locks);
+ } else {
+ // if dataset was accessed already by this job, return the registered version
+ Integer version = locks.get(lockKey);
+ if (version != null) {
+ return version;
+ }
+ }
+
+ HashMap<String, ExternalDatasetAccessManager> dataverseRegister;
+ ExternalDatasetAccessManager datasetAccessMgr;
+ dataverseRegister = globalRegister.get(dataset.getDataverseName());
+ if (dataverseRegister == null) {
+ synchronized (this) {
+ // A second time synchronized just to make sure
+ dataverseRegister = globalRegister.get(dataset.getDataverseName());
+ if (dataverseRegister == null) {
+ // Create a register for the dataverse, and put the dataset their with the initial value of 0
+ dataverseRegister = new HashMap<String, ExternalDatasetAccessManager>();
+ globalRegister.put(dataset.getDataverseName(), dataverseRegister);
+ }
+ }
+ }
+
+ datasetAccessMgr = dataverseRegister.get(dataset.getDatasetName());
+ if (datasetAccessMgr == null) {
+ synchronized (this) {
+ // a second time synchronized just to make sure
+ datasetAccessMgr = dataverseRegister.get(dataset.getDatasetName());
+ if (datasetAccessMgr == null) {
+ datasetAccessMgr = new ExternalDatasetAccessManager();
+ dataverseRegister.put(dataset.getDatasetName(), datasetAccessMgr);
+ }
+ }
+ }
+
+ // aquire the correct lock
+ int version = datasetAccessMgr.queryBegin();
+ locks.put(lockKey, version);
+ return version;
+ }
+
+ public void refreshBegin(Dataset dataset) {
+ HashMap<String, ExternalDatasetAccessManager> dataverseRegister;
+ ExternalDatasetAccessManager datasetAccessMgr;
+ synchronized (this) {
+ dataverseRegister = globalRegister.get(dataset.getDataverseName());
+ if (dataverseRegister == null) {
+ // Create a register for the dataverse, and put the dataset their with the initial value of 0
+ dataverseRegister = new HashMap<String, ExternalDatasetAccessManager>();
+ datasetAccessMgr = new ExternalDatasetAccessManager();
+ dataverseRegister.put(dataset.getDatasetName(), datasetAccessMgr);
+ globalRegister.put(dataset.getDataverseName(), dataverseRegister);
+ } else {
+ datasetAccessMgr = dataverseRegister.get(dataset.getDatasetName());
+ if (datasetAccessMgr == null) {
+ datasetAccessMgr = new ExternalDatasetAccessManager();
+ dataverseRegister.put(dataset.getDatasetName(), datasetAccessMgr);
+ }
+ }
+ }
+ // aquire the correct lock
+ datasetAccessMgr.refreshBegin();
+ }
+
+ public synchronized void removeDatasetInfo(Dataset dataset) {
+ HashMap<String, ExternalDatasetAccessManager> dataverseRegister = globalRegister
+ .get(dataset.getDataverseName());
+ if (dataverseRegister != null) {
+ dataverseRegister.remove(dataset.getDatasetName());
+ }
+ }
+
+ public synchronized void removeDataverse(Dataverse dataverse) {
+ globalRegister.remove(dataverse.getDataverseName());
+ }
+
+ public void refreshEnd(Dataset dataset, boolean success) {
+ HashMap<String, ExternalDatasetAccessManager> dataverseRegistry = globalRegister
+ .get(dataset.getDataverseName());
+ ExternalDatasetAccessManager datasetLockManager = dataverseRegistry.get(dataset.getDatasetName());
+ datasetLockManager.refreshEnd(success);
+ }
+
+ public void buildIndexBegin(Dataset dataset) {
+ HashMap<String, ExternalDatasetAccessManager> dataverseRegister;
+ ExternalDatasetAccessManager datasetAccessMgr;
+ synchronized (this) {
+ dataverseRegister = globalRegister.get(dataset.getDataverseName());
+ if (dataverseRegister == null) {
+ // Create a register for the dataverse, and put the dataset their with the initial value of 0
+ dataverseRegister = new HashMap<String, ExternalDatasetAccessManager>();
+ datasetAccessMgr = new ExternalDatasetAccessManager();
+ dataverseRegister.put(dataset.getDatasetName(), datasetAccessMgr);
+ globalRegister.put(dataset.getDataverseName(), dataverseRegister);
+ } else {
+ datasetAccessMgr = dataverseRegister.get(dataset.getDatasetName());
+ if (datasetAccessMgr == null) {
+ datasetAccessMgr = new ExternalDatasetAccessManager();
+ dataverseRegister.put(dataset.getDatasetName(), datasetAccessMgr);
+ }
+ }
+ }
+ datasetAccessMgr.buildIndexBegin();
+ }
+
+ public void buildIndexEnd(Dataset dataset) {
+ HashMap<String, ExternalDatasetAccessManager> dataverseRegistry = globalRegister
+ .get(dataset.getDataverseName());
+ ExternalDatasetAccessManager datasetLockManager = dataverseRegistry.get(dataset.getDatasetName());
+ datasetLockManager.buildIndexEnd();
+ }
+
+ public void releaseAcquiredLocks(AqlMetadataProvider metadataProvider) {
+ Map<String, Integer> locks = metadataProvider.getLocks();
+ if (locks == null) {
+ return;
+ } else {
+ // if dataset was accessed already by this job, return the registered version
+ Set<Entry<String, Integer>> aquiredLocks = locks.entrySet();
+ for (Entry<String, Integer> entry : aquiredLocks) {
+ //Get dataverse name
+ String dvName = entry.getKey().substring(0, entry.getKey().indexOf("."));
+ String dsName = entry.getKey().substring(entry.getKey().indexOf(".") + 1);
+ HashMap<String, ExternalDatasetAccessManager> dataverseRegistry = globalRegister.get(dvName);
+ ExternalDatasetAccessManager datasetLockManager = dataverseRegistry.get(dsName);
+ datasetLockManager.queryEnd(entry.getValue());
+ }
+ locks.clear();
+ }
+ }
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java
index ecfc170..004c09a 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java
@@ -660,6 +660,9 @@
public static final FunctionIdentifier COLLECTION_TO_SEQUENCE = new FunctionIdentifier(
FunctionConstants.ASTERIX_NS, "" + "collection-to-sequence", 1);
+ public static final FunctionIdentifier EXTERNAL_LOOKUP = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+ "external-lookup", FunctionIdentifier.VARARGS);
+
public static IFunctionInfo getAsterixFunctionInfo(FunctionIdentifier fid) {
return registeredFunctions.get(fid);
}
@@ -991,6 +994,18 @@
addPrivateFunction(COLLECTION_TO_SEQUENCE, CollectionToSequenceTypeComputer.INSTANCE, true);
+ // external lookup
+ addPrivateFunction(EXTERNAL_LOOKUP, new IResultTypeComputer() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+ IMetadataProvider<?, ?> mp) throws AlgebricksException {
+ return BuiltinType.ANY;
+ }
+ }, false);
+
String metadataFunctionLoaderClassName = "edu.uci.ics.asterix.metadata.functions.MetadataBuiltinFunctions";
try {
Class.forName(metadataFunctionLoaderClassName);
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixClusterProperties.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixClusterProperties.java
index 0eef5db..9362844 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixClusterProperties.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixClusterProperties.java
@@ -52,6 +52,8 @@
private AlgebricksAbsolutePartitionConstraint clusterPartitionConstraint;
+ private boolean globalRecoveryCompleted = false;
+
private AsterixClusterProperties() {
InputStream is = this.getClass().getClassLoader().getResourceAsStream(CLUSTER_CONFIGURATION_FILE);
if (is != null) {
@@ -239,4 +241,12 @@
}
}
+
+ public boolean isGlobalRecoveryCompleted() {
+ return globalRecoveryCompleted;
+ }
+
+ public void setGlobalRecoveryCompleted(boolean globalRecoveryCompleted) {
+ this.globalRecoveryCompleted = globalRecoveryCompleted;
+ }
}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalBTreeSearchOperatorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalBTreeSearchOperatorDescriptor.java
new file mode 100644
index 0000000..2c8c901
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalBTreeSearchOperatorDescriptor.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.runtime.external;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+
+public class ExternalBTreeSearchOperatorDescriptor extends BTreeSearchOperatorDescriptor {
+
+ private static final long serialVersionUID = 1L;
+
+ public ExternalBTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
+ IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
+ IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
+ IBinaryComparatorFactory[] comparatorFactories, int[] bloomFilterKeyFields, int[] lowKeyFields,
+ int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
+ IIndexDataflowHelperFactory dataflowHelperFactory, boolean retainInput, boolean retainNull,
+ INullWriterFactory iNullWriterFactory, ISearchOperationCallbackFactory searchOpCallbackProvider) {
+ super(spec, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
+ comparatorFactories, bloomFilterKeyFields, lowKeyFields, highKeyFields, lowKeyInclusive,
+ highKeyInclusive, dataflowHelperFactory, retainInput, retainNull, iNullWriterFactory,
+ searchOpCallbackProvider);
+ }
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+ return new ExternalBTreeSearchOperatorNodePushable(this, ctx, partition, recordDescProvider, lowKeyFields,
+ highKeyFields, lowKeyInclusive, highKeyInclusive);
+ }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalBTreeSearchOperatorNodePushable.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalBTreeSearchOperatorNodePushable.java
new file mode 100644
index 0000000..258c739
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalBTreeSearchOperatorNodePushable.java
@@ -0,0 +1,87 @@
+/*
+ * 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.runtime.external;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorNodePushable;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeWithBuddyDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.ExternalBTreeWithBuddy;
+
+public class ExternalBTreeSearchOperatorNodePushable extends BTreeSearchOperatorNodePushable{
+
+ public ExternalBTreeSearchOperatorNodePushable(ExternalBTreeSearchOperatorDescriptor opDesc, IHyracksTaskContext ctx,
+ int partition, IRecordDescriptorProvider recordDescProvider, int[] lowKeyFields, int[] highKeyFields,
+ boolean lowKeyInclusive, boolean highKeyInclusive) {
+ super(opDesc, ctx, partition, recordDescProvider, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive);
+ }
+
+ // We override the open function to search a specific version of the index
+ @Override
+ public void open() throws HyracksDataException {
+ ExternalBTreeWithBuddyDataflowHelper dataFlowHelper = (ExternalBTreeWithBuddyDataflowHelper) indexHelper;
+ accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRecDesc);
+ writer.open();
+ dataFlowHelper.open();
+ index = indexHelper.getIndexInstance();
+
+ if (retainNull) {
+ int fieldCount = getFieldCount();
+ nullTupleBuild = new ArrayTupleBuilder(fieldCount);
+ DataOutput out = nullTupleBuild.getDataOutput();
+ for (int i = 0; i < fieldCount; i++) {
+ try {
+ nullWriter.writeNull(out);
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ nullTupleBuild.addFieldEndOffset();
+ }
+ } else {
+ nullTupleBuild = null;
+ }
+
+ ExternalBTreeWithBuddy externalIndex = (ExternalBTreeWithBuddy) index;
+ try {
+ searchPred = createSearchPredicate();
+ writeBuffer = ctx.allocateFrame();
+ tb = new ArrayTupleBuilder(recordDesc.getFieldCount());
+ dos = tb.getDataOutput();
+ appender = new FrameTupleAppender(ctx.getFrameSize());
+ appender.reset(writeBuffer, true);
+ ISearchOperationCallback searchCallback = opDesc.getSearchOpCallbackFactory()
+ .createSearchOperationCallback(indexHelper.getResourceID(), ctx);
+ // The next line is the reason we override this method
+ indexAccessor = externalIndex.createAccessor(searchCallback, dataFlowHelper.getTargetVersion());
+ cursor = createCursor();
+ if (retainInput) {
+ frameTuple = new FrameTupleReference();
+ }
+ } catch (Exception e) {
+ indexHelper.close();
+ throw new HyracksDataException(e);
+ }
+ }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalRTreeSearchOperatorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalRTreeSearchOperatorDescriptor.java
new file mode 100644
index 0000000..fcaaa5a
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalRTreeSearchOperatorDescriptor.java
@@ -0,0 +1,53 @@
+/*
+ * 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.runtime.external;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.dataflow.ExternalRTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.dataflow.RTreeSearchOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+
+public class ExternalRTreeSearchOperatorDescriptor extends RTreeSearchOperatorDescriptor {
+
+ private static final long serialVersionUID = 1L;
+
+ public ExternalRTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
+ IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
+ IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
+ IBinaryComparatorFactory[] comparatorFactories, int[] keyFields,
+ ExternalRTreeDataflowHelperFactory dataflowHelperFactory, boolean retainInput, boolean retainNull,
+ INullWriterFactory iNullWriterFactory, ISearchOperationCallbackFactory searchOpCallbackFactory) {
+ super(spec, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
+ comparatorFactories, keyFields, dataflowHelperFactory, retainInput, retainNull, iNullWriterFactory,
+ searchOpCallbackFactory);
+ }
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+ return new ExternalRTreeSearchOperatorNodePushable(this, ctx, partition, recordDescProvider, keyFields);
+ }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalRTreeSearchOperatorNodePushable.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalRTreeSearchOperatorNodePushable.java
new file mode 100644
index 0000000..d9352f4
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalRTreeSearchOperatorNodePushable.java
@@ -0,0 +1,87 @@
+/*
+ * 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.runtime.external;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.dataflow.ExternalRTreeDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.ExternalRTree;
+import edu.uci.ics.hyracks.storage.am.rtree.dataflow.RTreeSearchOperatorNodePushable;
+
+public class ExternalRTreeSearchOperatorNodePushable extends RTreeSearchOperatorNodePushable{
+
+ public ExternalRTreeSearchOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
+ int partition, IRecordDescriptorProvider recordDescProvider, int[] keyFields) {
+ super(opDesc, ctx, partition, recordDescProvider, keyFields);
+ }
+
+ // We override this method to specify the searched version of the index
+ @Override
+ public void open() throws HyracksDataException {
+ accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRecDesc);
+ writer.open();
+ indexHelper.open();
+ ExternalRTreeDataflowHelper rTreeDataflowHelper = (ExternalRTreeDataflowHelper) indexHelper;
+ index = indexHelper.getIndexInstance();
+
+ if (retainNull) {
+ int fieldCount = getFieldCount();
+ nullTupleBuild = new ArrayTupleBuilder(fieldCount);
+ DataOutput out = nullTupleBuild.getDataOutput();
+ for (int i = 0; i < fieldCount; i++) {
+ try {
+ nullWriter.writeNull(out);
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ nullTupleBuild.addFieldEndOffset();
+ }
+ } else {
+ nullTupleBuild = null;
+ }
+
+ ExternalRTree rTreeIndex = (ExternalRTree) index;
+ try {
+ searchPred = createSearchPredicate();
+ writeBuffer = ctx.allocateFrame();
+ tb = new ArrayTupleBuilder(recordDesc.getFieldCount());
+ dos = tb.getDataOutput();
+ appender = new FrameTupleAppender(ctx.getFrameSize());
+ appender.reset(writeBuffer, true);
+ ISearchOperationCallback searchCallback = opDesc.getSearchOpCallbackFactory()
+ .createSearchOperationCallback(indexHelper.getResourceID(), ctx);
+ // The next line is the reason we override this method
+ indexAccessor = rTreeIndex.createAccessor(searchCallback, rTreeDataflowHelper.getTargetVersion());
+ cursor = createCursor();
+ if (retainInput) {
+ frameTuple = new FrameTupleReference();
+ }
+ } catch (Exception e) {
+ indexHelper.close();
+ throw new HyracksDataException(e);
+ }
+ }
+
+}
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/ExternalFilesIndexOperatorDescriptor.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/ExternalFilesIndexOperatorDescriptor.java
new file mode 100644
index 0000000..4ce142e
--- /dev/null
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/ExternalFilesIndexOperatorDescriptor.java
@@ -0,0 +1,161 @@
+/*
+ * 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.tools.external.data;
+
+import java.io.IOException;
+import java.util.List;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.external.indexing.dataflow.FileIndexTupleTranslator;
+import edu.uci.ics.asterix.metadata.entities.ExternalFile;
+import edu.uci.ics.asterix.metadata.external.FilesIndexDescription;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+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;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorNodePushable;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+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.impls.NoOpOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.ExternalBTree;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.ExternalBTree.LSMTwoPCBTreeBulkLoader;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactoryProvider;
+
+/**
+ * This operator is intended solely for external dataset files replicated index.
+ * It either create and bulkload when used for a new index
+ * or bulkmodify the index creating a hidden transaction component which later might be committed or deleted by another operator
+ *
+ * @author alamouda
+ */
+public class ExternalFilesIndexOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ private boolean createNewIndex;
+ private List<ExternalFile> files;
+
+ public ExternalFilesIndexOperatorDescriptor(IOperatorDescriptorRegistry spec,
+ IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
+ IFileSplitProvider fileSplitProvider, IIndexDataflowHelperFactory dataflowHelperFactory,
+ ILocalResourceFactoryProvider localResourceFactoryProvider, List<ExternalFile> files, boolean createNewIndex) {
+ super(spec, 0, 0, null, storageManager, lifecycleManagerProvider, fileSplitProvider,
+ FilesIndexDescription.EXTERNAL_FILE_INDEX_TYPE_TRAITS,
+ FilesIndexDescription.FILES_INDEX_COMP_FACTORIES, FilesIndexDescription.BLOOM_FILTER_FIELDS,
+ dataflowHelperFactory, null, false, false, null, localResourceFactoryProvider,
+ NoOpOperationCallbackFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE);
+ this.createNewIndex = createNewIndex;
+ this.files = files;
+ }
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
+ final IIndexDataflowHelper indexHelper = getIndexDataflowHelperFactory().createIndexDataflowHelper(this, ctx,
+ partition);
+ return new AbstractOperatorNodePushable() {
+
+ @SuppressWarnings("incomplete-switch")
+ @Override
+ public void initialize() throws HyracksDataException {
+ FileIndexTupleTranslator filesTupleTranslator = new FileIndexTupleTranslator();
+ if (createNewIndex) {
+ // Create
+ indexHelper.create();
+ // Open and get
+ indexHelper.open();
+ try {
+ IIndex index = indexHelper.getIndexInstance();
+ // Create bulk loader
+
+ IIndexBulkLoader bulkLoader = index.createBulkLoader(BTree.DEFAULT_FILL_FACTOR, false,
+ files.size(), false);
+ // Load files
+ for (ExternalFile file : files) {
+ bulkLoader.add(filesTupleTranslator.getTupleFromFile(file));
+ }
+ bulkLoader.end();
+ } catch (IndexException | IOException | AsterixException e) {
+ throw new HyracksDataException(e);
+ } finally {
+ indexHelper.close();
+ }
+ } else {
+ ///////// Bulk modify //////////
+ // Open and get
+ indexHelper.open();
+ IIndex index = indexHelper.getIndexInstance();
+ LSMTwoPCBTreeBulkLoader bulkLoader = null;
+ try {
+ bulkLoader = (LSMTwoPCBTreeBulkLoader) ((ExternalBTree) index).createTransactionBulkLoader(
+ BTree.DEFAULT_FILL_FACTOR, false, files.size(), false);
+ // Load files
+ // The files must be ordered according to their numbers
+ for (ExternalFile file : files) {
+ switch (file.getPendingOp()) {
+ case PENDING_ADD_OP:
+ case PENDING_APPEND_OP:
+ bulkLoader.add(filesTupleTranslator.getTupleFromFile(file));
+ break;
+ case PENDING_DROP_OP:
+ bulkLoader.delete(filesTupleTranslator.getTupleFromFile(file));
+ break;
+ }
+ }
+ bulkLoader.end();
+ } catch (IndexException | IOException | AsterixException e) {
+ if (bulkLoader != null) {
+ bulkLoader.abort();
+ }
+ throw new HyracksDataException(e);
+ } finally {
+ indexHelper.close();
+ }
+ }
+ }
+
+ @Override
+ public void deinitialize() throws HyracksDataException {
+ }
+
+ @Override
+ public int getInputArity() {
+ return 0;
+ }
+
+ @Override
+ public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc)
+ throws HyracksDataException {
+ }
+
+ @Override
+ public IFrameWriter getInputFrameWriter(int index) {
+ return null;
+ }
+
+ };
+ }
+
+}
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 ce14bb2..1f5de77 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
@@ -23,12 +23,14 @@
import org.apache.commons.lang3.StringUtils;
import edu.uci.ics.asterix.external.adapter.factory.StreamBasedAdapterFactory;
+import edu.uci.ics.asterix.metadata.entities.ExternalFile;
import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
import edu.uci.ics.asterix.metadata.feeds.IGenericAdapterFactory;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.util.AsterixRuntimeUtil;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
@@ -143,4 +145,9 @@
sockets.add(p);
}
}
+
+ @Override
+ public void setFiles(List<ExternalFile> files) throws AlgebricksException {
+ throw new AlgebricksException("files access not supported for this adapter");
+ }
}
\ No newline at end of file
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 dc558d7..d3b2120 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
@@ -16,6 +16,7 @@
import java.io.IOException;
import java.io.InputStream;
+import java.util.List;
import java.util.Map;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
@@ -23,6 +24,7 @@
import edu.uci.ics.asterix.external.adapter.factory.NCFileSystemAdapterFactory;
import edu.uci.ics.asterix.external.adapter.factory.StreamBasedAdapterFactory;
import edu.uci.ics.asterix.external.dataset.adapter.FileSystemBasedAdapter;
+import edu.uci.ics.asterix.metadata.entities.ExternalFile;
import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
import edu.uci.ics.asterix.metadata.feeds.IGenericAdapterFactory;
import edu.uci.ics.asterix.om.types.ARecordType;
@@ -32,6 +34,7 @@
import edu.uci.ics.asterix.runtime.operators.file.DelimitedDataParser;
import edu.uci.ics.asterix.runtime.operators.file.IDataParser;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
@@ -157,6 +160,11 @@
return fieldParserFactories;
}
+ @Override
+ public void setFiles(List<ExternalFile> files) throws AlgebricksException{
+ throw new AlgebricksException("can't set files for this Adapter");
+ }
+
}
class RateControlledTupleParserFactory implements ITupleParserFactory {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ExternalBTreeLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ExternalBTreeLocalResourceMetadata.java
new file mode 100644
index 0000000..f4a50fc
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ExternalBTreeLocalResourceMetadata.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.transaction.management.resource;
+
+import java.io.File;
+import java.util.Map;
+
+import edu.uci.ics.asterix.common.context.BaseOperationTracker;
+import edu.uci.ics.asterix.common.context.DatasetLifecycleManager;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
+import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTree;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
+
+public class ExternalBTreeLocalResourceMetadata extends LSMBTreeLocalResourceMetadata {
+
+ private static final long serialVersionUID = 1L;
+
+ public ExternalBTreeLocalResourceMetadata(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
+ int[] bloomFilterKeyFields, boolean isPrimary, int datasetID, ILSMMergePolicyFactory mergePolicyFactory,
+ Map<String, String> mergePolicyProperties) {
+ super(typeTraits, cmpFactories, bloomFilterKeyFields, isPrimary, datasetID, mergePolicyFactory,
+ mergePolicyProperties);
+ }
+
+ @Override
+ public ILSMIndex createIndexInstance(IAsterixAppRuntimeContextProvider runtimeContextProvider, String filePath,
+ int partition) {
+ FileReference file = new FileReference(new File(filePath));
+ LSMBTree lsmBTree = LSMBTreeUtils.createExternalBTree(file, runtimeContextProvider.getBufferCache(),
+ runtimeContextProvider.getFileMapManager(), typeTraits, cmpFactories, bloomFilterKeyFields,
+ runtimeContextProvider.getBloomFilterFalsePositiveRate(), mergePolicyFactory
+ .createMergePolicy(mergePolicyProperties), new BaseOperationTracker(
+ (DatasetLifecycleManager) runtimeContextProvider.getIndexLifecycleManager(), datasetID),
+ runtimeContextProvider.getLSMIOScheduler(), LSMBTreeIOOperationCallbackFactory.INSTANCE
+ .createIOOperationCallback(), -1);
+ return lsmBTree;
+ }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ExternalBTreeWithBuddyLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ExternalBTreeWithBuddyLocalResourceMetadata.java
new file mode 100644
index 0000000..e86da4a
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ExternalBTreeWithBuddyLocalResourceMetadata.java
@@ -0,0 +1,68 @@
+/*
+ * 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.transaction.management.resource;
+
+import java.io.File;
+import java.util.Map;
+
+import edu.uci.ics.asterix.common.context.BaseOperationTracker;
+import edu.uci.ics.asterix.common.context.DatasetLifecycleManager;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeWithBuddyIOOperationCallbackFactory;
+import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
+
+/**
+ * The local resource for disk only lsm btree with buddy tree
+ */
+public class ExternalBTreeWithBuddyLocalResourceMetadata extends AbstractLSMLocalResourceMetadata {
+
+ private static final long serialVersionUID = 1L;
+
+ private final ITypeTraits[] typeTraits;
+ private final IBinaryComparatorFactory[] btreeCmpFactories;
+ private final ILSMMergePolicyFactory mergePolicyFactory;
+ private final Map<String, String> mergePolicyProperties;
+ private final int[] buddyBtreeFields;
+
+ public ExternalBTreeWithBuddyLocalResourceMetadata(int datasetID, IBinaryComparatorFactory[] btreeCmpFactories,
+ ITypeTraits[] typeTraits, ILSMMergePolicyFactory mergePolicyFactory,
+ Map<String, String> mergePolicyProperties, int[] buddyBtreeFields) {
+ super(datasetID);
+ this.btreeCmpFactories = btreeCmpFactories;
+ this.typeTraits = typeTraits;
+ this.mergePolicyFactory = mergePolicyFactory;
+ this.mergePolicyProperties = mergePolicyProperties;
+ this.buddyBtreeFields = buddyBtreeFields;
+ }
+
+ @Override
+ public ILSMIndex createIndexInstance(IAsterixAppRuntimeContextProvider runtimeContextProvider, String filePath,
+ int partition) throws HyracksDataException {
+ FileReference file = new FileReference(new File(filePath));
+ return LSMBTreeUtils.createExternalBTreeWithBuddy(file, runtimeContextProvider.getBufferCache(),
+ runtimeContextProvider.getFileMapManager(), typeTraits, btreeCmpFactories, runtimeContextProvider
+ .getBloomFilterFalsePositiveRate(),
+ mergePolicyFactory.createMergePolicy(mergePolicyProperties), new BaseOperationTracker(
+ (DatasetLifecycleManager) runtimeContextProvider.getIndexLifecycleManager(), datasetID),
+ runtimeContextProvider.getLSMIOScheduler(), LSMBTreeWithBuddyIOOperationCallbackFactory.INSTANCE
+ .createIOOperationCallback(), buddyBtreeFields, -1);
+ }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ExternalRTreeLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ExternalRTreeLocalResourceMetadata.java
new file mode 100644
index 0000000..e88ec08
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ExternalRTreeLocalResourceMetadata.java
@@ -0,0 +1,69 @@
+/*
+ * 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.transaction.management.resource;
+
+import java.io.File;
+import java.util.Map;
+
+import edu.uci.ics.asterix.common.context.BaseOperationTracker;
+import edu.uci.ics.asterix.common.context.DatasetLifecycleManager;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
+import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.utils.LSMRTreeUtils;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+
+/**
+ * The local resource class for disk only lsm r-tree
+ */
+public class ExternalRTreeLocalResourceMetadata extends LSMRTreeLocalResourceMetadata {
+
+ private static final long serialVersionUID = 1L;
+
+ public ExternalRTreeLocalResourceMetadata(ITypeTraits[] typeTraits,
+ IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
+ IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
+ ILinearizeComparatorFactory linearizeCmpFactory, int datasetID, ILSMMergePolicyFactory mergePolicyFactory,
+ Map<String, String> mergePolicyProperties, int[] btreeFields) {
+ super(typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
+ linearizeCmpFactory, datasetID, mergePolicyFactory, mergePolicyProperties, btreeFields);
+ }
+
+ @Override
+ public ILSMIndex createIndexInstance(IAsterixAppRuntimeContextProvider runtimeContextProvider, String filePath,
+ int partition) throws HyracksDataException {
+ FileReference file = new FileReference(new File(filePath));
+ try {
+ return LSMRTreeUtils.createExternalRTree(file, runtimeContextProvider.getBufferCache(),
+ runtimeContextProvider.getFileMapManager(), typeTraits, rtreeCmpFactories, btreeCmpFactories,
+ valueProviderFactories, rtreePolicyType, runtimeContextProvider.getBloomFilterFalsePositiveRate(),
+ mergePolicyFactory.createMergePolicy(mergePolicyProperties), new BaseOperationTracker(
+ (DatasetLifecycleManager) runtimeContextProvider.getIndexLifecycleManager(), datasetID),
+ runtimeContextProvider.getLSMIOScheduler(), LSMRTreeIOOperationCallbackFactory.INSTANCE
+ .createIOOperationCallback(), linearizeCmpFactory, btreeFields, -1);
+ } catch (TreeIndexException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java
index 3951226..e6cc42e 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java
@@ -35,12 +35,12 @@
private static final long serialVersionUID = 1L;
- private final ITypeTraits[] typeTraits;
- private final IBinaryComparatorFactory[] cmpFactories;
- private final int[] bloomFilterKeyFields;
- private final boolean isPrimary;
- private final ILSMMergePolicyFactory mergePolicyFactory;
- private final Map<String, String> mergePolicyProperties;
+ protected final ITypeTraits[] typeTraits;
+ protected final IBinaryComparatorFactory[] cmpFactories;
+ protected final int[] bloomFilterKeyFields;
+ protected final boolean isPrimary;
+ protected final ILSMMergePolicyFactory mergePolicyFactory;
+ protected final Map<String, String> mergePolicyProperties;
public LSMBTreeLocalResourceMetadata(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
int[] bloomFilterKeyFields, boolean isPrimary, int datasetID, ILSMMergePolicyFactory mergePolicyFactory,
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java
index 0776db8..06e92c8 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java
@@ -39,15 +39,15 @@
private static final long serialVersionUID = 1L;
- private final ITypeTraits[] typeTraits;
- private final IBinaryComparatorFactory[] rtreeCmpFactories;
- private final IBinaryComparatorFactory[] btreeCmpFactories;
- private final IPrimitiveValueProviderFactory[] valueProviderFactories;
- private final RTreePolicyType rtreePolicyType;
- private final ILinearizeComparatorFactory linearizeCmpFactory;
- private final ILSMMergePolicyFactory mergePolicyFactory;
- private final Map<String, String> mergePolicyProperties;
- private final int[] btreeFields;
+ protected final ITypeTraits[] typeTraits;
+ protected final IBinaryComparatorFactory[] rtreeCmpFactories;
+ protected final IBinaryComparatorFactory[] btreeCmpFactories;
+ protected final IPrimitiveValueProviderFactory[] valueProviderFactories;
+ protected final RTreePolicyType rtreePolicyType;
+ protected final ILinearizeComparatorFactory linearizeCmpFactory;
+ protected final ILSMMergePolicyFactory mergePolicyFactory;
+ protected final Map<String, String> mergePolicyProperties;
+ protected final int[] btreeFields;
public LSMRTreeLocalResourceMetadata(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,