checkpoint towards fixing diskComponents' LSN
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 693a19e..ba940d6 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,7 +39,6 @@
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.ExternalDataAccessByRIDOperator;
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;
@@ -388,13 +387,6 @@
public ILogicalOperator visitUnnestMapOperator(UnnestMapOperator op, ILogicalOperator arg) {
throw new UnsupportedOperationException();
}
-
- @Override
- public ILogicalOperator visitExternalDataAccessByRIDOperator(
- ExternalDataAccessByRIDOperator op, ILogicalOperator arg)
- throws AlgebricksException {
- throw new UnsupportedOperationException();
- }
@Override
public ILogicalOperator visitUnnestOperator(UnnestOperator op, ILogicalOperator arg) throws AlgebricksException {
@@ -446,5 +438,4 @@
public Map<LogicalVariable, LogicalVariable> getVariableMapping() {
return outVarMapping;
}
-
}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntime.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntime.java
index 6dd11bd..32bfa58 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntime.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntime.java
@@ -72,7 +72,7 @@
@Override
public void open() throws HyracksDataException {
try {
- transactionContext = transactionManager.getTransactionContext(jobId);
+ transactionContext = transactionManager.getTransactionContext(jobId, false);
transactionContext.setWriteTxn(isWriteTransaction);
} catch (ACIDException e) {
throw new HyracksDataException(e);
@@ -89,7 +89,11 @@
pkHash = computePrimaryKeyHashValue(frameTupleReference, primaryKeyFields);
logRecord.formEntityCommitLogRecord(transactionContext, datasetId, pkHash, frameTupleReference,
primaryKeyFields);
- logMgr.log(logRecord);
+ try {
+ logMgr.log(logRecord);
+ } catch (ACIDException e) {
+ throw new HyracksDataException(e);
+ }
}
}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/ExternalDataAccessByRIDPOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/ExternalDataAccessByRIDPOperator.java
deleted file mode 100644
index 447555e..0000000
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/ExternalDataAccessByRIDPOperator.java
+++ /dev/null
@@ -1,110 +0,0 @@
-package edu.uci.ics.asterix.algebra.operators.physical;
-
-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.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.PhysicalOperatorTag;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
-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.ExternalDataAccessByRIDOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AbstractScanPOperator;
-import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
-
-public class ExternalDataAccessByRIDPOperator extends AbstractScanPOperator{
-
- private AqlSourceId datasetId;
- private Dataset dataset;
- private ARecordType recordType;
- private Index secondaryIndex;
- public ExternalDataAccessByRIDPOperator(AqlSourceId datasetId, Dataset dataset, ARecordType recordType,Index secondaryIndex)
- {
- this.datasetId = datasetId;
- this.dataset = dataset;
- this.recordType = recordType;
- this.secondaryIndex = secondaryIndex;
- }
-
- 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_ACCESS_BY_RID;
- }
-
- @Override
- public void computeDeliveredProperties(ILogicalOperator op,
- IOptimizationContext context) throws AlgebricksException {
- AqlDataSource ds = new AqlDataSource(datasetId, dataset, recordType);
- IDataSourcePropertiesProvider dspp = ds.getPropertiesProvider();
- AbstractScanOperator as = (AbstractScanOperator) op;
- deliveredProperties = dspp.computePropertiesVector(as.getVariables());
- }
-
- @Override
- public void contributeRuntimeOperator(IHyracksJobBuilder builder,
- JobGenContext context, ILogicalOperator op,
- IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas,
- IOperatorSchema outerPlanSchema) throws AlgebricksException {
- ExternalDataAccessByRIDOperator edabro = (ExternalDataAccessByRIDOperator) 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_ACCESS_BY_RID)) {
- return;
- }
- AqlMetadataProvider metadataProvider = (AqlMetadataProvider) context.getMetadataProvider();
- Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> externalAccessByRID = metadataProvider.buildExternalDataAccesByRIDRuntime(
- builder.getJobSpec(), dataset,secondaryIndex);
- builder.contributeHyracksOperator(edabro, externalAccessByRID.first);
- builder.contributeAlgebricksPartitionConstraint(externalAccessByRID.first, externalAccessByRID.second);
- ILogicalOperator srcExchange = edabro.getInputs().get(0).getValue();
- builder.contributeGraphEdge(srcExchange, 0, edabro, 0);
- }
-
- @Override
- public boolean isMicroOperator() {
- return false;
- }
-
-}
\ No newline at end of file
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 95d29e3..ab0fd79 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,12 +22,8 @@
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.commons.lang3.mutable.MutableObject;
-import edu.uci.ics.asterix.algebra.operators.physical.ExternalDataAccessByRIDPOperator;
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.metadata.declared.AqlMetadataProvider;
-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.utils.DatasetUtils;
@@ -38,7 +34,6 @@
import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -57,7 +52,6 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
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.ExternalDataAccessByRIDOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
@@ -66,384 +60,290 @@
* Static helper functions for rewriting plans using indexes.
*/
public class AccessMethodUtils {
- public static void appendPrimaryIndexTypes(Dataset dataset, IAType itemType, List<Object> target)
- throws IOException {
- ARecordType recordType = (ARecordType) itemType;
- List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
- for (String partitioningKey : partitioningKeys) {
- target.add(recordType.getFieldType(partitioningKey));
- }
- target.add(itemType);
- }
+ public static void appendPrimaryIndexTypes(Dataset dataset, IAType itemType, List<Object> target)
+ throws IOException {
+ ARecordType recordType = (ARecordType) itemType;
+ List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+ for (String partitioningKey : partitioningKeys) {
+ target.add(recordType.getFieldType(partitioningKey));
+ }
+ target.add(itemType);
+ }
- public static void appendExternalRecTypes(Dataset dataset, IAType itemType, List<Object> target){
- target.add(itemType);
- }
+ public static ConstantExpression createStringConstant(String str) {
+ return new ConstantExpression(new AsterixConstantValue(new AString(str)));
+ }
- public static void appendExternalRecPrimaryKey(Dataset dataset, List<Object> target){
- target.add(BuiltinType.ASTRING);
- target.add(BuiltinType.AINT64);
- if(DatasetUtils.getExternalRIDSize(dataset) == 3)
- {
- target.add(BuiltinType.AINT32);
- }
- }
+ public static ConstantExpression createInt32Constant(int i) {
+ return new ConstantExpression(new AsterixConstantValue(new AInt32(i)));
+ }
- public static ConstantExpression createStringConstant(String str) {
- return new ConstantExpression(new AsterixConstantValue(new AString(str)));
- }
+ public static ConstantExpression createBooleanConstant(boolean b) {
+ if (b) {
+ return new ConstantExpression(new AsterixConstantValue(ABoolean.TRUE));
+ } else {
+ return new ConstantExpression(new AsterixConstantValue(ABoolean.FALSE));
+ }
+ }
- public static ConstantExpression createInt32Constant(int i) {
- return new ConstantExpression(new AsterixConstantValue(new AInt32(i)));
- }
+ public static String getStringConstant(Mutable<ILogicalExpression> expr) {
+ IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
+ return ((AString) obj).getStringValue();
+ }
- public static ConstantExpression createBooleanConstant(boolean b) {
- if (b) {
- return new ConstantExpression(new AsterixConstantValue(ABoolean.TRUE));
- } else {
- return new ConstantExpression(new AsterixConstantValue(ABoolean.FALSE));
- }
- }
+ public static int getInt32Constant(Mutable<ILogicalExpression> expr) {
+ IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
+ return ((AInt32) obj).getIntegerValue();
+ }
- public static String getStringConstant(Mutable<ILogicalExpression> expr) {
- IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
- return ((AString) obj).getStringValue();
- }
+ public static boolean getBooleanConstant(Mutable<ILogicalExpression> expr) {
+ IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
+ return ((ABoolean) obj).getBoolean();
+ }
- public static int getInt32Constant(Mutable<ILogicalExpression> expr) {
- IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
- return ((AInt32) obj).getIntegerValue();
- }
+ public static boolean analyzeFuncExprArgsForOneConstAndVar(AbstractFunctionCallExpression funcExpr,
+ AccessMethodAnalysisContext analysisCtx) {
+ IAlgebricksConstantValue constFilterVal = null;
+ LogicalVariable fieldVar = null;
+ ILogicalExpression arg1 = funcExpr.getArguments().get(0).getValue();
+ ILogicalExpression arg2 = funcExpr.getArguments().get(1).getValue();
+ // One of the args must be a constant, and the other arg must be a variable.
+ if (arg1.getExpressionTag() == LogicalExpressionTag.CONSTANT
+ && arg2.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ ConstantExpression constExpr = (ConstantExpression) arg1;
+ constFilterVal = constExpr.getValue();
+ VariableReferenceExpression varExpr = (VariableReferenceExpression) arg2;
+ fieldVar = varExpr.getVariableReference();
+ } else if (arg1.getExpressionTag() == LogicalExpressionTag.VARIABLE
+ && arg2.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+ ConstantExpression constExpr = (ConstantExpression) arg2;
+ constFilterVal = constExpr.getValue();
+ VariableReferenceExpression varExpr = (VariableReferenceExpression) arg1;
+ fieldVar = varExpr.getVariableReference();
+ } else {
+ return false;
+ }
+ analysisCtx.matchedFuncExprs.add(new OptimizableFuncExpr(funcExpr, fieldVar, constFilterVal));
+ return true;
+ }
- public static boolean getBooleanConstant(Mutable<ILogicalExpression> expr) {
- IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
- return ((ABoolean) obj).getBoolean();
- }
+ public static boolean analyzeFuncExprArgsForTwoVars(AbstractFunctionCallExpression funcExpr,
+ AccessMethodAnalysisContext analysisCtx) {
+ LogicalVariable fieldVar1 = null;
+ LogicalVariable fieldVar2 = null;
+ ILogicalExpression arg1 = funcExpr.getArguments().get(0).getValue();
+ ILogicalExpression arg2 = funcExpr.getArguments().get(1).getValue();
+ if (arg1.getExpressionTag() == LogicalExpressionTag.VARIABLE
+ && arg2.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ fieldVar1 = ((VariableReferenceExpression) arg1).getVariableReference();
+ fieldVar2 = ((VariableReferenceExpression) arg2).getVariableReference();
+ } else {
+ return false;
+ }
+ analysisCtx.matchedFuncExprs.add(new OptimizableFuncExpr(funcExpr,
+ new LogicalVariable[] { fieldVar1, fieldVar2 }, null));
+ return true;
+ }
- public static boolean analyzeFuncExprArgsForOneConstAndVar(AbstractFunctionCallExpression funcExpr,
- AccessMethodAnalysisContext analysisCtx) {
- IAlgebricksConstantValue constFilterVal = null;
- LogicalVariable fieldVar = null;
- ILogicalExpression arg1 = funcExpr.getArguments().get(0).getValue();
- ILogicalExpression arg2 = funcExpr.getArguments().get(1).getValue();
- // One of the args must be a constant, and the other arg must be a variable.
- if (arg1.getExpressionTag() == LogicalExpressionTag.CONSTANT
- && arg2.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
- ConstantExpression constExpr = (ConstantExpression) arg1;
- constFilterVal = constExpr.getValue();
- VariableReferenceExpression varExpr = (VariableReferenceExpression) arg2;
- fieldVar = varExpr.getVariableReference();
- } else if (arg1.getExpressionTag() == LogicalExpressionTag.VARIABLE
- && arg2.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
- ConstantExpression constExpr = (ConstantExpression) arg2;
- constFilterVal = constExpr.getValue();
- VariableReferenceExpression varExpr = (VariableReferenceExpression) arg1;
- fieldVar = varExpr.getVariableReference();
- } else {
- return false;
- }
- analysisCtx.matchedFuncExprs.add(new OptimizableFuncExpr(funcExpr, fieldVar, constFilterVal));
- return true;
- }
+ public static int getNumSecondaryKeys(Index index, ARecordType recordType) throws AlgebricksException {
+ switch (index.getIndexType()) {
+ case BTREE:
+ case SINGLE_PARTITION_WORD_INVIX:
+ case SINGLE_PARTITION_NGRAM_INVIX:
+ case LENGTH_PARTITIONED_WORD_INVIX:
+ case LENGTH_PARTITIONED_NGRAM_INVIX: {
+ return index.getKeyFieldNames().size();
+ }
+ case RTREE: {
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(index.getKeyFieldNames().get(0),
+ recordType);
+ IAType keyType = keyPairType.first;
+ int numDimensions = NonTaggedFormatUtil.getNumDimensions(keyType.getTypeTag());
+ return numDimensions * 2;
+ }
+ default: {
+ throw new AlgebricksException("Unknown index kind: " + index.getIndexType());
+ }
+ }
+ }
- public static boolean analyzeFuncExprArgsForTwoVars(AbstractFunctionCallExpression funcExpr,
- AccessMethodAnalysisContext analysisCtx) {
- LogicalVariable fieldVar1 = null;
- LogicalVariable fieldVar2 = null;
- ILogicalExpression arg1 = funcExpr.getArguments().get(0).getValue();
- ILogicalExpression arg2 = funcExpr.getArguments().get(1).getValue();
- if (arg1.getExpressionTag() == LogicalExpressionTag.VARIABLE
- && arg2.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
- fieldVar1 = ((VariableReferenceExpression) arg1).getVariableReference();
- fieldVar2 = ((VariableReferenceExpression) arg2).getVariableReference();
- } else {
- return false;
- }
- analysisCtx.matchedFuncExprs.add(new OptimizableFuncExpr(funcExpr,
- new LogicalVariable[] { fieldVar1, fieldVar2 }, null));
- return true;
- }
+ /**
+ * Appends the types of the fields produced by the given secondary index to dest.
+ */
+ public static void appendSecondaryIndexTypes(Dataset dataset, ARecordType recordType, Index index,
+ boolean primaryKeysOnly, List<Object> dest) throws AlgebricksException {
+ if (!primaryKeysOnly) {
+ switch (index.getIndexType()) {
+ case BTREE:
+ case SINGLE_PARTITION_WORD_INVIX:
+ case SINGLE_PARTITION_NGRAM_INVIX: {
+ for (String sk : index.getKeyFieldNames()) {
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(sk, recordType);
+ dest.add(keyPairType.first);
+ }
+ break;
+ }
+ case RTREE: {
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(index.getKeyFieldNames()
+ .get(0), recordType);
+ IAType keyType = keyPairType.first;
+ IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(keyType.getTypeTag());
+ int numKeys = getNumSecondaryKeys(index, recordType);
+ for (int i = 0; i < numKeys; i++) {
+ dest.add(nestedKeyType);
+ }
+ break;
+ }
+ }
+ }
+ // Primary keys.
+ 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 int getNumSecondaryKeys(Index index, ARecordType recordType) throws AlgebricksException {
- switch (index.getIndexType()) {
- case BTREE:
- case SINGLE_PARTITION_WORD_INVIX:
- case SINGLE_PARTITION_NGRAM_INVIX:
- case LENGTH_PARTITIONED_WORD_INVIX:
- case LENGTH_PARTITIONED_NGRAM_INVIX: {
- return index.getKeyFieldNames().size();
- }
- case RTREE: {
- Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(index.getKeyFieldNames().get(0),
- recordType);
- IAType keyType = keyPairType.first;
- int numDimensions = NonTaggedFormatUtil.getNumDimensions(keyType.getTypeTag());
- return numDimensions * 2;
- }
- default: {
- throw new AlgebricksException("Unknown index kind: " + index.getIndexType());
- }
- }
- }
+ 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 numSecondaryKeys = getNumSecondaryKeys(index, recordType);
+ int numVars = (primaryKeysOnly) ? numPrimaryKeys : numPrimaryKeys + numSecondaryKeys;
+ for (int i = 0; i < numVars; i++) {
+ dest.add(context.newVar());
+ }
+ }
- /**
- * Appends the types of the fields produced by the given secondary index to dest.
- */
- public static void appendSecondaryIndexTypes(Dataset dataset, ARecordType recordType, Index index,
- boolean primaryKeysOnly, List<Object> dest) throws AlgebricksException {
- if (!primaryKeysOnly) {
- switch (index.getIndexType()) {
- case BTREE:
- case SINGLE_PARTITION_WORD_INVIX:
- case SINGLE_PARTITION_NGRAM_INVIX: {
- for (String sk : index.getKeyFieldNames()) {
- Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(sk, recordType);
- dest.add(keyPairType.first);
- }
- break;
- }
- case RTREE: {
- Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(index.getKeyFieldNames()
- .get(0), recordType);
- IAType keyType = keyPairType.first;
- IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(keyType.getTypeTag());
- int numKeys = getNumSecondaryKeys(index, recordType);
- for (int i = 0; i < numKeys; i++) {
- dest.add(nestedKeyType);
- }
- break;
- }
- }
- }
- // Primary keys.
- if(dataset.getDatasetType() == DatasetType.EXTERNAL)
- {
- //add primary keys
- appendExternalRecPrimaryKey(dataset, dest);
- }
- 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 List<LogicalVariable> getPrimaryKeyVarsFromUnnestMap(Dataset dataset, ILogicalOperator unnestMapOp) {
+ int 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.
+ int start = sourceVars.size() - numPrimaryKeys;
+ int stop = sourceVars.size();
+ for (int i = start; i < stop; i++) {
+ primaryKeyVars.add(sourceVars.get(i));
+ }
+ return primaryKeyVars;
+ }
- public static void appendSecondaryIndexOutputVars(Dataset dataset, ARecordType recordType, Index index,
- boolean primaryKeysOnly, IOptimizationContext context, List<LogicalVariable> dest)
- throws AlgebricksException {
- int numPrimaryKeys=0;
- if(dataset.getDatasetType() == DatasetType.EXTERNAL)
- {
- numPrimaryKeys = DatasetUtils.getExternalRIDSize(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++) {
- dest.add(context.newVar());
- }
- }
+ /**
+ * Returns the search key expression which feeds a secondary-index search. If we are optimizing a selection query then this method returns
+ * the a ConstantExpression from the first constant value in the optimizable function expression.
+ * If we are optimizing a join, then this method returns the VariableReferenceExpression that should feed the secondary index probe.
+ */
+ public static ILogicalExpression createSearchKeyExpr(IOptimizableFuncExpr optFuncExpr,
+ OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree) {
+ if (probeSubTree == null) {
+ // We are optimizing a selection query. Search key is a constant.
+ return new ConstantExpression(optFuncExpr.getConstantVal(0));
+ } else {
+ // We are optimizing a join query. Determine which variable feeds the secondary index.
+ if (optFuncExpr.getOperatorSubTree(0) == null || optFuncExpr.getOperatorSubTree(0) == probeSubTree) {
+ return new VariableReferenceExpression(optFuncExpr.getLogicalVar(0));
+ } else {
+ return new VariableReferenceExpression(optFuncExpr.getLogicalVar(1));
+ }
+ }
+ }
- public static List<LogicalVariable> getPrimaryKeyVarsFromUnnestMap(Dataset dataset, ILogicalOperator unnestMapOp) {
- int numPrimaryKeys;
- if(dataset.getDatasetType() == DatasetType.EXTERNAL)
- {
- numPrimaryKeys = DatasetUtils.getExternalRIDSize(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.
- int start = sourceVars.size() - numPrimaryKeys;
- int stop = sourceVars.size();
- for (int i = start; i < stop; i++) {
- primaryKeyVars.add(sourceVars.get(i));
- }
- return primaryKeyVars;
- }
+ /**
+ * Returns the first expr optimizable by this index.
+ */
+ public static IOptimizableFuncExpr chooseFirstOptFuncExpr(Index chosenIndex, AccessMethodAnalysisContext analysisCtx) {
+ List<Integer> indexExprs = analysisCtx.getIndexExprs(chosenIndex);
+ int firstExprIndex = indexExprs.get(0);
+ return analysisCtx.matchedFuncExprs.get(firstExprIndex);
+ }
- /**
- * Returns the search key expression which feeds a secondary-index search. If we are optimizing a selection query then this method returns
- * the a ConstantExpression from the first constant value in the optimizable function expression.
- * If we are optimizing a join, then this method returns the VariableReferenceExpression that should feed the secondary index probe.
- */
- public static ILogicalExpression createSearchKeyExpr(IOptimizableFuncExpr optFuncExpr,
- OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree) {
- if (probeSubTree == null) {
- // We are optimizing a selection query. Search key is a constant.
- return new ConstantExpression(optFuncExpr.getConstantVal(0));
- } else {
- // We are optimizing a join query. Determine which variable feeds the secondary index.
- if (optFuncExpr.getOperatorSubTree(0) == null || optFuncExpr.getOperatorSubTree(0) == probeSubTree) {
- return new VariableReferenceExpression(optFuncExpr.getLogicalVar(0));
- } else {
- return new VariableReferenceExpression(optFuncExpr.getLogicalVar(1));
- }
- }
- }
+ public static UnnestMapOperator createSecondaryIndexUnnestMap(Dataset dataset, ARecordType recordType, Index index,
+ ILogicalOperator inputOp, AccessMethodJobGenParams jobGenParams, IOptimizationContext context,
+ boolean outputPrimaryKeysOnly, boolean retainInput) throws AlgebricksException {
+ // The job gen parameters are transferred to the actual job gen via the UnnestMapOperator's function arguments.
+ ArrayList<Mutable<ILogicalExpression>> secondaryIndexFuncArgs = new ArrayList<Mutable<ILogicalExpression>>();
+ jobGenParams.writeToFuncArgs(secondaryIndexFuncArgs);
+ // Variables and types coming out of the secondary-index search.
+ List<LogicalVariable> secondaryIndexUnnestVars = new ArrayList<LogicalVariable>();
+ List<Object> secondaryIndexOutputTypes = new ArrayList<Object>();
+ // Append output variables/types generated by the secondary-index search (not forwarded from input).
+ appendSecondaryIndexOutputVars(dataset, recordType, index, outputPrimaryKeysOnly, context,
+ secondaryIndexUnnestVars);
+ appendSecondaryIndexTypes(dataset, recordType, index, outputPrimaryKeysOnly, secondaryIndexOutputTypes);
+ // An index search is expressed as an unnest over an index-search function.
+ IFunctionInfo secondaryIndexSearch = FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.INDEX_SEARCH);
+ UnnestingFunctionCallExpression secondaryIndexSearchFunc = new UnnestingFunctionCallExpression(
+ secondaryIndexSearch, secondaryIndexFuncArgs);
+ secondaryIndexSearchFunc.setReturnsUniqueValues(true);
+ // This is the operator that jobgen will be looking for. It contains an unnest function that has all necessary arguments to determine
+ // which index to use, which variables contain the index-search keys, what is the original dataset, etc.
+ UnnestMapOperator secondaryIndexUnnestOp = new UnnestMapOperator(secondaryIndexUnnestVars,
+ new MutableObject<ILogicalExpression>(secondaryIndexSearchFunc), secondaryIndexOutputTypes, retainInput);
+ secondaryIndexUnnestOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
+ context.computeAndSetTypeEnvironmentForOperator(secondaryIndexUnnestOp);
+ secondaryIndexUnnestOp.setExecutionMode(ExecutionMode.PARTITIONED);
+ return secondaryIndexUnnestOp;
+ }
- /**
- * Returns the first expr optimizable by this index.
- */
- public static IOptimizableFuncExpr chooseFirstOptFuncExpr(Index chosenIndex, AccessMethodAnalysisContext analysisCtx) {
- List<Integer> indexExprs = analysisCtx.getIndexExprs(chosenIndex);
- int firstExprIndex = indexExprs.get(0);
- return analysisCtx.matchedFuncExprs.get(firstExprIndex);
- }
-
- public 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);
- }
- }
-
- public static ExternalDataAccessByRIDOperator createExternalDataAccessByRIDUnnestMap(DataSourceScanOperator dataSourceScan, Dataset dataset,
- ARecordType recordType, ILogicalOperator inputOp, IOptimizationContext context, Index secondaryIndex) throws AlgebricksException {
- List<LogicalVariable> primaryKeyVars = AccessMethodUtils.getPrimaryKeyVarsFromUnnestMap(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>>();
- 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_ACCESS_BY_RID);
- AbstractFunctionCallExpression externalAccessFunc = new ScalarFunctionCallExpression(externalAccessByRID,
- externalRIDAccessFuncArgs);
-
- ExternalDataAccessByRIDOperator externalAccessByRIDOp = new ExternalDataAccessByRIDOperator(externalAccessByRIDVars,
- new MutableObject<ILogicalExpression>(externalAccessFunc), externalAccessOutputTypes);
- // Fed by the order operator or the secondaryIndexUnnestOp.
- externalAccessByRIDOp.getInputs().add(new MutableObject<ILogicalOperator>(order));
-
- context.computeAndSetTypeEnvironmentForOperator(externalAccessByRIDOp);
- externalAccessByRIDOp.setExecutionMode(ExecutionMode.PARTITIONED);
-
- //set the physical operator
- AqlSourceId dataSourceId = new AqlSourceId(dataset.getDataverseName(),
- dataset.getDatasetName());
- externalAccessByRIDOp.setPhysicalOperator(new ExternalDataAccessByRIDPOperator(dataSourceId, dataset, recordType, secondaryIndex));
- return externalAccessByRIDOp;
- }
-
- public static UnnestMapOperator createSecondaryIndexUnnestMap(Dataset dataset, ARecordType recordType, Index index,
- ILogicalOperator inputOp, AccessMethodJobGenParams jobGenParams, IOptimizationContext context,
- boolean outputPrimaryKeysOnly, boolean retainInput) throws AlgebricksException {
- // The job gen parameters are transferred to the actual job gen via the UnnestMapOperator's function arguments.
- ArrayList<Mutable<ILogicalExpression>> secondaryIndexFuncArgs = new ArrayList<Mutable<ILogicalExpression>>();
- jobGenParams.writeToFuncArgs(secondaryIndexFuncArgs);
- // Variables and types coming out of the secondary-index search.
- List<LogicalVariable> secondaryIndexUnnestVars = new ArrayList<LogicalVariable>();
- List<Object> secondaryIndexOutputTypes = new ArrayList<Object>();
- // Append output variables/types generated by the secondary-index search (not forwarded from input).
- appendSecondaryIndexOutputVars(dataset, recordType, index, outputPrimaryKeysOnly, context,
- secondaryIndexUnnestVars);
- appendSecondaryIndexTypes(dataset, recordType, index, outputPrimaryKeysOnly, secondaryIndexOutputTypes);
- // An index search is expressed as an unnest over an index-search function.
- IFunctionInfo secondaryIndexSearch = FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.INDEX_SEARCH);
- UnnestingFunctionCallExpression secondaryIndexSearchFunc = new UnnestingFunctionCallExpression(
- secondaryIndexSearch, secondaryIndexFuncArgs);
- secondaryIndexSearchFunc.setReturnsUniqueValues(true);
- // This is the operator that jobgen will be looking for. It contains an unnest function that has all necessary arguments to determine
- // which index to use, which variables contain the index-search keys, what is the original dataset, etc.
- UnnestMapOperator secondaryIndexUnnestOp = new UnnestMapOperator(secondaryIndexUnnestVars,
- new MutableObject<ILogicalExpression>(secondaryIndexSearchFunc), secondaryIndexOutputTypes, retainInput);
- secondaryIndexUnnestOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
- context.computeAndSetTypeEnvironmentForOperator(secondaryIndexUnnestOp);
- secondaryIndexUnnestOp.setExecutionMode(ExecutionMode.PARTITIONED);
- return secondaryIndexUnnestOp;
- }
-
- public static UnnestMapOperator createPrimaryIndexUnnestMap(DataSourceScanOperator dataSourceScan, Dataset dataset,
- ARecordType recordType, ILogicalOperator inputOp, IOptimizationContext context, boolean sortPrimaryKeys,
- boolean retainInput, boolean requiresBroadcast) throws AlgebricksException {
- List<LogicalVariable> primaryKeyVars = AccessMethodUtils.getPrimaryKeyVarsFromUnnestMap(dataset, inputOp);
- // Optionally add a sort on the primary-index keys before searching the primary index.
- OrderOperator order = null;
- if (sortPrimaryKeys) {
- 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);
- }
- // The job gen parameters are transferred to the actual job gen via the UnnestMapOperator's function arguments.
- List<Mutable<ILogicalExpression>> primaryIndexFuncArgs = new ArrayList<Mutable<ILogicalExpression>>();
- BTreeJobGenParams jobGenParams = new BTreeJobGenParams(dataset.getDatasetName(), IndexType.BTREE,
- dataset.getDataverseName(), dataset.getDatasetName(), retainInput, requiresBroadcast);
- // Set low/high inclusive to true for a point lookup.
- jobGenParams.setLowKeyInclusive(true);
- jobGenParams.setHighKeyInclusive(true);
- jobGenParams.setLowKeyVarList(primaryKeyVars, 0, primaryKeyVars.size());
- jobGenParams.setHighKeyVarList(primaryKeyVars, 0, primaryKeyVars.size());
- jobGenParams.writeToFuncArgs(primaryIndexFuncArgs);
- // Variables and types coming out of the primary-index search.
- List<LogicalVariable> primaryIndexUnnestVars = new ArrayList<LogicalVariable>();
- List<Object> primaryIndexOutputTypes = new ArrayList<Object>();
- // Append output variables/types generated by the primary-index search (not forwarded from input).
- primaryIndexUnnestVars.addAll(dataSourceScan.getVariables());
- try {
- appendPrimaryIndexTypes(dataset, recordType, primaryIndexOutputTypes);
- } catch (IOException e) {
- throw new AlgebricksException(e);
- }
- // An index search is expressed as an unnest over an index-search function.
- IFunctionInfo primaryIndexSearch = FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.INDEX_SEARCH);
- AbstractFunctionCallExpression primaryIndexSearchFunc = new ScalarFunctionCallExpression(primaryIndexSearch,
- primaryIndexFuncArgs);
- // This is the operator that jobgen will be looking for. It contains an unnest function that has all necessary arguments to determine
- // which index to use, which variables contain the index-search keys, what is the original dataset, etc.
- UnnestMapOperator primaryIndexUnnestOp = new UnnestMapOperator(primaryIndexUnnestVars,
- new MutableObject<ILogicalExpression>(primaryIndexSearchFunc), primaryIndexOutputTypes, retainInput);
- // Fed by the order operator or the secondaryIndexUnnestOp.
- if (sortPrimaryKeys) {
- primaryIndexUnnestOp.getInputs().add(new MutableObject<ILogicalOperator>(order));
- } else {
- primaryIndexUnnestOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
- }
- context.computeAndSetTypeEnvironmentForOperator(primaryIndexUnnestOp);
- primaryIndexUnnestOp.setExecutionMode(ExecutionMode.PARTITIONED);
- return primaryIndexUnnestOp;
- }
+ public static UnnestMapOperator createPrimaryIndexUnnestMap(DataSourceScanOperator dataSourceScan, Dataset dataset,
+ ARecordType recordType, ILogicalOperator inputOp, IOptimizationContext context, boolean sortPrimaryKeys,
+ boolean retainInput, boolean requiresBroadcast) throws AlgebricksException {
+ List<LogicalVariable> primaryKeyVars = AccessMethodUtils.getPrimaryKeyVarsFromUnnestMap(dataset, inputOp);
+ // Optionally add a sort on the primary-index keys before searching the primary index.
+ OrderOperator order = null;
+ if (sortPrimaryKeys) {
+ 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);
+ }
+ // The job gen parameters are transferred to the actual job gen via the UnnestMapOperator's function arguments.
+ List<Mutable<ILogicalExpression>> primaryIndexFuncArgs = new ArrayList<Mutable<ILogicalExpression>>();
+ BTreeJobGenParams jobGenParams = new BTreeJobGenParams(dataset.getDatasetName(), IndexType.BTREE,
+ dataset.getDataverseName(), dataset.getDatasetName(), retainInput, requiresBroadcast);
+ // Set low/high inclusive to true for a point lookup.
+ jobGenParams.setLowKeyInclusive(true);
+ jobGenParams.setHighKeyInclusive(true);
+ jobGenParams.setLowKeyVarList(primaryKeyVars, 0, primaryKeyVars.size());
+ jobGenParams.setHighKeyVarList(primaryKeyVars, 0, primaryKeyVars.size());
+ jobGenParams.writeToFuncArgs(primaryIndexFuncArgs);
+ // Variables and types coming out of the primary-index search.
+ List<LogicalVariable> primaryIndexUnnestVars = new ArrayList<LogicalVariable>();
+ List<Object> primaryIndexOutputTypes = new ArrayList<Object>();
+ // Append output variables/types generated by the primary-index search (not forwarded from input).
+ primaryIndexUnnestVars.addAll(dataSourceScan.getVariables());
+ try {
+ appendPrimaryIndexTypes(dataset, recordType, primaryIndexOutputTypes);
+ } catch (IOException e) {
+ throw new AlgebricksException(e);
+ }
+ // An index search is expressed as an unnest over an index-search function.
+ IFunctionInfo primaryIndexSearch = FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.INDEX_SEARCH);
+ AbstractFunctionCallExpression primaryIndexSearchFunc = new ScalarFunctionCallExpression(primaryIndexSearch,
+ primaryIndexFuncArgs);
+ // This is the operator that jobgen will be looking for. It contains an unnest function that has all necessary arguments to determine
+ // which index to use, which variables contain the index-search keys, what is the original dataset, etc.
+ UnnestMapOperator primaryIndexUnnestOp = new UnnestMapOperator(primaryIndexUnnestVars,
+ new MutableObject<ILogicalExpression>(primaryIndexSearchFunc), primaryIndexOutputTypes, retainInput);
+ // Fed by the order operator or the secondaryIndexUnnestOp.
+ if (sortPrimaryKeys) {
+ primaryIndexUnnestOp.getInputs().add(new MutableObject<ILogicalOperator>(order));
+ } else {
+ primaryIndexUnnestOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
+ }
+ context.computeAndSetTypeEnvironmentForOperator(primaryIndexUnnestOp);
+ primaryIndexUnnestOp.setExecutionMode(ExecutionMode.PARTITIONED);
+ return primaryIndexUnnestOp;
+ }
}
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 ce2a1f7..ddcf768 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
@@ -27,7 +27,6 @@
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.commons.lang3.mutable.MutableObject;
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
import edu.uci.ics.asterix.metadata.entities.Dataset;
import edu.uci.ics.asterix.metadata.entities.Index;
@@ -52,7 +51,6 @@
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.ExternalDataAccessByRIDOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
@@ -418,16 +416,10 @@
// Generate the rest of the upstream plan which feeds the search results into the primary index.
UnnestMapOperator primaryIndexUnnestOp;
boolean isPrimaryIndex = chosenIndex.getIndexName().equals(dataset.getDatasetName());
- if(dataset.getDatasetType() == DatasetType.EXTERNAL)
- {
- ExternalDataAccessByRIDOperator externalDataAccessOp = AccessMethodUtils.createExternalDataAccessByRIDUnnestMap(dataSourceScan, dataset,
- recordType, secondaryIndexUnnestOp, context, chosenIndex);
- indexSubTree.dataSourceScanRef.setValue(externalDataAccessOp);
- return externalDataAccessOp;
- }
- else if (!isPrimaryIndex) {
+ if (!isPrimaryIndex) {
primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceScan, dataset, recordType,
secondaryIndexUnnestOp, context, true, retainInput, false);
+
// Replace the datasource scan with the new plan rooted at
// primaryIndexUnnestMap.
indexSubTree.dataSourceScanRef.setValue(primaryIndexUnnestOp); //kisskys
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 ea2adad..28aee7a 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
@@ -114,8 +114,7 @@
if (dataset == null) {
throw new AlgebricksException("No metadata for dataset " + datasetName);
}
- if (dataset.getDatasetType() != DatasetType.INTERNAL && dataset.getDatasetType() != DatasetType.FEED
- && dataset.getDatasetType() != DatasetType.EXTERNAL ) {
+ if (dataset.getDatasetType() != DatasetType.INTERNAL && dataset.getDatasetType() != DatasetType.FEED) {
return false;
}
// Get the record type for that dataset.
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 8af4ac1..b8125aa 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
@@ -21,7 +21,6 @@
import org.apache.commons.lang3.mutable.MutableObject;
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.metadata.entities.Dataset;
import edu.uci.ics.asterix.metadata.entities.Index;
@@ -45,7 +44,6 @@
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.ExternalDataAccessByRIDOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
@@ -198,19 +196,10 @@
UnnestMapOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
chosenIndex, assignSearchKeys, jobGenParams, context, false, retainInput);
// Generate the rest of the upstream plan which feeds the search results into the primary index.
- if(dataset.getDatasetType() == DatasetType.EXTERNAL)
- {
- ExternalDataAccessByRIDOperator externalDataAccessOp = AccessMethodUtils.createExternalDataAccessByRIDUnnestMap(dataSourceScan, dataset,
- recordType, secondaryIndexUnnestOp, context, chosenIndex);
- return externalDataAccessOp;
- }
- else
- {
- UnnestMapOperator primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceScan, dataset,
- recordType, secondaryIndexUnnestOp, context, true, retainInput, false);
+ UnnestMapOperator primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceScan, dataset,
+ recordType, secondaryIndexUnnestOp, context, true, retainInput, false);
- return primaryIndexUnnestOp;
- }
+ return primaryIndexUnnestOp;
}
@Override
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 9c3a853..f22d2fb 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
@@ -414,8 +414,7 @@
case EXTERNAL: {
String adapter = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl()).getAdapter();
Map<String, String> properties = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl()).getProperties();
- String ngName = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl()).getNodegroupName().getValue();
- datasetDetails = new ExternalDatasetDetails(adapter, properties,ngName);
+ datasetDetails = new ExternalDatasetDetails(adapter, properties);
break;
}
case FEED: {
@@ -591,18 +590,6 @@
//#. create the index artifact in NC.
runJob(hcc, spec, true);
- //if external data and optimization is turned on, load file names
- if(ds.getDatasetType() == DatasetType.EXTERNAL && AqlMetadataProvider.isOptimizeExternalIndexes())
- {
- //load the file names into external files index
- mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
- bActiveTxn = true;
- metadataProvider.setMetadataTxnContext(mdTxnCtx);
- IndexOperations.addExternalDatasetFilesToMetadata(metadataProvider, ds);
- MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
- bActiveTxn = false;
- }
-
mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
bActiveTxn = true;
metadataProvider.setMetadataTxnContext(mdTxnCtx);
@@ -667,8 +654,6 @@
throw new IllegalStateException("System is inconsistent state: pending index(" + dataverseName
+ "." + datasetName + "." + indexName + ") couldn't be removed from the metadata", e);
}
-
- //if external dataset, remove external files from metadata
}
throw e;
} finally {
@@ -745,7 +730,7 @@
for (int j = 0; j < datasets.size(); j++) {
String datasetName = datasets.get(j).getDatasetName();
DatasetType dsType = datasets.get(j).getDatasetType();
- if (dsType == DatasetType.INTERNAL || dsType == DatasetType.FEED || dsType == DatasetType.EXTERNAL) {
+ if (dsType == DatasetType.INTERNAL || dsType == DatasetType.FEED) {
List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName,
datasetName);
@@ -858,7 +843,7 @@
}
}
- if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED || ds.getDatasetType() == DatasetType.EXTERNAL) {
+ if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED) {
//#. prepare jobs to drop the datatset and the indexes in NC
List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
@@ -959,7 +944,7 @@
+ dataverseName);
}
- if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED || ds.getDatasetType() == DatasetType.EXTERNAL) {
+ if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED) {
indexName = stmtIndexDrop.getIndexName().getValue();
Index index = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataverseName, datasetName, indexName);
if (index == null) {
@@ -997,8 +982,8 @@
//#. 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.");
+ throw new AlgebricksException(datasetName
+ + " is an external dataset. Indexes are not maintained for external datasets.");
}
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
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 e090c2c..bf31db7 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,18 +14,14 @@
*/
package edu.uci.ics.asterix.file;
-import java.util.ArrayList;
-
import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
import edu.uci.ics.asterix.common.config.OptimizationConfUtil;
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.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.ExternalFile;
import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
@@ -60,16 +56,6 @@
return secondaryIndexCreator.buildLoadingJobSpec();
}
- public static void addExternalDatasetFilesToMetadata(AqlMetadataProvider metadataProvider, Dataset dataset)
- throws AlgebricksException, MetadataException {
- //get the file list
- ArrayList<ExternalFile> files = metadataProvider.getExternalDatasetFiles(dataset);
- //add files to metadata
- for (int i = 0; i < files.size(); i++) {
- MetadataManager.INSTANCE.addExternalFile(metadataProvider.getMetadataTxnContext(), files.get(i));
- }
- }
-
public static JobSpecification buildDropSecondaryIndexJobSpec(CompiledIndexDropStatement indexDropStmt,
AqlMetadataProvider metadataProvider, Dataset dataset) throws AlgebricksException, MetadataException {
String dataverseName = indexDropStmt.getDataverseName() == null ? metadataProvider.getDefaultDataverseName()
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
index 55cccbc..3ae5f6d 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
@@ -16,31 +16,21 @@
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.external.data.operator.ExternalDataIndexingOperatorDescriptor;
-import edu.uci.ics.asterix.external.util.ExternalIndexHashPartitionComputerFactory;
-import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
-import edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat;
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.PersistentLocalResourceFactoryProvider;
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.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.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
import edu.uci.ics.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
@@ -62,6 +52,7 @@
@Override
public JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException {
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(
@@ -69,6 +60,7 @@
dataset.getDatasetId());
ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
localResourceMetadata, LocalResource.LSMBTreeResource);
+
TreeIndexCreateOperatorDescriptor secondaryIndexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
@@ -87,110 +79,49 @@
@Override
public JobSpecification buildLoadingJobSpec() throws AsterixException, AlgebricksException {
- if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
- JobSpecification spec = JobSpecificationUtils.createJobSpecification();
- Pair<ExternalDataIndexingOperatorDescriptor, AlgebricksPartitionConstraint> RIDScanOpAndConstraints;
- AlgebricksMetaOperatorDescriptor asterixAssignOp;
- try {
- //create external indexing scan operator
- RIDScanOpAndConstraints = createExternalIndexingOp(spec);
+ JobSpecification spec = JobSpecificationUtils.createJobSpecification();
- //create assign operator
- asterixAssignOp = createExternalAssignOp(spec);
- AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, asterixAssignOp,
- RIDScanOpAndConstraints.second);
- } catch (Exception e) {
- throw new AsterixException("Failed to create external index scanning and loading job");
- }
+ // 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,
- RIDScanOpAndConstraints.second);
- AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
- // Create secondary BTree bulk load op.
- TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(
- spec,
- numSecondaryKeys,
- new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
- new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
- LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties
- .getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
- IBinaryHashFunctionFactory[] hashFactories = DatasetUtils.computeExternalDataKeysBinaryHashFunFactories(
- dataset, NonTaggedDataFormat.INSTANCE.getBinaryHashFunctionFactoryProvider());
+ // Assign op.
+ AlgebricksMetaOperatorDescriptor asterixAssignOp = createAssignOp(spec, primaryScanOp, numSecondaryKeys);
- //select partitioning keys (always the first 2 after secondary keys)
- int[] keys = new int[2];
- keys[0] = numSecondaryKeys;
- keys[1] = numSecondaryKeys + 1;
-
- IConnectorDescriptor hashConn = new MToNPartitioningConnectorDescriptor(spec,
- new ExternalIndexHashPartitionComputerFactory(keys, hashFactories));
-
- spec.connect(new OneToOneConnectorDescriptor(spec), RIDScanOpAndConstraints.first, 0, asterixAssignOp, 0);
- if (anySecondaryKeyIsNullable) {
- spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, selectOp, 0);
- spec.connect(hashConn, selectOp, 0, sortOp, 0);
- } else {
- spec.connect(hashConn, asterixAssignOp, 0, sortOp, 0);
- }
- spec.connect(new OneToOneConnectorDescriptor(spec), sortOp, 0, secondaryBulkLoadOp, 0);
- spec.addRoot(secondaryBulkLoadOp);
- spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
- return spec;
- } else {
- JobSpecification spec = JobSpecificationUtils.createJobSpecification();
-
- // 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()),
- AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
- new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
- LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties
- .getBloomFilterFalsePositiveRate()), 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;
+ // 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()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new SecondaryIndexOperationTrackerProvider(
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties
+ .getBloomFilterFalsePositiveRate()), 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;
}
}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java
index 1418c32..9f80568 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java
@@ -29,9 +29,6 @@
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.adapter.factory.HDFSAdapterFactory;
-import edu.uci.ics.asterix.external.data.operator.ExternalDataIndexingOperatorDescriptor;
-import edu.uci.ics.asterix.external.dataset.adapter.HiveAdapter;
import edu.uci.ics.asterix.formats.nontagged.AqlBinaryBooleanInspectorImpl;
import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
@@ -39,17 +36,14 @@
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.ExternalDatasetDetails;
import edu.uci.ics.asterix.metadata.entities.Index;
import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
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.asterix.om.util.AsterixAppContextInfo;
import edu.uci.ics.asterix.runtime.evaluators.functions.AndDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.IsNullDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.NotDescriptor;
-import edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat;
import edu.uci.ics.asterix.runtime.job.listener.JobEventListenerFactory;
import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexInstantSearchOperationCallbackFactory;
import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexOperationTrackerProvider;
@@ -109,11 +103,11 @@
protected ISerializerDeserializer payloadSerde;
protected IFileSplitProvider primaryFileSplitProvider;
protected AlgebricksPartitionConstraint primaryPartitionConstraint;
- protected List<String> secondaryKeyFields;
protected IFileSplitProvider secondaryFileSplitProvider;
protected AlgebricksPartitionConstraint secondaryPartitionConstraint;
protected String secondaryIndexName;
protected boolean anySecondaryKeyIsNullable = false;
+
protected long numElementsHint;
protected IBinaryComparatorFactory[] primaryComparatorFactories;
protected int[] primaryBloomFilterKeyFields;
@@ -122,6 +116,7 @@
protected int[] secondaryBloomFilterKeyFields;
protected RecordDescriptor secondaryRecDesc;
protected ICopyEvaluatorFactory[] secondaryFieldAccessEvalFactories;
+
protected IAsterixPropertiesProvider propertiesProvider;
// Prevent public construction. Should be created via createIndexCreator().
@@ -176,55 +171,25 @@
throw new AsterixException("Unknown dataset " + datasetName);
}
if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
- //get external dataset details
- ExternalDatasetDetails edsd = ((ExternalDatasetDetails) dataset.getDatasetDetails());
- //get adapter name
- String adapter = edsd.getAdapter();
- //if not an hdfs adapter, throw an exception
- if (!adapter.equals(HDFSAdapterFactory.HDFS_ADAPTER_NAME) && !adapter.equals(HiveAdapter.class.getName())) {
- throw new AsterixException("Cannot index an external dataset with adapter type(" + adapter + ").");
- }
- //get the item type
- ARecordType externalItemType = (ARecordType) metadataProvider.findType(dataset.getDataverseName(),
- dataset.getItemTypeName());
- //number of primary keys here depends on the file input, 3 for rcfiles and 2 for text and sequence files.
- numPrimaryKeys = DatasetUtils.getExternalRIDSize(dataset);
- itemType = createExternalItemTypeWithRID(externalItemType);
- payloadSerde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType);
- numSecondaryKeys = createIndexStmt.getKeyFields().size();
- //splits and constraints <--They don't exist-->
- primaryFileSplitProvider = null;
- primaryPartitionConstraint = null;
- //create secondary split and constraints
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadataProvider
- .splitProviderAndPartitionConstraintsForExternalDataset(dataverseName, datasetName,
- secondaryIndexName);
- secondaryFileSplitProvider = secondarySplitsAndConstraint.first;
- secondaryPartitionConstraint = secondarySplitsAndConstraint.second;
- // Must be called in this order.
- setExternalRIDDescAndComparators();
- setExternalSecondaryRecDescAndComparators(createIndexStmt, metadataProvider);
- numElementsHint = metadataProvider.getCardinalityPerPartitionHint(dataset);
- } else {
- itemType = (ARecordType) metadataProvider.findType(dataset.getDataverseName(), dataset.getItemTypeName());
- payloadSerde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType);
- numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
- numSecondaryKeys = createIndexStmt.getKeyFields().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);
- secondaryFileSplitProvider = secondarySplitsAndConstraint.first;
- secondaryPartitionConstraint = secondarySplitsAndConstraint.second;
- // Must be called in this order.
- setPrimaryRecDescAndComparators();
- setSecondaryRecDescAndComparators(createIndexStmt, metadataProvider);
- numElementsHint = metadataProvider.getCardinalityPerPartitionHint(dataset);
+ 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 = createIndexStmt.getKeyFields().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);
+ secondaryFileSplitProvider = secondarySplitsAndConstraint.first;
+ secondaryPartitionConstraint = secondarySplitsAndConstraint.second;
+ // Must be called in this order.
+ setPrimaryRecDescAndComparators();
+ setSecondaryRecDescAndComparators(createIndexStmt, metadataProvider);
+ numElementsHint = metadataProvider.getCardinalityPerPartitionHint(dataset);
}
protected void setPrimaryRecDescAndComparators() throws AlgebricksException {
@@ -309,196 +274,6 @@
return keyProviderOp;
}
- protected ARecordType createExternalItemTypeWithRID(ARecordType externalItemType) throws AsterixException {
-
- String[] fieldsNames = new String[externalItemType.getFieldNames().length + numPrimaryKeys];
- IAType[] fieldsTypes = new IAType[externalItemType.getFieldTypes().length + numPrimaryKeys];
-
- //add RID fields names and types
- if (AqlMetadataProvider.isOptimizeExternalIndexes()) {
- fieldsNames[0] = "_file-number";
- fieldsTypes[0] = BuiltinType.AINT32;
- } else {
- fieldsNames[0] = "_file-name";
- fieldsTypes[0] = BuiltinType.ASTRING;
- }
- fieldsNames[1] = "_byte-location";
- fieldsTypes[1] = BuiltinType.AINT64;
- if (numPrimaryKeys == 3) {
- //add the row number for rc files
- fieldsNames[2] = "_row-number";
- fieldsTypes[2] = BuiltinType.AINT32;
- }
-
- //add the original fields names and types
- for (int i = 0; i < externalItemType.getFieldNames().length; i++) {
- fieldsNames[i + numPrimaryKeys] = externalItemType.getFieldNames()[i];
- fieldsTypes[i + numPrimaryKeys] = externalItemType.getFieldTypes()[i];
- }
- return new ARecordType(externalItemType.getTypeName(), fieldsNames, fieldsTypes, externalItemType.isOpen());
- }
-
- protected void setExternalRIDDescAndComparators() throws AlgebricksException {
-
- ISerializerDeserializer[] externalRecFields = new ISerializerDeserializer[itemType.getFieldNames().length];
- ITypeTraits[] externalTypeTraits = new ITypeTraits[itemType.getFieldNames().length];
-
- primaryComparatorFactories = new IBinaryComparatorFactory[numPrimaryKeys];
- primaryBloomFilterKeyFields = new int[numPrimaryKeys];
- ISerializerDeserializerProvider serdeProvider = metadataProvider.getFormat().getSerdeProvider();
-
- if (AqlMetadataProvider.isOptimizeExternalIndexes()) {
- primaryComparatorFactories[0] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(
- BuiltinType.AINT32, true);
- } else {
- primaryComparatorFactories[0] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(
- BuiltinType.ASTRING, true);
- }
- primaryComparatorFactories[1] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(
- BuiltinType.AINT64, true);
-
- primaryBloomFilterKeyFields[0] = 0;
- primaryBloomFilterKeyFields[1] = 1;
-
- if (numPrimaryKeys == 3) {
- primaryComparatorFactories[2] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(
- BuiltinType.AINT32, true);
- primaryBloomFilterKeyFields[2] = 2;
- }
-
- for (int i = 0; i < itemType.getFieldNames().length; i++) {
- externalRecFields[i] = serdeProvider.getSerializerDeserializer(itemType.getFieldTypes()[i]);
- externalTypeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(itemType.getFieldTypes()[i]);
- }
- primaryRecDesc = new RecordDescriptor(externalRecFields, externalTypeTraits);
- }
-
- protected void setExternalSecondaryRecDescAndComparators(CompiledCreateIndexStatement createIndexStmt,
- AqlMetadataProvider metadataProvider) throws AlgebricksException, AsterixException {
- secondaryKeyFields = createIndexStmt.getKeyFields();
- secondaryFieldAccessEvalFactories = new ICopyEvaluatorFactory[numSecondaryKeys + numPrimaryKeys];
- secondaryComparatorFactories = new IBinaryComparatorFactory[numSecondaryKeys + numPrimaryKeys];
- secondaryBloomFilterKeyFields = new int[numSecondaryKeys];
- ISerializerDeserializer[] secondaryRecFields = new ISerializerDeserializer[numPrimaryKeys + numSecondaryKeys];
- ITypeTraits[] secondaryTypeTraits = new ITypeTraits[numSecondaryKeys + numPrimaryKeys];
- ISerializerDeserializerProvider serdeProvider = metadataProvider.getFormat().getSerdeProvider();
- ITypeTraitProvider typeTraitProvider = metadataProvider.getFormat().getTypeTraitProvider();
- IBinaryComparatorFactoryProvider comparatorFactoryProvider = metadataProvider.getFormat()
- .getBinaryComparatorFactoryProvider();
-
- for (int i = 0; i < numSecondaryKeys; i++) {
- secondaryFieldAccessEvalFactories[i] = metadataProvider.getFormat().getFieldAccessEvaluatorFactory(
- itemType, secondaryKeyFields.get(i), 0);
- Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(secondaryKeyFields.get(i), itemType);
- IAType keyType = keyTypePair.first;
- anySecondaryKeyIsNullable = anySecondaryKeyIsNullable || keyTypePair.second;
- ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(keyType);
- secondaryRecFields[i] = keySerde;
- secondaryComparatorFactories[i] = comparatorFactoryProvider.getBinaryComparatorFactory(keyType, true);
- secondaryTypeTraits[i] = typeTraitProvider.getTypeTrait(keyType);
- secondaryBloomFilterKeyFields[i] = i;
- }
-
- if (AqlMetadataProvider.isOptimizeExternalIndexes()) {
- secondaryFieldAccessEvalFactories[numSecondaryKeys] = metadataProvider.getFormat()
- .getFieldAccessEvaluatorFactory(itemType, "_file-number", 0);
- } else {
- secondaryFieldAccessEvalFactories[numSecondaryKeys] = metadataProvider.getFormat()
- .getFieldAccessEvaluatorFactory(itemType, "_file-name", 0);
- }
- secondaryFieldAccessEvalFactories[numSecondaryKeys + 1] = metadataProvider.getFormat()
- .getFieldAccessEvaluatorFactory(itemType, "_byte-location", 0);
- if (numPrimaryKeys == 3) {
- secondaryFieldAccessEvalFactories[numSecondaryKeys + 2] = metadataProvider.getFormat()
- .getFieldAccessEvaluatorFactory(itemType, "_row-number", 0);
- }
-
- for (int i = 0; i < numPrimaryKeys; i++) {
- secondaryRecFields[numSecondaryKeys + i] = primaryRecDesc.getFields()[i];
- secondaryTypeTraits[numSecondaryKeys + i] = primaryRecDesc.getTypeTraits()[i];
- secondaryComparatorFactories[numSecondaryKeys + i] = primaryComparatorFactories[i];
- }
- secondaryRecDesc = new RecordDescriptor(secondaryRecFields, secondaryTypeTraits);
- }
-
- protected Pair<ExternalDataIndexingOperatorDescriptor, AlgebricksPartitionConstraint> createExternalIndexingOp(
- JobSpecification spec) throws Exception {
- Pair<ExternalDataIndexingOperatorDescriptor, AlgebricksPartitionConstraint> indexingOpAndConstraints = metadataProvider
- .buildExternalDataIndexingRuntime(spec, itemType, dataset, NonTaggedDataFormat.INSTANCE);
- AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, indexingOpAndConstraints.first,
- indexingOpAndConstraints.second);
- return indexingOpAndConstraints;
- }
-
- protected AlgebricksMetaOperatorDescriptor createExternalAssignOp(JobSpecification spec) throws AlgebricksException {
- int[] outColumns = new int[numSecondaryKeys + numPrimaryKeys];
- int[] projectionList = new int[numSecondaryKeys + numPrimaryKeys];
- for (int i = 0; i < numSecondaryKeys + numPrimaryKeys; i++) {
- outColumns[i] = i;
- projectionList[i] = i;
- }
-
- IScalarEvaluatorFactory[] sefs = new IScalarEvaluatorFactory[secondaryFieldAccessEvalFactories.length];
- for (int i = 0; i < secondaryFieldAccessEvalFactories.length; ++i) {
- sefs[i] = new LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.ScalarEvaluatorFactoryAdapter(
- secondaryFieldAccessEvalFactories[i]);
- }
- AssignRuntimeFactory assign = new AssignRuntimeFactory(outColumns, sefs, projectionList);
- AlgebricksMetaOperatorDescriptor asterixAssignOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
- new IPushRuntimeFactory[] { assign }, new RecordDescriptor[] { secondaryRecDesc });
- return asterixAssignOp;
- }
-
- protected ExternalSortOperatorDescriptor createSortOp(JobSpecification spec,
- IBinaryComparatorFactory[] secondaryComparatorFactories, RecordDescriptor secondaryRecDesc,
- AlgebricksPartitionConstraint partitionConstraints) {
- int[] sortFields = new int[secondaryComparatorFactories.length];
- for (int i = 0; i < secondaryComparatorFactories.length; i++) {
- sortFields[i] = i;
- }
- ExternalSortOperatorDescriptor sortOp = new ExternalSortOperatorDescriptor(spec,
- physOptConf.getMaxFramesExternalSort(), sortFields, secondaryComparatorFactories, secondaryRecDesc);
- AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, sortOp, partitionConstraints);
- return sortOp;
- }
-
- protected ARecordType createSecondaryItemType(ARecordType externalItemType, boolean isRCFile)
- throws AsterixException {
-
- String[] fieldsNames = new String[numSecondaryKeys + numPrimaryKeys];
- IAType[] fieldsTypes = new IAType[numSecondaryKeys + numPrimaryKeys];
-
- //first create the secondary index fields
- for (int i = 0; i < numSecondaryKeys; i++) {
- fieldsNames[i] = secondaryKeyFields.get(i);
- try {
- fieldsTypes[i] = externalItemType.getFieldType(fieldsNames[i]);
- } catch (IOException e) {
- // TODO Auto-generated catch block
- throw new AsterixException(e);
- }
- }
-
- //second add RID fields (File name or number and byte location)
- if (AqlMetadataProvider.isOptimizeExternalIndexes()) {
- fieldsNames[numSecondaryKeys] = "_file-number";
- fieldsTypes[numSecondaryKeys] = BuiltinType.ASTRING;
- } else {
- fieldsNames[numSecondaryKeys] = "_file-name";
- fieldsTypes[numSecondaryKeys] = BuiltinType.ASTRING;
- }
- fieldsNames[numSecondaryKeys + 1] = "_byte-location";
- fieldsTypes[numSecondaryKeys + 1] = BuiltinType.AINT64;
-
- if (isRCFile) {
- fieldsNames[numSecondaryKeys + 2] = "_row-Number";
- fieldsTypes[numSecondaryKeys + 2] = BuiltinType.AINT32;
- }
-
- //return type
- return new ARecordType(externalItemType.getTypeName(), fieldsNames, fieldsTypes, externalItemType.isOpen());
- }
-
protected BTreeSearchOperatorDescriptor createPrimaryIndexScanOp(JobSpecification spec) throws AlgebricksException {
// -Infinity
int[] lowKeyFields = null;
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java
index a3261e6..72d0d70 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java
@@ -296,10 +296,4 @@
storageProperties.getBloomFilterFalsePositiveRate());
}
}
-
- @Override
- protected void setExternalSecondaryRecDescAndComparators(CompiledCreateIndexStatement createIndexStmt,
- AqlMetadataProvider metadataProvider) throws AlgebricksException, AsterixException {
- throw new AsterixException("Cannot create inverted index on external dataset due to composite RID Fields.");
- }
}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java
index ebfee66..4d8118d 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java
@@ -18,47 +18,36 @@
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.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.data.operator.ExternalDataIndexingOperatorDescriptor;
-import edu.uci.ics.asterix.external.util.ExternalIndexHashPartitionComputerFactory;
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.declared.AqlMetadataProvider;
-import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
import edu.uci.ics.asterix.metadata.entities.Index;
-import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
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.runtime.formats.NonTaggedDataFormat;
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.PersistentLocalResourceFactoryProvider;
import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
import edu.uci.ics.asterix.translator.CompiledStatements.CompiledCreateIndexStatement;
-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.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
import edu.uci.ics.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
@@ -160,159 +149,47 @@
}
@Override
- protected void setExternalSecondaryRecDescAndComparators(CompiledCreateIndexStatement createIndexStmt,
- AqlMetadataProvider metadataProvider) throws AlgebricksException, AsterixException {
- secondaryKeyFields = createIndexStmt.getKeyFields();
- if (numSecondaryKeys != 1) {
- throw new AsterixException(
- "Cannot use "
- + numSecondaryKeys
- + " fields as a key for the R-tree index. There can be only one field as a key for the R-tree index.");
- }
- Pair<IAType, Boolean> spatialTypePair = Index.getNonNullableKeyFieldType(secondaryKeyFields.get(0), itemType);
- IAType spatialType = spatialTypePair.first;
- anySecondaryKeyIsNullable = spatialTypePair.second;
- if (spatialType == null) {
- throw new AsterixException("Could not find field " + secondaryKeyFields.get(0) + " in the schema.");
- }
- int numDimensions = NonTaggedFormatUtil.getNumDimensions(spatialType.getTypeTag());
- numNestedSecondaryKeyFields = numDimensions * 2;
- secondaryFieldAccessEvalFactories = metadataProvider.getFormat().createMBRFactory(itemType,
- secondaryKeyFields.get(0), numPrimaryKeys, numDimensions);
- secondaryComparatorFactories = new IBinaryComparatorFactory[numNestedSecondaryKeyFields];
- valueProviderFactories = new IPrimitiveValueProviderFactory[numNestedSecondaryKeyFields];
- ISerializerDeserializer[] secondaryRecFields = new ISerializerDeserializer[numPrimaryKeys
- + numNestedSecondaryKeyFields];
- ITypeTraits[] secondaryTypeTraits = new ITypeTraits[numNestedSecondaryKeyFields + numPrimaryKeys];
- IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(spatialType.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] = primaryRecDesc.getFields()[i];
- secondaryTypeTraits[numNestedSecondaryKeyFields + i] = primaryRecDesc.getTypeTraits()[i];
- }
- secondaryRecDesc = new RecordDescriptor(secondaryRecFields, secondaryTypeTraits);
- }
-
- @Override
public JobSpecification buildLoadingJobSpec() throws AsterixException, AlgebricksException {
JobSpecification spec = JobSpecificationUtils.createJobSpecification();
- if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
- Pair<ExternalDataIndexingOperatorDescriptor, AlgebricksPartitionConstraint> RIDScanOpAndConstraints;
- AlgebricksMetaOperatorDescriptor asterixAssignOp;
- try {
- //create external indexing scan operator
- RIDScanOpAndConstraints = createExternalIndexingOp(spec);
- //create assign operator
- asterixAssignOp = createExternalAssignOp(spec);
- AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, asterixAssignOp,
- RIDScanOpAndConstraints.second);
- } catch (Exception e) {
- throw new AsterixException("Failed to create external index scanning and loading job");
- }
- // 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 dummy key provider for feeding the primary index scan.
+ AbstractOperatorDescriptor keyProviderOp = createDummyKeyProviderOp(spec);
- // Create secondary RTree bulk load op.
- AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
- TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(
- spec,
- numNestedSecondaryKeyFields,
- new LSMRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
- primaryComparatorFactories, new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
- new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
- LSMRTreeIOOperationCallbackFactory.INSTANCE, AqlMetadataProvider.proposeLinearizer(keyType,
- secondaryComparatorFactories.length), storageProperties
- .getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
- // Connect the operators.
- // Create a hash partitioning connector
- ExternalDatasetDetails edsd = (ExternalDatasetDetails) dataset.getDatasetDetails();
- IBinaryHashFunctionFactory[] hashFactories = null;
- if (edsd.getProperties().get(HDFSAdapterFactory.KEY_INPUT_FORMAT).trim()
- .equals(HDFSAdapterFactory.INPUT_FORMAT_RC)) {
- hashFactories = DatasetUtils.computeExternalDataKeysBinaryHashFunFactories(dataset,
- NonTaggedDataFormat.INSTANCE.getBinaryHashFunctionFactoryProvider());
- } else {
- hashFactories = DatasetUtils.computeExternalDataKeysBinaryHashFunFactories(dataset,
- NonTaggedDataFormat.INSTANCE.getBinaryHashFunctionFactoryProvider());
- }
- //select partitioning keys (always the first 2 after secondary keys)
- int[] keys = new int[2];
- keys[0] = numSecondaryKeys;
- keys[1] = numSecondaryKeys + 1;
+ // Create primary index scan op.
+ BTreeSearchOperatorDescriptor primaryScanOp = createPrimaryIndexScanOp(spec);
- IConnectorDescriptor hashConn = new MToNPartitioningConnectorDescriptor(spec,
- new ExternalIndexHashPartitionComputerFactory(keys, hashFactories));
- spec.connect(new OneToOneConnectorDescriptor(spec), RIDScanOpAndConstraints.first, 0, asterixAssignOp, 0);
- if (anySecondaryKeyIsNullable) {
- spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, selectOp, 0);
- spec.connect(hashConn, selectOp, 0, secondaryBulkLoadOp, 0);
- } else {
- spec.connect(hashConn, asterixAssignOp, 0, secondaryBulkLoadOp, 0);
- }
- spec.addRoot(secondaryBulkLoadOp);
- spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
- return spec;
- } else {
+ // Assign op.
+ AlgebricksMetaOperatorDescriptor asterixAssignOp = createAssignOp(spec, primaryScanOp,
+ numNestedSecondaryKeyFields);
- // 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,
- 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);
- }
-
- 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()),
- AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
- new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
- LSMRTreeIOOperationCallbackFactory.INSTANCE, AqlMetadataProvider.proposeLinearizer(keyType,
- secondaryComparatorFactories.length), storageProperties
- .getBloomFilterFalsePositiveRate()), 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, secondaryBulkLoadOp, 0);
- } else {
- spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, secondaryBulkLoadOp, 0);
- }
- spec.addRoot(secondaryBulkLoadOp);
- spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
- return 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, numNestedSecondaryKeyFields);
}
+
+ 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()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMRTreeIOOperationCallbackFactory.INSTANCE,
+ AqlMetadataProvider.proposeLinearizer(keyType, secondaryComparatorFactories.length),
+ storageProperties.getBloomFilterFalsePositiveRate()), 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, secondaryBulkLoadOp, 0);
+ } else {
+ spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, secondaryBulkLoadOp, 0);
+ }
+ spec.addRoot(secondaryBulkLoadOp);
+ spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+ return spec;
}
}
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 8935d5d..b56fe7c 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,9 +1,8 @@
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sat Aug 31 20:36:01 AST 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" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sat Aug 31 20:36:01 AST 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" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 3, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "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" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 9, "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" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sat Aug 31 20:36:01 AST 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" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 4, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 5, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 6, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
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 a44cbc0..d193dd1 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,63 +1,62 @@
-{ "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": "FeedDetails", "FieldType": "Field_FeedDetails_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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:02 AST 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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 2013" }
-{ "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": "FileModDate", "FieldType": "datetime" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:02 AST 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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:02 AST 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": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:02 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:02 AST 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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:02 AST 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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 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" }, { "FieldName": "GroupName", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_FeedDetails_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_FeedDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Function", "FieldType": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Status", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 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" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
\ No newline at end of file
+{ "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": "FeedDetails", "FieldType": "Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Hints", "FieldType": "Field_Hints_in_DatasetRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:30 PST 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": "Fri Feb 08 15:49:29 PST 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" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Feb 08 15:49:30 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:30 PST 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" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_FeedDetails_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_FeedDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Function", "FieldType": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Status", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 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" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 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": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
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 6037036..607bfd1 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,12 +1,11 @@
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Sat Aug 31 20:36:02 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Sat Aug 31 20:36:02 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Sat Aug 31 20:36:02 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "ExternalFile", "IndexName": "ExternalFile", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "FileNumber" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
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 8935d5d..0078603 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,9 +1,9 @@
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sat Aug 31 20:36:01 AST 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" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sat Aug 31 20:36:01 AST 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" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 3, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "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" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 9, "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" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sat Aug 31 20:36:01 AST 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" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 4, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 5, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 6, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+
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 a44cbc0..cecdb85 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,63 +1,62 @@
-{ "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": "FeedDetails", "FieldType": "Field_FeedDetails_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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:02 AST 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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 2013" }
-{ "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": "FileModDate", "FieldType": "datetime" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:02 AST 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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:02 AST 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": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:02 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:02 AST 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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:02 AST 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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 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" }, { "FieldName": "GroupName", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_FeedDetails_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_FeedDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Function", "FieldType": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Status", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 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" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 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": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
\ No newline at end of file
+{ "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": "FeedDetails", "FieldType": "Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Hints", "FieldType": "Field_Hints_in_DatasetRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 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": "Fri Mar 29 11:19:47 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": "Fri Mar 29 11:19:47 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" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 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": "Fri Mar 29 11:19:47 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": "Fri Mar 29 11:19:47 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": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 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": "Fri Mar 29 11:19:47 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": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 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": "Fri Mar 29 11:19:47 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": "Fri Mar 29 11:19:47 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": "Fri Mar 29 11:19:47 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": "Fri Mar 29 11:19:47 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": "Fri Mar 29 11:19:47 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": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Mar 29 11:19:47 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": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Mar 29 11:19:47 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": "Fri Mar 29 11:19:47 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": "Fri Mar 29 11:19:47 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": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_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": "Fri Mar 29 11:19:47 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": "Fri Mar 29 11:19:47 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": "Fri Mar 29 11:19:47 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": "Fri Mar 29 11:19:47 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": "Fri Mar 29 11:19:47 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": "Fri Mar 29 11:19:47 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" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 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": "Fri Mar 29 11:19:47 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": "Fri Mar 29 11:19:47 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": "Fri Mar 29 11:19:47 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": "Fri Mar 29 11:19:47 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": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_FeedDetails_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_FeedDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Function", "FieldType": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Status", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 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" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 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": "Fri Mar 29 11:19:47 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": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
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 6037036..607bfd1 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,12 +1,11 @@
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Sat Aug 31 20:36:02 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Sat Aug 31 20:36:02 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Sat Aug 31 20:36:02 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "ExternalFile", "IndexName": "ExternalFile", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "FileNumber" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
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 445122f..7be9c57 100644
--- a/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv19/cross-dv19.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv19/cross-dv19.1.adm
@@ -1,7 +1,7 @@
-{ "DataverseName": "test1", "DatasetName": "TwitterData", "DataTypeName": "Tweet", "DatasetType": "EXTERNAL", "InternalDetails": null, "ExternalDetails": { "DatasourceAdapter": "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter", "Properties": [ { "Name": "path", "Value": "nc1://data/twitter/extrasmalltweets.txt" }, { "Name": "format", "Value": "adm" } ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Mon Sep 02 08:36:37 AST 2013", "DatasetId": 125, "PendingOp": 0 }
-{ "DataverseName": "test1", "DatasetName": "t1", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Mon Sep 02 08:36:36 AST 2013", "DatasetId": 119, "PendingOp": 0 }
-{ "DataverseName": "test1", "DatasetName": "t2", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Mon Sep 02 08:36:36 AST 2013", "DatasetId": 122, "PendingOp": 0 }
-{ "DataverseName": "test1", "DatasetName": "t3", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Mon Sep 02 08:36:37 AST 2013", "DatasetId": 123, "PendingOp": 0 }
-{ "DataverseName": "test2", "DatasetName": "t2", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Mon Sep 02 08:36:36 AST 2013", "DatasetId": 120, "PendingOp": 0 }
-{ "DataverseName": "test2", "DatasetName": "t3", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Mon Sep 02 08:36:36 AST 2013", "DatasetId": 121, "PendingOp": 0 }
-{ "DataverseName": "test2", "DatasetName": "t4", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Mon Sep 02 08:36:37 AST 2013", "DatasetId": 124, "PendingOp": 0 }
+{ "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" } ] }, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jan 29 19:04:36 PST 2013" }
+{ "DataverseName": "test1", "DatasetName": "t1", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jan 29 19:04:36 PST 2013" }
+{ "DataverseName": "test1", "DatasetName": "t2", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jan 29 19:04:36 PST 2013" }
+{ "DataverseName": "test1", "DatasetName": "t3", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jan 29 19:04:36 PST 2013" }
+{ "DataverseName": "test2", "DatasetName": "t2", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jan 29 19:04:36 PST 2013" }
+{ "DataverseName": "test2", "DatasetName": "t3", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jan 29 19:04:36 PST 2013" }
+{ "DataverseName": "test2", "DatasetName": "t4", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jan 29 19:04:36 PST 2013" }
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 2d97f0a..4233225 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
@@ -16,20 +16,10 @@
import java.util.Map;
-import edu.uci.ics.asterix.metadata.bootstrap.MetadataConstants;
-
public class ExternalDetailsDecl implements IDatasetDetailsDecl {
private Map<String, String> properties;
private String adapter;
- private Identifier nodegroupName;
- public ExternalDetailsDecl(Map<String, String> properties, String adapter, Identifier nodegroupName) {
- this.properties = properties;
- this.adapter = adapter;
- this.nodegroupName = nodegroupName == null ? new Identifier(MetadataConstants.METADATA_DEFAULT_NODEGROUP_NAME)
- : nodegroupName;
- }
-
public void setAdapter(String adapter) {
this.adapter = adapter;
}
@@ -45,12 +35,4 @@
public Map<String, String> getProperties() {
return properties;
}
-
- public void setNodegroupName(Identifier nodegroupName) {
- this.nodegroupName = nodegroupName;
- }
-
- public Identifier getNodegroupName() {
- return nodegroupName;
- }
}
diff --git a/asterix-aql/src/main/javacc/AQL.jj b/asterix-aql/src/main/javacc/AQL.jj
index 8dd7bfb..cb6336b 100644
--- a/asterix-aql/src/main/javacc/AQL.jj
+++ b/asterix-aql/src/main/javacc/AQL.jj
@@ -308,11 +308,11 @@
<LEFTPAREN> typeName = Identifier() <RIGHTPAREN>
ifNotExists = IfNotExists()
"using" adapterName = AdapterName() properties = Configuration()
- ("on" nodeGroupName = Identifier() )?
( "hints" hints = Properties() )?
{
- ExternalDetailsDecl edd = new ExternalDetailsDecl(properties,
- adapterName,nodeGroupName != null? new Identifier(nodeGroupName): null);
+ ExternalDetailsDecl edd = new ExternalDetailsDecl();
+ edd.setAdapter(adapterName);
+ edd.setProperties(properties);
dsetDecl = new DatasetDecl(nameComponents.first,
nameComponents.second,
new Identifier(typeName),
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/PrimaryIndexOperationTracker.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/PrimaryIndexOperationTracker.java
index 92a33cd..5adb361 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/PrimaryIndexOperationTracker.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/PrimaryIndexOperationTracker.java
@@ -18,6 +18,7 @@
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
+import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
@@ -41,7 +42,7 @@
public void beforeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
IModificationOperationCallback modificationCallback) throws HyracksDataException {
if (opType == LSMOperationType.MODIFICATION || opType == LSMOperationType.FORCE_MODIFICATION) {
- numActiveOperations.incrementAndGet();
+ incrementNumActiveOperations(modificationCallback);
} else if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
datasetLifecycleManager.declareActiveIOOperation(datasetID);
}
@@ -60,14 +61,11 @@
public void completeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
IModificationOperationCallback modificationCallback) throws HyracksDataException {
if (opType == LSMOperationType.MODIFICATION || opType == LSMOperationType.FORCE_MODIFICATION) {
- numActiveOperations.decrementAndGet();
+ decrementNumActiveOperations(modificationCallback);
+ flushIfFull();
} else if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
datasetLifecycleManager.undeclareActiveIOOperation(datasetID);
}
-
- if (opType == LSMOperationType.MODIFICATION || opType == LSMOperationType.FORCE_MODIFICATION) {
- flushIfFull();
- }
}
private void flushIfFull() throws HyracksDataException {
@@ -100,4 +98,25 @@
public int getNumActiveOperations() {
return numActiveOperations.get();
}
+
+ private void incrementNumActiveOperations(IModificationOperationCallback modificationCallback) {
+ if (modificationCallback != null && modificationCallback != NoOpOperationCallback.INSTANCE) {
+ numActiveOperations.incrementAndGet();
+ ((AbstractOperationCallback) modificationCallback).incrementLocalNumActiveOperations();
+ }
+ }
+
+ private void decrementNumActiveOperations(IModificationOperationCallback modificationCallback) {
+ if (modificationCallback != null && modificationCallback != NoOpOperationCallback.INSTANCE) {
+ numActiveOperations.decrementAndGet();
+ ((AbstractOperationCallback) modificationCallback).decrementLocalNumActiveOperations();
+ }
+ }
+
+ public void cleanupNumActiveOperationsForAbortedJob(AbstractOperationCallback callback) {
+ int delta = callback.getLocalNumActiveOperations() * -1;
+ numActiveOperations.getAndAdd(delta);
+ callback.resetLocalNumActiveOperations();
+ }
+
}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
index d2eb5ba..87c497c 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
@@ -17,7 +17,6 @@
import java.util.List;
-import edu.uci.ics.asterix.common.context.BaseOperationTracker;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTreeDiskComponent;
@@ -32,7 +31,7 @@
@Override
public void afterOperation(List<ILSMComponent> oldComponents, ILSMComponent newComponent)
throws HyracksDataException {
- if (oldComponents != null && newComponent != null) {
+ if (newComponent != null) {
LSMBTreeDiskComponent btreeComponent = (LSMBTreeDiskComponent) newComponent;
putLSNIntoMetadata(btreeComponent.getBTree(), oldComponents);
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/AbstractOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/AbstractOperationCallback.java
index d4b26f7..c549e7d 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/AbstractOperationCallback.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/AbstractOperationCallback.java
@@ -15,6 +15,8 @@
package edu.uci.ics.asterix.common.transactions;
+import java.util.concurrent.atomic.AtomicInteger;
+
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.MurmurHash128Bit;
@@ -27,6 +29,7 @@
protected final ITransactionContext txnCtx;
protected final ILockManager lockManager;
protected final long[] longHashes;
+ protected final AtomicInteger transactorLocalNumActiveOperations;
public AbstractOperationCallback(int datasetId, int[] primaryKeyFields, ITransactionContext txnCtx,
ILockManager lockManager) {
@@ -34,6 +37,7 @@
this.primaryKeyFields = primaryKeyFields;
this.txnCtx = txnCtx;
this.lockManager = lockManager;
+ this.transactorLocalNumActiveOperations = new AtomicInteger(0);
this.longHashes = new long[2];
}
@@ -41,4 +45,21 @@
MurmurHash128Bit.hash3_x64_128(tuple, primaryKeyFields, SEED, longHashes);
return Math.abs((int) longHashes[0]);
}
+
+ public void resetLocalNumActiveOperations() {
+ transactorLocalNumActiveOperations.set(0);
+ }
+
+ public int getLocalNumActiveOperations() {
+ return transactorLocalNumActiveOperations.get();
+ }
+
+ public void incrementLocalNumActiveOperations() {
+ transactorLocalNumActiveOperations.incrementAndGet();
+ }
+
+ public void decrementLocalNumActiveOperations() {
+ transactorLocalNumActiveOperations.decrementAndGet();
+ }
+
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java
index 8913f8a..27a91a4 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java
@@ -14,9 +14,11 @@
*/
package edu.uci.ics.asterix.common.transactions;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+
public interface ILogManager {
- public void log(ILogRecord logRecord);
+ public void log(ILogRecord logRecord) throws ACIDException;
public ILogReader getLogReader(boolean isRecoveryMode);
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecord.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecord.java
index d13ef6c..3068867 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecord.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecord.java
@@ -20,15 +20,15 @@
public interface ILogRecord {
- public static final int JOB_COMMIT_LOG_SIZE = 13;
- public static final int ENTITY_COMMIT_LOG_BASE_SIZE = 29;
- public static final int UPDATE_LOG_BASE_SIZE = 64;
+ public static final int JOB_TERMINATE_LOG_SIZE = 13; //JOB_COMMIT or ABORT log type
+ public static final int ENTITY_COMMIT_LOG_BASE_SIZE = 25;
+ public static final int UPDATE_LOG_BASE_SIZE = 60;
public boolean readLogRecord(ByteBuffer buffer);
public void writeLogRecord(ByteBuffer buffer);
- public void formJobCommitLogRecord(ITransactionContext txnCtx);
+ public void formJobTerminateLogRecord(ITransactionContext txnCtx, boolean isCommit);
public void formEntityCommitLogRecord(ITransactionContext txnCtx, int datasetId, int PKHashValue,
ITupleReference tupleReference, int[] primaryKeyFields);
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionManager.java
index 77e960b..ffd4cc2 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionManager.java
@@ -52,10 +52,11 @@
*
* @param jobId
* a unique value for the transaction id.
+ * @param createIfNotExist TODO
* @return
* @throws ACIDException
*/
- public ITransactionContext getTransactionContext(JobId jobId) throws ACIDException;
+ public ITransactionContext getTransactionContext(JobId jobId, boolean createIfNotExist) throws ACIDException;
/**
* Commits a transaction.
diff --git a/asterix-external-data/pom.xml b/asterix-external-data/pom.xml
index ffc0eac..f8d5ea2 100644
--- a/asterix-external-data/pom.xml
+++ b/asterix-external-data/pom.xml
@@ -153,17 +153,6 @@
<artifactId>jdom</artifactId>
<version>1.0</version>
</dependency>
- <dependency>
- <groupId>javax.jdo</groupId>
- <artifactId>jdo2-api</artifactId>
- <version>2.3-20090302111651</version>
- </dependency>
- <dependency>
- <groupId>org.apache.hive</groupId>
- <artifactId>hive-exec</artifactId>
- <version>0.11.0</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 e9d2175..4ca3d72 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
@@ -20,21 +20,12 @@
import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.conf.Configuration;
-import edu.uci.ics.asterix.external.dataset.adapter.HDFSAccessByRIDAdapter;
+
import edu.uci.ics.asterix.external.dataset.adapter.HDFSAdapter;
-import edu.uci.ics.asterix.external.dataset.adapter.HDFSIndexingAdapter;
-import edu.uci.ics.asterix.external.dataset.adapter.IControlledAdapter;
import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
-import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
-import edu.uci.ics.asterix.om.types.BuiltinType;
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.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.hdfs.dataflow.ConfFactory;
import edu.uci.ics.hyracks.hdfs.dataflow.InputSplitsFactory;
import edu.uci.ics.hyracks.hdfs.scheduler.Scheduler;
@@ -44,190 +35,76 @@
*/
@SuppressWarnings("deprecation")
public class HDFSAdapterFactory implements IGenericDatasetAdapterFactory {
- private static final long serialVersionUID = 1L;
+ private static final long serialVersionUID = 1L;
- public static final String HDFS_ADAPTER_NAME = "hdfs";
- public static final String CLUSTER_LOCATIONS = "cluster-locations";
- public static transient String SCHEDULER = "hdfs-scheduler";
+ public static final String HDFS_ADAPTER_NAME = "hdfs";
+ public static final String CLUSTER_LOCATIONS = "cluster-locations";
+ public static transient String SCHEDULER = "hdfs-scheduler";
- public static final String KEY_HDFS_URL = "hdfs";
- public static final String KEY_PATH = "path";
- 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";
- public static final String INPUT_FORMAT_RC = "rc-input-format";
- public static final String KEY_DELIMITER = "delimiter";
- public static final String KEY_FORMAT = "format";
- public static final String FORMAT_DELIMITED_TEXT = "delimited-text";
+ public static final String KEY_HDFS_URL = "hdfs";
+ public static final String KEY_PATH = "path";
+ 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";
- private transient AlgebricksPartitionConstraint clusterLocations;
- private String[] readSchedule;
- private boolean executed[];
- private InputSplitsFactory inputSplitsFactory;
- private ConfFactory confFactory;
- private boolean setup = false;
+ private transient AlgebricksPartitionConstraint clusterLocations;
+ private String[] readSchedule;
+ private boolean executed[];
+ private InputSplitsFactory inputSplitsFactory;
+ private ConfFactory confFactory;
+ private boolean setup = false;
- private static final Map<String, String> formatClassNames = initInputFormatMap();
+ private static final Map<String, String> formatClassNames = initInputFormatMap();
- private 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;
- }
+ private 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");
+ return formatClassNames;
+ }
- @Override
- public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception {
- if (!setup) {
- /** set up the factory --serializable stuff --- this if-block should be called only once for each factory instance */
- configureJobConf(configuration);
- JobConf conf = configureJobConf(configuration);
- confFactory = new ConfFactory(conf);
+ @Override
+ public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception {
+ if (!setup) {
+ /** set up the factory --serializable stuff --- this if-block should be called only once for each factory instance */
+ configureJobConf(configuration);
+ JobConf conf = configureJobConf(configuration);
+ confFactory = new ConfFactory(conf);
- clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
- int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
+ clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
+ int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
- InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
- inputSplitsFactory = new InputSplitsFactory(inputSplits);
+ InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
+ inputSplitsFactory = new InputSplitsFactory(inputSplits);
- Scheduler scheduler = (Scheduler) configuration.get(SCHEDULER);
- readSchedule = scheduler.getLocationConstraints(inputSplits);
- executed = new boolean[readSchedule.length];
- Arrays.fill(executed, false);
+ Scheduler scheduler = (Scheduler) configuration.get(SCHEDULER);
+ readSchedule = scheduler.getLocationConstraints(inputSplits);
+ executed = new boolean[readSchedule.length];
+ Arrays.fill(executed, false);
- setup = true;
- }
- JobConf conf = confFactory.getConf();
- InputSplit[] inputSplits = inputSplitsFactory.getSplits();
- HDFSAdapter hdfsAdapter = new HDFSAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations);
+ setup = true;
+ }
+ JobConf conf = confFactory.getConf();
+ InputSplit[] inputSplits = inputSplitsFactory.getSplits();
+ HDFSAdapter hdfsAdapter = new HDFSAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations);
+ hdfsAdapter.configure(configuration);
+ return hdfsAdapter;
+ }
- //If input format is rcfile, configure parser expected format to delimeted text with 0x01 (default ) as delimiter
- if(((String)configuration.get(KEY_INPUT_FORMAT)).equals(INPUT_FORMAT_RC))
- {
- char delimeter = 0x01;
- configuration.put(KEY_FORMAT, FORMAT_DELIMITED_TEXT);
- configuration.put(KEY_DELIMITER, Character.toString(delimeter));
- }
+ @Override
+ public String getName() {
+ return HDFS_ADAPTER_NAME;
+ }
- hdfsAdapter.configure(configuration);
- return hdfsAdapter;
- }
-
- @Override
- public IControlledAdapter createAccessByRIDAdapter(
- Map<String, Object> configuration, IAType atype, HashMap<Integer, String> files) throws Exception {
- Configuration conf = configureHadoopConnection(configuration);
- clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
-
- //Create RID record desc
- RecordDescriptor ridRecordDesc = null;
-
- //If input format is rcfile, configure parser expected format to delimeted text with control char 0x01 as delimiter
- if(((String)configuration.get(KEY_INPUT_FORMAT)).equals(INPUT_FORMAT_RC))
- {
- char delimeter = 0x01;
- configuration.put(KEY_FORMAT, FORMAT_DELIMITED_TEXT);
- configuration.put(KEY_DELIMITER, Character.toString(delimeter));
- ridRecordDesc = getRIDRecDesc(true, files != null);
- }
- else
- {
- ridRecordDesc = getRIDRecDesc(false, files != null);
- }
- HDFSAccessByRIDAdapter adapter = new HDFSAccessByRIDAdapter(atype, ((String)configuration.get(KEY_INPUT_FORMAT)), clusterLocations,ridRecordDesc, conf, files);
- adapter.configure(configuration);
- return adapter;
- }
-
- @Override
- public IDatasourceAdapter createIndexingAdapter(Map<String, Object> configuration, IAType atype, Map<String,Integer> files) throws Exception {
- if (!setup) {
- /** set up the factory --serializable stuff --- this if-block should be called only once for each factory instance */
- configureJobConf(configuration);
- JobConf conf = configureJobConf(configuration);
- confFactory = new ConfFactory(conf);
-
- clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
- int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
-
- InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
- inputSplitsFactory = new InputSplitsFactory(inputSplits);
-
- Scheduler scheduler = (Scheduler) configuration.get(SCHEDULER);
- readSchedule = scheduler.getLocationConstraints(inputSplits);
- executed = new boolean[readSchedule.length];
- Arrays.fill(executed, false);
-
- setup = true;
- }
- JobConf conf = confFactory.getConf();
- InputSplit[] inputSplits = inputSplitsFactory.getSplits();
- //If input format is rcfile, configure parser expected format to delimeted text with 0x01 (default) as delimiter
- if(((String)configuration.get(KEY_INPUT_FORMAT)).equals(INPUT_FORMAT_RC))
- {
- char delimeter = 0x01;
- configuration.put(KEY_FORMAT, FORMAT_DELIMITED_TEXT);
- configuration.put(KEY_DELIMITER, Character.toString(delimeter));
- }
- HDFSIndexingAdapter hdfsIndexingAdapter = new HDFSIndexingAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations, files);
- hdfsIndexingAdapter.configure(configuration);
- return hdfsIndexingAdapter;
- }
-
- @Override
- public String getName() {
- return HDFS_ADAPTER_NAME;
- }
-
- private JobConf configureJobConf(Map<String, Object> configuration) throws Exception {
- JobConf conf = new JobConf();
- 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()));
- return conf;
- }
-
- public static Configuration configureHadoopConnection(Map<String, Object> configuration)
- {
- Configuration conf = new Configuration();
- conf.set("fs.default.name", ((String) configuration.get(KEY_HDFS_URL)).trim());
- conf.set("fs.hdfs.impl", "org.apache.hadoop.hdfs.DistributedFileSystem");
- return conf;
- }
-
- public static RecordDescriptor getRIDRecDesc(boolean isRCFile, boolean optimize){
- int numOfPrimaryKeys = 2;
- if(isRCFile)
- {
- numOfPrimaryKeys++;
- }
- @SuppressWarnings("rawtypes")
- ISerializerDeserializer[] serde = new ISerializerDeserializer[numOfPrimaryKeys];
- ITypeTraits[] tt = new ITypeTraits[numOfPrimaryKeys];
- if(optimize)
- {
- serde[0] = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
- tt[0] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(BuiltinType.AINT32);
- }
- else
- {
- serde[0] = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING);
- tt[0] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(BuiltinType.ASTRING);
- }
- serde[1] = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
- tt[1] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(BuiltinType.AINT64);
- if(isRCFile)
- {
- //we add the row number for rc-files
- serde[2] = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
- tt[2] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(BuiltinType.AINT32);
- }
- return new RecordDescriptor(serde, tt);
- }
-
+ private JobConf configureJobConf(Map<String, Object> configuration) throws Exception {
+ JobConf conf = new JobConf();
+ 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()));
+ return conf;
+ }
}
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 64c8153..409eb7a 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
@@ -18,20 +18,15 @@
import java.util.HashMap;
import java.util.Map;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.JobConf;
-import edu.uci.ics.asterix.external.dataset.adapter.HDFSAccessByRIDAdapter;
import edu.uci.ics.asterix.external.dataset.adapter.HDFSAdapter;
import edu.uci.ics.asterix.external.dataset.adapter.HiveAdapter;
-import edu.uci.ics.asterix.external.dataset.adapter.HiveIndexingAdapter;
import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
-import edu.uci.ics.asterix.external.dataset.adapter.IControlledAdapter;
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.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.hdfs.dataflow.ConfFactory;
import edu.uci.ics.hyracks.hdfs.dataflow.InputSplitsFactory;
import edu.uci.ics.hyracks.hdfs.scheduler.Scheduler;
@@ -41,185 +36,109 @@
*/
@SuppressWarnings("deprecation")
public class HiveAdapterFactory implements IGenericDatasetAdapterFactory {
- private static final long serialVersionUID = 1L;
+ private static final long serialVersionUID = 1L;
- public static final String HDFS_ADAPTER_NAME = "hdfs";
- public static final String CLUSTER_LOCATIONS = "cluster-locations";
- public static transient String SCHEDULER = "hdfs-scheduler";
+ public static final String HDFS_ADAPTER_NAME = "hdfs";
+ public static final String CLUSTER_LOCATIONS = "cluster-locations";
+ public static transient String SCHEDULER = "hdfs-scheduler";
- public static final String KEY_HDFS_URL = "hdfs";
- public static final String KEY_PATH = "path";
- 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";
- public static final String INPUT_FORMAT_RC = "rc-input-format";
+ public static final String KEY_HDFS_URL = "hdfs";
+ public static final String KEY_PATH = "path";
+ 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";
- public static final String KEY_FORMAT = "format";
- public static final String KEY_PARSER_FACTORY = "parser";
- public static final String FORMAT_DELIMITED_TEXT = "delimited-text";
- public static final String FORMAT_ADM = "adm";
- public static final String KEY_DELIMITER = "delimiter";
+ public static final String KEY_FORMAT = "format";
+ public static final String KEY_PARSER_FACTORY = "parser";
+ public static final String FORMAT_DELIMITED_TEXT = "delimited-text";
+ public static final String FORMAT_ADM = "adm";
- public static final String HIVE_DATABASE = "database";
- public static final String HIVE_TABLE = "table";
- public static final String HIVE_HOME = "hive-home";
- public static final String HIVE_METASTORE_URI = "metastore-uri";
- public static final String HIVE_WAREHOUSE_DIR = "warehouse-dir";
- public static final String HIVE_METASTORE_RAWSTORE_IMPL = "rawstore-impl";
+ public static final String HIVE_DATABASE = "database";
+ public static final String HIVE_TABLE = "table";
+ public static final String HIVE_HOME = "hive-home";
+ public static final String HIVE_METASTORE_URI = "metastore-uri";
+ public static final String HIVE_WAREHOUSE_DIR = "warehouse-dir";
+ public static final String HIVE_METASTORE_RAWSTORE_IMPL = "rawstore-impl";
- private String[] readSchedule;
- private boolean executed[];
- private InputSplitsFactory inputSplitsFactory;
- private ConfFactory confFactory;
- private transient AlgebricksPartitionConstraint clusterLocations;
- private boolean setup = false;
+ private String[] readSchedule;
+ private boolean executed[];
+ private InputSplitsFactory inputSplitsFactory;
+ private ConfFactory confFactory;
+ private transient AlgebricksPartitionConstraint clusterLocations;
+ private boolean setup = false;
- private static final Map<String, String> formatClassNames = initInputFormatMap();
+ private static final Map<String, String> formatClassNames = initInputFormatMap();
- private 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;
- }
+ private 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");
+ return formatClassNames;
+ }
+ @Override
+ public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception {
+ if (!setup) {
+ /** set up the factory --serializable stuff --- this if-block should be called only once for each factory instance */
+ configureJobConf(configuration);
+ JobConf conf = configureJobConf(configuration);
+ confFactory = new ConfFactory(conf);
- @Override
- public IControlledAdapter createAccessByRIDAdapter(Map<String, Object> configuration, IAType atype, HashMap<Integer, String> files) throws Exception {
- Configuration conf = HDFSAdapterFactory.configureHadoopConnection(configuration);
- clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
- //Create RID record desc
- RecordDescriptor ridRecordDesc = null;
+ clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
+ int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
- //If input format is rcfile, configure parser expected format to delimeted text with control char 0x01 as delimiter
- if(((String)configuration.get(KEY_INPUT_FORMAT)).equals(INPUT_FORMAT_RC))
- {
- char delimeter = 0x01;
- configuration.put(KEY_FORMAT, FORMAT_DELIMITED_TEXT);
- configuration.put(KEY_DELIMITER, Character.toString(delimeter));
- ridRecordDesc = HDFSAdapterFactory.getRIDRecDesc(true, files != null);
- }
- else
- {
- ridRecordDesc = HDFSAdapterFactory.getRIDRecDesc(false, files != null);
- }
- HDFSAccessByRIDAdapter adapter = new HDFSAccessByRIDAdapter(atype, ((String)configuration.get(KEY_INPUT_FORMAT)), clusterLocations,ridRecordDesc, conf, files);
- adapter.configure(configuration);
- return adapter;
- }
+ InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
+ inputSplitsFactory = new InputSplitsFactory(inputSplits);
- @Override
- public IDatasourceAdapter createIndexingAdapter(
- Map<String, Object> configuration, IAType atype, Map<String,Integer> files) throws Exception {
- if (!setup) {
- /** set up the factory --serializable stuff --- this if-block should be called only once for each factory instance */
- configureJobConf(configuration);
- JobConf conf = configureJobConf(configuration);
- confFactory = new ConfFactory(conf);
+ Scheduler scheduler = (Scheduler) configuration.get(SCHEDULER);
+ readSchedule = scheduler.getLocationConstraints(inputSplits);
+ executed = new boolean[readSchedule.length];
+ Arrays.fill(executed, false);
- clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
- int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
+ setup = true;
+ }
+ JobConf conf = confFactory.getConf();
+ InputSplit[] inputSplits = inputSplitsFactory.getSplits();
+ HiveAdapter hiveAdapter = new HiveAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations);
+ hiveAdapter.configure(configuration);
+ return hiveAdapter;
+ }
- InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
- inputSplitsFactory = new InputSplitsFactory(inputSplits);
+ @Override
+ public String getName() {
+ return "hive";
+ }
- Scheduler scheduler = (Scheduler) configuration.get(SCHEDULER);
- readSchedule = scheduler.getLocationConstraints(inputSplits);
- executed = new boolean[readSchedule.length];
- Arrays.fill(executed, false);
+ private JobConf configureJobConf(Map<String, Object> configuration) throws Exception {
+ JobConf conf = new JobConf();
- setup = true;
- }
- JobConf conf = confFactory.getConf();
- InputSplit[] inputSplits = inputSplitsFactory.getSplits();
- HiveIndexingAdapter hiveIndexingAdapter = new HiveIndexingAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations, files);
+ /** configure hive */
+ String database = (String) configuration.get(HIVE_DATABASE);
+ String tablePath = null;
+ if (database == null) {
+ tablePath = configuration.get(HIVE_WAREHOUSE_DIR) + "/" + configuration.get(HIVE_TABLE);
+ } else {
+ tablePath = configuration.get(HIVE_WAREHOUSE_DIR) + "/" + tablePath + ".db" + "/"
+ + configuration.get(HIVE_TABLE);
+ }
+ configuration.put(HDFSAdapter.KEY_PATH, tablePath);
+ if (!configuration.get(KEY_FORMAT).equals(FORMAT_DELIMITED_TEXT)) {
+ throw new IllegalArgumentException("format" + configuration.get(KEY_FORMAT) + " is not supported");
+ }
- //If input format is rcfile, configure parser expected format to delimeted text with 0x01 (default ) as delimiter
- if(((String)configuration.get(KEY_INPUT_FORMAT)).equals(INPUT_FORMAT_RC))
- {
- char delimeter = 0x01;
- configuration.put(KEY_FORMAT, FORMAT_DELIMITED_TEXT);
- configuration.put(KEY_DELIMITER, Character.toString(delimeter));
- }
+ if (!(configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT).equals(HDFSAdapterFactory.INPUT_FORMAT_TEXT) || configuration
+ .get(HDFSAdapterFactory.KEY_INPUT_FORMAT).equals(HDFSAdapterFactory.INPUT_FORMAT_SEQUENCE))) {
+ throw new IllegalArgumentException("file input format"
+ + configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT) + " is not supported");
+ }
- hiveIndexingAdapter.configure(configuration);
- return hiveIndexingAdapter;
- }
-
- @Override
- public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception {
- if (!setup) {
- /** set up the factory --serializable stuff --- this if-block should be called only once for each factory instance */
- configureJobConf(configuration);
- JobConf conf = configureJobConf(configuration);
- confFactory = new ConfFactory(conf);
-
- clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
- int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
-
- InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
- inputSplitsFactory = new InputSplitsFactory(inputSplits);
-
- Scheduler scheduler = (Scheduler) configuration.get(SCHEDULER);
- readSchedule = scheduler.getLocationConstraints(inputSplits);
- executed = new boolean[readSchedule.length];
- Arrays.fill(executed, false);
-
- setup = true;
- }
- JobConf conf = confFactory.getConf();
- InputSplit[] inputSplits = inputSplitsFactory.getSplits();
- HiveAdapter hiveAdapter = new HiveAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations);
-
- //If input format is rcfile, configure parser expected format to delimeted text with 0x01 (default ) as delimiter
- if(((String)configuration.get(KEY_INPUT_FORMAT)).equals(INPUT_FORMAT_RC))
- {
- char delimeter = 0x01;
- configuration.put(KEY_FORMAT, FORMAT_DELIMITED_TEXT);
- configuration.put(KEY_DELIMITER, Character.toString(delimeter));
- }
-
- hiveAdapter.configure(configuration);
- return hiveAdapter;
- }
-
- @Override
- public String getName() {
- return "hive";
- }
-
- private JobConf configureJobConf(Map<String, Object> configuration) throws Exception {
- JobConf conf = new JobConf();
-
- /** configure hive */
- String database = (String) configuration.get(HIVE_DATABASE);
- String tablePath = null;
- if (database == null) {
- tablePath = configuration.get(HIVE_WAREHOUSE_DIR) + "/" + configuration.get(HIVE_TABLE);
- } else {
- tablePath = configuration.get(HIVE_WAREHOUSE_DIR) + "/" + tablePath + ".db" + "/"
- + configuration.get(HIVE_TABLE);
- }
- configuration.put(HDFSAdapter.KEY_PATH, tablePath);
- if (!configuration.get(KEY_FORMAT).equals(FORMAT_DELIMITED_TEXT)) {
- throw new IllegalArgumentException("format" + configuration.get(KEY_FORMAT) + " is not supported");
- }
-
- if (!(configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT).equals(HDFSAdapterFactory.INPUT_FORMAT_TEXT) || configuration
- .get(HDFSAdapterFactory.KEY_INPUT_FORMAT).equals(HDFSAdapterFactory.INPUT_FORMAT_SEQUENCE) || configuration
- .get(HDFSAdapterFactory.KEY_INPUT_FORMAT).equals(HDFSAdapterFactory.INPUT_FORMAT_RC))) {
- throw new IllegalArgumentException("file input format"
- + configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT) + " is not supported");
- }
-
- /** configure hdfs */
- 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()));
- return conf;
- }
-}
\ No newline at end of file
+ /** configure hdfs */
+ 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()));
+ return conf;
+ }
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IGenericDatasetAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IGenericDatasetAdapterFactory.java
index f046f88..0a178a7 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IGenericDatasetAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IGenericDatasetAdapterFactory.java
@@ -14,10 +14,8 @@
*/
package edu.uci.ics.asterix.external.adapter.factory;
-import java.util.HashMap;
import java.util.Map;
-import edu.uci.ics.asterix.external.dataset.adapter.IControlledAdapter;
import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
import edu.uci.ics.asterix.om.types.IAType;
@@ -42,29 +40,4 @@
*/
public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception;
- /**
- * Creates an instance of IDatasourceAdapter that is used to read records and their RIDs.
- *
- * @param configuration
- * The configuration parameters for the adapter that is instantiated.
- * The passed-in configuration is used to configure the created instance of the adapter.
- * @param atype
- * The type for the ADM records that are returned by the adapter (contains both original fields and RID fields).
- * @return An instance of IDatasourceAdapter.
- * @throws Exception
- */
- public IDatasourceAdapter createIndexingAdapter(Map<String, Object> configuration, IAType atype, Map<String,Integer> files) throws Exception;
-
- /**
- * Creates an instance of IDatasourceAdapter that is used to read records using their RIDs.
- *
- * @param configuration
- * The configuration parameters for the adapter that is instantiated.
- * The passed-in configuration is used to configure the created instance of the adapter.
- * @param atype
- * The type for the ADM records that are returned by the adapter.
- * @return An instance of IControlledAdapter.
- * @throws Exception
- */
- public IControlledAdapter createAccessByRIDAdapter(Map<String, Object> configuration, IAType atype, HashMap<Integer, String> files) throws Exception;
}
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 4fae7e7..e680232 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
@@ -14,14 +14,11 @@
*/
package edu.uci.ics.asterix.external.adapter.factory;
-import java.util.HashMap;
import java.util.Map;
-import edu.uci.ics.asterix.external.dataset.adapter.IControlledAdapter;
import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
import edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter;
import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
/**
* Factory class for creating an instance of NCFileSystemAdapter. An
@@ -29,31 +26,18 @@
* an NC.
*/
public class NCFileSystemAdapterFactory implements IGenericDatasetAdapterFactory {
- private static final long serialVersionUID = 1L;
- public static final String NC_FILE_SYSTEM_ADAPTER_NAME = "localfs";
+ private static final long serialVersionUID = 1L;
+ public static final String NC_FILE_SYSTEM_ADAPTER_NAME = "localfs";
- @Override
- public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception {
- NCFileSystemAdapter fsAdapter = new NCFileSystemAdapter(atype);
- fsAdapter.configure(configuration);
- return fsAdapter;
- }
+ @Override
+ public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception {
+ NCFileSystemAdapter fsAdapter = new NCFileSystemAdapter(atype);
+ fsAdapter.configure(configuration);
+ return fsAdapter;
+ }
- @Override
- public String getName() {
- return NC_FILE_SYSTEM_ADAPTER_NAME;
- }
-
- @Override
- public IDatasourceAdapter createIndexingAdapter(
- Map<String, Object> configuration, IAType atype, Map<String,Integer> files) throws Exception {
- throw new NotImplementedException("Indexing Adapter is not implemented for NC FileSystem Data");
- }
-
-
-
- @Override
- public IControlledAdapter createAccessByRIDAdapter(Map<String, Object> configuration, IAType atype, HashMap<Integer, String> files) throws Exception {
- throw new NotImplementedException("Access by RID Adapter is not implemented for NC FileSystem Data");
- }
+ @Override
+ public String getName() {
+ return NC_FILE_SYSTEM_ADAPTER_NAME;
+ }
}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataAccessByRIDOperatorDescriptor.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataAccessByRIDOperatorDescriptor.java
deleted file mode 100644
index aa91a56..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataAccessByRIDOperatorDescriptor.java
+++ /dev/null
@@ -1,78 +0,0 @@
-package edu.uci.ics.asterix.external.data.operator;
-
-import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.Map;
-
-import edu.uci.ics.asterix.external.adapter.factory.IGenericDatasetAdapterFactory;
-import edu.uci.ics.asterix.external.dataset.adapter.IControlledAdapter;
-import edu.uci.ics.asterix.om.types.IAType;
-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.AbstractSingleActivityOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
-
-public class ExternalDataAccessByRIDOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
-
- /**
- * This operator is used to access external data residing in hdfs using record ids pushed in frame buffers
- */
- private static final long serialVersionUID = 1L;
- private final Map<String, Object> adapterConfiguration;
- private final IAType atype;
- private IGenericDatasetAdapterFactory datasourceAdapterFactory;
- private IControlledAdapter adapter;
- private final HashMap<Integer, String> files;
-
- public ExternalDataAccessByRIDOperatorDescriptor(
- IOperatorDescriptorRegistry spec, Map<String, Object> arguments, IAType atype,
- RecordDescriptor outRecDesc,IGenericDatasetAdapterFactory dataSourceAdapterFactory, HashMap<Integer, String> files) {
- super(spec, 1, 1);
- this.atype = atype;
- this.adapterConfiguration = arguments;
- this.datasourceAdapterFactory = dataSourceAdapterFactory;
- this.recordDescriptors[0] = outRecDesc;
- this.files = files;
- }
-
- @Override
- public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
- IRecordDescriptorProvider recordDescProvider, int partition,
- int nPartitions) throws HyracksDataException {
- return new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
- @Override
- public void open() throws HyracksDataException {
- //create the access by index adapter
- try {
- adapter = datasourceAdapterFactory.createAccessByRIDAdapter(adapterConfiguration, atype, files);
- adapter.initialize(ctx);
- } catch (Exception e) {
- throw new HyracksDataException("error during creation of external read by RID adapter", e);
- }
- writer.open();
- }
-
- @Override
- public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
- adapter.processNextFrame(buffer, writer);
- }
-
- @Override
- public void close() throws HyracksDataException {
- //close adapter and flush remaining frame if needed
- adapter.close(writer);
- //close writer
- writer.close();
- }
-
- @Override
- public void fail() throws HyracksDataException {
- writer.fail();
- }
- };
- }
-}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataIndexingOperatorDescriptor.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataIndexingOperatorDescriptor.java
deleted file mode 100644
index 9ff1f06..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataIndexingOperatorDescriptor.java
+++ /dev/null
@@ -1,64 +0,0 @@
-package edu.uci.ics.asterix.external.data.operator;
-
-import java.util.Map;
-
-import edu.uci.ics.asterix.external.adapter.factory.IGenericDatasetAdapterFactory;
-import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
-import edu.uci.ics.asterix.om.types.IAType;
-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.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
-
-/*
- * A single activity operator that provides the functionality of scanning data along
- * with their RIDs using an instance of the configured adapter.
- */
-
-public class ExternalDataIndexingOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor{
-
- private static final long serialVersionUID = 1L;
-
- private final Map<String, Object> adapterConfiguration;
- private final Map<String,Integer> files;
- private final IAType atype;
- private IGenericDatasetAdapterFactory datasourceAdapterFactory;
-
- public ExternalDataIndexingOperatorDescriptor(JobSpecification spec, Map<String, Object> arguments, IAType atype,
- RecordDescriptor rDesc, IGenericDatasetAdapterFactory dataSourceAdapterFactory, Map<String,Integer> files) {
- super(spec, 0, 1);
- recordDescriptors[0] = rDesc;
- this.adapterConfiguration = arguments;
- this.atype = atype;
- this.datasourceAdapterFactory = dataSourceAdapterFactory;
- this.files = files;
- }
-
- @Override
- public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
- IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
- throws HyracksDataException {
-
- return new AbstractUnaryOutputSourceOperatorNodePushable() {
- @Override
- public void initialize() throws HyracksDataException {
- writer.open();
- IDatasourceAdapter adapter = null;
- try {
- adapter = ((IGenericDatasetAdapterFactory) datasourceAdapterFactory).createIndexingAdapter(
- adapterConfiguration, atype, files);
- adapter.initialize(ctx);
- adapter.start(partition, writer);
- } catch (Exception e) {
- throw new HyracksDataException("exception during reading from external data source", e);
- } finally {
- writer.close();
- }
- }
- };
- }
-}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAccessByRIDAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAccessByRIDAdapter.java
deleted file mode 100644
index 86a060c..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAccessByRIDAdapter.java
+++ /dev/null
@@ -1,1170 +0,0 @@
-/*
- * 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.DataInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.ql.io.RCFile;
-import org.apache.hadoop.hive.ql.io.RCFile.Reader;
-import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable;
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory;
-import edu.uci.ics.asterix.om.base.AInt32;
-import edu.uci.ics.asterix.om.base.AInt64;
-import edu.uci.ics.asterix.om.base.AString;
-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.runtime.operators.file.ControlledADMTupleParserFactory;
-import edu.uci.ics.asterix.runtime.operators.file.ControlledDelimitedDataTupleParserFactory;
-import edu.uci.ics.asterix.runtime.operators.file.ControlledTupleParser;
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
-import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
-
-/**
- * Provides functionality for fetching specific external data records stored in an HDFS instance
- * using their RID.
- */
-@SuppressWarnings({ "deprecation" })
-public class HDFSAccessByRIDAdapter extends FileSystemBasedAdapter implements IControlledAdapter{
-
- private static final long serialVersionUID = 1L;
- private boolean newFrame;
- private transient ByteBuffer frameBuffer;
- private String inputFormat;
- private Configuration conf;
- private transient FileSystem fs;
- private RecordDescriptor inRecDesc;
- private final HashMap<Integer, String> files;
-
- public HDFSAccessByRIDAdapter(IAType atype, String inputFormat, AlgebricksPartitionConstraint clusterLocations, RecordDescriptor inRecDesc, Configuration conf, HashMap<Integer,String> files) {
- super(atype);
- this.inputFormat = inputFormat;
- this.conf = conf;
- this.inRecDesc = inRecDesc;
- this.files = files;
- }
-
- @Override
- public void configure(Map<String, Object> arguments) throws Exception {
- this.configuration = arguments;
- fs = FileSystem.get(conf);
- 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) atype);
- } else if (FORMAT_ADM.equalsIgnoreCase((String)configuration.get(KEY_FORMAT))) {
- parserFactory = new ControlledADMTupleParserFactory((ARecordType) atype);
- } else {
- throw new IllegalArgumentException(" format " + configuration.get(KEY_FORMAT) + " not supported");
- }
- }
-
- @Override
- protected ITupleParserFactory getDelimitedDataTupleParserFactory(ARecordType recordType) throws AsterixException {
- int n = recordType.getFieldTypes().length;
- IValueParserFactory[] fieldParserFactories = new IValueParserFactory[n];
- for (int i = 0; i < n; i++) {
- ATypeTag tag = recordType.getFieldTypes()[i].getTypeTag();
- IValueParserFactory vpf = typeToValueParserFactMap.get(tag);
- if (vpf == null) {
- throw new NotImplementedException("No value parser factory for delimited fields of type " + tag);
- }
- fieldParserFactories[i] = vpf;
- }
- String delimiterValue = (String) configuration.get(KEY_DELIMITER);
- if (delimiterValue != null && delimiterValue.length() > 1) {
- throw new AsterixException("improper delimiter");
- }
-
- Character delimiter = delimiterValue.charAt(0);
- return new ControlledDelimitedDataTupleParserFactory(recordType, fieldParserFactories, delimiter);
- }
-
- @Override
- public void start(int partition, IFrameWriter writer) throws Exception {
- throw new NotImplementedException("Access by RID adapter doesn't support start function");
- }
-
- public void processNextFrame(ByteBuffer buffer, IFrameWriter writer) throws HyracksDataException
- {
- frameBuffer = buffer;
- newFrame = true;
- ((ControlledTupleParser)parser).parseNext(writer);
- }
-
- public void close(IFrameWriter writer) throws HyracksDataException
- {
- ((ControlledTupleParser)parser).close(writer);
- }
-
- public AdapterType getAdapterType() {
- return AdapterType.READ;
- }
-
- @Override
- public void initialize(IHyracksTaskContext ctx) throws Exception {
- this.ctx = ctx;
- //create parser and initialize it with an instance of the inputStream
- parser = parserFactory.createTupleParser(ctx);
- ((ControlledTupleParser)parser).initialize(getInputStream(0));
- }
-
- @Override
- public InputStream getInputStream(int partition) throws IOException {
-
- //if files map is not null, then it is optimized and we should return optimized inputStream, else return regular
- if(files == null)
- {
-
- //different input stream implementation based on the input format
- if(inputFormat.equals(HDFSAdapterFactory.INPUT_FORMAT_RC))
- {
- return new InputStream() {
- private RCFile.Reader reader;
- private int rowDifference;
- private String lastFileName = "";
- private String newFileName;
- private long lastByteLocation = 0;
- private long newByteLocation = 0;
- private int lastRowNumber = 0;
- private int newRowNumber = 0;
- private LongWritable key;
- private BytesRefArrayWritable value;
- private int EOL = "\n".getBytes()[0];
- private byte delimiter = 0x01;
- private boolean pendingValue = false;
- private int currentTupleIdx;
- private int numberOfTuplesInCurrentFrame;
- private IFrameTupleAccessor tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(),inRecDesc);
- private ByteBufferInputStream bbis = new ByteBufferInputStream();
- private DataInputStream dis = new DataInputStream(bbis);
-
- @Override
- public void close()
- {
- if (reader != null)
- {
- reader.close();
- }
- try {
- super.close();
- } catch (IOException e) {
- e.printStackTrace();
- }
- }
-
- @Override
- public int read(byte[] buffer, int offset, int len) throws IOException {
- if(newFrame)
- {
- //first time called with this frame
- //reset frame buffer
- tupleAccessor.reset(frameBuffer);
- //get number of tuples in frame
- numberOfTuplesInCurrentFrame = tupleAccessor.getTupleCount();
- //set tuple index to first tuple
- currentTupleIdx = 0;
- //set new frame to false
- newFrame = false;
- pendingValue = false;
- }
-
- //check and see if there is a pending value
- //Double check this
- int numBytes = 0;
- if (pendingValue) {
- //last value didn't fit into buffer
- int sizeOfNextTuple = getTupleSize(value) + 1;
- if(sizeOfNextTuple > len)
- {
- return 0;
- }
- copyCurrentTuple(buffer, offset + numBytes);
- buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
- numBytes += sizeOfNextTuple;
- //set pending to false
- pendingValue = false;
- //move to next tuple
- currentTupleIdx++;
- }
-
- //No pending value or done with pending value
- //check if there are more tuples in the frame
- while(currentTupleIdx < numberOfTuplesInCurrentFrame)
- {
- //get 3 things from the current tuple in the frame(File name, byte location and row number)
- //get the fileName
- bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 0));
- newFileName = ((AString) inRecDesc.getFields()[0].deserialize(dis)).getStringValue();
- //check if it is a new file
- if(!lastFileName.equals(newFileName))//stringBuilder.toString()))
- {
- //new file
- lastFileName = newFileName;
- //close old file
- if(reader != null)
- {
- reader.close();
- }
- //open new file
- reader = new Reader(fs, new Path(lastFileName), conf);
- //read and save byte location
- bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
- lastByteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
- //seek
- reader.seek(lastByteLocation);
- //read and save rowNumber
- bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 2));
- lastRowNumber = ((AInt32)(inRecDesc.getFields()[2].deserialize(dis))).getIntegerValue();
- //loop until row
- for(int i=0; i < lastRowNumber; i++)
- {
- //this loop perform a single I/O and move to the next record in the block which is already in memory
- //if no more records in the current block, it perform another I/O and get the next block
- //<this should never happen here>
- reader.next(key);
- }
- //read record
- reader.getCurrentRow(value);
- //copy it to the buffer if there is enough space
- int sizeOfNextTuple = getTupleSize(value) + 1;
- if(sizeOfNextTuple + numBytes > len)
- {
- //mark waiting value
- pendingValue = true;
- return numBytes;
- }
- copyCurrentTuple(buffer, offset + numBytes);
- buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
- numBytes += sizeOfNextTuple;
- }
- else
- {
- //same file
- //get the byte location
- bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
- newByteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
-
- //check if same block
- if(lastByteLocation != newByteLocation)
- {
- //new block
- lastByteLocation = newByteLocation;
- //seek
- reader.seek(lastByteLocation);
- //read and save rowNumber
- bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 2));
- lastRowNumber = ((AInt32)(inRecDesc.getFields()[2].deserialize(dis))).getIntegerValue();
- //loop until row
- for(int i=0; i < lastRowNumber; i++)
- {
- reader.next(key);
- }
- //read record
- reader.getCurrentRow(value);
- //copy it to the buffer if there is enough space
- int sizeOfNextTuple = getTupleSize(value) + 1;
- if(sizeOfNextTuple + numBytes > len)
- {
- //mark waiting value
- pendingValue = true;
- return numBytes;
- }
- copyCurrentTuple(buffer, offset + numBytes);
- buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
- numBytes += sizeOfNextTuple;
- }
- else
- {
- //same block
- //get the row number
- bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 2));
- newRowNumber = ((AInt32)(inRecDesc.getFields()[2].deserialize(dis))).getIntegerValue();
-
- //calculate row difference
- rowDifference = newRowNumber - lastRowNumber;
-
- //update last row number
- lastRowNumber = newRowNumber;
-
- //move to the new row
- for(int i=0; i < rowDifference; i++)
- {
- reader.next(key);
- }
- //read record
- reader.getCurrentRow(value);
-
- //copy it to the buffer if there is enough space
- int sizeOfNextTuple = getTupleSize(value) + 1;
- if(sizeOfNextTuple + numBytes > len)
- {
- //mark waiting value
- pendingValue = true;
- return numBytes;
- }
- copyCurrentTuple(buffer, offset + numBytes);
- buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
- numBytes += sizeOfNextTuple;
- }
- }
- //move to next tuple
- currentTupleIdx++;
- }
- //no more tuples in frame
- return (numBytes == 0) ? -1 : numBytes;
- }
-
- private void copyCurrentTuple(byte[] buffer, int offset) throws IOException {
- int rcOffset = 0;
- for(int i=0; i< value.size(); i++)
- {
- System.arraycopy(value.get(i).getData(), value.get(i).getStart(), buffer, offset + rcOffset, value.get(i).getLength());
- rcOffset += value.get(i).getLength() + 1;
- buffer[rcOffset - 1] = delimiter;
- }
- }
-
- private int getTupleSize(BytesRefArrayWritable value2) {
- int size=0;
- //loop over rc column and add lengths
- for(int i=0; i< value.size(); i++)
- {
- size += value.get(i).getLength();
- }
- //add delimeters bytes sizes
- size += value.size() -1;
- return size;
- }
-
- @Override
- public int read() throws IOException {
- throw new NotImplementedException("Use read(byte[], int, int");
- }
- };
- }
- else if (inputFormat.equals(HDFSAdapterFactory.INPUT_FORMAT_TEXT))
- {
- return new InputStream() {
- private FSDataInputStream reader;
- private String lastFileName = "";
- private String newFileName;
- private int EOL = "\n".getBytes()[0];
- private int currentTupleIdx;
- private int numberOfTuplesInCurrentFrame;
- private long byteLocation;
- private IFrameTupleAccessor tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(),inRecDesc);
- private String value;
- private String pendingValue = null;
- private ByteBufferInputStream bbis = new ByteBufferInputStream();
- private DataInputStream dis = new DataInputStream(bbis);
-
- @Override
- public int read(byte[] buffer, int offset, int len) throws IOException {
- if(newFrame)
- {
- //first time called with this frame
- //reset frame buffer
- tupleAccessor.reset(frameBuffer);
- //get number of tuples in frame
- numberOfTuplesInCurrentFrame = tupleAccessor.getTupleCount();
- //set tuple index to first tuple
- currentTupleIdx = 0;
- //set new frame to false
- newFrame = false;
- }
-
- //check and see if there is a pending value
- int numBytes = 0;
- if (pendingValue != null) {
- //last value didn't fit into buffer
- int sizeOfNextTuple = pendingValue.length() + 1;
- if(sizeOfNextTuple > len)
- {
- return 0;
- }
- //there is enough space
- System.arraycopy(pendingValue.getBytes(), 0, buffer, offset + numBytes, pendingValue.length());
- buffer[offset + numBytes + pendingValue.length()] = (byte) EOL;
- numBytes += sizeOfNextTuple;
- //set pending to false
- pendingValue = null;
- //move to next tuple
- currentTupleIdx++;
- }
-
- //No pending value or done with pending value
- //check if there are more tuples in the frame
- while(currentTupleIdx < numberOfTuplesInCurrentFrame)
- {
- //get the fileName
- bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 0));
- newFileName = ((AString) inRecDesc.getFields()[0].deserialize(dis)).getStringValue();
- //check if it is a new file
- if(!lastFileName.equals(newFileName))
- {
- //new file
- lastFileName = newFileName;
- //close old file
- if(reader != null)
- {
- reader.close();
- }
- //open new file
- reader = fs.open(new Path(lastFileName));
- //read byte location
- bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
- byteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
- //seek
- reader.seek(byteLocation);
- //read record
- value = reader.readLine();
- //copy it to the buffer if there is enough space
- int sizeOfNextTuple = value.length() + 1;
- if(sizeOfNextTuple + numBytes > len)
- {
- //mark waiting value
- pendingValue = value;
- return numBytes;
- }
- System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.length());
- buffer[offset + numBytes + value.length()] = (byte) EOL;
- numBytes += sizeOfNextTuple;
- }
- else
- {
- //same file, just seek and read
- //read byte location
- bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
- byteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
- //seek
- reader.seek(byteLocation);
- //read record
- value = reader.readLine();
- //copy it to the buffer if there is enough space
- int sizeOfNextTuple = value.length() + 1;
- if(sizeOfNextTuple + numBytes > len)
- {
- //mark waiting value
- pendingValue = value;
- return numBytes;
- }
- System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.length());
- buffer[offset + numBytes + value.length()] = (byte) EOL;
- numBytes += sizeOfNextTuple;
- }
- currentTupleIdx++;
- }
- return (numBytes == 0) ? -1 : numBytes;
- }
-
- @Override
- public int read() throws IOException {
- throw new NotImplementedException("Use read(byte[], int, int");
- }
-
- @Override
- public void close(){
- try {
- if (reader != null)
- {
- reader.close();
- }
- super.close();
- } catch (IOException e) {
- e.printStackTrace();
- }
- }
-
- };
- }
- else if (inputFormat.equals(HDFSAdapterFactory.INPUT_FORMAT_SEQUENCE))
- {
- return new InputStream() {
- private SequenceFile.Reader reader;
- private Writable key;
- private Text value;
- private String lastFileName = "";
- private String newFileName;
- private long byteLocation;
- private int EOL = "\n".getBytes()[0];
- private int currentTupleIdx;
- private int numberOfTuplesInCurrentFrame;
- private IFrameTupleAccessor tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(),inRecDesc);
- private Text pendingValue = null;
- private ByteBufferInputStream bbis = new ByteBufferInputStream();
- private DataInputStream dis = new DataInputStream(bbis);
-
- @Override
- public int read(byte[] buffer, int offset, int len) throws IOException {
-
- if(newFrame)
- {
- //first time called with this frame
- //reset frame buffer
- tupleAccessor.reset(frameBuffer);
- //get number of tuples in frame
- numberOfTuplesInCurrentFrame = tupleAccessor.getTupleCount();
- //set tuple index to first tuple
- currentTupleIdx = 0;
- //set new frame to false
- newFrame = false;
- }
-
- //check and see if there is a pending value
- //Double check this
- int numBytes = 0;
- if (pendingValue != null) {
- //last value didn't fit into buffer
- int sizeOfNextTuple = pendingValue.getLength() + 1;
- if(sizeOfNextTuple > len)
- {
- return 0;
- }
- //there is enough space
- System.arraycopy(pendingValue.getBytes(), 0, buffer, offset + numBytes, pendingValue.getLength());
- buffer[offset + numBytes + pendingValue.getLength()] = (byte) EOL;
- numBytes += sizeOfNextTuple;
- //set pending to false
- pendingValue = null;
- //move to next tuple
- currentTupleIdx++;
- }
-
- //No pending value or done with pending value
- //check if there are more tuples in the frame
- while(currentTupleIdx < numberOfTuplesInCurrentFrame)
- {
- //get the fileName]
- bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 0));
- newFileName = ((AString) inRecDesc.getFields()[0].deserialize(dis)).getStringValue();
- //check if it is a new file
- if(!lastFileName.equals(newFileName))
- {
- //new file
- lastFileName = newFileName;
- //close old file
- if(reader != null)
- {
- reader.close();
- }
- //open new file
- reader = new SequenceFile.Reader(fs,new Path(lastFileName),conf);
- //read byte location
- bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
- byteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
- //seek
- reader.seek(byteLocation);
- //read record
- reader.next(key, value);
- //copy it to the buffer if there is enough space
- int sizeOfNextTuple = value.getLength() + 1;
- if(sizeOfNextTuple + numBytes > len)
- {
- //mark waiting value
- pendingValue = value;
- return numBytes;
- }
- System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.getLength());
- buffer[offset + numBytes + value.getLength()] = (byte) EOL;
- numBytes += sizeOfNextTuple;
- }
- else
- {
- //same file, just seek and read
- //read byte location
- bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
- byteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
- //seek
- reader.seek(byteLocation);
- //read record
- reader.next(key, value);
- //copy it to the buffer if there is enough space
- int sizeOfNextTuple = value.getLength() + 1;
- if(sizeOfNextTuple + numBytes > len)
- {
- //mark waiting value
- pendingValue = value;
- return numBytes;
- }
- System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.getLength());
- buffer[offset + numBytes + value.getLength()] = (byte) EOL;
- numBytes += sizeOfNextTuple;
- }
- currentTupleIdx++;
- }
- return (numBytes == 0) ? -1 : numBytes;
- }
-
- @Override
- public int read() throws IOException {
- throw new NotImplementedException("Use read(byte[], int, int");
- }
-
- @Override
- public void close(){
- try {
- if (reader != null)
- {
- reader.close();
- }
- super.close();
- } catch (IOException e) {
- e.printStackTrace();
- }
- }
- };
- }
- //unknow format
- throw new IOException("Unknown input format");
- }
- else
- {
- //optimized
- //different input stream implementation based on the input format
- if(inputFormat.equals(HDFSAdapterFactory.INPUT_FORMAT_RC))
- {
- return new InputStream() {
- private RCFile.Reader reader;
- private int rowDifference;
- private int lastFileNumber = -1;
- private int newFileNumber = 0;
- private long lastByteLocation = 0;
- private long newByteLocation = 0;
- private int lastRowNumber = 0;
- private int newRowNumber = 0;
- private LongWritable key;
- private BytesRefArrayWritable value;
- private int EOL = "\n".getBytes()[0];
- private byte delimiter = 0x01;
- private boolean pendingValue = false;
- private int currentTupleIdx;
- private int numberOfTuplesInCurrentFrame;
- private IFrameTupleAccessor tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(),inRecDesc);
- private ByteBufferInputStream bbis = new ByteBufferInputStream();
- private DataInputStream dis = new DataInputStream(bbis);
-
- @Override
- public void close()
- {
- if (reader != null)
- {
- reader.close();
- }
- try {
- super.close();
- } catch (IOException e) {
- e.printStackTrace();
- }
- }
-
- @Override
- public int read(byte[] buffer, int offset, int len) throws IOException {
- if(newFrame)
- {
- //first time called with this frame
- //reset frame buffer
- tupleAccessor.reset(frameBuffer);
- //get number of tuples in frame
- numberOfTuplesInCurrentFrame = tupleAccessor.getTupleCount();
- //set tuple index to first tuple
- currentTupleIdx = 0;
- //set new frame to false
- newFrame = false;
- pendingValue = false;
- }
-
- //check and see if there is a pending value
- //Double check this
- int numBytes = 0;
- if (pendingValue) {
- //last value didn't fit into buffer
- int sizeOfNextTuple = getTupleSize(value) + 1;
- if(sizeOfNextTuple > len)
- {
- return 0;
- }
- copyCurrentTuple(buffer, offset + numBytes);
- buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
- numBytes += sizeOfNextTuple;
- //set pending to false
- pendingValue = false;
- //move to next tuple
- currentTupleIdx++;
- }
-
- //No pending value or done with pending value
- //check if there are more tuples in the frame
- while(currentTupleIdx < numberOfTuplesInCurrentFrame)
- {
- //get 3 things from the current tuple in the frame(File name, byte location and row number)
- //get the fileName
- bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 0));
- newFileNumber = ((AInt32) inRecDesc.getFields()[0].deserialize(dis)).getIntegerValue();
- //check if it is a new file
- if(lastFileNumber != newFileNumber)
- {
- //new file
- lastFileNumber = newFileNumber;
- //close old file
- if(reader != null)
- {
- reader.close();
- }
- //open new file
- reader = new Reader(fs, new Path(files.get(newFileNumber)), conf);
- //read and save byte location
- bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
- lastByteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
- //seek
- reader.seek(lastByteLocation);
- //read and save rowNumber
- bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 2));
- lastRowNumber = ((AInt32)(inRecDesc.getFields()[2].deserialize(dis))).getIntegerValue();
- //loop until row
- for(int i=0; i < lastRowNumber; i++)
- {
- //this loop perform a single I/O and move to the next record in the block which is already in memory
- //if no more records in the current block, it perform another I/O and get the next block
- //<this should never happen here>
- reader.next(key);
- }
- //read record
- reader.getCurrentRow(value);
- //copy it to the buffer if there is enough space
- int sizeOfNextTuple = getTupleSize(value) + 1;
- if(sizeOfNextTuple + numBytes > len)
- {
- //mark waiting value
- pendingValue = true;
- return numBytes;
- }
- copyCurrentTuple(buffer, offset + numBytes);
- buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
- numBytes += sizeOfNextTuple;
- }
- else
- {
- //same file
- //get the byte location
- bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
- newByteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
-
- //check if same block
- if(lastByteLocation != newByteLocation)
- {
- //new block
- lastByteLocation = newByteLocation;
- //seek
- reader.seek(lastByteLocation);
- //read and save rowNumber
- bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 2));
- lastRowNumber = ((AInt32)(inRecDesc.getFields()[2].deserialize(dis))).getIntegerValue();
- //loop until row
- for(int i=0; i < lastRowNumber; i++)
- {
- reader.next(key);
- }
- //read record
- reader.getCurrentRow(value);
- //copy it to the buffer if there is enough space
- int sizeOfNextTuple = getTupleSize(value) + 1;
- if(sizeOfNextTuple + numBytes > len)
- {
- //mark waiting value
- pendingValue = true;
- return numBytes;
- }
- copyCurrentTuple(buffer, offset + numBytes);
- buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
- numBytes += sizeOfNextTuple;
- }
- else
- {
- //same block
- //get the row number
- bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 2));
- newRowNumber = ((AInt32)(inRecDesc.getFields()[2].deserialize(dis))).getIntegerValue();
-
- //calculate row difference
- rowDifference = newRowNumber - lastRowNumber;
-
- //update last row number
- lastRowNumber = newRowNumber;
-
- //move to the new row
- for(int i=0; i < rowDifference; i++)
- {
- reader.next(key);
- }
- //read record
- reader.getCurrentRow(value);
-
- //copy it to the buffer if there is enough space
- int sizeOfNextTuple = getTupleSize(value) + 1;
- if(sizeOfNextTuple + numBytes > len)
- {
- //mark waiting value
- pendingValue = true;
- return numBytes;
- }
- copyCurrentTuple(buffer, offset + numBytes);
- buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
- numBytes += sizeOfNextTuple;
- }
- }
- //move to next tuple
- currentTupleIdx++;
- }
- //no more tuples in frame
- return (numBytes == 0) ? -1 : numBytes;
- }
-
- private void copyCurrentTuple(byte[] buffer, int offset) throws IOException {
- int rcOffset = 0;
- for(int i=0; i< value.size(); i++)
- {
- System.arraycopy(value.get(i).getData(), value.get(i).getStart(), buffer, offset + rcOffset, value.get(i).getLength());
- rcOffset += value.get(i).getLength() + 1;
- buffer[rcOffset - 1] = delimiter;
- }
- }
-
- private int getTupleSize(BytesRefArrayWritable value2) {
- int size=0;
- //loop over rc column and add lengths
- for(int i=0; i< value.size(); i++)
- {
- size += value.get(i).getLength();
- }
- //add delimeters bytes sizes
- size += value.size() -1;
- return size;
- }
-
- @Override
- public int read() throws IOException {
- throw new NotImplementedException("Use read(byte[], int, int");
- }
- };
- }
- else if (inputFormat.equals(HDFSAdapterFactory.INPUT_FORMAT_TEXT))
- {
- return new InputStream() {
- private FSDataInputStream reader;
- private int lastFileNumber = -1;
- private int newFileNumber = 0;
- private int EOL = "\n".getBytes()[0];
- private int currentTupleIdx;
- private int numberOfTuplesInCurrentFrame;
- private long byteLocation;
- private IFrameTupleAccessor tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(),inRecDesc);
- private String value;
- private String pendingValue = null;
- private ByteBufferInputStream bbis = new ByteBufferInputStream();
- private DataInputStream dis = new DataInputStream(bbis);
-
- @Override
- public int read(byte[] buffer, int offset, int len) throws IOException {
- if(newFrame)
- {
- //first time called with this frame
- //reset frame buffer
- tupleAccessor.reset(frameBuffer);
- //get number of tuples in frame
- numberOfTuplesInCurrentFrame = tupleAccessor.getTupleCount();
- //set tuple index to first tuple
- currentTupleIdx = 0;
- //set new frame to false
- newFrame = false;
- }
-
- //check and see if there is a pending value
- int numBytes = 0;
- if (pendingValue != null) {
- //last value didn't fit into buffer
- int sizeOfNextTuple = pendingValue.length() + 1;
- if(sizeOfNextTuple > len)
- {
- return 0;
- }
- //there is enough space
- System.arraycopy(pendingValue.getBytes(), 0, buffer, offset + numBytes, pendingValue.length());
- buffer[offset + numBytes + pendingValue.length()] = (byte) EOL;
- numBytes += sizeOfNextTuple;
- //set pending to false
- pendingValue = null;
- //move to next tuple
- currentTupleIdx++;
- }
-
- //No pending value or done with pending value
- //check if there are more tuples in the frame
- while(currentTupleIdx < numberOfTuplesInCurrentFrame)
- {
- //get the file number
- bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 0));
- newFileNumber = ((AInt32) inRecDesc.getFields()[0].deserialize(dis)).getIntegerValue();
- //check if it is a new file
- if(lastFileNumber != newFileNumber)
- {
- //new file
- lastFileNumber = newFileNumber;
- //close old file
- if(reader != null)
- {
- reader.close();
- }
-
- //open new file
- reader = fs.open(new Path(files.get(newFileNumber)));
- //read byte location
- bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
- byteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
- //seek
- reader.seek(byteLocation);
- //read record
- value = reader.readLine();
- //copy it to the buffer if there is enough space
- int sizeOfNextTuple = value.length() + 1;
- if(sizeOfNextTuple + numBytes > len)
- {
- //mark waiting value
- pendingValue = value;
- return numBytes;
- }
- System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.length());
- buffer[offset + numBytes + value.length()] = (byte) EOL;
- numBytes += sizeOfNextTuple;
- }
- else
- {
- //same file, just seek and read
- //read byte location
- bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
- byteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
- //seek
- reader.seek(byteLocation);
- //read record
- value = reader.readLine();
- //copy it to the buffer if there is enough space
- int sizeOfNextTuple = value.length() + 1;
- if(sizeOfNextTuple + numBytes > len)
- {
- //mark waiting value
- pendingValue = value;
- return numBytes;
- }
- System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.length());
- buffer[offset + numBytes + value.length()] = (byte) EOL;
- numBytes += sizeOfNextTuple;
- }
- currentTupleIdx++;
- }
- return (numBytes == 0) ? -1 : numBytes;
- }
-
- @Override
- public int read() throws IOException {
- throw new NotImplementedException("Use read(byte[], int, int");
- }
-
- @Override
- public void close(){
- try {
- if (reader != null)
- {
- reader.close();
- }
- super.close();
- } catch (IOException e) {
- e.printStackTrace();
- }
- }
-
- };
- }
- else if (inputFormat.equals(HDFSAdapterFactory.INPUT_FORMAT_SEQUENCE))
- {
- return new InputStream() {
- private SequenceFile.Reader reader;
- private Writable key;
- private Text value;
- private int lastFileNumber = -1;
- private int newFileNumber = 0;
- private long byteLocation;
- private int EOL = "\n".getBytes()[0];
- private int currentTupleIdx;
- private int numberOfTuplesInCurrentFrame;
- private IFrameTupleAccessor tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(),inRecDesc);
- private Text pendingValue = null;
- private ByteBufferInputStream bbis = new ByteBufferInputStream();
- private DataInputStream dis = new DataInputStream(bbis);
-
- @Override
- public int read(byte[] buffer, int offset, int len) throws IOException {
-
- if(newFrame)
- {
- //first time called with this frame
- //reset frame buffer
- tupleAccessor.reset(frameBuffer);
- //get number of tuples in frame
- numberOfTuplesInCurrentFrame = tupleAccessor.getTupleCount();
- //set tuple index to first tuple
- currentTupleIdx = 0;
- //set new frame to false
- newFrame = false;
- }
-
- //check and see if there is a pending value
- //Double check this
- int numBytes = 0;
- if (pendingValue != null) {
- //last value didn't fit into buffer
- int sizeOfNextTuple = pendingValue.getLength() + 1;
- if(sizeOfNextTuple > len)
- {
- return 0;
- }
- //there is enough space
- System.arraycopy(pendingValue.getBytes(), 0, buffer, offset + numBytes, pendingValue.getLength());
- buffer[offset + numBytes + pendingValue.getLength()] = (byte) EOL;
- numBytes += sizeOfNextTuple;
- //set pending to false
- pendingValue = null;
- //move to next tuple
- currentTupleIdx++;
- }
-
- //No pending value or done with pending value
- //check if there are more tuples in the frame
- while(currentTupleIdx < numberOfTuplesInCurrentFrame)
- {
- //get the fileName]
- bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 0));
- newFileNumber = ((AInt32) inRecDesc.getFields()[0].deserialize(dis)).getIntegerValue();
- //check if it is a new file
- if(lastFileNumber != newFileNumber)
- {
- //new file
- lastFileNumber = newFileNumber;
- //close old file
- if(reader != null)
- {
- reader.close();
- }
- //open new file
- reader = new SequenceFile.Reader(fs,new Path(files.get(newFileNumber)),conf);
- //read byte location
- bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
- byteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
- //seek
- reader.seek(byteLocation);
- //read record
- reader.next(key, value);
- //copy it to the buffer if there is enough space
- int sizeOfNextTuple = value.getLength() + 1;
- if(sizeOfNextTuple + numBytes > len)
- {
- //mark waiting value
- pendingValue = value;
- return numBytes;
- }
- System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.getLength());
- buffer[offset + numBytes + value.getLength()] = (byte) EOL;
- numBytes += sizeOfNextTuple;
- }
- else
- {
- //same file, just seek and read
- //read byte location
- bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
- byteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
- //seek
- reader.seek(byteLocation);
- //read record
- reader.next(key, value);
- //copy it to the buffer if there is enough space
- int sizeOfNextTuple = value.getLength() + 1;
- if(sizeOfNextTuple + numBytes > len)
- {
- //mark waiting value
- pendingValue = value;
- return numBytes;
- }
- System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.getLength());
- buffer[offset + numBytes + value.getLength()] = (byte) EOL;
- numBytes += sizeOfNextTuple;
- }
- currentTupleIdx++;
- }
- return (numBytes == 0) ? -1 : numBytes;
- }
-
- @Override
- public int read() throws IOException {
- throw new NotImplementedException("Use read(byte[], int, int");
- }
-
- @Override
- public void close(){
- try {
- if (reader != null)
- {
- reader.close();
- }
- super.close();
- } catch (IOException e) {
- e.printStackTrace();
- }
- }
- };
- }
- //unknow format
- throw new IOException("Unknown input format");
- }
- }
-
- @Override
- public AlgebricksPartitionConstraint getPartitionConstraint()
- throws Exception {
- return partitionConstraint;
- }
-}
\ No newline at end of file
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 2b355ff..f8b381b 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
@@ -18,7 +18,6 @@
import java.io.InputStream;
import java.util.Map;
-import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.Counters.Counter;
import org.apache.hadoop.mapred.InputSplit;
@@ -27,8 +26,6 @@
import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.mapred.SequenceFileInputFormat;
import org.apache.hadoop.mapred.TextInputFormat;
-import org.apache.hadoop.hive.ql.io.RCFileInputFormat;
-import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable;
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
@@ -37,348 +34,202 @@
/**
* Provides functionality for fetching external data stored in an HDFS instance.
- * Note: reader are never closed in adapters. should we make sure they are closed before returning or switching to a different reader?
*/
@SuppressWarnings({ "deprecation", "rawtypes" })
public class HDFSAdapter extends FileSystemBasedAdapter {
- private static final long serialVersionUID = 1L;
+ private static final long serialVersionUID = 1L;
- private transient String[] readSchedule;
- private transient boolean executed[];
- private transient InputSplit[] inputSplits;
- private transient JobConf conf;
- private transient AlgebricksPartitionConstraint clusterLocations;
+ private transient String[] readSchedule;
+ private transient boolean executed[];
+ private transient InputSplit[] inputSplits;
+ private transient JobConf conf;
+ private transient AlgebricksPartitionConstraint clusterLocations;
- private transient String nodeName;
+ private transient String nodeName;
- public HDFSAdapter(IAType atype, String[] readSchedule, boolean[] executed, InputSplit[] inputSplits, JobConf conf,
- AlgebricksPartitionConstraint clusterLocations) {
- super(atype);
- this.readSchedule = readSchedule;
- this.executed = executed;
- this.inputSplits = inputSplits;
- this.conf = conf;
- this.clusterLocations = clusterLocations;
- }
+ public HDFSAdapter(IAType atype, String[] readSchedule, boolean[] executed, InputSplit[] inputSplits, JobConf conf,
+ AlgebricksPartitionConstraint clusterLocations) {
+ super(atype);
+ this.readSchedule = readSchedule;
+ this.executed = executed;
+ this.inputSplits = inputSplits;
+ this.conf = conf;
+ this.clusterLocations = clusterLocations;
+ }
- @Override
- public void configure(Map<String, Object> arguments) throws Exception {
- this.configuration = arguments;
- configureFormat();
- }
+ @Override
+ public void configure(Map<String, Object> arguments) throws Exception {
+ this.configuration = arguments;
+ configureFormat();
+ }
- public AdapterType getAdapterType() {
- return AdapterType.READ_WRITE;
- }
+ public AdapterType getAdapterType() {
+ return AdapterType.READ_WRITE;
+ }
- @Override
- public void initialize(IHyracksTaskContext ctx) throws Exception {
- this.ctx = ctx;
- this.nodeName = ctx.getJobletContext().getApplicationContext().getNodeId();
- }
+ @Override
+ public void initialize(IHyracksTaskContext ctx) throws Exception {
+ this.ctx = ctx;
+ this.nodeName = ctx.getJobletContext().getApplicationContext().getNodeId();
+ }
- private Reporter getReporter() {
- Reporter reporter = new Reporter() {
+ private Reporter getReporter() {
+ Reporter reporter = new Reporter() {
- @Override
- public Counter getCounter(Enum<?> arg0) {
- return null;
- }
+ @Override
+ public Counter getCounter(Enum<?> arg0) {
+ return null;
+ }
- @Override
- public Counter getCounter(String arg0, String arg1) {
- return null;
- }
+ @Override
+ public Counter getCounter(String arg0, String arg1) {
+ return null;
+ }
- @Override
- public InputSplit getInputSplit() throws UnsupportedOperationException {
- return null;
- }
+ @Override
+ public InputSplit getInputSplit() throws UnsupportedOperationException {
+ return null;
+ }
- @Override
- public void incrCounter(Enum<?> arg0, long arg1) {
- }
+ @Override
+ public void incrCounter(Enum<?> arg0, long arg1) {
+ }
- @Override
- public void incrCounter(String arg0, String arg1, long arg2) {
- }
+ @Override
+ public void incrCounter(String arg0, String arg1, long arg2) {
+ }
- @Override
- public void setStatus(String arg0) {
- }
+ @Override
+ public void setStatus(String arg0) {
+ }
- @Override
- public void progress() {
- }
- };
+ @Override
+ public void progress() {
+ }
+ };
- return reporter;
- }
+ return reporter;
+ }
- @Override
- public InputStream getInputStream(int partition) throws IOException {
+ @Override
+ public InputStream getInputStream(int partition) throws IOException {
- if(conf.getInputFormat() instanceof RCFileInputFormat)
- {
- //if hdfs input format is rc-input-format, we return a different InputStream
- return new InputStream() {
+ return new InputStream() {
- private RecordReader<LongWritable, BytesRefArrayWritable> reader;
- private LongWritable key;
- private BytesRefArrayWritable value;
- private boolean hasMore = false;
- private int EOL = "\n".getBytes()[0];
- private byte delimiter = 0x01;
- private boolean pendingValue = false;
- private int currentSplitIndex = 0;
+ 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;
- }
- }
+ @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 = reader.createValue();
- return true;
- }
- }
- return false;
- }
+ /**
+ * 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;
- }
- }
+ @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) {
- //last value didn't fit into buffer
- int sizeOfNextTuple = getTupleSize(value) + 1;
- if(sizeOfNextTuple > len)
- {
- return 0;
- }
- copyCurrentTuple(buffer, offset + numBytes);
- buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
- numBytes += sizeOfNextTuple;
- //set pending to false
- pendingValue = false;
- }
+ 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 = getTupleSize(value) + 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 = true;
- break;
- } else {
- //copy
- copyCurrentTuple(buffer, offset + numBytes);
- buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
- numBytes += sizeOfNextTuple;
- }
- }
- return numBytes;
- }
+ 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;
+ }
- private void copyCurrentTuple(byte[] buffer, int offset) throws IOException {
- int rcOffset = 0;
- for(int i=0; i< value.size(); i++)
- {
- System.arraycopy(value.get(i).getData(), value.get(i).getStart(), buffer, offset + rcOffset, value.get(i).getLength());
- rcOffset += value.get(i).getLength() + 1;
- buffer[rcOffset - 1] = delimiter;
- }
- }
+ @Override
+ public int read() throws IOException {
+ throw new NotImplementedException("Use read(byte[], int, int");
+ }
- private int getTupleSize(BytesRefArrayWritable value2) {
- int size=0;
- //loop over rc column and add lengths
- for(int i=0; i< value.size(); i++)
- {
- size += value.get(i).getLength();
- }
- //add delimeters bytes sizes
- size += value.size() -1;
- return size;
- }
+ 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;
+ }
+ }
- @Override
- public int read() throws IOException {
- throw new NotImplementedException("Use read(byte[], int, int");
- }
+ };
- private RecordReader getRecordReader(int slitIndex) throws IOException {
- RCFileInputFormat format = (RCFileInputFormat) conf.getInputFormat();
- RecordReader reader = format.getRecordReader(
- (org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
- return reader;
- }
+ }
- };
- }
- else
- {
- 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;
- }
-
- @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");
- }
-
- 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;
- }
- }
-
- };
- }
-
- }
-
- @Override
- public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
- return clusterLocations;
- }
+ @Override
+ public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+ return clusterLocations;
+ }
}
\ 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
deleted file mode 100644
index 59b39c5..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSIndexingAdapter.java
+++ /dev/null
@@ -1,1208 +0,0 @@
-package edu.uci.ics.asterix.external.dataset.adapter;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.Map;
-
-import org.apache.hadoop.hive.ql.io.RCFileInputFormat;
-import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.FileSplit;
-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.asterix.om.types.IAType;
-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.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
-
-
-/**
- * Provides functionality for reading external files and attach RID info to them before they are sent to the parser
- *
- * Room for optimization: get numbers of indexing fields (or fields names for adm) and do a quick filtering before sending to parser
- *
- */
-@SuppressWarnings({ "deprecation", "rawtypes" })
-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 transient AlgebricksPartitionConstraint clusterLocations;
- private final Map<String,Integer> files;
- private transient String nodeName;
- public static final byte[] fileNameFieldNameWithRecOpeningBraces = "{\"_file-name\":\"".getBytes();
- public static final byte[] fileNameFieldClosingQuotation = "\"".getBytes();
- public static final byte[] fileNumberFieldNameWithRecOpeningBraces = "{\"_file-number\":".getBytes();
- public static final byte[] bytelocationFieldName = ",\"_byte-location\":".getBytes();
- public static final byte[] bytelocationValueEnd = "i64,".getBytes();
-
- public HDFSIndexingAdapter(IAType atype, String[] readSchedule, boolean[] executed, InputSplit[] inputSplits, JobConf conf,
- AlgebricksPartitionConstraint clusterLocations, Map<String,Integer> files) {
- super(atype);
- this.readSchedule = readSchedule;
- this.executed = executed;
- this.inputSplits = inputSplits;
- this.conf = conf;
- this.clusterLocations = clusterLocations;
- this.files = files;
- }
-
- @Override
- public void configure(Map<String, Object> arguments) throws Exception {
- this.configuration = arguments;
- configureFormat();
- }
-
- public AdapterType getAdapterType() {
- return AdapterType.READ;
- }
-
- @Override
- public void initialize(IHyracksTaskContext ctx) throws Exception {
- this.ctx = ctx;
- this.nodeName = ctx.getJobletContext().getApplicationContext().getNodeId();
- }
-
- 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;
- }
-
- @Override
- public InputStream getInputStream(int partition) throws IOException {
- if(files == null)
- {
- if(conf.getInputFormat() instanceof RCFileInputFormat)
- {
- //indexing rc input format
- return new InputStream() {
-
- private RecordReader<LongWritable, BytesRefArrayWritable> reader;
- private LongWritable key;
- private BytesRefArrayWritable value;
- private boolean hasMore = false;
- private int EOL = "\n".getBytes()[0];
- private byte delimiter = 0x01;
- private boolean pendingValue = false;
- private int currentSplitIndex = 0;
- private byte[] fileName;
- private byte[] byteLocation;
- private byte[] rowNumberBytes;
- private long blockByteLocation;
- private long NextblockByteLocation;
- private int rowNumber;
-
- @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 = reader.createValue();
- fileName = ((FileSplit)(inputSplits[currentSplitIndex])).getPath().toUri().getPath().getBytes();
- blockByteLocation = reader.getPos();
- pendingValue = reader.next(key, value);
- NextblockByteLocation = reader.getPos();
- rowNumber = 1;
- byteLocation = String.valueOf(blockByteLocation).getBytes("UTF-8");
- rowNumberBytes = String.valueOf(rowNumber).getBytes("UTF-8");
- 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) {
- //last value didn't fit into buffer
- // 1 for EOL
- int sizeOfNextTuple = getTupleSize(value) + 1;
- if (numBytes + sizeOfNextTuple + rowNumberBytes.length + byteLocation.length + fileName.length + 3 > len) {
- return 0;
- }
-
- //copy filename
- System.arraycopy(fileName, 0, buffer, offset + numBytes, fileName.length);
- buffer[offset + numBytes + fileName.length] = delimiter;
- numBytes += fileName.length + 1;
-
- //copy byte location
- System.arraycopy(byteLocation, 0, buffer, offset + numBytes, byteLocation.length);
- buffer[offset + numBytes + byteLocation.length] = delimiter;
- numBytes += byteLocation.length + 1;
-
- //copy row number
- System.arraycopy(rowNumberBytes, 0, buffer, offset + numBytes, rowNumberBytes.length);
- buffer[offset + numBytes + rowNumberBytes.length] = delimiter;
- numBytes += rowNumberBytes.length + 1;
-
- copyCurrentTuple(buffer, offset + numBytes);
- buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
- numBytes += sizeOfNextTuple;
- //set pending to false
- pendingValue = false;
- }
-
- 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;
- }
-
- //check if moved to next block
- blockByteLocation = reader.getPos();
- if(blockByteLocation != NextblockByteLocation)
- {
- //moved to a new block, reset stuff
- //row number
- rowNumber = 1;
- rowNumberBytes = String.valueOf(rowNumber).getBytes("UTF-8");
-
- //block location
- byteLocation = String.valueOf(NextblockByteLocation).getBytes("UTF-8");
- NextblockByteLocation = blockByteLocation;
- }
- else
- {
- rowNumber += 1;
- rowNumberBytes = String.valueOf(rowNumber).getBytes("UTF-8");
- }
-
- int sizeOfNextTuple = getTupleSize(value) + 1;
- if (numBytes + sizeOfNextTuple + rowNumberBytes.length + byteLocation.length + fileName.length + 3 > 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 = true;
- break;
- } else {
- //copy filename
- System.arraycopy(fileName, 0, buffer, offset + numBytes, fileName.length);
- buffer[offset + numBytes + fileName.length] = delimiter;
- numBytes += fileName.length + 1;
-
- //copy byte location
- System.arraycopy(byteLocation, 0, buffer, offset + numBytes, byteLocation.length);
- buffer[offset + numBytes + byteLocation.length] = delimiter;
- numBytes += byteLocation.length + 1;
-
- //copy row number
- System.arraycopy(rowNumberBytes, 0, buffer, offset + numBytes, rowNumberBytes.length);
- buffer[offset + numBytes + rowNumberBytes.length] = delimiter;
- numBytes += rowNumberBytes.length + 1;
-
- copyCurrentTuple(buffer, offset + numBytes);
- buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
- numBytes += sizeOfNextTuple;
- }
- }
- return numBytes;
- }
-
- private void copyCurrentTuple(byte[] buffer, int offset) throws IOException {
- int rcOffset = 0;
- for(int i=0; i< value.size(); i++)
- {
- System.arraycopy(value.get(i).getData(), value.get(i).getStart(), buffer, offset + rcOffset, value.get(i).getLength());
- rcOffset += value.get(i).getLength() + 1;
- buffer[rcOffset - 1] = delimiter;
- }
- }
-
- private int getTupleSize(BytesRefArrayWritable value2) {
- int size=0;
- //loop over rc column and add lengths
- for(int i=0; i< value.size(); i++)
- {
- size += value.get(i).getLength();
- }
- //add delimeters bytes sizes
- size += value.size() -1;
- return size;
- }
-
- @Override
- public int read() throws IOException {
- throw new NotImplementedException("Use read(byte[], int, int");
- }
-
- private RecordReader getRecordReader(int slitIndex) throws IOException {
- RCFileInputFormat format = (RCFileInputFormat) conf.getInputFormat();
- RecordReader reader = format.getRecordReader(
- (org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
- return reader;
- }
-
- };
- }
- else
- {
- //get content format
- if(configuration.get(KEY_FORMAT).equals(FORMAT_DELIMITED_TEXT))
- {
- //reading data and RIDs for delimited text
- 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;
- private byte[] fileName;
- private byte[] byteLocation;
- private byte delimiter = ((String)configuration.get(KEY_DELIMITER)).getBytes()[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();
- fileName = ((FileSplit)(inputSplits[currentSplitIndex])).getPath().toUri().getPath().getBytes();
- 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 (numBytes + sizeOfNextTuple +byteLocation.length + fileName.length + 2> len)
- {
- return numBytes;
- }
- //copy filename
- System.arraycopy(fileName, 0, buffer, offset + numBytes, fileName.length);
- buffer[offset + numBytes + fileName.length] = delimiter;
- numBytes += fileName.length + 1;
-
- //copy byte location
- System.arraycopy(byteLocation, 0, buffer, offset + numBytes, byteLocation.length);
- buffer[offset + numBytes + byteLocation.length] = delimiter;
- numBytes += byteLocation.length + 1;
-
- //copy actual value
- 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) {
- //get reader position before you actually read
- byteLocation = String.valueOf(reader.getPos()).getBytes();
- hasMore = reader.next(key, value);
- if (!hasMore) {
- while (moveToNext()) {
- //get reader position before you actually read
- byteLocation = String.valueOf(reader.getPos()).getBytes("UTF-8");
- 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 +byteLocation.length + fileName.length + 2> 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 {
- //copy filename
- System.arraycopy(fileName, 0, buffer, offset + numBytes, fileName.length);
- buffer[offset + numBytes + fileName.length] = delimiter;
- numBytes += fileName.length + 1;
-
- //copy byte location
- System.arraycopy(byteLocation, 0, buffer, offset + numBytes, byteLocation.length);
- buffer[offset + numBytes + byteLocation.length] = delimiter;
- numBytes += byteLocation.length + 1;
-
- //Copy actual value
- 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");
- }
-
- 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;
- }
- }
-
- };
- }
- else if((configuration.get(KEY_FORMAT).equals(FORMAT_ADM)))
- {
- //reading data and RIDs for adm formatted data
- 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;
- private byte[] fileName;
- private byte[] byteLocation;
-
- @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();
- fileName = ((FileSplit)(inputSplits[currentSplitIndex])).getPath().toUri().getPath().getBytes();
- 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 firstFieldLocation = value.find("\"");
- int admValueSize = value.getLength();
- if(firstFieldLocation >= 0)
- {
- int sizeOfNextTuple = value.getLength() - firstFieldLocation + 1;
- int sizeOfNextTupleAndRID = fileNameFieldNameWithRecOpeningBraces.length + fileName.length + fileNameFieldClosingQuotation.length + bytelocationFieldName.length + byteLocation.length + bytelocationValueEnd.length + sizeOfNextTuple;
- if (numBytes + sizeOfNextTupleAndRID > len) {
- // still cannot add tuple to current buffer
- // return 0 so parser would double the buffer size.
- return 0;
- } else {
- //copy fileNameFieldNameWithRecOpeningBraces
- System.arraycopy(fileNameFieldNameWithRecOpeningBraces, 0, buffer, offset + numBytes,fileNameFieldNameWithRecOpeningBraces.length);
- numBytes += fileNameFieldNameWithRecOpeningBraces.length;
- //copy fileName
- System.arraycopy(fileName, 0, buffer, offset + numBytes,fileName.length);
- numBytes += fileName.length;
- //copy fileName closing quotation
- System.arraycopy(fileNameFieldClosingQuotation, 0, buffer, offset + numBytes,fileNameFieldClosingQuotation.length);
- numBytes += fileNameFieldClosingQuotation.length;
- //copy bytelocationFieldName
- System.arraycopy(bytelocationFieldName, 0, buffer, offset + numBytes,bytelocationFieldName.length);
- numBytes += bytelocationFieldName.length;
- //copy byte location value
- System.arraycopy(byteLocation, 0, buffer, offset + numBytes,byteLocation.length);
- numBytes += byteLocation.length;
- //copy byte location field end
- System.arraycopy(bytelocationValueEnd, 0, buffer, offset + numBytes,bytelocationValueEnd.length);
- numBytes += bytelocationValueEnd.length;
- //copy the actual adm instance
- System.arraycopy(value.getBytes(), firstFieldLocation, buffer, offset + numBytes,admValueSize - firstFieldLocation);
- buffer[offset + numBytes + admValueSize - firstFieldLocation] = (byte) EOL;
- numBytes += admValueSize - firstFieldLocation +1;
- }
- }
- pendingValue = null;
- }
-
- while (numBytes < len) {
- //get reader position before you actually read
- byteLocation = String.valueOf(reader.getPos()).getBytes("UTF-8");
- hasMore = reader.next(key, value);
- if (!hasMore) {
- while (moveToNext()) {
- //get reader position before you actually read
- byteLocation = String.valueOf(reader.getPos()).getBytes("UTF-8");
- hasMore = reader.next(key, value);
- if (hasMore) {
- //move to the next non-empty split
- break;
- }
- }
- }
- if (!hasMore) {
- return (numBytes == 0) ? -1 : numBytes;
- }
- //get the index of the first field name
- int firstFieldLocation = value.find("\"");
- int admValueSize = value.getLength();
- if(firstFieldLocation >= 0)
- {
- int sizeOfNextTuple = value.getLength() - firstFieldLocation + 1;
- int sizeOfNextTupleAndRID = fileNameFieldNameWithRecOpeningBraces.length + fileName.length + fileNameFieldClosingQuotation.length + bytelocationFieldName.length + byteLocation.length + bytelocationValueEnd.length + sizeOfNextTuple;
- if (numBytes + sizeOfNextTupleAndRID > 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 {
- //copy fileNameFieldNameWithRecOpeningBraces
- System.arraycopy(fileNameFieldNameWithRecOpeningBraces, 0, buffer, offset + numBytes,fileNameFieldNameWithRecOpeningBraces.length);
- numBytes += fileNameFieldNameWithRecOpeningBraces.length;
- //copy fileName
- System.arraycopy(fileName, 0, buffer, offset + numBytes,fileName.length);
- numBytes += fileName.length;
- //copy fileName closing quotation
- System.arraycopy(fileNameFieldClosingQuotation, 0, buffer, offset + numBytes,fileNameFieldClosingQuotation.length);
- numBytes += fileNameFieldClosingQuotation.length;
- //copy bytelocationFieldName
- System.arraycopy(bytelocationFieldName, 0, buffer, offset + numBytes,bytelocationFieldName.length);
- numBytes += bytelocationFieldName.length;
- //copy byte location value
- System.arraycopy(byteLocation, 0, buffer, offset + numBytes,byteLocation.length);
- numBytes += byteLocation.length;
- //copy byte location field end
- System.arraycopy(bytelocationValueEnd, 0, buffer, offset + numBytes,bytelocationValueEnd.length);
- numBytes += bytelocationValueEnd.length;
- //copy the actual adm instance
- System.arraycopy(value.getBytes(), firstFieldLocation, buffer, offset + numBytes,admValueSize - firstFieldLocation);
- buffer[offset + numBytes + admValueSize - firstFieldLocation] = (byte) EOL;
- numBytes += admValueSize - firstFieldLocation +1;
- }
- }
- }
- return numBytes;
- }
-
- @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;
- }
- }
-
- };
- }
- else
- {
- throw new IOException("Can't index " +configuration.get(KEY_FORMAT)+" input");
- }
- }
- }
- else
- {
- if(conf.getInputFormat() instanceof RCFileInputFormat)
- {
- //indexing rc input format
- return new InputStream() {
-
- private RecordReader<LongWritable, BytesRefArrayWritable> reader;
- private LongWritable key;
- private BytesRefArrayWritable value;
- private boolean hasMore = false;
- private int EOL = "\n".getBytes()[0];
- private byte delimiter = 0x01;
- private boolean pendingValue = false;
- private int currentSplitIndex = 0;
- private byte[] fileNumber;
- private byte[] byteLocation;
- private byte[] rowNumberBytes;
- private Integer file;
- private long blockByteLocation;
- private long NextblockByteLocation;
- private int rowNumber;
-
- @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 = reader.createValue();
- //getting the file number
- file = files.get(((FileSplit)(inputSplits[currentSplitIndex])).getPath().toUri().getPath());
- if(file == null)
- {
- throw new HyracksException("a file was not found in the map while indexing");
- }
- fileNumber = String.valueOf(file).getBytes("UTF-8");
- blockByteLocation = reader.getPos();
- pendingValue = reader.next(key, value);
- NextblockByteLocation = reader.getPos();
- rowNumber = 1;
- byteLocation = String.valueOf(blockByteLocation).getBytes("UTF-8");
- rowNumberBytes = String.valueOf(rowNumber).getBytes("UTF-8");
- 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) {
- //last value didn't fit into buffer
- // 1 for EOL
- int sizeOfNextTuple = getTupleSize(value) + 1;
- if (numBytes + sizeOfNextTuple + rowNumberBytes.length + byteLocation.length + fileNumber.length + 3 > len) {
- return 0;
- }
- //copy file number
- System.arraycopy(fileNumber, 0, buffer, offset + numBytes, fileNumber.length);
- buffer[offset + numBytes + fileNumber.length] = delimiter;
- numBytes += fileNumber.length + 1;
-
- //copy byte location
- System.arraycopy(byteLocation, 0, buffer, offset + numBytes, byteLocation.length);
- buffer[offset + numBytes + byteLocation.length] = delimiter;
- numBytes += byteLocation.length + 1;
-
- //copy row number
- System.arraycopy(rowNumberBytes, 0, buffer, offset + numBytes, rowNumberBytes.length);
- buffer[offset + numBytes + rowNumberBytes.length] = delimiter;
- numBytes += rowNumberBytes.length + 1;
-
- copyCurrentTuple(buffer, offset + numBytes);
- buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
- numBytes += sizeOfNextTuple;
- //set pending to false
- pendingValue = false;
- }
-
- 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;
- }
-
- //check if moved to next block
- blockByteLocation = reader.getPos();
- if(blockByteLocation != NextblockByteLocation)
- {
- //moved to a new block, reset stuff
- //row number
- rowNumber = 1;
- rowNumberBytes = String.valueOf(rowNumber).getBytes("UTF-8");
-
- //block location
- byteLocation = String.valueOf(NextblockByteLocation).getBytes("UTF-8");
- NextblockByteLocation = blockByteLocation;
- }
- else
- {
- rowNumber += 1;
- rowNumberBytes = String.valueOf(rowNumber).getBytes("UTF-8");
- }
-
- int sizeOfNextTuple = getTupleSize(value) + 1;
- if (numBytes + sizeOfNextTuple + rowNumberBytes.length + byteLocation.length + fileNumber.length + 3 > 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 = true;
- break;
- } else {
- //copy file number
- System.arraycopy(fileNumber, 0, buffer, offset + numBytes, fileNumber.length);
- buffer[offset + numBytes + fileNumber.length] = delimiter;
- numBytes += fileNumber.length + 1;
-
- //copy byte location
- System.arraycopy(byteLocation, 0, buffer, offset + numBytes, byteLocation.length);
- buffer[offset + numBytes + byteLocation.length] = delimiter;
- numBytes += byteLocation.length + 1;
-
- //copy row number
- System.arraycopy(rowNumberBytes, 0, buffer, offset + numBytes, rowNumberBytes.length);
- buffer[offset + numBytes + rowNumberBytes.length] = delimiter;
- numBytes += rowNumberBytes.length + 1;
-
- copyCurrentTuple(buffer, offset + numBytes);
- buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
- numBytes += sizeOfNextTuple;
- }
- }
- return numBytes;
- }
-
- private void copyCurrentTuple(byte[] buffer, int offset) throws IOException {
- int rcOffset = 0;
- for(int i=0; i< value.size(); i++)
- {
- System.arraycopy(value.get(i).getData(), value.get(i).getStart(), buffer, offset + rcOffset, value.get(i).getLength());
- rcOffset += value.get(i).getLength() + 1;
- buffer[rcOffset - 1] = delimiter;
- }
- }
-
- private int getTupleSize(BytesRefArrayWritable value2) {
- int size=0;
- //loop over rc column and add lengths
- for(int i=0; i< value.size(); i++)
- {
- size += value.get(i).getLength();
- }
- //add delimeters bytes sizes
- size += value.size() -1;
- return size;
- }
-
- @Override
- public int read() throws IOException {
- throw new NotImplementedException("Use read(byte[], int, int");
- }
-
- private RecordReader getRecordReader(int slitIndex) throws IOException {
- RCFileInputFormat format = (RCFileInputFormat) conf.getInputFormat();
- RecordReader reader = format.getRecordReader(
- (org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
- return reader;
- }
-
- };
- }
- else
- {
- //get content format
- if(configuration.get(KEY_FORMAT).equals(FORMAT_DELIMITED_TEXT))
- {
- //reading data and RIDs for delimited text
- 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;
- private Integer file;
- private byte[] fileNumber;
- private byte[] byteLocation;
- private byte delimiter = ((String)configuration.get(KEY_DELIMITER)).getBytes()[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();
- file = files.get(((FileSplit)(inputSplits[currentSplitIndex])).getPath().toUri().getPath());
- if(file == null)
- {
- throw new HyracksException("The file:"+((FileSplit)(inputSplits[currentSplitIndex])).getPath().toUri().getPath()+" was not found in the map while indexing");
- }
- fileNumber = String.valueOf(file).getBytes("UTF-8");
- 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 (numBytes + sizeOfNextTuple +byteLocation.length + fileNumber.length + 2> len)
- {
- return numBytes;
- }
- //copy file number
- System.arraycopy(fileNumber, 0, buffer, offset + numBytes, fileNumber.length);
- buffer[offset + numBytes + fileNumber.length] = delimiter;
- numBytes += fileNumber.length + 1;
-
- //copy byte location
- System.arraycopy(byteLocation, 0, buffer, offset + numBytes, byteLocation.length);
- buffer[offset + numBytes + byteLocation.length] = delimiter;
- numBytes += byteLocation.length + 1;
-
- //copy actual value
- 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) {
- //get reader position before you actually read
- byteLocation = String.valueOf(reader.getPos()).getBytes();
- hasMore = reader.next(key, value);
- if (!hasMore) {
- while (moveToNext()) {
- //get reader position before you actually read
- byteLocation = String.valueOf(reader.getPos()).getBytes("UTF-8");
- 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 +byteLocation.length + fileNumber.length + 2> 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 {
- //copy file number
- System.arraycopy(fileNumber, 0, buffer, offset + numBytes, fileNumber.length);
- buffer[offset + numBytes + fileNumber.length] = delimiter;
- numBytes += fileNumber.length + 1;
-
- //copy byte location
- System.arraycopy(byteLocation, 0, buffer, offset + numBytes, byteLocation.length);
- buffer[offset + numBytes + byteLocation.length] = delimiter;
- numBytes += byteLocation.length + 1;
-
- //Copy actual value
- 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");
- }
-
- 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;
- }
- }
-
- };
- }
- else if((configuration.get(KEY_FORMAT).equals(FORMAT_ADM)))
- {
- //reading data and RIDs for adm formatted data
- 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;
- private Integer file;
- private byte[] fileNumber;
- private byte[] byteLocation;
-
- @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();
- file = files.get(((FileSplit)(inputSplits[currentSplitIndex])).getPath().toUri().getPath());
- if(file == null)
- {
- throw new HyracksException("a file was not found in the map while indexing");
- }
- fileNumber = String.valueOf(file).getBytes("UTF-8");
- 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 firstFieldLocation = value.find("\"");
- int admValueSize = value.getLength();
- if(firstFieldLocation >= 0)
- {
- int sizeOfNextTuple = value.getLength() - firstFieldLocation + 1;
- int sizeOfNextTupleAndRID = fileNumberFieldNameWithRecOpeningBraces.length + fileNumber.length + bytelocationFieldName.length + byteLocation.length + bytelocationValueEnd.length + sizeOfNextTuple;
- if (numBytes + sizeOfNextTupleAndRID > len) {
- // still cannot add tuple to current buffer
- // return 0 so parser would double the buffer size.
- return 0;
- } else {
- //copy fileNumberFieldNameWithRecOpeningBraces
- System.arraycopy(fileNumberFieldNameWithRecOpeningBraces, 0, buffer, offset + numBytes,fileNumberFieldNameWithRecOpeningBraces.length);
- numBytes += fileNumberFieldNameWithRecOpeningBraces.length;
- //copy file Number
- System.arraycopy(fileNumber, 0, buffer, offset + numBytes,fileNumber.length);
- numBytes += fileNumber.length;
- //copy bytelocationFieldName
- System.arraycopy(bytelocationFieldName, 0, buffer, offset + numBytes,bytelocationFieldName.length);
- numBytes += bytelocationFieldName.length;
- //copy byte location value
- System.arraycopy(byteLocation, 0, buffer, offset + numBytes,byteLocation.length);
- numBytes += byteLocation.length;
- //copy byte location field end
- System.arraycopy(bytelocationValueEnd, 0, buffer, offset + numBytes,bytelocationValueEnd.length);
- numBytes += bytelocationValueEnd.length;
- //copy the actual adm instance
- System.arraycopy(value.getBytes(), firstFieldLocation, buffer, offset + numBytes,admValueSize - firstFieldLocation);
- buffer[offset + numBytes + admValueSize - firstFieldLocation] = (byte) EOL;
- numBytes += admValueSize - firstFieldLocation +1;
- }
- }
- pendingValue = null;
- }
-
- while (numBytes < len) {
- //get reader position before you actually read
- byteLocation = String.valueOf(reader.getPos()).getBytes("UTF-8");
- hasMore = reader.next(key, value);
- if (!hasMore) {
- while (moveToNext()) {
- //get reader position before you actually read
- byteLocation = String.valueOf(reader.getPos()).getBytes("UTF-8");
- hasMore = reader.next(key, value);
- if (hasMore) {
- //move to the next non-empty split
- break;
- }
- }
- }
- if (!hasMore) {
- return (numBytes == 0) ? -1 : numBytes;
- }
- //get the index of the first field name
- int firstFieldLocation = value.find("\"");
- int admValueSize = value.getLength();
- if(firstFieldLocation >= 0)
- {
- int sizeOfNextTuple = value.getLength() - firstFieldLocation + 1;
- int sizeOfNextTupleAndRID = fileNumberFieldNameWithRecOpeningBraces.length + fileNumber.length + bytelocationFieldName.length + byteLocation.length + bytelocationValueEnd.length + sizeOfNextTuple;
- if (numBytes + sizeOfNextTupleAndRID > 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 {
- //copy fileNumberFieldNameWithRecOpeningBraces
- System.arraycopy(fileNumberFieldNameWithRecOpeningBraces, 0, buffer, offset + numBytes,fileNumberFieldNameWithRecOpeningBraces.length);
- numBytes += fileNumberFieldNameWithRecOpeningBraces.length;
- //copy fileNumber
- System.arraycopy(fileNumber, 0, buffer, offset + numBytes,fileNumber.length);
- numBytes += fileNumber.length;
- //copy bytelocationFieldName
- System.arraycopy(bytelocationFieldName, 0, buffer, offset + numBytes,bytelocationFieldName.length);
- numBytes += bytelocationFieldName.length;
- //copy byte location value
- System.arraycopy(byteLocation, 0, buffer, offset + numBytes,byteLocation.length);
- numBytes += byteLocation.length;
- //copy byte location field end
- System.arraycopy(bytelocationValueEnd, 0, buffer, offset + numBytes,bytelocationValueEnd.length);
- numBytes += bytelocationValueEnd.length;
- //copy the actual adm instance
- System.arraycopy(value.getBytes(), firstFieldLocation, buffer, offset + numBytes,admValueSize - firstFieldLocation);
- buffer[offset + numBytes + admValueSize - firstFieldLocation] = (byte) EOL;
- numBytes += admValueSize - firstFieldLocation +1;
- }
- }
- }
- return numBytes;
- }
-
- @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;
- }
- }
-
- };
- }
- else
- {
- throw new IOException("Can't index " +configuration.get(KEY_FORMAT)+" input");
- }
- }
- }
- }
-
- @Override
- public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
- return clusterLocations;
- }
-}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HiveIndexingAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HiveIndexingAdapter.java
deleted file mode 100644
index 178b106..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HiveIndexingAdapter.java
+++ /dev/null
@@ -1,62 +0,0 @@
-package edu.uci.ics.asterix.external.dataset.adapter;
-
-import java.util.Map;
-
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-
-import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-
-/**
- * Provides the functionality of fetching data in form of ADM records from a Hive dataset.
- */
-@SuppressWarnings("deprecation")
-public class HiveIndexingAdapter extends AbstractDatasourceAdapter{
-
- private static final long serialVersionUID = 1L;
-
- public static final String HIVE_DATABASE = "database";
- public static final String HIVE_TABLE = "table";
- public static final String HIVE_HOME = "hive-home";
- public static final String HIVE_METASTORE_URI = "metastore-uri";
- public static final String HIVE_WAREHOUSE_DIR = "warehouse-dir";
- public static final String HIVE_METASTORE_RAWSTORE_IMPL = "rawstore-impl";
-
- private HDFSIndexingAdapter hdfsIndexingAdapter;
-
- public HiveIndexingAdapter(IAType atype, String[] readSchedule, boolean[] executed, InputSplit[] inputSplits, JobConf conf,
- AlgebricksPartitionConstraint clusterLocations, Map<String,Integer> files) {
- this.hdfsIndexingAdapter = new HDFSIndexingAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations, files);
- this.atype = atype;
- }
-
- @Override
- public AdapterType getAdapterType() {
- return AdapterType.READ;
- }
-
- @Override
- public void configure(Map<String, Object> arguments) throws Exception {
- this.configuration = arguments;
- this.hdfsIndexingAdapter.configure(arguments);
- }
-
- @Override
- public void initialize(IHyracksTaskContext ctx) throws Exception {
- hdfsIndexingAdapter.initialize(ctx);
- }
-
- @Override
- public void start(int partition, IFrameWriter writer) throws Exception {
- hdfsIndexingAdapter.start(partition, writer);
- }
-
- @Override
- public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
- return hdfsIndexingAdapter.getPartitionConstraint();
- }
-
-}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IControlledAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IControlledAdapter.java
deleted file mode 100644
index 253f675..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IControlledAdapter.java
+++ /dev/null
@@ -1,17 +0,0 @@
-package edu.uci.ics.asterix.external.dataset.adapter;
-
-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.exceptions.HyracksDataException;
-
-public interface IControlledAdapter extends Serializable{
-
- public void initialize(IHyracksTaskContext ctx) throws Exception;
-
- public void processNextFrame(ByteBuffer buffer, IFrameWriter writer) throws HyracksDataException;
-
- public void close(IFrameWriter writer) throws HyracksDataException;
-}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/util/ExternalDataFilesMetadataProvider.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/util/ExternalDataFilesMetadataProvider.java
deleted file mode 100644
index 47550a4..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/util/ExternalDataFilesMetadataProvider.java
+++ /dev/null
@@ -1,38 +0,0 @@
-package edu.uci.ics.asterix.external.util;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.StringTokenizer;
-
-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.external.adapter.factory.HDFSAdapterFactory;
-import edu.uci.ics.asterix.external.dataset.adapter.AbstractDatasourceAdapter;
-import edu.uci.ics.asterix.external.dataset.adapter.HDFSAdapter;
-
-public class ExternalDataFilesMetadataProvider {
- public static ArrayList<FileStatus> getHDFSFileStatus(AbstractDatasourceAdapter adapter) throws IOException
- {
- ArrayList<FileStatus> files = new ArrayList<FileStatus>();
- //Configure hadoop connection
- Configuration conf = HDFSAdapterFactory.configureHadoopConnection(adapter.getConfiguration());
- FileSystem fs = FileSystem.get(conf);
- //get the list of paths from the adapter
- StringTokenizer tokenizer = new StringTokenizer(((String)adapter.getConfiguration().get(HDFSAdapter.KEY_PATH)),",");
- Path inputPath = null;
- FileStatus[] fileStatuses;
- while(tokenizer.hasMoreTokens())
- {
- inputPath = new Path(tokenizer.nextToken().trim());
- fileStatuses = fs.listStatus(inputPath);
- for(int i=0; i < fileStatuses.length; i++)
- {
- files.add(fileStatuses[i]);
- }
- }
- return files;
- }
-}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/util/ExternalIndexHashPartitionComputerFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/util/ExternalIndexHashPartitionComputerFactory.java
deleted file mode 100644
index 29fcfb1..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/util/ExternalIndexHashPartitionComputerFactory.java
+++ /dev/null
@@ -1,94 +0,0 @@
-package edu.uci.ics.asterix.external.util;
-
-import java.io.DataInputStream;
-import java.nio.ByteBuffer;
-
-import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import edu.uci.ics.asterix.om.base.AInt64;
-import edu.uci.ics.asterix.om.types.BuiltinType;
-import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
-
-/* A class that is used to partition external data tuples when building an index over them
- * the computer it returns, computes the HDFS block value before using the actual hash partitioning
- * function. this way we ensure that records within ranges of 64MB sizes are partitioned together to the same
- * data node.
- */
-
-public class ExternalIndexHashPartitionComputerFactory implements ITuplePartitionComputerFactory{
- private static final long serialVersionUID = 1L;
- private final int[] hashFields;
- private final int bytesInHDFSBlock = 67108864;
- private final IBinaryHashFunctionFactory[] hashFunctionFactories;
- @SuppressWarnings("unchecked")
- private final ISerializerDeserializer<AInt64> longSerde = AqlSerializerDeserializerProvider.INSTANCE.getNonTaggedSerializerDeserializer(BuiltinType.AINT64);
-
- public ExternalIndexHashPartitionComputerFactory(int[] hashFields, IBinaryHashFunctionFactory[] hashFunctionFactories) {
- this.hashFields = hashFields;
- this.hashFunctionFactories = hashFunctionFactories;
- }
-
- @Override
- public ITuplePartitionComputer createPartitioner() {
- final IBinaryHashFunction[] hashFunctions = new IBinaryHashFunction[hashFunctionFactories.length];
- for (int i = 0; i < hashFunctionFactories.length; ++i) {
- hashFunctions[i] = hashFunctionFactories[i].createBinaryHashFunction();
- }
- return new ITuplePartitionComputer() {
- private ByteBuffer serializedLong = ByteBuffer.allocate(8);;
- private AInt64 byteLocation;
- private ByteBufferInputStream bbis = new ByteBufferInputStream();
- private DataInputStream dis = new DataInputStream(bbis);
- @Override
- public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts){
- if (nParts == 1) {
- return 0;
- }
- int h = 0;
- int startOffset = accessor.getTupleStartOffset(tIndex);
- int slotLength = accessor.getFieldSlotsLength();
- for (int j = 0; j < hashFields.length; ++j) {
- int fIdx = hashFields[j];
- IBinaryHashFunction hashFn = hashFunctions[j];
- int fStart = accessor.getFieldStartOffset(tIndex, fIdx);
- int fEnd = accessor.getFieldEndOffset(tIndex, fIdx);
- if(j == 1)
- {
- //reset the buffer
- serializedLong.clear();
- //read byte location
- bbis.setByteBuffer(accessor.getBuffer() , accessor.getTupleStartOffset(tIndex) + accessor.getFieldSlotsLength() + accessor.getFieldStartOffset(tIndex, hashFields[1]));
- try {
- byteLocation = ((AInt64) longSerde.deserialize(dis));
- //compute the block number, serialize it into a new array and call the hash function
- serializedLong.putLong(byteLocation.getLongValue()/bytesInHDFSBlock);
- //call the hash function
- int fh = hashFn
- .hash(serializedLong.array(), 0,serializedLong.capacity());
- h = h * 31 + fh;
- } catch (HyracksDataException e) {
- System.err.println("could not serialize the long byte position value!!!");
- e.printStackTrace();
- }
- }
- else
- {
- int fh = hashFn
- .hash(accessor.getBuffer().array(), startOffset + slotLength + fStart, fEnd - fStart);
- h = h * 31 + fh;
- }
- }
- if (h < 0) {
- h = -(h + 1);
- }
- return h % nParts;
- }
- };
- }
-}
\ No newline at end of file
diff --git a/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/resources/Lexer.java b/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/resources/Lexer.java
index 92d1c8e..dae1fb1 100644
--- a/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/resources/Lexer.java
+++ b/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/resources/Lexer.java
@@ -19,235 +19,215 @@
public class [LEXER_NAME] {
- public static final int
- TOKEN_EOF = 0, TOKEN_AUX_NOT_FOUND = 1 [TOKENS_CONSTANTS];
+ public static final int
+ TOKEN_EOF = 0, TOKEN_AUX_NOT_FOUND = 1 [TOKENS_CONSTANTS];
- // Human representation of tokens. Useful for debug.
- // Is possible to convert a TOKEN_CONSTANT in its image through
- // [LEXER_NAME].tokenKindToString(TOKEN_CONSTANT);
- private static final String[] tokenImage = {
- "<EOF>", "<AUX_NOT_FOUND>" [TOKENS_IMAGES]
- };
+ // Human representation of tokens. Useful for debug.
+ // Is possible to convert a TOKEN_CONSTANT in its image through
+ // [LEXER_NAME].tokenKindToString(TOKEN_CONSTANT);
+ private static final String[] tokenImage = {
+ "<EOF>", "<AUX_NOT_FOUND>" [TOKENS_IMAGES]
+ };
+
+ private static final char EOF_CHAR = 4;
+ protected java.io.Reader inputStream;
+ protected int column;
+ protected int line;
+ protected boolean prevCharIsCR;
+ protected boolean prevCharIsLF;
+ protected char[] buffer;
+ protected int bufsize;
+ protected int bufpos;
+ protected int tokenBegin;
+ protected int endOf_USED_Buffer;
+ protected int endOf_UNUSED_Buffer;
+ protected int maxUnusedBufferSize;
- private static final char EOF_CHAR = 4;
- protected java.io.Reader inputStream;
- protected int column;
- protected int line;
- protected boolean prevCharIsCR;
- protected boolean prevCharIsLF;
- protected char[] buffer;
- protected int bufsize;
- protected int bufpos;
- protected int tokenBegin;
- protected int endOf_USED_Buffer;
- protected int endOf_UNUSED_Buffer;
- protected int maxUnusedBufferSize;
+// ================================================================================
+// Auxiliary functions. Can parse the tokens used in the grammar as partial/auxiliary
+// ================================================================================
- // ================================================================================
- // Auxiliary functions. Can parse the tokens used in the grammar as partial/auxiliary
- // ================================================================================
+ [LEXER_AUXFUNCTIONS]
- [LEXER_AUXFUNCTIONS]
+// ================================================================================
+// Main method. Return a TOKEN_CONSTANT
+// ================================================================================
+
+ public int next() throws [LEXER_NAME]Exception, IOException{
+ char currentChar = buffer[bufpos];
+ while (currentChar == ' ' || currentChar=='\t' || currentChar == '\n' || currentChar=='\r')
+ currentChar = readNextChar();
+ tokenBegin = bufpos;
+ if (currentChar==EOF_CHAR) return TOKEN_EOF;
- // ================================================================================
- // Main method. Return a TOKEN_CONSTANT
- // ================================================================================
+ [LEXER_LOGIC]
+ }
- public int next() throws [LEXER_NAME]Exception, IOException{
- if(bufpos < 0)
- readNextChar();
- char currentChar = buffer[bufpos];
- while (currentChar == ' ' || currentChar=='\t' || currentChar == '\n' || currentChar=='\r')
- currentChar = readNextChar();
- tokenBegin = bufpos;
- if (currentChar==EOF_CHAR) return TOKEN_EOF;
+// ================================================================================
+// Public interface
+// ================================================================================
+
+ public [LEXER_NAME](java.io.Reader stream) throws IOException{
+ reInit(stream);
+ }
- [LEXER_LOGIC]
- }
+ public void reInit(java.io.Reader stream) throws IOException{
+ done();
+ inputStream = stream;
+ bufsize = 4096;
+ line = 1;
+ column = 0;
+ bufpos = -1;
+ endOf_UNUSED_Buffer = bufsize;
+ endOf_USED_Buffer = 0;
+ prevCharIsCR = false;
+ prevCharIsLF = false;
+ buffer = new char[bufsize];
+ tokenBegin = -1;
+ maxUnusedBufferSize = 4096/2;
+ readNextChar();
+ }
- //used when done with stream, must be called exiplicitly now.
- public void close()throws IOException
- {
- inputStream.close();
- }
+ public String getLastTokenImage() {
+ if (bufpos >= tokenBegin)
+ return new String(buffer, tokenBegin, bufpos - tokenBegin);
+ else
+ return new String(buffer, tokenBegin, bufsize - tokenBegin) +
+ new String(buffer, 0, bufpos);
+ }
+
+ public static String tokenKindToString(int token) {
+ return tokenImage[token];
+ }
- //used before processing a new patch in the inputStream
- public void reset(){
- line = 1;
- column = 0;
- bufpos = -1;
- endOf_UNUSED_Buffer = bufsize;
- endOf_USED_Buffer = 0;
- prevCharIsCR = false;
- prevCharIsLF = false;
- tokenBegin = -1;
- }
+ public void done(){
+ buffer = null;
+ }
- // ================================================================================
- // Public interface
- // ================================================================================
+// ================================================================================
+// Parse error management
+// ================================================================================
+
+ protected int parseError(String reason) throws [LEXER_NAME]Exception {
+ StringBuilder message = new StringBuilder();
+ message.append(reason).append("\n");
+ message.append("Line: ").append(line).append("\n");
+ message.append("Row: ").append(column).append("\n");
+ throw new [LEXER_NAME]Exception(message.toString());
+ }
- public [LEXER_NAME](java.io.Reader stream) throws IOException{
- reInit(stream);
- }
+ protected int parseError(int ... tokens) throws [LEXER_NAME]Exception {
+ StringBuilder message = new StringBuilder();
+ message.append("Error while parsing. ");
+ message.append(" Line: ").append(line);
+ message.append(" Row: ").append(column);
+ message.append(" Expecting:");
+ for (int tokenId : tokens){
+ message.append(" ").append([LEXER_NAME].tokenKindToString(tokenId));
+ }
+ throw new [LEXER_NAME]Exception(message.toString());
+ }
+
+ protected void updateLineColumn(char c){
+ column++;
+
+ if (prevCharIsLF)
+ {
+ prevCharIsLF = false;
+ line += (column = 1);
+ }
+ else if (prevCharIsCR)
+ {
+ prevCharIsCR = false;
+ if (c == '\n')
+ {
+ prevCharIsLF = true;
+ }
+ else
+ {
+ line += (column = 1);
+ }
+ }
+
+ if (c=='\r') {
+ prevCharIsCR = true;
+ } else if(c == '\n') {
+ prevCharIsLF = true;
+ }
+ }
+
+// ================================================================================
+// Read data, buffer management. It uses a circular (and expandable) buffer
+// ================================================================================
- public void reInit(java.io.Reader stream) throws IOException{
- done();
- inputStream = stream;
- bufsize = 4096;
- line = 1;
- column = 0;
- bufpos = -1;
- endOf_UNUSED_Buffer = bufsize;
- endOf_USED_Buffer = 0;
- prevCharIsCR = false;
- prevCharIsLF = false;
- buffer = new char[bufsize];
- tokenBegin = -1;
- maxUnusedBufferSize = 4096/2;
- }
+ protected char readNextChar() throws IOException {
+ if (++bufpos >= endOf_USED_Buffer)
+ fillBuff();
+ char c = buffer[bufpos];
+ updateLineColumn(c);
+ return c;
+ }
- public String getLastTokenImage() {
- if (bufpos >= tokenBegin)
- return new String(buffer, tokenBegin, bufpos - tokenBegin);
- else
- return new String(buffer, tokenBegin, bufsize - tokenBegin) +
- new String(buffer, 0, bufpos);
- }
-
- public static String tokenKindToString(int token) {
- return tokenImage[token];
- }
-
- public void done(){
- buffer = null;
- }
-
- // ================================================================================
- // Parse error management
- // ================================================================================
-
- protected int parseError(String reason) throws [LEXER_NAME]Exception {
- StringBuilder message = new StringBuilder();
- message.append(reason).append("\n");
- message.append("Line: ").append(line).append("\n");
- message.append("Row: ").append(column).append("\n");
- throw new [LEXER_NAME]Exception(message.toString());
- }
-
- protected int parseError(int ... tokens) throws [LEXER_NAME]Exception {
- StringBuilder message = new StringBuilder();
- message.append("Error while parsing. ");
- message.append(" Line: ").append(line);
- message.append(" Row: ").append(column);
- message.append(" Expecting:");
- for (int tokenId : tokens){
- message.append(" ").append([LEXER_NAME].tokenKindToString(tokenId));
- }
- throw new [LEXER_NAME]Exception(message.toString());
- }
-
- protected void updateLineColumn(char c){
- column++;
-
- if (prevCharIsLF)
- {
- prevCharIsLF = false;
- line += (column = 1);
- }
- else if (prevCharIsCR)
- {
- prevCharIsCR = false;
- if (c == '\n')
- {
- prevCharIsLF = true;
- }
- else
- {
- line += (column = 1);
- }
- }
-
- if (c=='\r') {
- prevCharIsCR = true;
- } else if(c == '\n') {
- prevCharIsLF = true;
- }
- }
-
- // ================================================================================
- // Read data, buffer management. It uses a circular (and expandable) buffer
- // ================================================================================
-
- protected char readNextChar() throws IOException {
- if (++bufpos >= endOf_USED_Buffer)
- fillBuff();
- char c = buffer[bufpos];
- updateLineColumn(c);
- return c;
- }
-
- protected boolean fillBuff() throws IOException {
- if (endOf_UNUSED_Buffer == endOf_USED_Buffer) // If no more unused buffer space
- {
- if (endOf_UNUSED_Buffer == bufsize) // -- If the previous unused space was
- { // -- at the end of the buffer
- if (tokenBegin > maxUnusedBufferSize) // -- -- If the first N bytes before
- { // the current token are enough
- bufpos = endOf_USED_Buffer = 0; // -- -- -- setup buffer to use that fragment
- endOf_UNUSED_Buffer = tokenBegin;
- }
- else if (tokenBegin < 0) // -- -- If no token yet
- bufpos = endOf_USED_Buffer = 0; // -- -- -- reuse the whole buffer
- else
- ExpandBuff(false); // -- -- Otherwise expand buffer after its end
- }
- else if (endOf_UNUSED_Buffer > tokenBegin) // If the endOf_UNUSED_Buffer is after the token
- endOf_UNUSED_Buffer = bufsize; // -- set endOf_UNUSED_Buffer to the end of the buffer
- else if ((tokenBegin - endOf_UNUSED_Buffer) < maxUnusedBufferSize)
- { // If between endOf_UNUSED_Buffer and the token
- ExpandBuff(true); // there is NOT enough space expand the buffer
- } // reorganizing it
- else
- endOf_UNUSED_Buffer = tokenBegin; // Otherwise there is enough space at the start
- } // so we set the buffer to use that fragment
- int i;
- if ((i = inputStream.read(buffer, endOf_USED_Buffer, endOf_UNUSED_Buffer - endOf_USED_Buffer)) == -1)
- {
- //moved outside
- //inputStream.close();
- buffer[endOf_USED_Buffer]=(char)EOF_CHAR;
- endOf_USED_Buffer++;
- return false;
- }
- else
- endOf_USED_Buffer += i;
- return true;
- }
+ protected boolean fillBuff() throws IOException {
+ if (endOf_UNUSED_Buffer == endOf_USED_Buffer) // If no more unused buffer space
+ {
+ if (endOf_UNUSED_Buffer == bufsize) // -- If the previous unused space was
+ { // -- at the end of the buffer
+ if (tokenBegin > maxUnusedBufferSize) // -- -- If the first N bytes before
+ { // the current token are enough
+ bufpos = endOf_USED_Buffer = 0; // -- -- -- setup buffer to use that fragment
+ endOf_UNUSED_Buffer = tokenBegin;
+ }
+ else if (tokenBegin < 0) // -- -- If no token yet
+ bufpos = endOf_USED_Buffer = 0; // -- -- -- reuse the whole buffer
+ else
+ ExpandBuff(false); // -- -- Otherwise expand buffer after its end
+ }
+ else if (endOf_UNUSED_Buffer > tokenBegin) // If the endOf_UNUSED_Buffer is after the token
+ endOf_UNUSED_Buffer = bufsize; // -- set endOf_UNUSED_Buffer to the end of the buffer
+ else if ((tokenBegin - endOf_UNUSED_Buffer) < maxUnusedBufferSize)
+ { // If between endOf_UNUSED_Buffer and the token
+ ExpandBuff(true); // there is NOT enough space expand the buffer
+ } // reorganizing it
+ else
+ endOf_UNUSED_Buffer = tokenBegin; // Otherwise there is enough space at the start
+ } // so we set the buffer to use that fragment
+ int i;
+ if ((i = inputStream.read(buffer, endOf_USED_Buffer, endOf_UNUSED_Buffer - endOf_USED_Buffer)) == -1)
+ {
+ inputStream.close();
+ buffer[endOf_USED_Buffer]=(char)EOF_CHAR;
+ endOf_USED_Buffer++;
+ return false;
+ }
+ else
+ endOf_USED_Buffer += i;
+ return true;
+ }
- protected void ExpandBuff(boolean wrapAround)
- {
- char[] newbuffer = new char[bufsize + maxUnusedBufferSize];
+ protected void ExpandBuff(boolean wrapAround)
+ {
+ char[] newbuffer = new char[bufsize + maxUnusedBufferSize];
- try {
- if (wrapAround) {
- System.arraycopy(buffer, tokenBegin, newbuffer, 0, bufsize - tokenBegin);
- System.arraycopy(buffer, 0, newbuffer, bufsize - tokenBegin, bufpos);
- buffer = newbuffer;
- endOf_USED_Buffer = (bufpos += (bufsize - tokenBegin));
- }
- else {
- System.arraycopy(buffer, tokenBegin, newbuffer, 0, bufsize - tokenBegin);
- buffer = newbuffer;
- endOf_USED_Buffer = (bufpos -= tokenBegin);
- }
- } catch (Throwable t) {
- throw new Error(t.getMessage());
- }
+ try {
+ if (wrapAround) {
+ System.arraycopy(buffer, tokenBegin, newbuffer, 0, bufsize - tokenBegin);
+ System.arraycopy(buffer, 0, newbuffer, bufsize - tokenBegin, bufpos);
+ buffer = newbuffer;
+ endOf_USED_Buffer = (bufpos += (bufsize - tokenBegin));
+ }
+ else {
+ System.arraycopy(buffer, tokenBegin, newbuffer, 0, bufsize - tokenBegin);
+ buffer = newbuffer;
+ endOf_USED_Buffer = (bufpos -= tokenBegin);
+ }
+ } catch (Throwable t) {
+ throw new Error(t.getMessage());
+ }
- bufsize += maxUnusedBufferSize;
- endOf_UNUSED_Buffer = bufsize;
- tokenBegin = 0;
- }
+ bufsize += maxUnusedBufferSize;
+ endOf_UNUSED_Buffer = bufsize;
+ tokenBegin = 0;
+ }
}
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 2de5d78..8f0eedb 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
@@ -31,7 +31,6 @@
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.Function;
import edu.uci.ics.asterix.metadata.entities.Index;
import edu.uci.ics.asterix.metadata.entities.Node;
@@ -269,39 +268,6 @@
}
return dataset;
}
-
- @Override
- public List<ExternalFile> getDatasetExternalFiles(
- MetadataTransactionContext mdTxnCtx, Dataset dataset)
- throws MetadataException {
- List<ExternalFile> externalFiles;
- try {
- externalFiles = metadataNode.getExternalDatasetFiles(mdTxnCtx.getJobId(), dataset);
- } catch (RemoteException e) {
- throw new MetadataException(e);
- }
- return externalFiles;
- }
-
- @Override
- public void addExternalFile(MetadataTransactionContext mdTxnCtx,
- ExternalFile externalFile) throws MetadataException {
- try {
- metadataNode.addExternalDatasetFile(mdTxnCtx.getJobId(), externalFile);
- } catch (RemoteException e) {
- throw new MetadataException(e);
- }
- }
-
- @Override
- public void dropExternalFile(MetadataTransactionContext mdTxnCtx,
- ExternalFile externalFile) throws MetadataException {
- try {
- metadataNode.dropExternalFile(mdTxnCtx.getJobId(), externalFile.getDataverseName(), externalFile.getDatasetName(), externalFile.getFileNumber());
- } catch (RemoteException e) {
- throw new MetadataException(e);
- }
- }
@Override
public List<Index> getDatasetIndexes(MetadataTransactionContext ctx, String dataverseName, String datasetName)
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 81ce4f8..ee0791b 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
@@ -41,8 +41,6 @@
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.Function;
import edu.uci.ics.asterix.metadata.entities.Index;
import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
@@ -52,7 +50,6 @@
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.FunctionTupleTranslator;
import edu.uci.ics.asterix.metadata.entitytupletranslators.IndexTupleTranslator;
import edu.uci.ics.asterix.metadata.entitytupletranslators.NodeGroupTupleTranslator;
@@ -62,7 +59,6 @@
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;
@@ -119,14 +115,15 @@
@Override
public void commitTransaction(JobId jobId) throws RemoteException, ACIDException {
- ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+ ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
transactionSubsystem.getTransactionManager().commitTransaction(txnCtx, new DatasetId(-1), -1);
}
@Override
public void abortTransaction(JobId jobId) throws RemoteException, ACIDException {
try {
- ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+ ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId,
+ false);
transactionSubsystem.getTransactionManager().abortTransaction(txnCtx, new DatasetId(-1), -1);
} catch (ACIDException e) {
e.printStackTrace();
@@ -136,13 +133,13 @@
@Override
public void lock(JobId jobId, byte lockMode) throws ACIDException, RemoteException {
- ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+ ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
transactionSubsystem.getLockManager().lock(METADATA_DATASET_ID, -1, lockMode, txnCtx);
}
@Override
public void unlock(JobId jobId) throws ACIDException, RemoteException {
- ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+ ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
transactionSubsystem.getLockManager().unlock(METADATA_DATASET_ID, -1, txnCtx);
}
@@ -173,56 +170,48 @@
Index primaryIndex = new Index(dataset.getDataverseName(), dataset.getDatasetName(),
dataset.getDatasetName(), IndexType.BTREE, id.getPrimaryKey(), true, dataset.getPendingOp());
- addIndex(jobId, primaryIndex);
- ITupleReference nodeGroupTuple = createTuple(id.getNodeGroupName(), dataset.getDataverseName(),
- dataset.getDatasetName());
- insertTupleIntoIndex(jobId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, nodeGroupTuple);
- }
- else
- {
- //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);
- }
- // Add entry in datatype secondary index.
- ITupleReference dataTypeTuple = createTuple(dataset.getDataverseName(), dataset.getItemTypeName(),
- dataset.getDatasetName());
- insertTupleIntoIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeTuple);
- } catch (TreeIndexDuplicateKeyException e) {
- throw new MetadataException("A dataset with this name " + dataset.getDatasetName()
- + " already exists in dataverse '" + dataset.getDataverseName() + "'.", e);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ addIndex(jobId, primaryIndex);
+ ITupleReference nodeGroupTuple = createTuple(id.getNodeGroupName(), dataset.getDataverseName(),
+ dataset.getDatasetName());
+ insertTupleIntoIndex(jobId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, nodeGroupTuple);
+ }
+ // Add entry in datatype secondary index.
+ ITupleReference dataTypeTuple = createTuple(dataset.getDataverseName(), dataset.getItemTypeName(),
+ dataset.getDatasetName());
+ insertTupleIntoIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeTuple);
+ } catch (TreeIndexDuplicateKeyException e) {
+ throw new MetadataException("A dataset with this name " + dataset.getDatasetName()
+ + " already exists in dataverse '" + dataset.getDataverseName() + "'.", e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public void addIndex(JobId jobId, Index index) throws MetadataException, RemoteException {
- try {
- IndexTupleTranslator tupleWriter = new IndexTupleTranslator(true);
- ITupleReference tuple = tupleWriter.getTupleFromMetadataEntity(index);
- insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, tuple);
- } catch (TreeIndexDuplicateKeyException e) {
- throw new MetadataException("An index with name '" + index.getIndexName() + "' already exists.", e);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public void addIndex(JobId jobId, Index index) throws MetadataException, RemoteException {
+ try {
+ IndexTupleTranslator tupleWriter = new IndexTupleTranslator(true);
+ ITupleReference tuple = tupleWriter.getTupleFromMetadataEntity(index);
+ insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, tuple);
+ } catch (TreeIndexDuplicateKeyException e) {
+ throw new MetadataException("An index with name '" + index.getIndexName() + "' already exists.", e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public void addNode(JobId jobId, Node node) throws MetadataException, RemoteException {
- try {
- NodeTupleTranslator tupleReaderWriter = new NodeTupleTranslator(true);
- ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(node);
- insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.NODE_DATASET, tuple);
- } catch (TreeIndexDuplicateKeyException e) {
- throw new MetadataException("A node with name '" + node.getNodeName() + "' already exists.", e);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public void addNode(JobId jobId, Node node) throws MetadataException, RemoteException {
+ try {
+ NodeTupleTranslator tupleReaderWriter = new NodeTupleTranslator(true);
+ ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(node);
+ insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.NODE_DATASET, tuple);
+ } catch (TreeIndexDuplicateKeyException e) {
+ throw new MetadataException("A node with name '" + node.getNodeName() + "' already exists.", e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
@Override
public void addNodeGroup(JobId jobId, NodeGroup nodeGroup) throws MetadataException, RemoteException {
@@ -285,7 +274,7 @@
ILSMIndexAccessor indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
- ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+ ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
txnCtx.setWriteTxn(true);
txnCtx.registerIndexAndCallback(resourceID, lsmIndex, (AbstractOperationCallback) modCallback,
metadataIndex.isPrimaryIndex());
@@ -298,7 +287,7 @@
private IModificationOperationCallback createIndexModificationCallback(JobId jobId, long resourceId,
IMetadataIndex metadataIndex, ILSMIndex lsmIndex, IndexOperation indexOp) throws Exception {
- ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+ ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
if (metadataIndex.isPrimaryIndex()) {
return new PrimaryIndexModificationOperationCallback(metadataIndex.getDatasetId().getId(),
@@ -396,60 +385,47 @@
// artifacts.
}
- // Delete entry from secondary index 'group'.
- if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
- 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.
- }
- }
- else
- {
- ExternalDatasetDetails id = (ExternalDatasetDetails) 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.
- }
- }
- // 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
- // lock on the DATATYPENAME_ON_DATASET_INDEX index.
- try {
- ITupleReference dataTypeTuple = getTupleToBeDeleted(jobId,
- MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeSearchKey);
- deleteTupleFromIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeTuple);
- } catch (TreeIndexException tie) {
- // ignore this exception and continue deleting all relevant
- // artifacts.
- }
+ // Delete entry from secondary index 'group'.
+ if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
+ 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.
+ }
+ }
+ // 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
+ // lock on the DATATYPENAME_ON_DATASET_INDEX index.
+ try {
+ ITupleReference dataTypeTuple = getTupleToBeDeleted(jobId,
+ MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeSearchKey);
+ deleteTupleFromIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeTuple);
+ } catch (TreeIndexException tie) {
+ // ignore this exception and continue deleting all relevant
+ // artifacts.
+ }
- // Delete entry(s) from the 'indexes' dataset.
- List<Index> datasetIndexes = getDatasetIndexes(jobId, dataverseName, datasetName);
- if (datasetIndexes != null) {
- for (Index index : datasetIndexes) {
- dropIndex(jobId, dataverseName, datasetName, index.getIndexName());
- }
- }
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ // Delete entry(s) from the 'indexes' dataset.
+ if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
+ List<Index> datasetIndexes = getDatasetIndexes(jobId, dataverseName, datasetName);
+ if (datasetIndexes != null) {
+ for (Index index : datasetIndexes) {
+ dropIndex(jobId, dataverseName, datasetName, index.getIndexName());
+ }
+ }
+ }
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
@Override
public void dropIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
@@ -606,7 +582,7 @@
lsmIndex, IndexOperation.DELETE);
ILSMIndexAccessor indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
- ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+ ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
txnCtx.setWriteTxn(true);
txnCtx.registerIndexAndCallback(resourceID, lsmIndex, (AbstractOperationCallback) modCallback,
metadataIndex.isPrimaryIndex());
@@ -1160,116 +1136,8 @@
}
}
- @Override
- public List<ExternalFile> getExternalDatasetFiles(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);
- }
- }
-
- @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;
- }
-
- public ExternalFile getExternalDatasetFile(JobId jobId,String dataverseName, String datasetName,
- int fileNumber) throws MetadataException, RemoteException {
- try {
- //create the search key
- 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);
- return results.get(0);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
-
- @Override
- public void dropExternalFile(JobId jobId, String dataverseName,
- String datasetName, int fileNumber) throws MetadataException,
- RemoteException {
- ExternalFile externalFile;
- try {
- externalFile = getExternalDatasetFile(jobId, dataverseName, datasetName,fileNumber);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- if (externalFile == null) {
- throw new MetadataException("Cannot drop external file because it doesn't exist.");
- }
- 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 addExternalDatasetFile(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 int getMostRecentDatasetId() throws MetadataException, RemoteException {
+ return DatasetIdFactory.getMostRecentDatasetId();
}
-
-
- @Override
- public int getMostRecentDatasetId() throws MetadataException, RemoteException {
- return DatasetIdFactory.getMostRecentDatasetId();
- }
}
-
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 53f72dd..22c5e46 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
@@ -26,7 +26,6 @@
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.Function;
import edu.uci.ics.asterix.metadata.entities.Index;
import edu.uci.ics.asterix.metadata.entities.Node;
@@ -440,36 +439,6 @@
*/
public List<Function> getDataverseFunctions(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;
public void initializeDatasetIdFactory(MetadataTransactionContext ctx) throws MetadataException;
@@ -484,5 +453,4 @@
public void releaseReadLatch();
-
}
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 206ef8a..d1e63e1 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
@@ -28,7 +28,6 @@
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.Function;
import edu.uci.ics.asterix.metadata.entities.Index;
import edu.uci.ics.asterix.metadata.entities.Node;
@@ -472,45 +471,6 @@
* @throws RemoteException
*/
public void addAdapter(JobId jobId, DatasourceAdapter adapter) 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 addExternalDatasetFile(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> getExternalDatasetFiles(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;
public void initializeDatasetIdFactory(JobId jobId) 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 0c64497..7be4e8e 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
@@ -60,7 +60,6 @@
import edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat;
import edu.uci.ics.asterix.transaction.management.resource.LSMBTreeLocalResourceMetadata;
import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
-import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
import edu.uci.ics.hyracks.api.application.INCApplicationContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -117,7 +116,7 @@
MetadataPrimaryIndexes.DATASET_DATASET, MetadataPrimaryIndexes.DATATYPE_DATASET,
MetadataPrimaryIndexes.INDEX_DATASET, MetadataPrimaryIndexes.NODE_DATASET,
MetadataPrimaryIndexes.NODEGROUP_DATASET, MetadataPrimaryIndexes.FUNCTION_DATASET,
- MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET };
+ MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET };
secondaryIndexes = new IMetadataIndex[] { MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX,
MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX,
MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX };
@@ -333,7 +332,6 @@
int[] bloomFilterKeyFields = index.getBloomFilterKeyFields();
LSMBTree lsmBtree = null;
long resourceID = -1;
- AsterixRuntimeComponentsProvider rtcProvider = AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
ILSMOperationTracker opTracker = index.isPrimaryIndex() ? runtimeContext.getLSMBTreeOperationTracker(index
.getDatasetId().getId()) : new BaseOperationTracker((DatasetLifecycleManager) indexLifecycleManager,
index.getDatasetId().getId());
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 100ec40..8bdd92b 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
@@ -41,11 +41,9 @@
public static final int NODEGROUP_DATASET_ID = 6;
public static final int FUNCTION_DATASET_ID = 7;
public static final int DATASOURCE_ADAPTER_DATASET_ID = 8;
- public static final int EXTERNAL_FILE_DATASET_ID = 9;
public static final int FIRST_AVAILABLE_USER_DATASET_ID = 100;
public static IMetadataIndex DATASOURCE_ADAPTER_DATASET;
- public static IMetadataIndex EXTERNAL_FILE_DATASET;
/**
* Create all metadata primary index descriptors. MetadataRecordTypes must
@@ -94,11 +92,5 @@
BuiltinType.ASTRING, BuiltinType.ASTRING }, new String[] { "DataverseName", "Name" }, 0,
MetadataRecordTypes.DATASOURCE_ADAPTER_RECORDTYPE, DATASOURCE_ADAPTER_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 11f9c91..50681ee 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
@@ -47,7 +47,6 @@
public static ARecordType NODEGROUP_RECORDTYPE;
public static ARecordType FUNCTION_RECORDTYPE;
public static ARecordType DATASOURCE_ADAPTER_RECORDTYPE;
- public static ARecordType EXTERNAL_FILE_RECORDTYPE;
/**
* Create all metadata record types.
@@ -77,7 +76,6 @@
NODEGROUP_RECORDTYPE = createNodeGroupRecordType();
FUNCTION_RECORDTYPE = createFunctionRecordType();
DATASOURCE_ADAPTER_RECORDTYPE = createDatasourceAdapterRecordType();
- EXTERNAL_FILE_RECORDTYPE = createExternalFileRecordType();
} catch (AsterixException e) {
throw new MetadataException(e);
}
@@ -127,13 +125,12 @@
// 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;
-
+
private static final ARecordType createExternalDetailsRecordType() throws AsterixException {
AOrderedListType orderedPropertyListType = new AOrderedListType(DATASOURCE_ADAPTER_PROPERTIES_RECORDTYPE, null);
- String[] fieldNames = { "DatasourceAdapter", "Properties", "GroupName" };
- IAType[] fieldTypes = { BuiltinType.ASTRING, orderedPropertyListType, BuiltinType.ASTRING };
+ String[] fieldNames = { "DatasourceAdapter", "Properties" };
+ IAType[] fieldTypes = { BuiltinType.ASTRING, orderedPropertyListType };
return new ARecordType(null, fieldNames, fieldTypes, true);
}
@@ -359,19 +356,5 @@
BuiltinType.ASTRING };
return new ARecordType("DatasourceAdapterRecordType", 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;
-
- private static ARecordType createExternalFileRecordType() throws AsterixException {
- String[] fieldNames = { "DataverseName", "DatasetName", "FileNumber", "FileName", "FileSize", "FileModDate"};
- IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.AINT32, BuiltinType.ASTRING, BuiltinType.AINT64,
- BuiltinType.ADATETIME};
- 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 de94045..4909e21 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
@@ -18,14 +18,11 @@
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
-import java.util.Date;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.logging.Logger;
-import org.apache.hadoop.fs.FileStatus;
-
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;
@@ -47,17 +44,13 @@
import edu.uci.ics.asterix.external.adapter.factory.IAdapterFactory;
import edu.uci.ics.asterix.external.adapter.factory.IGenericDatasetAdapterFactory;
import edu.uci.ics.asterix.external.adapter.factory.ITypedDatasetAdapterFactory;
-import edu.uci.ics.asterix.external.data.operator.ExternalDataAccessByRIDOperatorDescriptor;
-import edu.uci.ics.asterix.external.data.operator.ExternalDataIndexingOperatorDescriptor;
import edu.uci.ics.asterix.external.data.operator.ExternalDataScanOperatorDescriptor;
import edu.uci.ics.asterix.external.data.operator.FeedIntakeOperatorDescriptor;
import edu.uci.ics.asterix.external.data.operator.FeedMessageOperatorDescriptor;
-import edu.uci.ics.asterix.external.dataset.adapter.AbstractDatasourceAdapter;
import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
import edu.uci.ics.asterix.external.dataset.adapter.ITypedDatasourceAdapter;
import edu.uci.ics.asterix.external.feed.lifecycle.FeedId;
import edu.uci.ics.asterix.external.feed.lifecycle.IFeedMessage;
-import edu.uci.ics.asterix.external.util.ExternalDataFilesMetadataProvider;
import edu.uci.ics.asterix.formats.base.IDataFormat;
import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
@@ -71,7 +64,6 @@
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.FeedDatasetDetails;
import edu.uci.ics.asterix.metadata.entities.Index;
import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
@@ -168,7 +160,6 @@
private boolean asyncResults;
private ResultSetId resultSetId;
private IResultSerializerFactoryProvider resultSerializerFactoryProvider;
- private static boolean optimizeExternalIndexes = false;
private final Dataverse defaultDataverse;
private JobId jobId;
@@ -178,6 +169,22 @@
private static final Map<String, String> adapterFactoryMapping = initializeAdapterFactoryMapping();
private static Scheduler hdfsScheduler;
+ public String getPropertyValue(String propertyName) {
+ return config.get(propertyName);
+ }
+
+ public void setConfig(Map<String, String> config) {
+ this.config = config;
+ }
+
+ public Map<String, String[]> getAllStores() {
+ return stores;
+ }
+
+ public Map<String, String> getConfig() {
+ return config;
+ }
+
public AqlMetadataProvider(Dataverse defaultDataverse) {
this.defaultDataverse = defaultDataverse;
this.stores = AsterixAppContextInfo.getInstance().getMetadataProperties().getStores();
@@ -258,30 +265,6 @@
return resultSerializerFactoryProvider;
}
- public String getPropertyValue(String propertyName) {
- return config.get(propertyName);
- }
-
- public void setConfig(Map<String, String> config) {
- this.config = config;
- }
-
- public Map<String, String[]> getAllStores() {
- return stores;
- }
-
- public Map<String, String> getConfig() {
- return config;
- }
-
- public static boolean isOptimizeExternalIndexes() {
- return optimizeExternalIndexes;
- }
-
- public static void setOptimizeExternalIndexes(boolean optimizeExternalIndexes) {
- AqlMetadataProvider.optimizeExternalIndexes = optimizeExternalIndexes;
- }
-
@Override
public AqlDataSource findDataSource(AqlSourceId id) throws AlgebricksException {
AqlSourceId aqlId = (AqlSourceId) id;
@@ -421,206 +404,6 @@
}
@SuppressWarnings("rawtypes")
- public Pair<ExternalDataIndexingOperatorDescriptor, AlgebricksPartitionConstraint> buildExternalDataIndexingRuntime(
- JobSpecification jobSpec, IAType itemType, Dataset dataset, IDataFormat format) throws AlgebricksException {
- IGenericDatasetAdapterFactory adapterFactory;
- IDatasourceAdapter adapter;
- String adapterName;
- DatasourceAdapter adapterEntity;
- String adapterFactoryClassname;
- ExternalDatasetDetails datasetDetails = null;
- try {
- datasetDetails = (ExternalDatasetDetails) dataset.getDatasetDetails();
- adapterName = datasetDetails.getAdapter();
- adapterEntity = MetadataManager.INSTANCE.getAdapter(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME,
- adapterName);
- if (adapterEntity != null) {
- adapterFactoryClassname = adapterEntity.getClassname();
- adapterFactory = (IGenericDatasetAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
- } else {
- adapterFactoryClassname = adapterFactoryMapping.get(adapterName);
- if (adapterFactoryClassname == null) {
- throw new AlgebricksException(" Unknown adapter :" + adapterName);
- }
- adapterFactory = (IGenericDatasetAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
- }
-
- adapter = ((IGenericDatasetAdapterFactory) adapterFactory).createIndexingAdapter(
- wrapProperties(datasetDetails.getProperties()), itemType, null);
- } catch (AlgebricksException ae) {
- throw ae;
- } catch (Exception e) {
- e.printStackTrace();
- throw new AlgebricksException("Unable to create adapter " + e);
- }
- if (!(adapter.getAdapterType().equals(IDatasourceAdapter.AdapterType.READ) || adapter.getAdapterType().equals(
- IDatasourceAdapter.AdapterType.READ_WRITE))) {
- throw new AlgebricksException("external dataset adapter does not support read operation");
- }
- ARecordType rt = (ARecordType) itemType;
- ISerializerDeserializer payloadSerde = format.getSerdeProvider().getSerializerDeserializer(itemType);
- RecordDescriptor indexerDesc = new RecordDescriptor(new ISerializerDeserializer[] { payloadSerde });
- ExternalDataIndexingOperatorDescriptor dataIndexScanner = null;
- List<ExternalFile> files = null;
- HashMap<String, Integer> filesNumbers = null;
- if (optimizeExternalIndexes) {
- try {
- files = MetadataManager.INSTANCE.getDatasetExternalFiles(mdTxnCtx, dataset);
- } catch (MetadataException e) {
- e.printStackTrace();
- throw new AlgebricksException("Unable to get list of external files from metadata " + e);
- }
-
- filesNumbers = new HashMap<String, Integer>();
- for (int i = 0; i < files.size(); i++) {
- filesNumbers.put(files.get(i).getFileName(), files.get(i).getFileNumber());
- }
-
- dataIndexScanner = new ExternalDataIndexingOperatorDescriptor(jobSpec,
- wrapPropertiesEmpty(datasetDetails.getProperties()), rt, indexerDesc, adapterFactory, filesNumbers);
- } else {
- dataIndexScanner = new ExternalDataIndexingOperatorDescriptor(jobSpec,
- wrapPropertiesEmpty(datasetDetails.getProperties()), rt, indexerDesc, adapterFactory, filesNumbers);
- }
- AlgebricksPartitionConstraint constraint;
- try {
- constraint = adapter.getPartitionConstraint();
- } catch (Exception e) {
- throw new AlgebricksException(e);
- }
- return new Pair<ExternalDataIndexingOperatorDescriptor, AlgebricksPartitionConstraint>(dataIndexScanner,
- constraint);
- }
-
- public ArrayList<ExternalFile> getExternalDatasetFiles(Dataset dataset) throws AlgebricksException {
- ArrayList<ExternalFile> files = new ArrayList<ExternalFile>();
- if (dataset.getDatasetType() != DatasetType.EXTERNAL) {
- throw new AlgebricksException("Can only get external dataset files");
- }
- ExternalDatasetDetails datasetDetails = (ExternalDatasetDetails) dataset.getDatasetDetails();
- IGenericDatasetAdapterFactory adapterFactory;
- IDatasourceAdapter adapter;
- String adapterName;
- DatasourceAdapter adapterEntity;
- String adapterFactoryClassname;
- try {
- adapterName = datasetDetails.getAdapter();
- adapterEntity = MetadataManager.INSTANCE.getAdapter(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME,
- adapterName);
- if (adapterEntity != null) {
- adapterFactoryClassname = adapterEntity.getClassname();
- adapterFactory = (IGenericDatasetAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
- } else {
- adapterFactoryClassname = adapterFactoryMapping.get(adapterName);
- if (adapterFactoryClassname == null) {
- throw new AlgebricksException(" Unknown adapter :" + adapterName);
- }
- adapterFactory = (IGenericDatasetAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
- }
-
- adapter = ((IGenericDatasetAdapterFactory) adapterFactory).createAdapter(
- wrapProperties(datasetDetails.getProperties()), null);
- } catch (Exception e) {
- e.printStackTrace();
- throw new AlgebricksException("Unable to create adapter " + e);
- }
-
- try {
- ArrayList<FileStatus> fileStatuses = ExternalDataFilesMetadataProvider
- .getHDFSFileStatus((AbstractDatasourceAdapter) adapter);
- for (int i = 0; i < fileStatuses.size(); i++) {
- files.add(new ExternalFile(dataset.getDataverseName(), dataset.getDatasetName(), new Date(fileStatuses
- .get(i).getModificationTime()), fileStatuses.get(i).getLen(), fileStatuses.get(i).getPath()
- .toUri().getPath(), i));
- }
- return files;
- } catch (IOException e) {
- e.printStackTrace();
- throw new AlgebricksException("Unable to get list of HDFS files " + e);
- }
- }
-
- @SuppressWarnings("rawtypes")
- public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildExternalDataAccesByRIDRuntime(
- JobSpecification jobSpec, Dataset dataset, Index secondaryIndex) throws AlgebricksException {
- IAType itemType = null;
- try {
- itemType = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, 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.");
- }
-
- ExternalDatasetDetails datasetDetails = (ExternalDatasetDetails) dataset.getDatasetDetails();
- IGenericDatasetAdapterFactory adapterFactory;
- IDatasourceAdapter adapter;
- String adapterName;
- DatasourceAdapter adapterEntity;
- String adapterFactoryClassname;
- try {
- adapterName = datasetDetails.getAdapter();
- adapterEntity = MetadataManager.INSTANCE.getAdapter(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME,
- adapterName);
- if (adapterEntity != null) {
- adapterFactoryClassname = adapterEntity.getClassname();
- adapterFactory = (IGenericDatasetAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
- } else {
- adapterFactoryClassname = adapterFactoryMapping.get(adapterName);
- if (adapterFactoryClassname == null) {
- throw new AlgebricksException(" Unknown adapter :" + adapterName);
- }
- adapterFactory = (IGenericDatasetAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
- }
-
- adapter = ((IGenericDatasetAdapterFactory) adapterFactory).createAdapter(
- wrapProperties(datasetDetails.getProperties()), itemType);
- } catch (AlgebricksException ae) {
- throw ae;
- } catch (Exception e) {
- e.printStackTrace();
- throw new AlgebricksException("Unable to create adapter " + e);
- }
-
- if (!(adapter.getAdapterType().equals(IDatasourceAdapter.AdapterType.READ) || adapter.getAdapterType().equals(
- IDatasourceAdapter.AdapterType.READ_WRITE))) {
- throw new AlgebricksException("external dataset adapter does not support read operation");
- }
- IDataFormat format = NonTaggedDataFormat.INSTANCE;
- ISerializerDeserializer payloadSerde = format.getSerdeProvider().getSerializerDeserializer(itemType);
- RecordDescriptor outRecDesc = new RecordDescriptor(new ISerializerDeserializer[] { payloadSerde });
-
- ExternalDataAccessByRIDOperatorDescriptor dataAccessOperator = null;
- if (optimizeExternalIndexes) {
- //create the hashmap
- List<ExternalFile> files = null;
- try {
- files = MetadataManager.INSTANCE.getDatasetExternalFiles(mdTxnCtx, dataset);
- } catch (MetadataException e) {
- e.printStackTrace();
- throw new AlgebricksException("Couldn't get file names for access by optimized RIDs", e);
- }
- HashMap<Integer, String> filesMapping = new HashMap<Integer, String>();
- for (int i = 0; i < files.size(); i++) {
- filesMapping.put(files.get(i).getFileNumber(), files.get(i).getFileName());
- }
- dataAccessOperator = new ExternalDataAccessByRIDOperatorDescriptor(jobSpec,
- wrapPropertiesEmpty(datasetDetails.getProperties()), itemType, outRecDesc, adapterFactory,
- filesMapping);
- } else {
- dataAccessOperator = new ExternalDataAccessByRIDOperatorDescriptor(jobSpec,
- wrapPropertiesEmpty(datasetDetails.getProperties()), itemType, outRecDesc, adapterFactory, null);
- }
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraints = splitProviderAndPartitionConstraintsForExternalDataset(
- dataset.getDataverseName(), dataset.getDatasetName(), secondaryIndex.getIndexName());
- return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(dataAccessOperator,
- splitsAndConstraints.second);
- }
-
- @SuppressWarnings("rawtypes")
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildScannerRuntime(JobSpecification jobSpec,
IAType itemType, IParseFileSplitsDecl decl, IDataFormat format) throws AlgebricksException {
if (itemType.getTypeTag() != ATypeTag.RECORD) {
@@ -726,133 +509,85 @@
int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive, Object implConfig)
throws AlgebricksException {
boolean isSecondary = true;
- if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
- try {
- int numPrimaryKeys = DatasetUtils.getExternalRIDSize(dataset);
- RecordDescriptor outputRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
- int numKeys = numPrimaryKeys;;
- ITypeTraits[] typeTraits = null;
- int[] bloomFilterKeyFields;
+ try {
+ Index primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
+ dataset.getDatasetName(), dataset.getDatasetName());
+ if (primaryIndex != null) {
+ isSecondary = !indexName.equals(primaryIndex.getIndexName());
+ }
+ int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
+ RecordDescriptor outputRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
+ int numKeys = numPrimaryKeys;
+ int keysStartIndex = outputRecDesc.getFieldCount() - numKeys - 1;
+ ITypeTraits[] typeTraits = null;
+ int[] bloomFilterKeyFields;
+ if (isSecondary) {
Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
dataset.getDatasetName(), indexName);
int numSecondaryKeys = secondaryIndex.getKeyFieldNames().size();
numKeys += numSecondaryKeys;
- int keysStartIndex = outputVars.size() - numKeys;
+ keysStartIndex = outputVars.size() - numKeys;
typeTraits = JobGenHelper.variablesToTypeTraits(outputVars, keysStartIndex, numKeys, typeEnv, context);
bloomFilterKeyFields = new int[numSecondaryKeys];
for (int i = 0; i < numSecondaryKeys; i++) {
bloomFilterKeyFields[i] = i;
}
- IBinaryComparatorFactory[] comparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(
- outputVars, keysStartIndex, numKeys, typeEnv, context);
- IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc;
- try {
- spPc = splitProviderAndPartitionConstraintsForExternalDataset(dataset.getDataverseName(),
- dataset.getDatasetName(), indexName);
- } catch (Exception e) {
- throw new AlgebricksException(e);
+ } else {
+ typeTraits = JobGenHelper.variablesToTypeTraits(outputVars, keysStartIndex, numKeys + 1, typeEnv,
+ context);
+ bloomFilterKeyFields = new int[numPrimaryKeys];
+ for (int i = 0; i < numPrimaryKeys; i++) {
+ bloomFilterKeyFields[i] = i;
}
- ISearchOperationCallbackFactory searchCallbackFactory = null;
- searchCallbackFactory = new SecondaryIndexSearchOperationCallbackFactory();
- 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()), rtcProvider,
- isSecondary ? new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId())
- : new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
- rtcProvider, LSMBTreeIOOperationCallbackFactory.INSTANCE,
- storageProperties.getBloomFilterFalsePositiveRate()), retainInput,
- searchCallbackFactory);
- return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeSearchOp, spPc.second);
- } catch (MetadataException me) {
- throw new AlgebricksException(me);
}
- } else {
+ IBinaryComparatorFactory[] comparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(
+ outputVars, keysStartIndex, numKeys, typeEnv, context);
+
+ IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc;
try {
- Index primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
- dataset.getDatasetName(), dataset.getDatasetName());
- if (primaryIndex != null) {
- isSecondary = !indexName.equals(primaryIndex.getIndexName());
- }
- int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
- RecordDescriptor outputRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
- int numKeys = numPrimaryKeys;
- int keysStartIndex = outputRecDesc.getFieldCount() - numKeys - 1;
- ITypeTraits[] typeTraits = null;
- int[] bloomFilterKeyFields;
- if (isSecondary) {
- Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
- dataset.getDatasetName(), indexName);
- int numSecondaryKeys = secondaryIndex.getKeyFieldNames().size();
- numKeys += numSecondaryKeys;
- keysStartIndex = outputVars.size() - numKeys;
- typeTraits = JobGenHelper.variablesToTypeTraits(outputVars, keysStartIndex, numKeys, typeEnv,
- context);
- bloomFilterKeyFields = new int[numSecondaryKeys];
- for (int i = 0; i < numSecondaryKeys; i++) {
- bloomFilterKeyFields[i] = i;
- }
- } else {
- typeTraits = JobGenHelper.variablesToTypeTraits(outputVars, keysStartIndex, numKeys + 1, typeEnv,
- context);
- bloomFilterKeyFields = new int[numPrimaryKeys];
- for (int i = 0; i < numPrimaryKeys; i++) {
- bloomFilterKeyFields[i] = i;
- }
- }
- IBinaryComparatorFactory[] comparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(
- outputVars, keysStartIndex, numKeys, typeEnv, context);
-
- IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc;
- try {
- spPc = splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataset.getDataverseName(),
- dataset.getDatasetName(), indexName);
- } catch (Exception e) {
- throw new AlgebricksException(e);
- }
-
- ISearchOperationCallbackFactory searchCallbackFactory = null;
- if (isSecondary) {
- searchCallbackFactory = new SecondaryIndexSearchOperationCallbackFactory();
- } else {
- JobId jobId = ((JobEventListenerFactory) jobSpec.getJobletEventListenerFactory()).getJobId();
- int datasetId = dataset.getDatasetId();
- int[] primaryKeyFields = new int[numPrimaryKeys];
- for (int i = 0; i < numPrimaryKeys; i++) {
- primaryKeyFields[i] = i;
- }
-
- AqlMetadataImplConfig aqlMetadataImplConfig = (AqlMetadataImplConfig) implConfig;
- ITransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
- if (aqlMetadataImplConfig != null && aqlMetadataImplConfig.isInstantLock()) {
- searchCallbackFactory = new PrimaryIndexInstantSearchOperationCallbackFactory(jobId, datasetId,
- primaryKeyFields, txnSubsystemProvider, ResourceType.LSM_BTREE);
- } else {
- searchCallbackFactory = new PrimaryIndexSearchOperationCallbackFactory(jobId, datasetId,
- primaryKeyFields, txnSubsystemProvider, ResourceType.LSM_BTREE);
- }
- }
- 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()), rtcProvider,
- isSecondary ? new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId())
- : new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
- rtcProvider, LSMBTreeIOOperationCallbackFactory.INSTANCE,
- storageProperties.getBloomFilterFalsePositiveRate()), retainInput,
- searchCallbackFactory);
-
- return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeSearchOp, spPc.second);
-
- } catch (MetadataException me) {
- throw new AlgebricksException(me);
+ spPc = splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataset.getDataverseName(),
+ dataset.getDatasetName(), indexName);
+ } catch (Exception e) {
+ throw new AlgebricksException(e);
}
+
+ ISearchOperationCallbackFactory searchCallbackFactory = null;
+ if (isSecondary) {
+ searchCallbackFactory = new SecondaryIndexSearchOperationCallbackFactory();
+ } else {
+ JobId jobId = ((JobEventListenerFactory) jobSpec.getJobletEventListenerFactory()).getJobId();
+ int datasetId = dataset.getDatasetId();
+ int[] primaryKeyFields = new int[numPrimaryKeys];
+ for (int i = 0; i < numPrimaryKeys; i++) {
+ primaryKeyFields[i] = i;
+ }
+
+ AqlMetadataImplConfig aqlMetadataImplConfig = (AqlMetadataImplConfig) implConfig;
+ ITransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
+ if (aqlMetadataImplConfig != null && aqlMetadataImplConfig.isInstantLock()) {
+ searchCallbackFactory = new PrimaryIndexInstantSearchOperationCallbackFactory(jobId, datasetId,
+ primaryKeyFields, txnSubsystemProvider, ResourceType.LSM_BTREE);
+ } else {
+ searchCallbackFactory = new PrimaryIndexSearchOperationCallbackFactory(jobId, datasetId,
+ primaryKeyFields, txnSubsystemProvider, ResourceType.LSM_BTREE);
+ }
+ }
+ 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()), rtcProvider,
+ isSecondary ? new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId())
+ : new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()), rtcProvider,
+ LSMBTreeIOOperationCallbackFactory.INSTANCE,
+ storageProperties.getBloomFilterFalsePositiveRate()), retainInput, searchCallbackFactory);
+
+ return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeSearchOp, spPc.second);
+
+ } catch (MetadataException me) {
+ throw new AlgebricksException(me);
}
}
@@ -1609,17 +1344,9 @@
}
int numPartitions = 0;
- List<String> nodeGroup = null;
- if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
- ExternalDatasetDetails datasetDetails = (ExternalDatasetDetails) dataset.getDatasetDetails();
- nodeGroup = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, datasetDetails.getNodeGroupName())
- .getNodeNames();
- } else {
- InternalDatasetDetails datasetDetails = (InternalDatasetDetails) dataset.getDatasetDetails();
- nodeGroup = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, datasetDetails.getNodeGroupName())
- .getNodeNames();
- }
-
+ InternalDatasetDetails datasetDetails = (InternalDatasetDetails) dataset.getDatasetDetails();
+ List<String> nodeGroup = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, datasetDetails.getNodeGroupName())
+ .getNodeNames();
for (String nd : nodeGroup) {
numPartitions += AsterixClusterProperties.INSTANCE.getNumberOfIODevices(nd);
}
@@ -1637,12 +1364,6 @@
return splitProviderAndPartitionConstraints(splits);
}
- public Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitProviderAndPartitionConstraintsForExternalDataset(
- String dataverseName, String datasetName, String targetIdxName) throws AlgebricksException {
- FileSplit[] splits = splitsForExternalDataset(mdTxnCtx, dataverseName, datasetName, targetIdxName);
- return splitProviderAndPartitionConstraints(splits);
- }
-
public Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitProviderAndPartitionConstraintsForDataverse(
String dataverse) {
FileSplit[] splits = splitsForDataverse(mdTxnCtx, dataverse);
@@ -1733,56 +1454,6 @@
}
}
- private FileSplit[] splitsForExternalDataset(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.EXTERNAL) {
- throw new AlgebricksException("Not an external dataset");
- }
- 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 {
- int numIODevices;
- if (datasetDetails.getNodeGroupName().compareTo(MetadataConstants.METADATA_NODEGROUP_NAME) == 0) {
- numIODevices = 1;
- } else {
- numIODevices = AsterixClusterProperties.INSTANCE.getNumberOfIODevices(nd);
- }
- String[] ioDevices = AsterixClusterProperties.INSTANCE.getIODevices(nd);
- for (int j = 0; j < nodeStores.length; j++) {
- for (int k = 0; k < numIODevices; k++) {
- File f = new File(ioDevices[k] + File.separator + nodeStores[j] + File.separator
- + relPathFile);
- splitArray.add(new FileSplit(nd, new FileReference(f), k));
- }
- }
- }
- }
- 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);
- }
- }
-
private static Map<String, String> initializeAdapterFactoryMapping() {
Map<String, String> adapterFactoryMapping = new HashMap<String, String>();
adapterFactoryMapping.put("edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter",
@@ -1795,8 +1466,6 @@
"edu.uci.ics.asterix.external.dataset.adapter..RSSFeedAdapterFactory");
adapterFactoryMapping.put("edu.uci.ics.asterix.external.dataset.adapter.CNNFeedAdapter",
"edu.uci.ics.asterix.external.dataset.adapter.CNNFeedAdapterFactory");
- adapterFactoryMapping.put("edu.uci.ics.asterix.external.dataset.adapter.HiveAdapter",
- "edu.uci.ics.asterix.external.adapter.factory.HiveAdapterFactory");
return adapterFactoryMapping;
}
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 849c6bd..18cef340 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
@@ -40,25 +40,17 @@
private static final long serialVersionUID = 1L;
private final String adapter;
- private final Map<String,String> properties;
- private final String nodeGroupName;
+ private final Map<String, String> properties;
- private final static ARecordType externalRecordType = MetadataRecordTypes.EXTERNAL_DETAILS_RECORDTYPE;
+ 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, String nodeGroupName) {
+ public ExternalDatasetDetails(String adapter, Map<String, String> properties) {
this.properties = properties;
this.adapter = adapter;
- this.nodeGroupName = nodeGroupName;
- }
-
- public ExternalDatasetDetails(String adapter, Map<String,String> properties) {
- this.properties = properties;
- this.adapter = adapter;
- this.nodeGroupName = null;
}
- public String getAdapter() {
+ public String getAdapter() {
return adapter;
}
@@ -66,10 +58,6 @@
return properties;
}
- public String getNodeGroupName() {
- return nodeGroupName;
- }
-
@Override
public DatasetType getDatasetType() {
return DatasetType.EXTERNAL;
@@ -105,12 +93,6 @@
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);
try {
externalRecordBuilder.write(out, true);
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
deleted file mode 100644
index 0128783..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/ExternalFile.java
+++ /dev/null
@@ -1,110 +0,0 @@
-package edu.uci.ics.asterix.metadata.entities;
-
-import java.util.Date;
-
-import edu.uci.ics.asterix.metadata.MetadataCache;
-import edu.uci.ics.asterix.metadata.api.IMetadataEntity;
-
-public class ExternalFile implements IMetadataEntity{
-
- /**
- * A class for metadata entity externalFile
- * This class represents an external dataset file and is intended for use with external data indexes
- */
- private static final long serialVersionUID = 1L;
-
- private String dataverseName;
- private String datasetName;
- private Date lastModefiedTime;
- private long size;
- private String fileName;
- private int fileNumber;
-
-
- public ExternalFile(String dataverseName, String datasetName, Date lastModefiedTime, long size, String fileName,
- int fileNumber) {
- this.dataverseName = dataverseName;
- this.datasetName = datasetName;
- this.lastModefiedTime = lastModefiedTime;
- this.size = size;
- this.fileName = fileName;
- this.fileNumber = fileNumber;
- }
-
- 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;
- }
-
- @Override
- public Object addToCache(MetadataCache cache) {
- //return cache.addExternalFileIfNotExists(this);
- return null;
- }
-
- @Override
- public Object dropFromCache(MetadataCache cache) {
- //cache.dropExternalFile(this);
- return null;
- }
-
- @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(!fileName.equals(anotherFile.fileName))
- return false;
- return true;
- }
-
-}
\ No newline at end of file
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 d346a6c..61f856a 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
@@ -219,10 +219,7 @@
.getStringValue();
properties.put(key, value);
}
- String groupName = ((AString) datasetDetailsRecord
- .getValueByPos(MetadataRecordTypes.EXTERNAL_DETAILS_ARECORD_GROUPNAME_FIELD_INDEX ))
- .getStringValue();
- datasetDetails = new ExternalDatasetDetails(adapter, properties,groupName);
+ datasetDetails = new ExternalDatasetDetails(adapter, properties);
}
Map<String, String> hints = getDatasetHints(datasetRecord);
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
deleted file mode 100644
index 6837c72..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/ExternalFileTupleTranslator.java
+++ /dev/null
@@ -1,158 +0,0 @@
-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.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 getMetadataEntytiFromTuple(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();
- String FileName = ((AString) externalFileRecord
- .getValueByPos(MetadataRecordTypes.EXTERNAL_FILE_ARECORD_FILE_NAME_FIELD_INDEX)).getStringValue();
- int fileNumber = ((AInt32) externalFileRecord
- .getValueByPos(MetadataRecordTypes.EXTERNAL_FILE_ARECORD_FILE_NUMBER_FIELD_INDEX)).getIntegerValue();
- Date lastMoDifiedDate = new Date(((ADateTime) externalFileRecord
- .getValueByPos(MetadataRecordTypes.EXTERNAL_FILE_ARECORD_FILE_MOD_DATE_FIELD_INDEX)).getChrononTime());
- long fileSize = ((AInt64) externalFileRecord
- .getValueByPos(MetadataRecordTypes.EXTERNAL_FILE_ARECORD_FILE_SIZE_FIELD_INDEX)).getLongValue();
-
- return new ExternalFile(dataverseName, datasetName,lastMoDifiedDate,fileSize,FileName,fileNumber);
- }
-
- @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 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/utils/DatasetUtils.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/utils/DatasetUtils.java
index 95d26d9..820f277 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,12 +20,9 @@
import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
-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.InternalDatasetDetails;
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.algebricks.common.exceptions.AlgebricksException;
import edu.uci.ics.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
@@ -35,8 +32,6 @@
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
public class DatasetUtils {
- public static final String KEY_INPUT_FORMAT = "input-format";
- public static final String INPUT_FORMAT_RC = "rc-input-format";
public static IBinaryComparatorFactory[] computeKeysBinaryComparatorFactories(Dataset dataset,
ARecordType itemType, IBinaryComparatorFactoryProvider comparatorFactoryProvider)
throws AlgebricksException {
@@ -87,46 +82,6 @@
}
return bhffs;
}
-
- public static IBinaryHashFunctionFactory[] computeExternalDataKeysBinaryHashFunFactories(Dataset dataset,
- IBinaryHashFunctionFactoryProvider hashFunProvider) throws AlgebricksException {
- if (dataset.getDatasetType() != DatasetType.EXTERNAL) {
- throw new AlgebricksException("not implemented");
- }
- //get dataset details
- ExternalDatasetDetails edd = (ExternalDatasetDetails) dataset.getDatasetDetails();
- if (edd.getProperties().get(KEY_INPUT_FORMAT).trim().equals(INPUT_FORMAT_RC))
- {
- //RID: <String(File name) OR Int32(File number), Int64(Block byte location), Int32(row number)>
- IBinaryHashFunctionFactory[] bhffs = new IBinaryHashFunctionFactory[3];
- if(AqlMetadataProvider.isOptimizeExternalIndexes())
- {
- bhffs[0] = hashFunProvider.getBinaryHashFunctionFactory(BuiltinType.AINT32);
- }
- else
- {
- bhffs[0] = hashFunProvider.getBinaryHashFunctionFactory(BuiltinType.ASTRING);
- }
- bhffs[1] = hashFunProvider.getBinaryHashFunctionFactory(BuiltinType.AINT64);
- bhffs[2] = hashFunProvider.getBinaryHashFunctionFactory(BuiltinType.AINT32);
- return bhffs;
- }
- else
- {
- //RID: <String(File name) OR Int32(File number), Int64(Record byte location)>
- IBinaryHashFunctionFactory[] bhffs = new IBinaryHashFunctionFactory[2];
- if(AqlMetadataProvider.isOptimizeExternalIndexes())
- {
- bhffs[0] = hashFunProvider.getBinaryHashFunctionFactory(BuiltinType.AINT32);
- }
- else
- {
- bhffs[0] = hashFunProvider.getBinaryHashFunctionFactory(BuiltinType.ASTRING);
- }
- bhffs[1] = hashFunProvider.getBinaryHashFunctionFactory(BuiltinType.AINT64);
- return bhffs;
- }
- }
public static ITypeTraits[] computeTupleTypeTraits(Dataset dataset, ARecordType itemType)
throws AlgebricksException {
@@ -157,17 +112,6 @@
return (((InternalDatasetDetails) dataset.getDatasetDetails())).getNodeGroupName();
}
- public static int getExternalRIDSize(Dataset dataset) {
- ExternalDatasetDetails dsd = ((ExternalDatasetDetails) dataset.getDatasetDetails());
- if (dsd.getProperties().get(KEY_INPUT_FORMAT).equals(INPUT_FORMAT_RC))
- {
- return 3;
- }
- else{
- return 2;
- }
- }
-
public static int getPositionOfPartitioningKeyField(Dataset dataset, String fieldExpr) {
List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
for (int i = 0; i < partitioningKeys.size(); i++) {
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AInt64.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AInt64.java
index 00b7459..cc3fd04 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AInt64.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AInt64.java
@@ -33,10 +33,6 @@
public long getLongValue() {
return value;
}
-
- public void setValue(long value) {
- this.value = value;
- }
@Override
public IAType getType() {
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 b6bdb4b..3c8a0c4 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
@@ -243,8 +243,6 @@
public final static FunctionIdentifier INDEX_SEARCH = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
"index-search", FunctionIdentifier.VARARGS);
- public final static FunctionIdentifier EXTERNAL_ACCESS_BY_RID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
- "external-access-by-rid", FunctionIdentifier.VARARGS);
public final static FunctionIdentifier MAKE_FIELD_INDEX_HANDLE = new FunctionIdentifier(
FunctionConstants.ASTERIX_NS, "make-field-index-handle", 2);
@@ -674,14 +672,6 @@
return BuiltinType.ANY; // TODO
}
});
- addPrivateFunction(EXTERNAL_ACCESS_BY_RID, new IResultTypeComputer() {
-
- @Override
- public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
- IMetadataProvider<?, ?> mp) throws AlgebricksException {
- return BuiltinType.ANY;
- }
- });
addFunction(INT8_CONSTRUCTOR, OptionalAInt8TypeComputer.INSTANCE);
addFunction(INT16_CONSTRUCTOR, OptionalAInt16TypeComputer.INSTANCE);
addFunction(INT32_CONSTRUCTOR, OptionalAInt32TypeComputer.INSTANCE);
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/job/listener/JobEventListenerFactory.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/job/listener/JobEventListenerFactory.java
index b44755c..4f4eba2 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/job/listener/JobEventListenerFactory.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/job/listener/JobEventListenerFactory.java
@@ -49,7 +49,7 @@
try {
ITransactionManager txnManager = ((IAsterixAppRuntimeContext) jobletContext.getApplicationContext()
.getApplicationObject()).getTransactionSubsystem().getTransactionManager();
- ITransactionContext txnContext = txnManager.getTransactionContext(jobId);
+ ITransactionContext txnContext = txnManager.getTransactionContext(jobId, false);
txnContext.setWriteTxn(transactionalWrite);
txnManager.completedTransaction(txnContext, new DatasetId(-1), -1,
!(jobStatus == JobStatus.FAILURE));
@@ -62,7 +62,7 @@
public void jobletStart() {
try {
((IAsterixAppRuntimeContext) jobletContext.getApplicationContext().getApplicationObject())
- .getTransactionSubsystem().getTransactionManager().getTransactionContext(jobId);
+ .getTransactionSubsystem().getTransactionManager().getTransactionContext(jobId, true);
} catch (ACIDException e) {
throw new Error(e);
}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ADMDataParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ADMDataParser.java
index e08c509..b8c8659 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ADMDataParser.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ADMDataParser.java
@@ -92,16 +92,6 @@
throw new AsterixException(e);
}
}
-
- @Override
- public void close() throws IOException{
- admLexer.close();
- }
-
- @Override
- public void reset(){
- admLexer.reset();
- }
protected boolean parseAdmInstance(IAType objectType, boolean datasetRec, DataOutput out) throws AsterixException,
IOException {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractControlledTupleParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractControlledTupleParser.java
deleted file mode 100644
index a4f1691..0000000
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractControlledTupleParser.java
+++ /dev/null
@@ -1,123 +0,0 @@
-package edu.uci.ics.asterix.runtime.operators.file;
-
-import java.io.DataOutput;
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.ByteBuffer;
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
-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;
-
-/**
- * An Abstract class implementation for IControlledTupleParser. It provides common
- * functionality involved in parsing data in an external format in a pipelined manner and packing
- * frames with formed tuples.
- * (DONE)
- */
-public abstract class AbstractControlledTupleParser extends ControlledTupleParser{
-
- protected ArrayTupleBuilder tb = new ArrayTupleBuilder(1);
- protected transient DataOutput dos;
- protected final FrameTupleAppender appender;
- protected final ByteBuffer frame;
- protected final ARecordType recType;
- protected final IHyracksTaskContext ctx;
- protected IDataParser parser;
-
- public AbstractControlledTupleParser(IHyracksTaskContext ctx, ARecordType recType) throws HyracksDataException {
- appender = new FrameTupleAppender(ctx.getFrameSize());
- frame = ctx.allocateFrame();
- this.recType = recType;
- this.ctx = ctx;
- dos = tb.getDataOutput();
- }
-
- public abstract IDataParser getDataParser();
-
- @Override
- public void parse(InputStream in, IFrameWriter writer)
- throws HyracksDataException {
- //This function when used works as non-pipelined parser
- //This whole parser interface needs revisiting.
- appender.reset(frame, true);
- parser = getDataParser();
- try {
- parser.initialize(in, recType, true);
- while (true) {
- tb.reset();
- if (!parser.parse(tb.getDataOutput())) {
- parser.reset();
- break;
- }
- tb.addFieldEndOffset();
- addTupleToFrame(writer);
- }
- parser.close();
- if (appender.getTupleCount() > 0) {
- FrameUtils.flushFrame(frame, writer);
- }
- } catch (Exception e) {
- throw new HyracksDataException("Failed to initialize data parser");
- }
- }
-
- @Override
- public void initialize(InputStream in) throws HyracksDataException {
- appender.reset(frame, true);
- parser = getDataParser();
- try {
- parser.initialize(in, recType, true);
-
- } catch (Exception e) {
- throw new HyracksDataException("Failed to initialize data parser");
- }
- }
-
- @Override
- public void parseNext(IFrameWriter writer) throws HyracksDataException {
- try {
- while (true) {
- tb.reset();
- if (!parser.parse(tb.getDataOutput())) {
- parser.reset();
- break;
- }
- tb.addFieldEndOffset();
- addTupleToFrame(writer);
- }
- } catch (AsterixException ae) {
- throw new HyracksDataException(ae);
- } catch (IOException ioe) {
- throw new HyracksDataException(ioe);
- }
- }
-
- @Override
- public void close(IFrameWriter writer) throws HyracksDataException {
- try{
- parser.close();
- if (appender.getTupleCount() > 0) {
- FrameUtils.flushFrame(frame, writer);
- }
- } catch (IOException ioe) {
- throw new HyracksDataException(ioe);
- }
- }
-
- 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();
- }
- }
-
- }
-
-}
\ No newline at end of file
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractTupleParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractTupleParser.java
index 78159f5..2322338 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractTupleParser.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractTupleParser.java
@@ -62,7 +62,6 @@
while (true) {
tb.reset();
if (!parser.parse(tb.getDataOutput())) {
- parser.close();
break;
}
tb.addFieldEndOffset();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AdmTupleParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AdmTupleParser.java
index 34f75e6..7cb9bb0 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AdmTupleParser.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AdmTupleParser.java
@@ -20,7 +20,7 @@
/**
* An extension of AbstractTupleParser that provides functionality for
- * parsing adm formatted input files.
+ * parsing delimited files.
*/
public class AdmTupleParser extends AbstractTupleParser {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledADMTupleParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledADMTupleParser.java
deleted file mode 100644
index aa2d9ba..0000000
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledADMTupleParser.java
+++ /dev/null
@@ -1,23 +0,0 @@
-package edu.uci.ics.asterix.runtime.operators.file;
-
-import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-
-/**
- * An extension of AbstractControlledTupleParser that provides functionality for
- * parsing Adm formatted input.
- */
-public class ControlledADMTupleParser extends AbstractControlledTupleParser{
-
- public ControlledADMTupleParser(IHyracksTaskContext ctx, ARecordType recType)
- throws HyracksDataException {
- super(ctx, recType);
- }
-
- @Override
- public IDataParser getDataParser() {
- return new ADMDataParser();
- }
-
-}
\ No newline at end of file
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledADMTupleParserFactory.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledADMTupleParserFactory.java
deleted file mode 100644
index d72366c..0000000
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledADMTupleParserFactory.java
+++ /dev/null
@@ -1,28 +0,0 @@
-package edu.uci.ics.asterix.runtime.operators.file;
-
-
-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;
-
-/**
- * A Controlled tuple parser factory for creating a tuple parser capable of parsing
- * ADM data.
- */
-public class ControlledADMTupleParserFactory implements ITupleParserFactory{
- private static final long serialVersionUID = 1L;
-
- protected ARecordType recType;
-
- public ControlledADMTupleParserFactory(ARecordType recType){
- this.recType = recType;
- }
-
- @Override
- public ITupleParser createTupleParser(IHyracksTaskContext ctx)
- throws HyracksDataException {
- return new ControlledADMTupleParser(ctx, recType);
- }
-}
\ No newline at end of file
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledDelimitedDataTupleParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledDelimitedDataTupleParser.java
deleted file mode 100644
index 15643d4..0000000
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledDelimitedDataTupleParser.java
+++ /dev/null
@@ -1,23 +0,0 @@
-package edu.uci.ics.asterix.runtime.operators.file;
-
-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.data.parsers.IValueParserFactory;
-
-public class ControlledDelimitedDataTupleParser extends AbstractControlledTupleParser{
-
- private final DelimitedDataParser dataParser;
-
- public ControlledDelimitedDataTupleParser(IHyracksTaskContext ctx,
- ARecordType recType, IValueParserFactory[] valueParserFactories, char fieldDelimter) throws HyracksDataException {
- super(ctx, recType);
- dataParser = new DelimitedDataParser(recType, valueParserFactories, fieldDelimter);
- }
-
- @Override
- public IDataParser getDataParser() {
- return dataParser;
- }
-
-}
\ No newline at end of file
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledDelimitedDataTupleParserFactory.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledDelimitedDataTupleParserFactory.java
deleted file mode 100644
index ced33ef..0000000
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledDelimitedDataTupleParserFactory.java
+++ /dev/null
@@ -1,29 +0,0 @@
-package edu.uci.ics.asterix.runtime.operators.file;
-
-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.data.parsers.IValueParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
-import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
-
-public class ControlledDelimitedDataTupleParserFactory implements ITupleParserFactory{
- private static final long serialVersionUID = 1L;
- private IValueParserFactory[] valueParserFactories;
- private char fieldDelimiter;
- protected ARecordType recordType;
-
-
- public ControlledDelimitedDataTupleParserFactory(ARecordType recordType, IValueParserFactory[] fieldParserFactories, char fieldDelimiter) {
- this.recordType = recordType;
- this.valueParserFactories = fieldParserFactories;
- this.fieldDelimiter = fieldDelimiter;
- }
-
-
- @Override
- public ITupleParser createTupleParser(IHyracksTaskContext ctx)
- throws HyracksDataException {
- return new ControlledDelimitedDataTupleParser(ctx, recordType, valueParserFactories, fieldDelimiter);
- }
-}
\ No newline at end of file
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledTupleParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledTupleParser.java
deleted file mode 100644
index 10b09f5..0000000
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledTupleParser.java
+++ /dev/null
@@ -1,29 +0,0 @@
-package edu.uci.ics.asterix.runtime.operators.file;
-
-import java.io.InputStream;
-
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
-
-/**
- * This interface is to be implemented by parsers used in a pipelined hyracks job where input is not ready all at once
- */
-public abstract class ControlledTupleParser implements ITupleParser{
-
- /**
- * This function associate an input stream with the parser
- */
- public abstract void initialize(InputStream in) throws HyracksDataException;
-
- /**
- * This function should flush the tuples setting in the frame writer buffer
- * and free all resources
- */
- public abstract void close(IFrameWriter writer) throws HyracksDataException;
-
- /**
- * This function is called when there are more data ready for parsing in the input stream
- */
- public abstract void parseNext(IFrameWriter writer) throws HyracksDataException;
-}
\ No newline at end of file
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/DelimitedDataParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/DelimitedDataParser.java
index 22c43ec..5a639dc 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/DelimitedDataParser.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/DelimitedDataParser.java
@@ -100,17 +100,6 @@
}
@Override
- public void close() {
- cursor.close();
- }
-
- @Override
- public void reset()
- {
- cursor.reset();
- }
-
- @Override
public boolean parse(DataOutput out) throws AsterixException, IOException {
while (cursor.nextRecord()) {
recBuilder.reset(recordType);
@@ -176,7 +165,7 @@
private static final int INITIAL_BUFFER_SIZE = 4096;
private static final int INCREMENT = 4096;
- private Reader in;
+ private final Reader in;
private char[] buffer;
private int start;
@@ -193,21 +182,6 @@
end = 0;
state = State.INIT;
}
-
- public void close(){
- try {
- in.close();
- } catch (IOException e) {
- // TODO Auto-generated catch block
- e.printStackTrace();
- }
- }
-
- public void reset(){
- start = 0;
- end = 0;
- state = State.INIT;
- }
public boolean nextRecord() throws IOException {
while (true) {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/IDataParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/IDataParser.java
index cd9ae2e..46d281b 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/IDataParser.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/IDataParser.java
@@ -52,17 +52,4 @@
* @throws IOException
*/
public boolean parse(DataOutput out) throws AsterixException, IOException;
-
- /**
- * Close the underlying inputStream object.
- *
- */
- public void close() throws IOException;
-
- /**
- * Reset the parser before processing a new patch of input in the input stream
- */
- public void reset();
}
-
-
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapter.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapter.java
index ffd9edd..e037ec6 100644
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapter.java
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapter.java
@@ -224,8 +224,7 @@
while (continueIngestion) {
tb.reset();
if (!parser.parse(tb.getDataOutput())) {
- parser.close();
- break;
+ break;
}
tb.addFieldEndOffset();
if (delayConfigured) {
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 c576cf8..bf3c086 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
@@ -14,17 +14,14 @@
*/
package edu.uci.ics.asterix.tools.external.data;
-import java.util.HashMap;
import java.util.Map;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.external.adapter.factory.IGenericDatasetAdapterFactory;
import edu.uci.ics.asterix.external.dataset.adapter.FileSystemBasedAdapter;
-import edu.uci.ics.asterix.external.dataset.adapter.IControlledAdapter;
import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
/**
* Factory class for creating @see{RateControllerFileSystemBasedAdapter} The
@@ -70,12 +67,6 @@
public String getName() {
return "file_feed";
}
-
- @Override
- public IDatasourceAdapter createIndexingAdapter(
- Map<String, Object> configuration, IAType atype, Map<String,Integer> files) throws Exception {
- throw new NotImplementedException("Rate Controlled Indexing Adapter is not implemented for feeds");
- }
private void checkRequiredArgs(Map<String, Object> configuration) throws Exception {
if (configuration.get(KEY_FILE_SYSTEM) == null) {
@@ -92,10 +83,4 @@
}
}
- @Override
- public IControlledAdapter createAccessByRIDAdapter(
- Map<String, Object> configuration, IAType atype, HashMap<Integer,String> files) throws Exception {
- throw new NotImplementedException("Rate Controlled Access by RID Adapter is not implemented for feeds");
- }
-
}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallbackFactory.java
index a6537cd..00f0c4a 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallbackFactory.java
@@ -41,7 +41,7 @@
throws HyracksDataException {
ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
try {
- ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
+ ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
return new PrimaryIndexInstantSearchOperationCallback(datasetId, primaryKeyFields,
txnSubsystem.getLockManager(), txnCtx);
} catch (ACIDException e) {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
index abeec62..07daee4 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
@@ -58,7 +58,7 @@
}
try {
- ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
+ ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
IModificationOperationCallback modCallback = new PrimaryIndexModificationOperationCallback(datasetId,
primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId, resourceType,
indexOp);
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java
index b59eb75..01cb725 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java
@@ -41,7 +41,7 @@
throws HyracksDataException {
ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
try {
- ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
+ ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
return new PrimaryIndexSearchOperationCallback(datasetId, primaryKeyFields, txnSubsystem.getLockManager(),
txnCtx);
} catch (ACIDException e) {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
index d5bc877..563e9b7 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
@@ -55,7 +55,7 @@
}
try {
- ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
+ ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
IModificationOperationCallback modCallback = new SecondaryIndexModificationOperationCallback(datasetId, primaryKeyFields, txnCtx,
txnSubsystem.getLockManager(), txnSubsystem, resourceId, resourceType, indexOp);
txnCtx.registerIndexAndCallback(resourceId, index, (AbstractOperationCallback) modCallback, false);
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java
index 6d86f70..c7df2f2 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java
@@ -665,8 +665,6 @@
latchLockTable();
try {
- validateJob(txnContext);
-
if (IS_DEBUG_MODE) {
trackLockRequest("Requested", RequestType.UNLOCK, datasetId, entityHashValue, (byte) 0, txnContext,
dLockInfo, eLockInfo);
@@ -2212,14 +2210,14 @@
if (logRecord.getLogType() == LogType.ENTITY_COMMIT) {
tempDatasetIdObj.setId(logRecord.getDatasetId());
tempJobIdObj.setId(logRecord.getJobId());
- txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(tempJobIdObj);
+ txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(tempJobIdObj, false);
unlock(tempDatasetIdObj, logRecord.getPKHashValue(), txnCtx);
txnCtx.notifyOptracker(false);
- } else if (logRecord.getLogType() == LogType.JOB_COMMIT) {
+ } else if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT) {
tempJobIdObj.setId(logRecord.getJobId());
- txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(tempJobIdObj);
+ txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(tempJobIdObj, false);
txnCtx.notifyOptracker(true);
- ((LogPage) logPage).notifyJobCommitter();
+ ((LogPage) logPage).notifyJobTerminator();
}
logRecord = logPageReader.next();
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
index 4f0bb59..933afcd 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
@@ -38,6 +38,7 @@
import edu.uci.ics.asterix.common.transactions.ILogReader;
import edu.uci.ics.asterix.common.transactions.ILogRecord;
import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.asterix.common.transactions.ITransactionManager;
import edu.uci.ics.asterix.common.transactions.LogManagerProperties;
import edu.uci.ics.asterix.common.transactions.MutableLong;
import edu.uci.ics.asterix.transaction.management.service.locking.LockManager;
@@ -74,16 +75,16 @@
logDir = logManagerProperties.getLogDir();
logFilePrefix = logManagerProperties.getLogFilePrefix();
flushLSN = new MutableLong();
- initializeLogManager();
+ initializeLogManager(0);
}
- private void initializeLogManager() {
+ private void initializeLogManager(long nextLogFileId) {
emptyQ = new LinkedBlockingQueue<LogPage>(numLogPages);
flushQ = new LinkedBlockingQueue<LogPage>(numLogPages);
for (int i = 0; i < numLogPages; i++) {
emptyQ.offer(new LogPage((LockManager) txnSubsystem.getLockManager(), logPageSize, flushLSN));
}
- appendLSN = initializeLogAnchor();
+ appendLSN = initializeLogAnchor(nextLogFileId);
flushLSN.set(appendLSN);
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("LogManager starts logging in LSN: " + appendLSN);
@@ -95,12 +96,13 @@
}
@Override
- public void log(ILogRecord logRecord) {
+ public void log(ILogRecord logRecord) throws ACIDException {
if (logRecord.getLogSize() > logPageSize) {
throw new IllegalStateException();
}
syncLog(logRecord);
- if (logRecord.getLogType() == LogType.JOB_COMMIT && !logRecord.isFlushed()) {
+ if ((logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT)
+ && !logRecord.isFlushed()) {
synchronized (logRecord) {
while (!logRecord.isFlushed()) {
try {
@@ -113,13 +115,16 @@
}
}
- private synchronized void syncLog(ILogRecord logRecord) {
+ private synchronized void syncLog(ILogRecord logRecord) throws ACIDException {
ITransactionContext txnCtx = logRecord.getTxnCtx();
+ if (txnCtx.getTxnState() == ITransactionManager.ABORTED && logRecord.getLogType() != LogType.ABORT) {
+ throw new ACIDException("Aborted job(" + txnCtx.getJobId() + ") tried to write non-abort type log record.");
+ }
if (getLogFileOffset(appendLSN) + logRecord.getLogSize() > logFileSize) {
prepareNextLogFile();
appendPage.isFull(true);
getAndInitNewPage();
- } else if (!appendPage.hasSpace(logRecord.getLogSize(), getLogFileOffset(appendLSN))) {
+ } else if (!appendPage.hasSpace(logRecord.getLogSize())) {
appendPage.isFull(true);
getAndInitNewPage();
}
@@ -141,7 +146,6 @@
}
appendPage.reset();
appendPage.setFileChannel(appendChannel);
- appendPage.setInitialFlushOffset(getLogFileOffset(appendLSN));
flushQ.offer(appendPage);
}
@@ -229,7 +233,7 @@
return flushLSN;
}
- private long initializeLogAnchor() {
+ private long initializeLogAnchor(long nextLogFileId) {
long fileId = 0;
long offset = 0;
File fileLogDir = new File(logDir);
@@ -237,9 +241,10 @@
if (fileLogDir.exists()) {
List<Long> logFileIds = getLogFileIds();
if (logFileIds == null) {
- createFileIfNotExists(getLogFilePath(0));
+ fileId = nextLogFileId;
+ createFileIfNotExists(getLogFilePath(fileId));
if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("created a log file: " + getLogFilePath(0));
+ LOGGER.info("created a log file: " + getLogFilePath(fileId));
}
} else {
fileId = logFileIds.get(logFileIds.size() - 1);
@@ -247,13 +252,14 @@
offset = logFile.length();
}
} else {
+ fileId = nextLogFileId;
createNewDirectory(logDir);
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("created the log directory: " + logManagerProperties.getLogDir());
}
- createFileIfNotExists(getLogFilePath(0));
+ createFileIfNotExists(getLogFilePath(fileId));
if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("created a log file: " + getLogFilePath(0));
+ LOGGER.info("created a log file: " + getLogFilePath(fileId));
}
}
} catch (IOException ioe) {
@@ -267,8 +273,8 @@
public void renewLogFiles() {
terminateLogFlusher();
- deleteAllLogFiles();
- initializeLogManager();
+ long lastMaxLogFileId = deleteAllLogFiles();
+ initializeLogManager(lastMaxLogFileId + 1);
}
private void terminateLogFlusher() {
@@ -290,7 +296,7 @@
}
}
- private void deleteAllLogFiles() {
+ private long deleteAllLogFiles() {
if (appendChannel != null) {
try {
appendChannel.close();
@@ -305,6 +311,7 @@
throw new IllegalStateException("Failed to delete a file: " + file.getAbsolutePath());
}
}
+ return logFileIds.get(logFileIds.size() - 1);
}
private List<Long> getLogFileIds() {
@@ -384,10 +391,16 @@
}
return newFileChannel;
}
+
+ public long getReadableSmallestLSN() {
+ List<Long> logFileIds = getLogFileIds();
+ return logFileIds.get(0) * logFileSize;
+ }
}
class LogFlusher implements Callable<Boolean> {
- private final static LogPage POISON_PILL = new LogPage(null, ILogRecord.JOB_COMMIT_LOG_SIZE, null);
+ private static final Logger LOGGER = Logger.getLogger(LogFlusher.class.getName());
+ private final static LogPage POISON_PILL = new LogPage(null, ILogRecord.JOB_TERMINATE_LOG_SIZE, null);
private final LogManager logMgr;//for debugging
private final LinkedBlockingQueue<LogPage> emptyQ;
private final LinkedBlockingQueue<LogPage> flushQ;
@@ -402,13 +415,13 @@
flushPage = null;
isStarted = new AtomicBoolean(false);
terminateFlag = new AtomicBoolean(false);
-
+
}
public void terminate() {
//make sure the LogFlusher thread started before terminating it.
synchronized (isStarted) {
- while(!isStarted.get()) {
+ while (!isStarted.get()) {
try {
isStarted.wait();
} catch (InterruptedException e) {
@@ -416,7 +429,7 @@
}
}
}
-
+
terminateFlag.set(true);
if (flushPage != null) {
synchronized (flushPage) {
@@ -432,24 +445,34 @@
@Override
public Boolean call() {
- synchronized(isStarted) {
+ synchronized (isStarted) {
isStarted.set(true);
isStarted.notify();
}
- while (true) {
- flushPage = null;
- try {
- flushPage = flushQ.take();
- if (flushPage == POISON_PILL || terminateFlag.get()) {
- return true;
+ try {
+ while (true) {
+ flushPage = null;
+ try {
+ flushPage = flushQ.take();
+ if (flushPage == POISON_PILL || terminateFlag.get()) {
+ return true;
+ }
+ } catch (InterruptedException e) {
+ if (flushPage == null) {
+ continue;
+ }
}
- } catch (InterruptedException e) {
- if (flushPage == null) {
- continue;
- }
+ flushPage.flush();
+ emptyQ.offer(flushPage);
}
- flushPage.flush();
- emptyQ.offer(flushPage);
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("-------------------------------------------------------------------------");
+ LOGGER.info("LogFlusher is terminating abnormally. System is in unusalbe state.");
+ LOGGER.info("-------------------------------------------------------------------------");
+ }
+ e.printStackTrace();
+ throw e;
}
}
}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPage.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPage.java
index edfec69..8921f35 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPage.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPage.java
@@ -58,7 +58,7 @@
appendOffset = 0;
flushOffset = 0;
isLastPage = false;
- syncCommitQ = new LinkedBlockingQueue<ILogRecord>(logPageSize / ILogRecord.JOB_COMMIT_LOG_SIZE);
+ syncCommitQ = new LinkedBlockingQueue<ILogRecord>(logPageSize / ILogRecord.JOB_TERMINATE_LOG_SIZE);
}
////////////////////////////////////
@@ -76,7 +76,7 @@
if (IS_DEBUG_MODE) {
LOGGER.info("append()| appendOffset: " + appendOffset);
}
- if (logRecord.getLogType() == LogType.JOB_COMMIT) {
+ if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT) {
logRecord.isFlushed(false);
syncCommitQ.offer(logRecord);
}
@@ -105,7 +105,7 @@
this.isLastPage = isLastPage;
}
- public boolean hasSpace(int logSize, long logFileOffset) {
+ public boolean hasSpace(int logSize) {
return appendOffset + logSize <= logPageSize;
}
@@ -192,7 +192,7 @@
}
}
- public void notifyJobCommitter() {
+ public void notifyJobTerminator() {
ILogRecord logRecord = null;
while (logRecord == null) {
try {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecord.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecord.java
index 4b0e1f2..dd81df7 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecord.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecord.java
@@ -34,10 +34,9 @@
* LogType(1)
* JobId(4)
* ---------------------------
- * [Header2] (16 bytes + PKValueSize) : for entity_commit and update log types
+ * [Header2] (12 bytes + PKValueSize) : for entity_commit and update log types
* DatasetId(4) //stored in dataset_dataset in Metadata Node
* PKHashValue(4)
- * PKFieldCnt(4)
* PKValueSize(4)
* PKValue(PKValueSize)
* ---------------------------
@@ -61,10 +60,10 @@
* ---------------------------
* = LogSize =
* 1) JOB_COMMIT_LOG_SIZE: 13 bytes (5 + 8)
- * 2) ENTITY_COMMIT: 29 + PKSize (5 + 16 + PKSize + 8)
- * --> ENTITY_COMMIT_LOG_BASE_SIZE = 29
- * 3) UPDATE: 64 + PKSize + New/OldValueSize (5 + 16 + PKSize + 21 + 14 + New/OldValueSize + 8)
- * --> UPDATE_LOG_BASE_SIZE = 64
+ * 2) ENTITY_COMMIT: 25 + PKSize (5 + 12 + PKSize + 8)
+ * --> ENTITY_COMMIT_LOG_BASE_SIZE = 25
+ * 3) UPDATE: 64 + PKSize + New/OldValueSize (5 + 12 + PKSize + 21 + 14 + New/OldValueSize + 8)
+ * --> UPDATE_LOG_BASE_SIZE = 60
*/
public class LogRecord implements ILogRecord {
@@ -73,7 +72,6 @@
private int jobId;
private int datasetId;
private int PKHashValue;
- private int PKFieldCnt;
private int PKValueSize;
private ITupleReference PKValue;
private long prevLSN;
@@ -90,12 +88,13 @@
private long checksum;
//------------- fields in a log record (end) --------------//
+ private int PKFieldCnt;
private static final int CHECKSUM_SIZE = 8;
private ITransactionContext txnCtx;
private long LSN;
private final AtomicBoolean isFlushed;
private final SimpleTupleWriter tupleWriter;
- private final SimpleTupleReference readPKValue;
+ private final PrimaryKeyTupleReference readPKValue;
private final SimpleTupleReference readNewValue;
private final SimpleTupleReference readOldValue;
private final CRC32 checksumGen;
@@ -104,7 +103,7 @@
public LogRecord() {
isFlushed = new AtomicBoolean(false);
tupleWriter = new SimpleTupleWriter();
- readPKValue = (SimpleTupleReference) tupleWriter.createTupleReference();
+ readPKValue = new PrimaryKeyTupleReference();
readNewValue = (SimpleTupleReference) tupleWriter.createTupleReference();
readOldValue = (SimpleTupleReference) tupleWriter.createTupleReference();
checksumGen = new CRC32();
@@ -115,10 +114,9 @@
int beginOffset = buffer.position();
buffer.put(logType);
buffer.putInt(jobId);
- if (logType != LogType.JOB_COMMIT) {
+ if (logType == LogType.UPDATE || logType == LogType.ENTITY_COMMIT) {
buffer.putInt(datasetId);
buffer.putInt(PKHashValue);
- buffer.putInt(PKFieldCnt);
if (PKValueSize <= 0) {
throw new IllegalStateException("Primary Key Size is less than or equal to 0");
}
@@ -173,13 +171,12 @@
try {
logType = buffer.get();
jobId = buffer.getInt();
- if (logType == LogType.JOB_COMMIT) {
+ if (logType == LogType.JOB_COMMIT || logType == LogType.ABORT) {
datasetId = -1;
PKHashValue = -1;
} else {
- datasetId = buffer.getInt();
+ datasetId = buffer.getInt();
PKHashValue = buffer.getInt();
- PKFieldCnt = buffer.getInt();
PKValueSize = buffer.getInt();
if (PKValueSize <= 0) {
throw new IllegalStateException("Primary Key Size is less than or equal to 0");
@@ -217,12 +214,20 @@
}
return true;
}
-
+
private ITupleReference readPKValue(ByteBuffer buffer) {
- return readTuple(buffer, readPKValue, PKFieldCnt, PKValueSize);
+ if (buffer.position() + PKValueSize > buffer.limit()) {
+ throw new BufferUnderflowException();
+ }
+ readPKValue.reset(buffer.array(), buffer.position(), PKValueSize);
+ buffer.position(buffer.position() + PKValueSize);
+ return readPKValue;
}
private ITupleReference readTuple(ByteBuffer srcBuffer, SimpleTupleReference destTuple, int fieldCnt, int size) {
+ if (srcBuffer.position() + size > srcBuffer.limit()) {
+ throw new BufferUnderflowException();
+ }
destTuple.setFieldCount(fieldCnt);
destTuple.resetByTupleOffset(srcBuffer, srcBuffer.position());
srcBuffer.position(srcBuffer.position() + size);
@@ -230,9 +235,9 @@
}
@Override
- public void formJobCommitLogRecord(ITransactionContext txnCtx) {
+ public void formJobTerminateLogRecord(ITransactionContext txnCtx, boolean isCommit) {
this.txnCtx = txnCtx;
- this.logType = LogType.JOB_COMMIT;
+ this.logType = isCommit ? LogType.JOB_COMMIT : LogType.ABORT;
this.jobId = txnCtx.getJobId().getId();
this.datasetId = -1;
this.PKHashValue = -1;
@@ -281,7 +286,8 @@
setUpdateLogSize();
break;
case LogType.JOB_COMMIT:
- logSize = JOB_COMMIT_LOG_SIZE;
+ case LogType.ABORT:
+ logSize = JOB_TERMINATE_LOG_SIZE;
break;
case LogType.ENTITY_COMMIT:
logSize = ENTITY_COMMIT_LOG_BASE_SIZE + PKValueSize;
@@ -298,7 +304,7 @@
builder.append(" LogType : ").append(LogType.toString(logType));
builder.append(" LogSize : ").append(logSize);
builder.append(" JobId : ").append(jobId);
- if (logType != LogType.JOB_COMMIT) {
+ if (logType == LogType.ENTITY_COMMIT || logType == LogType.UPDATE) {
builder.append(" DatasetId : ").append(datasetId);
builder.append(" PKHashValue : ").append(PKHashValue);
builder.append(" PKFieldCnt : ").append(PKFieldCnt);
@@ -496,17 +502,17 @@
public void setLSN(long LSN) {
this.LSN = LSN;
}
-
+
@Override
public int getPKValueSize() {
return PKValueSize;
}
-
+
@Override
public ITupleReference getPKValue() {
return PKValue;
}
-
+
@Override
public void setPKFields(int[] primaryKeyFields) {
PKFields = primaryKeyFields;
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogType.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogType.java
index 823c8d3..f9e9304 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogType.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogType.java
@@ -19,11 +19,14 @@
public static final byte UPDATE = 0;
public static final byte JOB_COMMIT = 1;
public static final byte ENTITY_COMMIT = 2;
+ public static final byte ABORT = 3;
private static final String STRING_UPDATE = "UPDATE";
private static final String STRING_JOB_COMMIT = "JOB_COMMIT";
private static final String STRING_ENTITY_COMMIT = "ENTITY_COMMIT";
+ private static final String STRING_ABORT = "ABORT";
private static final String STRING_INVALID_LOG_TYPE = "INVALID_LOG_TYPE";
+
public static String toString(byte logType) {
switch (logType) {
case LogType.UPDATE:
@@ -32,6 +35,8 @@
return STRING_JOB_COMMIT;
case LogType.ENTITY_COMMIT:
return STRING_ENTITY_COMMIT;
+ case LogType.ABORT:
+ return STRING_ABORT;
default:
return STRING_INVALID_LOG_TYPE;
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/PrimaryKeyTupleReference.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/PrimaryKeyTupleReference.java
new file mode 100644
index 0000000..d45b209
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/PrimaryKeyTupleReference.java
@@ -0,0 +1,36 @@
+package edu.uci.ics.asterix.transaction.management.service.logging;
+
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public class PrimaryKeyTupleReference implements ITupleReference {
+ private byte[] fieldData;
+ private int start;
+ private int length;
+
+ public void reset(byte[] fieldData, int start, int length) {
+ this.fieldData = fieldData;
+ this.start = start;
+ this.length = length;
+ }
+
+ @Override
+ public int getFieldCount() {
+ return 1;
+ }
+
+ @Override
+ public byte[] getFieldData(int fIdx) {
+ return fieldData;
+ }
+
+ @Override
+ public int getFieldStart(int fIdx) {
+ return start;
+ }
+
+ @Override
+ public int getFieldLength(int fIdx) {
+ return length;
+ }
+
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
index d0e9348..fcb9aa1 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
@@ -76,6 +76,7 @@
private final TransactionSubsystem txnSubsystem;
private final LogManager logMgr;
private final int checkpointHistory;
+ private final long SHARP_CHECKPOINT_LSN = -1;
/**
* A file at a known location that contains the LSN of the last log record
@@ -114,18 +115,23 @@
return state;
}
- //#. if minMCTFirstLSN is equal to -1 &&
- // checkpointLSN in the checkpoint file is equal to the lastLSN in the log file,
- // then return healthy state. Otherwise, return corrupted.
- if ((checkpointObject.getMinMCTFirstLsn() == -2 && logMgr.getAppendLSN() == 0)
- || (checkpointObject.getMinMCTFirstLsn() == -1 && checkpointObject.getCheckpointLsn() == logMgr
- .getAppendLSN())) {
+ long readableSmallestLSN = logMgr.getReadableSmallestLSN();
+ if (logMgr.getAppendLSN() == readableSmallestLSN) {
+ if (checkpointObject.getMinMCTFirstLsn() != SHARP_CHECKPOINT_LSN) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("[Warning] ---------------------------------------------------");
+ LOGGER.info("[Warning] Some(or all) of transaction log files are lost.");
+ LOGGER.info("[Warning] ---------------------------------------------------");
+ //No choice but continuing when the log files are lost.
+ }
+ }
+ state = SystemState.HEALTHY;
+ return state;
+ } else if (checkpointObject.getCheckpointLsn() == logMgr.getAppendLSN()
+ && checkpointObject.getMinMCTFirstLsn() == SHARP_CHECKPOINT_LSN) {
state = SystemState.HEALTHY;
return state;
} else {
- if (logMgr.getAppendLSN() == 0) {
- throw new IllegalStateException("Transaction log files are lost.");
- }
state = SystemState.CORRUPTED;
return state;
}
@@ -153,10 +159,11 @@
TxnId winnerEntity = null;
//#. read checkpoint file and set lowWaterMark where anaylsis and redo start
+ long readableSmallestLSN = logMgr.getReadableSmallestLSN();
CheckpointObject checkpointObject = readCheckpoint();
- long lowWaterMarkLsn = checkpointObject.getMinMCTFirstLsn();
- if (lowWaterMarkLsn == -1 || lowWaterMarkLsn == -2) {
- lowWaterMarkLsn = 0;
+ long lowWaterMarkLSN = checkpointObject.getMinMCTFirstLsn();
+ if (lowWaterMarkLSN < readableSmallestLSN) {
+ lowWaterMarkLSN = readableSmallestLSN;
}
int maxJobId = checkpointObject.getMaxJobId();
@@ -170,11 +177,11 @@
//#. set log reader to the lowWaterMarkLsn
ILogReader logReader = logMgr.getLogReader(true);
- logReader.initializeScan(lowWaterMarkLsn);
+ logReader.initializeScan(lowWaterMarkLSN);
ILogRecord logRecord = logReader.next();
while (logRecord != null) {
if (IS_DEBUG_MODE) {
- System.out.println(logRecord.getLogRecordForDisplay());
+ LOGGER.info(logRecord.getLogRecordForDisplay());
}
//update max jobId
if (logRecord.getJobId() > maxJobId) {
@@ -182,16 +189,12 @@
}
switch (logRecord.getLogType()) {
case LogType.UPDATE:
- if (IS_DEBUG_MODE) {
- updateLogCount++;
- }
+ updateLogCount++;
break;
case LogType.JOB_COMMIT:
winnerJobSet.add(Integer.valueOf(logRecord.getJobId()));
jobId2WinnerEntitiesMap.remove(Integer.valueOf(logRecord.getJobId()));
- if (IS_DEBUG_MODE) {
- jobCommitLogCount++;
- }
+ jobCommitLogCount++;
break;
case LogType.ENTITY_COMMIT:
jobId = logRecord.getJobId();
@@ -204,9 +207,10 @@
winnerEntitySet = jobId2WinnerEntitiesMap.get(Integer.valueOf(jobId));
}
winnerEntitySet.add(winnerEntity);
- if (IS_DEBUG_MODE) {
- entityCommitLogCount++;
- }
+ entityCommitLogCount++;
+ break;
+ case LogType.ABORT:
+ //ignore
break;
default:
throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
@@ -238,11 +242,11 @@
ILocalResourceRepository localResourceRepository = appRuntimeContext.getLocalResourceRepository();
//#. set log reader to the lowWaterMarkLsn again.
- logReader.initializeScan(lowWaterMarkLsn);
+ logReader.initializeScan(lowWaterMarkLSN);
logRecord = logReader.next();
while (logRecord != null) {
- if (LogManager.IS_DEBUG_MODE) {
- System.out.println(logRecord.getLogRecordForDisplay());
+ if (IS_DEBUG_MODE) {
+ LOGGER.info(logRecord.getLogRecordForDisplay());
}
LSN = logRecord.getLSN();
jobId = logRecord.getJobId();
@@ -307,15 +311,14 @@
if (LSN > maxDiskLastLsn) {
redo(logRecord);
- if (IS_DEBUG_MODE) {
- redoCount++;
- }
+ redoCount++;
}
}
break;
case LogType.JOB_COMMIT:
case LogType.ENTITY_COMMIT:
+ case LogType.ABORT:
//do nothing
break;
@@ -335,9 +338,7 @@
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("[RecoveryMgr] recovery is completed.");
- }
- if (IS_DEBUG_MODE) {
- System.out.println("[RecoveryMgr] Count: Update/EntityCommit/JobCommit/Redo = " + updateLogCount + "/"
+ LOGGER.info("[RecoveryMgr] Count: Update/EntityCommit/JobCommit/Redo = " + updateLogCount + "/"
+ entityCommitLogCount + "/" + jobCommitLogCount + "/" + redoCount);
}
}
@@ -389,7 +390,7 @@
throw new ACIDException(e);
}
}
- minMCTFirstLSN = -2;
+ minMCTFirstLSN = SHARP_CHECKPOINT_LSN;
} else {
long firstLSN;
minMCTFirstLSN = Long.MAX_VALUE;
@@ -400,7 +401,7 @@
minMCTFirstLSN = Math.min(minMCTFirstLSN, firstLSN);
}
} else {
- minMCTFirstLSN = -1;
+ minMCTFirstLSN = SHARP_CHECKPOINT_LSN;
}
}
CheckpointObject checkpointObject = new CheckpointObject(logMgr.getAppendLSN(), minMCTFirstLSN,
@@ -577,7 +578,7 @@
break;
} else {
if (IS_DEBUG_MODE) {
- System.out.println(logRecord.getLogRecordForDisplay());
+ LOGGER.info(logRecord.getLogRecordForDisplay());
}
currentLSN = logRecord.getLSN();
}
@@ -597,9 +598,9 @@
loserTxnTable.put(loserEntity, undoLSNSet);
}
undoLSNSet.add(Long.valueOf(currentLSN));
+ updateLogCount++;
if (IS_DEBUG_MODE) {
- updateLogCount++;
- System.out.println("" + Thread.currentThread().getId() + "======> update[" + currentLSN + "]:"
+ LOGGER.info("" + Thread.currentThread().getId() + "======> update[" + currentLSN + "]:"
+ tempKeyTxnId);
}
break;
@@ -608,12 +609,19 @@
throw new ACIDException("Unexpected LogType(" + logRecord.getLogType() + ") during abort.");
case LogType.ENTITY_COMMIT:
- loserTxnTable.remove(tempKeyTxnId);
- if (IS_DEBUG_MODE) {
- entityCommitLogCount++;
- System.out.println("" + Thread.currentThread().getId() + "======> entity_commit[" + currentLSN
- + "]" + tempKeyTxnId);
+ undoLSNSet = loserTxnTable.remove(tempKeyTxnId);
+ if (undoLSNSet == null) {
+ undoLSNSet = loserTxnTable.remove(tempKeyTxnId);
}
+ entityCommitLogCount++;
+ if (IS_DEBUG_MODE) {
+ LOGGER.info("" + Thread.currentThread().getId() + "======> entity_commit[" + currentLSN + "]"
+ + tempKeyTxnId);
+ }
+ break;
+
+ case LogType.ABORT:
+ //ignore
break;
default:
@@ -647,23 +655,17 @@
throw new ACIDException("IllegalState exception during abort( " + txnContext.getJobId() + ")");
}
if (IS_DEBUG_MODE) {
- System.out.println(logRecord.getLogRecordForDisplay());
+ LOGGER.info(logRecord.getLogRecordForDisplay());
}
undo(logRecord);
- if (IS_DEBUG_MODE) {
- undoCount++;
- }
+ undoCount++;
}
}
-
logReader.close();
-
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info(" undone loser transaction's effect");
- }
- if (IS_DEBUG_MODE) {
- System.out.println("UpdateLogCount/CommitLogCount/UndoCount:" + updateLogCount + "/" + entityCommitLogCount
- + "/" + undoCount);
+ LOGGER.info("UpdateLogCount/CommitLogCount/UndoCount:" + updateLogCount + "/" + entityCommitLogCount + "/"
+ + undoCount);
}
}
@@ -762,6 +764,7 @@
this.datasetId = datasetId;
this.pkHashValue = pkHashValue;
this.tupleReferencePKValue = pkValue;
+ this.pkSize = pkSize;
isByteArrayPKValue = false;
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java
index c54cb7f..4468109 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java
@@ -220,4 +220,10 @@
public LogRecord getLogRecord() {
return logRecord;
}
+
+ public void cleanupForAbort() {
+ if (primaryIndexOpTracker != null) {
+ primaryIndexOpTracker.cleanupNumActiveOperationsForAbortedJob(primaryIndexCallback);
+ }
+ }
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManager.java
index 01b38c2..07fc152 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManager.java
@@ -53,7 +53,12 @@
txnCtx.setTxnState(ITransactionManager.ABORTED);
}
try {
- txnSubsystem.getRecoveryManager().rollbackTransaction(txnCtx);
+ if (txnCtx.isWriteTxn()) {
+ LogRecord logRecord = ((TransactionContext) txnCtx).getLogRecord();
+ logRecord.formJobTerminateLogRecord(txnCtx, false);
+ txnSubsystem.getLogManager().log(logRecord);
+ txnSubsystem.getRecoveryManager().rollbackTransaction(txnCtx);
+ }
} catch (Exception ae) {
String msg = "Could not complete rollback! System is in an inconsistent state";
if (LOGGER.isLoggable(Level.SEVERE)) {
@@ -62,6 +67,7 @@
ae.printStackTrace();
throw new ACIDException(msg, ae);
} finally {
+ ((TransactionContext) txnCtx).cleanupForAbort();
txnSubsystem.getLockManager().releaseLocks(txnCtx);
transactionContextRepository.remove(txnCtx.getJobId());
}
@@ -69,20 +75,24 @@
@Override
public ITransactionContext beginTransaction(JobId jobId) throws ACIDException {
- return getTransactionContext(jobId);
+ return getTransactionContext(jobId, true);
}
@Override
- public ITransactionContext getTransactionContext(JobId jobId) throws ACIDException {
+ public ITransactionContext getTransactionContext(JobId jobId, boolean createIfNotExist) throws ACIDException {
setMaxJobId(jobId.getId());
ITransactionContext txnCtx = transactionContextRepository.get(jobId);
if (txnCtx == null) {
- synchronized (this) {
- txnCtx = transactionContextRepository.get(jobId);
- if (txnCtx == null) {
- txnCtx = new TransactionContext(jobId, txnSubsystem);
- transactionContextRepository.put(jobId, txnCtx);
+ if (createIfNotExist) {
+ synchronized (this) {
+ txnCtx = transactionContextRepository.get(jobId);
+ if (txnCtx == null) {
+ txnCtx = new TransactionContext(jobId, txnSubsystem);
+ transactionContextRepository.put(jobId, txnCtx);
+ }
}
+ } else {
+ throw new ACIDException("TransactionContext of " + jobId + " doesn't exist.");
}
}
return txnCtx;
@@ -94,7 +104,7 @@
try {
if (txnCtx.isWriteTxn()) {
LogRecord logRecord = ((TransactionContext) txnCtx).getLogRecord();
- logRecord.formJobCommitLogRecord(txnCtx);
+ logRecord.formJobTerminateLogRecord(txnCtx, true);
txnSubsystem.getLogManager().log(logRecord);
}
} catch (Exception ae) {