Introduced changes needed for external data indexing
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 ba940d6..693a19e 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalOperatorDeepCopyVisitor.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalOperatorDeepCopyVisitor.java
@@ -39,6 +39,7 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.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;
@@ -387,6 +388,13 @@
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 {
@@ -438,4 +446,5 @@
public Map<LogicalVariable, LogicalVariable> getVariableMapping() {
return outVarMapping;
}
+
}
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
new file mode 100644
index 0000000..447555e
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/ExternalDataAccessByRIDPOperator.java
@@ -0,0 +1,110 @@
+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 ab0fd79..95d29e3 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,8 +22,12 @@
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;
@@ -34,6 +38,7 @@
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;
@@ -52,6 +57,7 @@
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;
@@ -60,290 +66,384 @@
* 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 ConstantExpression createStringConstant(String str) {
- return new ConstantExpression(new AsterixConstantValue(new AString(str)));
- }
+ public static void appendExternalRecTypes(Dataset dataset, IAType itemType, List<Object> target){
+ target.add(itemType);
+ }
- public static ConstantExpression createInt32Constant(int i) {
- return new ConstantExpression(new AsterixConstantValue(new AInt32(i)));
- }
+ 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 createBooleanConstant(boolean b) {
- if (b) {
- return new ConstantExpression(new AsterixConstantValue(ABoolean.TRUE));
- } else {
- return new ConstantExpression(new AsterixConstantValue(ABoolean.FALSE));
- }
- }
+ public static ConstantExpression createStringConstant(String str) {
+ return new ConstantExpression(new AsterixConstantValue(new AString(str)));
+ }
- public static String getStringConstant(Mutable<ILogicalExpression> expr) {
- IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
- return ((AString) obj).getStringValue();
- }
+ public static ConstantExpression createInt32Constant(int i) {
+ return new ConstantExpression(new AsterixConstantValue(new AInt32(i)));
+ }
- public static int getInt32Constant(Mutable<ILogicalExpression> expr) {
- IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
- return ((AInt32) obj).getIntegerValue();
- }
+ 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 boolean getBooleanConstant(Mutable<ILogicalExpression> expr) {
- IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
- return ((ABoolean) obj).getBoolean();
- }
+ public static String getStringConstant(Mutable<ILogicalExpression> expr) {
+ IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
+ return ((AString) obj).getStringValue();
+ }
- 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 getInt32Constant(Mutable<ILogicalExpression> expr) {
+ IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
+ return ((AInt32) obj).getIntegerValue();
+ }
- 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 getBooleanConstant(Mutable<ILogicalExpression> expr) {
+ IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
+ return ((ABoolean) obj).getBoolean();
+ }
- 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 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;
+ }
- /**
- * 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 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 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());
- }
- }
+ 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 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;
- }
+ /**
+ * 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);
+ }
+ }
+ }
+ }
- /**
- * 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 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 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 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;
+ }
- 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 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 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;
- }
+ /**
+ * 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;
+ }
}
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 ddcf768..ce2a1f7 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,6 +27,7 @@
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;
@@ -51,6 +52,7 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExternalDataAccessByRIDOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
@@ -416,10 +418,16 @@
// 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 (!isPrimaryIndex) {
+ if(dataset.getDatasetType() == DatasetType.EXTERNAL)
+ {
+ ExternalDataAccessByRIDOperator externalDataAccessOp = AccessMethodUtils.createExternalDataAccessByRIDUnnestMap(dataSourceScan, dataset,
+ recordType, secondaryIndexUnnestOp, context, chosenIndex);
+ indexSubTree.dataSourceScanRef.setValue(externalDataAccessOp);
+ return externalDataAccessOp;
+ }
+ else 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 28aee7a..ea2adad 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,7 +114,8 @@
if (dataset == null) {
throw new AlgebricksException("No metadata for dataset " + datasetName);
}
- if (dataset.getDatasetType() != DatasetType.INTERNAL && dataset.getDatasetType() != DatasetType.FEED) {
+ if (dataset.getDatasetType() != DatasetType.INTERNAL && dataset.getDatasetType() != DatasetType.FEED
+ && dataset.getDatasetType() != DatasetType.EXTERNAL ) {
return false;
}
// Get the record type for that dataset.
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 e931271..2198d67 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
@@ -413,7 +413,8 @@
case EXTERNAL: {
String adapter = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl()).getAdapter();
Map<String, String> properties = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl()).getProperties();
- datasetDetails = new ExternalDatasetDetails(adapter, properties);
+ String ngName = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl()).getNodegroupName().getValue();
+ datasetDetails = new ExternalDatasetDetails(adapter, properties,ngName);
break;
}
case FEED: {
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 052368a..fc407eb 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
@@ -14,21 +14,32 @@
*/
package edu.uci.ics.asterix.file;
+
import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
+import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
+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.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;
@@ -42,86 +53,148 @@
public class SecondaryBTreeCreator extends SecondaryIndexCreator {
- protected SecondaryBTreeCreator(PhysicalOptimizationConfig physOptConf,
- IAsterixPropertiesProvider propertiesProvider) {
- super(physOptConf, propertiesProvider);
- }
+ protected SecondaryBTreeCreator(PhysicalOptimizationConfig physOptConf,
+ IAsterixPropertiesProvider propertiesProvider) {
+ super(physOptConf, propertiesProvider);
+ }
- @Override
- public JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException {
- JobSpecification spec = JobSpecificationUtils.createJobSpecification();
+ @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(
+ secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields, true,
+ dataset.getDatasetId());
+ ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
+ localResourceMetadata, LocalResource.LSMBTreeResource);
- AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
- //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
- ILocalResourceMetadata localResourceMetadata = new LSMBTreeLocalResourceMetadata(
- secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields, true,
- dataset.getDatasetId());
- ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
- localResourceMetadata, LocalResource.LSMBTreeResource);
+ TreeIndexCreateOperatorDescriptor secondaryIndexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
+ AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+ secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
+ secondaryBloomFilterKeyFields, new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
+ storageProperties.getBloomFilterFalsePositiveRate()), localResourceFactoryProvider,
+ NoOpOperationCallbackFactory.INSTANCE);
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,
+ secondaryPartitionConstraint);
+ spec.addRoot(secondaryIndexCreateOp);
+ spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+ return spec;
+ }
- TreeIndexCreateOperatorDescriptor secondaryIndexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
- AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
- secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
- secondaryBloomFilterKeyFields, new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
- dataset.getDatasetId()), AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- storageProperties.getBloomFilterFalsePositiveRate()), localResourceFactoryProvider,
- NoOpOperationCallbackFactory.INSTANCE);
- AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,
- secondaryPartitionConstraint);
- spec.addRoot(secondaryIndexCreateOp);
- spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
- return spec;
- }
+ @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);
- @Override
- public JobSpecification buildLoadingJobSpec() throws AsterixException, AlgebricksException {
- 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,RIDScanOpAndConstraints.second);
+ }
- // Create primary index scan op.
- BTreeSearchOperatorDescriptor primaryScanOp = createPrimaryIndexScanOp(spec);
+ // Sort by secondary keys.
+ ExternalSortOperatorDescriptor sortOp = createSortOp(spec, secondaryComparatorFactories, secondaryRecDesc,RIDScanOpAndConstraints.second);
+ // Create secondary BTree bulk load op.
+ AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
+ TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(
+ spec,
+ numSecondaryKeys,
+ new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER, 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;
- // 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);
- }
+ IConnectorDescriptor hashConn = new MToNPartitioningConnectorDescriptor(spec,
+ new ExternalIndexHashPartitionComputerFactory(keys, hashFactories));
- // Sort by secondary keys.
- ExternalSortOperatorDescriptor sortOp = createSortOp(spec, secondaryComparatorFactories, secondaryRecDesc);
+ 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();
- AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
- // Create secondary BTree bulk load op.
- TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(
- spec,
- numSecondaryKeys,
- new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER, storageProperties
- .getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
+ // Create dummy key provider for feeding the primary index scan.
+ AbstractOperatorDescriptor keyProviderOp = createDummyKeyProviderOp(spec);
- // 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;
- }
-}
+ // 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.LSMBTREE_SECONDARY_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER, 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;
+ }
+ }
+}
\ No newline at end of file
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 6717b4d..0d1741a 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
@@ -19,11 +19,15 @@
import java.io.IOException;
import java.util.List;
+
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.external.adapter.factory.HDFSAdapterFactory;
+import edu.uci.ics.asterix.external.adapter.factory.HiveAdapterFactory;
+import edu.uci.ics.asterix.external.data.operator.ExternalDataIndexingOperatorDescriptor;
import edu.uci.ics.asterix.formats.nontagged.AqlBinaryBooleanInspectorImpl;
import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
@@ -31,14 +35,17 @@
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.transaction.management.opcallbacks.PrimaryIndexOperationTrackerProvider;
import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
import edu.uci.ics.asterix.translator.CompiledStatements.CompiledCreateIndexStatement;
@@ -81,298 +88,537 @@
// possible index job specs,
// not just for creation.
public abstract class SecondaryIndexCreator {
- protected final PhysicalOptimizationConfig physOptConf;
+ protected final PhysicalOptimizationConfig physOptConf;
+ protected int numPrimaryKeys;
+ protected int numSecondaryKeys;
+ protected AqlMetadataProvider metadataProvider;
+ protected String dataverseName;
+ protected String datasetName;
+ protected Dataset dataset;
+ protected ARecordType itemType;
+ 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;
+ protected RecordDescriptor primaryRecDesc;
+ protected IBinaryComparatorFactory[] secondaryComparatorFactories;
+ protected int[] secondaryBloomFilterKeyFields;
+ protected RecordDescriptor secondaryRecDesc;
+ protected ICopyEvaluatorFactory[] secondaryFieldAccessEvalFactories;
+ protected IAsterixPropertiesProvider propertiesProvider;
- protected int numPrimaryKeys;
- protected int numSecondaryKeys;
- protected AqlMetadataProvider metadataProvider;
- protected String dataverseName;
- protected String datasetName;
- protected Dataset dataset;
- protected ARecordType itemType;
- protected ISerializerDeserializer payloadSerde;
- protected IFileSplitProvider primaryFileSplitProvider;
- protected AlgebricksPartitionConstraint primaryPartitionConstraint;
- protected IFileSplitProvider secondaryFileSplitProvider;
- protected AlgebricksPartitionConstraint secondaryPartitionConstraint;
- protected String secondaryIndexName;
- protected boolean anySecondaryKeyIsNullable = false;
+ // Prevent public construction. Should be created via createIndexCreator().
+ protected SecondaryIndexCreator(PhysicalOptimizationConfig physOptConf,
+ IAsterixPropertiesProvider propertiesProvider) {
+ this.physOptConf = physOptConf;
+ this.propertiesProvider = propertiesProvider;
+ }
- protected long numElementsHint;
- protected IBinaryComparatorFactory[] primaryComparatorFactories;
- protected int[] primaryBloomFilterKeyFields;
- protected RecordDescriptor primaryRecDesc;
- protected IBinaryComparatorFactory[] secondaryComparatorFactories;
- protected int[] secondaryBloomFilterKeyFields;
- protected RecordDescriptor secondaryRecDesc;
- protected ICopyEvaluatorFactory[] secondaryFieldAccessEvalFactories;
+ public static SecondaryIndexCreator createIndexCreator(CompiledCreateIndexStatement createIndexStmt,
+ AqlMetadataProvider metadataProvider, PhysicalOptimizationConfig physOptConf) throws AsterixException,
+ AlgebricksException {
+ IAsterixPropertiesProvider asterixPropertiesProvider = AsterixAppContextInfo.getInstance();
+ SecondaryIndexCreator indexCreator = null;
+ switch (createIndexStmt.getIndexType()) {
+ case BTREE: {
+ indexCreator = new SecondaryBTreeCreator(physOptConf, asterixPropertiesProvider);
+ break;
+ }
+ case RTREE: {
+ indexCreator = new SecondaryRTreeCreator(physOptConf, asterixPropertiesProvider);
+ break;
+ }
+ case SINGLE_PARTITION_WORD_INVIX:
+ case SINGLE_PARTITION_NGRAM_INVIX:
+ case LENGTH_PARTITIONED_WORD_INVIX:
+ case LENGTH_PARTITIONED_NGRAM_INVIX: {
+ indexCreator = new SecondaryInvertedIndexCreator(physOptConf, asterixPropertiesProvider);
+ break;
+ }
+ default: {
+ throw new AsterixException("Unknown Index Type: " + createIndexStmt.getIndexType());
+ }
+ }
+ indexCreator.init(createIndexStmt, metadataProvider);
+ return indexCreator;
+ }
- protected IAsterixPropertiesProvider propertiesProvider;
+ public abstract JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException;
- // Prevent public construction. Should be created via createIndexCreator().
- protected SecondaryIndexCreator(PhysicalOptimizationConfig physOptConf,
- IAsterixPropertiesProvider propertiesProvider) {
- this.physOptConf = physOptConf;
- this.propertiesProvider = propertiesProvider;
- }
+ public abstract JobSpecification buildLoadingJobSpec() throws AsterixException, AlgebricksException;
- public static SecondaryIndexCreator createIndexCreator(CompiledCreateIndexStatement createIndexStmt,
- AqlMetadataProvider metadataProvider, PhysicalOptimizationConfig physOptConf) throws AsterixException,
- AlgebricksException {
- IAsterixPropertiesProvider asterixPropertiesProvider = AsterixAppContextInfo.getInstance();
- SecondaryIndexCreator indexCreator = null;
- switch (createIndexStmt.getIndexType()) {
- case BTREE: {
- indexCreator = new SecondaryBTreeCreator(physOptConf, asterixPropertiesProvider);
- break;
- }
- case RTREE: {
- indexCreator = new SecondaryRTreeCreator(physOptConf, asterixPropertiesProvider);
- break;
- }
- case SINGLE_PARTITION_WORD_INVIX:
- case SINGLE_PARTITION_NGRAM_INVIX:
- case LENGTH_PARTITIONED_WORD_INVIX:
- case LENGTH_PARTITIONED_NGRAM_INVIX: {
- indexCreator = new SecondaryInvertedIndexCreator(physOptConf, asterixPropertiesProvider);
- break;
- }
- default: {
- throw new AsterixException("Unknown Index Type: " + createIndexStmt.getIndexType());
- }
- }
- indexCreator.init(createIndexStmt, metadataProvider);
- return indexCreator;
- }
+ protected void init(CompiledCreateIndexStatement createIndexStmt, AqlMetadataProvider metadataProvider)
+ throws AsterixException, AlgebricksException {
+ this.metadataProvider = metadataProvider;
+ dataverseName = createIndexStmt.getDataverseName() == null ? metadataProvider.getDefaultDataverseName()
+ : createIndexStmt.getDataverseName();
+ datasetName = createIndexStmt.getDatasetName();
+ secondaryIndexName = createIndexStmt.getIndexName();
+ dataset = metadataProvider.findDataset(dataverseName, datasetName);
+ if (dataset == null) {
+ 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(HiveAdapterFactory.HDFS_ADAPTER_NAME))
+ {
+ 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);
+ }
+ }
- public abstract JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException;
+ protected ARecordType createSecondaryItemType(ARecordType externalItemType, boolean isRCFile) throws AsterixException
+ {
- public abstract JobSpecification buildLoadingJobSpec() throws AsterixException, AlgebricksException;
+ String[] fieldsNames = new String[numSecondaryKeys+numPrimaryKeys];
+ IAType[] fieldsTypes = new IAType[numSecondaryKeys+numPrimaryKeys];
- protected void init(CompiledCreateIndexStatement createIndexStmt, AqlMetadataProvider metadataProvider)
- throws AsterixException, AlgebricksException {
- this.metadataProvider = metadataProvider;
- dataverseName = createIndexStmt.getDataverseName() == null ? metadataProvider.getDefaultDataverseName()
- : createIndexStmt.getDataverseName();
- datasetName = createIndexStmt.getDatasetName();
- secondaryIndexName = createIndexStmt.getIndexName();
- dataset = metadataProvider.findDataset(dataverseName, datasetName);
- if (dataset == null) {
- throw new AsterixException("Unknown dataset " + datasetName);
- }
- if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
- throw new AsterixException("Cannot index an external dataset (" + datasetName + ").");
- }
- itemType = (ARecordType) metadataProvider.findType(dataset.getDataverseName(), dataset.getItemTypeName());
- payloadSerde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType);
- numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
- numSecondaryKeys = 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);
- }
+ //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
+ e.printStackTrace();
+ }
+ }
- protected void setPrimaryRecDescAndComparators() throws AlgebricksException {
- List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
- int numPrimaryKeys = partitioningKeys.size();
- ISerializerDeserializer[] primaryRecFields = new ISerializerDeserializer[numPrimaryKeys + 1];
- ITypeTraits[] primaryTypeTraits = new ITypeTraits[numPrimaryKeys + 1];
- primaryComparatorFactories = new IBinaryComparatorFactory[numPrimaryKeys];
- primaryBloomFilterKeyFields = new int[numPrimaryKeys];
- ISerializerDeserializerProvider serdeProvider = metadataProvider.getFormat().getSerdeProvider();
- for (int i = 0; i < numPrimaryKeys; i++) {
- IAType keyType;
- try {
- keyType = itemType.getFieldType(partitioningKeys.get(i));
- } catch (IOException e) {
- throw new AlgebricksException(e);
- }
- primaryRecFields[i] = serdeProvider.getSerializerDeserializer(keyType);
- primaryComparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(
- keyType, true);
- primaryTypeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
- primaryBloomFilterKeyFields[i] = i;
- }
- primaryRecFields[numPrimaryKeys] = payloadSerde;
- primaryTypeTraits[numPrimaryKeys] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(itemType);
- primaryRecDesc = new RecordDescriptor(primaryRecFields, primaryTypeTraits);
- }
+ //second add RID fields (File name and byte location)
+ fieldsNames[numSecondaryKeys] = "_file-name";
+ fieldsTypes[numSecondaryKeys] = BuiltinType.ASTRING;
- protected void setSecondaryRecDescAndComparators(CompiledCreateIndexStatement createIndexStmt,
- AqlMetadataProvider metadataProvider) throws AlgebricksException, AsterixException {
- List<String> secondaryKeyFields = createIndexStmt.getKeyFields();
- secondaryFieldAccessEvalFactories = new ICopyEvaluatorFactory[numSecondaryKeys];
- 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), numPrimaryKeys);
- 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;
- }
- // Add serializers and comparators for primary index fields.
- for (int i = 0; i < numPrimaryKeys; i++) {
- secondaryRecFields[numSecondaryKeys + i] = primaryRecDesc.getFields()[i];
- secondaryTypeTraits[numSecondaryKeys + i] = primaryRecDesc.getTypeTraits()[i];
- secondaryComparatorFactories[numSecondaryKeys + i] = primaryComparatorFactories[i];
- }
- secondaryRecDesc = new RecordDescriptor(secondaryRecFields, secondaryTypeTraits);
- }
+ fieldsNames[numSecondaryKeys+1] = "_byte-location";
+ fieldsTypes[numSecondaryKeys+1] = BuiltinType.AINT64;
- protected AbstractOperatorDescriptor createDummyKeyProviderOp(JobSpecification spec) throws AsterixException,
- AlgebricksException {
- // Build dummy tuple containing one field with a dummy value inside.
- ArrayTupleBuilder tb = new ArrayTupleBuilder(1);
- DataOutput dos = tb.getDataOutput();
- tb.reset();
- try {
- // Serialize dummy value into a field.
- IntegerSerializerDeserializer.INSTANCE.serialize(0, dos);
- } catch (HyracksDataException e) {
- throw new AsterixException(e);
- }
- // Add dummy field.
- tb.addFieldEndOffset();
- ISerializerDeserializer[] keyRecDescSers = { IntegerSerializerDeserializer.INSTANCE };
- RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
- ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
- keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
- AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, keyProviderOp,
- primaryPartitionConstraint);
- return keyProviderOp;
- }
+ if(isRCFile)
+ {
+ fieldsNames[numSecondaryKeys+2] = "_row-Number";
+ fieldsTypes[numSecondaryKeys+2] = BuiltinType.AINT32;
+ }
- protected BTreeSearchOperatorDescriptor createPrimaryIndexScanOp(JobSpecification spec) throws AlgebricksException {
- // -Infinity
- int[] lowKeyFields = null;
- // +Infinity
- int[] highKeyFields = null;
- AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
- BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
- AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
- primaryFileSplitProvider, primaryRecDesc.getTypeTraits(), primaryComparatorFactories,
- primaryBloomFilterKeyFields, lowKeyFields, highKeyFields, true, true,
- new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
- new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER, storageProperties
- .getBloomFilterFalsePositiveRate()), false,
- NoOpOperationCallbackFactory.INSTANCE);
- AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, primarySearchOp,
- primaryPartitionConstraint);
- return primarySearchOp;
- }
+ //return type
+ return new ARecordType(externalItemType.getTypeName(), fieldsNames, fieldsTypes, externalItemType.isOpen());
+ }
+ protected ARecordType createExternalItemTypeWithRID(
+ ARecordType externalItemType) throws AsterixException {
- protected AlgebricksMetaOperatorDescriptor createAssignOp(JobSpecification spec,
- BTreeSearchOperatorDescriptor primaryScanOp, int numSecondaryKeyFields) throws AlgebricksException {
- int[] outColumns = new int[numSecondaryKeyFields];
- int[] projectionList = new int[numSecondaryKeyFields + numPrimaryKeys];
- for (int i = 0; i < numSecondaryKeyFields; i++) {
- outColumns[i] = numPrimaryKeys + i + 1;
- }
- int projCount = 0;
- for (int i = 0; i < numSecondaryKeyFields; i++) {
- projectionList[projCount++] = numPrimaryKeys + i + 1;
- }
- for (int i = 0; i < numPrimaryKeys; i++) {
- projectionList[projCount++] = 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 });
- AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, asterixAssignOp,
- primaryPartitionConstraint);
- return asterixAssignOp;
- }
+ String[] fieldsNames = new String[externalItemType.getFieldNames().length+numPrimaryKeys];
+ IAType[] fieldsTypes = new IAType[externalItemType.getFieldTypes().length+numPrimaryKeys];
- protected ExternalSortOperatorDescriptor createSortOp(JobSpecification spec,
- IBinaryComparatorFactory[] secondaryComparatorFactories, RecordDescriptor secondaryRecDesc) {
- 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, primaryPartitionConstraint);
- return sortOp;
- }
+ //add RID fields names
+ fieldsNames[0] = "_file-name";
+ fieldsNames[1] = "_byte-location";
- protected TreeIndexBulkLoadOperatorDescriptor createTreeIndexBulkLoadOp(JobSpecification spec,
- int numSecondaryKeyFields, IIndexDataflowHelperFactory dataflowHelperFactory, float fillFactor)
- throws MetadataException, AlgebricksException {
- int[] fieldPermutation = new int[numSecondaryKeyFields + numPrimaryKeys];
- for (int i = 0; i < numSecondaryKeyFields + numPrimaryKeys; i++) {
- fieldPermutation[i] = i;
- }
- TreeIndexBulkLoadOperatorDescriptor treeIndexBulkLoadOp = new TreeIndexBulkLoadOperatorDescriptor(spec,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER, secondaryFileSplitProvider,
- secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields,
- fieldPermutation, fillFactor, false, numElementsHint, false, dataflowHelperFactory,
- NoOpOperationCallbackFactory.INSTANCE);
- AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, treeIndexBulkLoadOp,
- secondaryPartitionConstraint);
- return treeIndexBulkLoadOp;
- }
+ //add RID types
+ fieldsTypes[0] = BuiltinType.ASTRING;
+ fieldsTypes[1] = BuiltinType.AINT64;
- public AlgebricksMetaOperatorDescriptor createFilterNullsSelectOp(JobSpecification spec, int numSecondaryKeyFields)
- throws AlgebricksException {
- ICopyEvaluatorFactory[] andArgsEvalFactories = new ICopyEvaluatorFactory[numSecondaryKeyFields];
- NotDescriptor notDesc = new NotDescriptor();
- IsNullDescriptor isNullDesc = new IsNullDescriptor();
- for (int i = 0; i < numSecondaryKeyFields; i++) {
- // Access column i, and apply 'is not null'.
- ColumnAccessEvalFactory columnAccessEvalFactory = new ColumnAccessEvalFactory(i);
- ICopyEvaluatorFactory isNullEvalFactory = isNullDesc
- .createEvaluatorFactory(new ICopyEvaluatorFactory[] { columnAccessEvalFactory });
- ICopyEvaluatorFactory notEvalFactory = notDesc
- .createEvaluatorFactory(new ICopyEvaluatorFactory[] { isNullEvalFactory });
- andArgsEvalFactories[i] = notEvalFactory;
- }
- ICopyEvaluatorFactory selectCond = null;
- if (numSecondaryKeyFields > 1) {
- // Create conjunctive condition where all secondary index keys must
- // satisfy 'is not null'.
- AndDescriptor andDesc = new AndDescriptor();
- selectCond = andDesc.createEvaluatorFactory(andArgsEvalFactories);
- } else {
- selectCond = andArgsEvalFactories[0];
- }
- StreamSelectRuntimeFactory select = new StreamSelectRuntimeFactory(
- new LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.ScalarEvaluatorFactoryAdapter(selectCond),
- null, AqlBinaryBooleanInspectorImpl.FACTORY);
- AlgebricksMetaOperatorDescriptor asterixSelectOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
- new IPushRuntimeFactory[] { select }, new RecordDescriptor[] { secondaryRecDesc });
- AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, asterixSelectOp,
- primaryPartitionConstraint);
- return asterixSelectOp;
- }
-}
+
+ 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();
+
+ 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 setPrimaryRecDescAndComparators() throws AlgebricksException {
+ List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+ int numPrimaryKeys = partitioningKeys.size();
+ ISerializerDeserializer[] primaryRecFields = new ISerializerDeserializer[numPrimaryKeys + 1];
+ ITypeTraits[] primaryTypeTraits = new ITypeTraits[numPrimaryKeys + 1];
+ primaryComparatorFactories = new IBinaryComparatorFactory[numPrimaryKeys];
+ primaryBloomFilterKeyFields = new int[numPrimaryKeys];
+ ISerializerDeserializerProvider serdeProvider = metadataProvider.getFormat().getSerdeProvider();
+ for (int i = 0; i < numPrimaryKeys; i++) {
+ IAType keyType;
+ try {
+ keyType = itemType.getFieldType(partitioningKeys.get(i));
+ } catch (IOException e) {
+ throw new AlgebricksException(e);
+ }
+ primaryRecFields[i] = serdeProvider.getSerializerDeserializer(keyType);
+ primaryComparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(
+ keyType, true);
+ primaryTypeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
+ primaryBloomFilterKeyFields[i] = i;
+ }
+ primaryRecFields[numPrimaryKeys] = payloadSerde;
+ primaryTypeTraits[numPrimaryKeys] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(itemType);
+ primaryRecDesc = new RecordDescriptor(primaryRecFields, primaryTypeTraits);
+ }
+
+ protected void setSecondaryRecDescAndComparators(CompiledCreateIndexStatement createIndexStmt,
+ AqlMetadataProvider metadataProvider) throws AlgebricksException, AsterixException {
+ secondaryKeyFields = createIndexStmt.getKeyFields();
+ secondaryFieldAccessEvalFactories = new ICopyEvaluatorFactory[numSecondaryKeys];
+ 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), numPrimaryKeys);
+ 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;
+ }
+ // Add serializers and comparators for primary index fields.
+ for (int i = 0; i < numPrimaryKeys; i++) {
+ secondaryRecFields[numSecondaryKeys + i] = primaryRecDesc.getFields()[i];
+ secondaryTypeTraits[numSecondaryKeys + i] = primaryRecDesc.getTypeTraits()[i];
+ secondaryComparatorFactories[numSecondaryKeys + i] = primaryComparatorFactories[i];
+ }
+ secondaryRecDesc = new RecordDescriptor(secondaryRecFields, secondaryTypeTraits);
+ }
+
+ 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;
+ }
+
+ 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 AbstractOperatorDescriptor createDummyKeyProviderOp(JobSpecification spec) throws AsterixException,
+ AlgebricksException {
+ // Build dummy tuple containing one field with a dummy value inside.
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(1);
+ DataOutput dos = tb.getDataOutput();
+ tb.reset();
+ try {
+ // Serialize dummy value into a field.
+ IntegerSerializerDeserializer.INSTANCE.serialize(0, dos);
+ } catch (HyracksDataException e) {
+ throw new AsterixException(e);
+ }
+ // Add dummy field.
+ tb.addFieldEndOffset();
+ ISerializerDeserializer[] keyRecDescSers = { IntegerSerializerDeserializer.INSTANCE };
+ RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
+ ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
+ keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, keyProviderOp,
+ primaryPartitionConstraint);
+ return keyProviderOp;
+ }
+
+ protected Pair<ExternalDataIndexingOperatorDescriptor, AlgebricksPartitionConstraint> createExternalIndexingOp(JobSpecification spec) throws Exception {
+ Pair<ExternalDataIndexingOperatorDescriptor,AlgebricksPartitionConstraint> indexingOpAndConstraints = metadataProvider.buildExternalDataIndexingRuntime(spec, itemType, (ExternalDatasetDetails)dataset.getDatasetDetails(), NonTaggedDataFormat.INSTANCE);
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, indexingOpAndConstraints.first,
+ indexingOpAndConstraints.second);
+ return indexingOpAndConstraints;
+ }
+
+ protected BTreeSearchOperatorDescriptor createPrimaryIndexScanOp(JobSpecification spec) throws AlgebricksException {
+ // -Infinity
+ int[] lowKeyFields = null;
+ // +Infinity
+ int[] highKeyFields = null;
+ AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
+ BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
+ AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+ primaryFileSplitProvider, primaryRecDesc.getTypeTraits(), primaryComparatorFactories,
+ primaryBloomFilterKeyFields, lowKeyFields, highKeyFields, true, true,
+ new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+ new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER, storageProperties
+ .getBloomFilterFalsePositiveRate()), false,
+ NoOpOperationCallbackFactory.INSTANCE);
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, primarySearchOp,
+ primaryPartitionConstraint);
+ return primarySearchOp;
+ }
+
+ protected AlgebricksMetaOperatorDescriptor createAssignOp(JobSpecification spec,
+ BTreeSearchOperatorDescriptor primaryScanOp, int numSecondaryKeyFields) throws AlgebricksException {
+ int[] outColumns = new int[numSecondaryKeyFields];
+ int[] projectionList = new int[numSecondaryKeyFields + numPrimaryKeys];
+ for (int i = 0; i < numSecondaryKeyFields; i++) {
+ outColumns[i] = numPrimaryKeys + i + 1;
+ }
+ int projCount = 0;
+ for (int i = 0; i < numSecondaryKeyFields; i++) {
+ projectionList[projCount++] = numPrimaryKeys + i + 1;
+ }
+ for (int i = 0; i < numPrimaryKeys; i++) {
+ projectionList[projCount++] = 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 });
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, asterixAssignOp,
+ primaryPartitionConstraint);
+ return asterixAssignOp;
+ }
+
+ 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) {
+ 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, primaryPartitionConstraint);
+ return sortOp;
+ }
+
+ 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 TreeIndexBulkLoadOperatorDescriptor createTreeIndexBulkLoadOp(JobSpecification spec,
+ int numSecondaryKeyFields, IIndexDataflowHelperFactory dataflowHelperFactory, float fillFactor)
+ throws MetadataException, AlgebricksException {
+ int[] fieldPermutation = new int[numSecondaryKeyFields + numPrimaryKeys];
+ for (int i = 0; i < numSecondaryKeyFields + numPrimaryKeys; i++) {
+ fieldPermutation[i] = i;
+ }
+ TreeIndexBulkLoadOperatorDescriptor treeIndexBulkLoadOp = new TreeIndexBulkLoadOperatorDescriptor(spec,
+ AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER, secondaryFileSplitProvider,
+ secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields,
+ fieldPermutation, fillFactor, false, numElementsHint, false, dataflowHelperFactory,
+ NoOpOperationCallbackFactory.INSTANCE);
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, treeIndexBulkLoadOp,
+ secondaryPartitionConstraint);
+ return treeIndexBulkLoadOp;
+ }
+
+ public AlgebricksMetaOperatorDescriptor createFilterNullsSelectOp(JobSpecification spec, int numSecondaryKeyFields,AlgebricksPartitionConstraint partitionConstraints)
+ throws AlgebricksException {
+ ICopyEvaluatorFactory[] andArgsEvalFactories = new ICopyEvaluatorFactory[numSecondaryKeyFields];
+ NotDescriptor notDesc = new NotDescriptor();
+ IsNullDescriptor isNullDesc = new IsNullDescriptor();
+ for (int i = 0; i < numSecondaryKeyFields; i++) {
+ // Access column i, and apply 'is not null'.
+ ColumnAccessEvalFactory columnAccessEvalFactory = new ColumnAccessEvalFactory(i);
+ ICopyEvaluatorFactory isNullEvalFactory = isNullDesc
+ .createEvaluatorFactory(new ICopyEvaluatorFactory[] { columnAccessEvalFactory });
+ ICopyEvaluatorFactory notEvalFactory = notDesc
+ .createEvaluatorFactory(new ICopyEvaluatorFactory[] { isNullEvalFactory });
+ andArgsEvalFactories[i] = notEvalFactory;
+ }
+ ICopyEvaluatorFactory selectCond = null;
+ if (numSecondaryKeyFields > 1) {
+ // Create conjunctive condition where all secondary index keys must
+ // satisfy 'is not null'.
+ AndDescriptor andDesc = new AndDescriptor();
+ selectCond = andDesc.createEvaluatorFactory(andArgsEvalFactories);
+ } else {
+ selectCond = andArgsEvalFactories[0];
+ }
+ StreamSelectRuntimeFactory select = new StreamSelectRuntimeFactory(
+ new LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.ScalarEvaluatorFactoryAdapter(selectCond),
+ null, AqlBinaryBooleanInspectorImpl.FACTORY);
+ AlgebricksMetaOperatorDescriptor asterixSelectOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
+ new IPushRuntimeFactory[] { select }, new RecordDescriptor[] { secondaryRecDesc });
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, asterixSelectOp,
+ partitionConstraints);
+ return asterixSelectOp;
+ }
+
+
+ public AlgebricksMetaOperatorDescriptor createFilterNullsSelectOp(JobSpecification spec, int numSecondaryKeyFields)
+ throws AlgebricksException {
+ ICopyEvaluatorFactory[] andArgsEvalFactories = new ICopyEvaluatorFactory[numSecondaryKeyFields];
+ NotDescriptor notDesc = new NotDescriptor();
+ IsNullDescriptor isNullDesc = new IsNullDescriptor();
+ for (int i = 0; i < numSecondaryKeyFields; i++) {
+ // Access column i, and apply 'is not null'.
+ ColumnAccessEvalFactory columnAccessEvalFactory = new ColumnAccessEvalFactory(i);
+ ICopyEvaluatorFactory isNullEvalFactory = isNullDesc
+ .createEvaluatorFactory(new ICopyEvaluatorFactory[] { columnAccessEvalFactory });
+ ICopyEvaluatorFactory notEvalFactory = notDesc
+ .createEvaluatorFactory(new ICopyEvaluatorFactory[] { isNullEvalFactory });
+ andArgsEvalFactories[i] = notEvalFactory;
+ }
+ ICopyEvaluatorFactory selectCond = null;
+ if (numSecondaryKeyFields > 1) {
+ // Create conjunctive condition where all secondary index keys must
+ // satisfy 'is not null'.
+ AndDescriptor andDesc = new AndDescriptor();
+ selectCond = andDesc.createEvaluatorFactory(andArgsEvalFactories);
+ } else {
+ selectCond = andArgsEvalFactories[0];
+ }
+ StreamSelectRuntimeFactory select = new StreamSelectRuntimeFactory(
+ new LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.ScalarEvaluatorFactoryAdapter(selectCond),
+ null, AqlBinaryBooleanInspectorImpl.FACTORY);
+ AlgebricksMetaOperatorDescriptor asterixSelectOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
+ new IPushRuntimeFactory[] { select }, new RecordDescriptor[] { secondaryRecDesc });
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, asterixSelectOp,
+ primaryPartitionConstraint);
+ return asterixSelectOp;
+ }
+}
\ No newline at end of file
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/ExternalDetailsDecl.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/ExternalDetailsDecl.java
index 4233225..2d97f0a 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,10 +16,20 @@
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;
}
@@ -35,4 +45,12 @@
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 cb6336b..8dd7bfb 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();
- edd.setAdapter(adapterName);
- edd.setProperties(properties);
+ ExternalDetailsDecl edd = new ExternalDetailsDecl(properties,
+ adapterName,nodeGroupName != null? new Identifier(nodeGroupName): null);
dsetDecl = new DatasetDecl(nameComponents.first,
nameComponents.second,
new Identifier(typeName),
diff --git a/asterix-external-data/pom.xml b/asterix-external-data/pom.xml
index f8d5ea2..7966b52 100644
--- a/asterix-external-data/pom.xml
+++ b/asterix-external-data/pom.xml
@@ -153,6 +153,11 @@
<artifactId>jdom</artifactId>
<version>1.0</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 4ca3d72..ccab0f4 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,12 +20,21 @@
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;
@@ -35,76 +44,182 @@
*/
@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 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";
- 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");
- 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");
+ formatClassNames.put(INPUT_FORMAT_RC, "org.apache.hadoop.hive.ql.io.RCFileInputFormat");
+ 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);
- hdfsAdapter.configure(configuration);
- return hdfsAdapter;
- }
+ setup = true;
+ }
+ JobConf conf = confFactory.getConf();
+ InputSplit[] inputSplits = inputSplitsFactory.getSplits();
+ HDFSAdapter hdfsAdapter = new HDFSAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations);
- @Override
- public String getName() {
- return HDFS_ADAPTER_NAME;
- }
+ //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));
+ }
- 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;
- }
+ hdfsAdapter.configure(configuration);
+ return hdfsAdapter;
+ }
+
+ @Override
+ public IControlledAdapter createAccessByRIDAdapter(
+ Map<String, Object> configuration, IAType atype) 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);
+ }
+ else
+ {
+ ridRecordDesc = getRIDRecDesc(false);
+ }
+ HDFSAccessByRIDAdapter adapter = new HDFSAccessByRIDAdapter(atype, ((String)configuration.get(KEY_INPUT_FORMAT)), clusterLocations,ridRecordDesc, conf);
+ adapter.configure(configuration);
+ return adapter;
+ }
+
+ @Override
+ public IDatasourceAdapter createIndexingAdapter(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();
+ //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);
+ 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){
+ int numOfPrimaryKeys = 2;
+ if(isRCFile)
+ {
+ numOfPrimaryKeys++;
+ }
+ @SuppressWarnings("rawtypes")
+ ISerializerDeserializer[] serde = new ISerializerDeserializer[numOfPrimaryKeys];
+ ITypeTraits[] tt = new ITypeTraits[numOfPrimaryKeys];
+ serde[0] = AqlSerializerDeserializerProvider.INSTANCE.getNonTaggedSerializerDeserializer(BuiltinType.ASTRING);
+ tt[0] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(BuiltinType.ASTRING);
+ serde[1] = AqlSerializerDeserializerProvider.INSTANCE.getNonTaggedSerializerDeserializer(BuiltinType.AINT64);
+ tt[1] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(BuiltinType.AINT64);
+ if(isRCFile)
+ {
+ //we add the row number for rc-files
+ serde[2] = AqlSerializerDeserializerProvider.INSTANCE.getNonTaggedSerializerDeserializer(BuiltinType.AINT32);
+ tt[2] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(BuiltinType.AINT32);
+ }
+ return new RecordDescriptor(serde, tt);
+ }
+
}
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 409eb7a..e4a1570 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,15 +18,20 @@
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;
@@ -36,109 +41,185 @@
*/
@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 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_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_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 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");
- 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");
+ formatClassNames.put(INPUT_FORMAT_RC, "org.apache.hadoop.hive.ql.io.RCFileInputFormat");
+ 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);
- clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
- int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
+ @Override
+ public IControlledAdapter createAccessByRIDAdapter(Map<String, Object> configuration, IAType atype) throws Exception {
+ Configuration conf = HDFSAdapterFactory.configureHadoopConnection(configuration);
+ clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
+ //Create RID record desc
+ RecordDescriptor ridRecordDesc = null;
- InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
- inputSplitsFactory = new InputSplitsFactory(inputSplits);
+ //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);
+ }
+ else
+ {
+ ridRecordDesc = HDFSAdapterFactory.getRIDRecDesc(false);
+ }
+ HDFSAccessByRIDAdapter adapter = new HDFSAccessByRIDAdapter(atype, ((String)configuration.get(KEY_INPUT_FORMAT)), clusterLocations,ridRecordDesc, conf);
+ adapter.configure(configuration);
+ return adapter;
+ }
- Scheduler scheduler = (Scheduler) configuration.get(SCHEDULER);
- readSchedule = scheduler.getLocationConstraints(inputSplits);
- executed = new boolean[readSchedule.length];
- Arrays.fill(executed, false);
+ @Override
+ public IDatasourceAdapter createIndexingAdapter(
+ 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);
- 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;
- }
+ clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
+ int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
- @Override
- public String getName() {
- return "hive";
- }
+ InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
+ inputSplitsFactory = new InputSplitsFactory(inputSplits);
- private JobConf configureJobConf(Map<String, Object> configuration) throws Exception {
- JobConf conf = new JobConf();
+ Scheduler scheduler = (Scheduler) configuration.get(SCHEDULER);
+ readSchedule = scheduler.getLocationConstraints(inputSplits);
+ executed = new boolean[readSchedule.length];
+ Arrays.fill(executed, false);
- /** 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");
- }
+ setup = true;
+ }
+ JobConf conf = confFactory.getConf();
+ InputSplit[] inputSplits = inputSplitsFactory.getSplits();
+ HiveIndexingAdapter hiveIndexingAdapter = new HiveIndexingAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations);
- 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");
- }
+ //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));
+ }
- /** 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;
- }
-}
+ 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
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 0a178a7..75d972e 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
@@ -16,6 +16,7 @@
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;
@@ -40,4 +41,29 @@
*/
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) 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) 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 e680232..3dc6cc8 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
@@ -16,9 +16,11 @@
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
@@ -26,18 +28,31 @@
* 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 String getName() {
+ return NC_FILE_SYSTEM_ADAPTER_NAME;
+ }
+
+ @Override
+ public IDatasourceAdapter createIndexingAdapter(
+ Map<String, Object> configuration, IAType atype) throws Exception {
+ throw new NotImplementedException("Indexing Adapter is not implemented for NC FileSystem Data");
+ }
+
+
+
+ @Override
+ public IControlledAdapter createAccessByRIDAdapter(Map<String, Object> configuration, IAType atype) throws Exception {
+ throw new NotImplementedException("Access by RID Adapter is not implemented for NC FileSystem Data");
+ }
}
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
new file mode 100644
index 0000000..36968ef
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataAccessByRIDOperatorDescriptor.java
@@ -0,0 +1,75 @@
+package edu.uci.ics.asterix.external.data.operator;
+
+import java.nio.ByteBuffer;
+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;
+
+ public ExternalDataAccessByRIDOperatorDescriptor(
+ IOperatorDescriptorRegistry spec, Map<String, Object> arguments, IAType atype,
+ RecordDescriptor outRecDesc,IGenericDatasetAdapterFactory dataSourceAdapterFactory) {
+ super(spec, 1, 1);
+ this.atype = atype;
+ this.adapterConfiguration = arguments;
+ this.datasourceAdapterFactory = dataSourceAdapterFactory;
+ this.recordDescriptors[0] = outRecDesc;
+ }
+
+ @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);
+ 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
new file mode 100644
index 0000000..436c7cc
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataIndexingOperatorDescriptor.java
@@ -0,0 +1,62 @@
+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 IAType atype;
+ private IGenericDatasetAdapterFactory datasourceAdapterFactory;
+
+ public ExternalDataIndexingOperatorDescriptor(JobSpecification spec, Map<String, Object> arguments, IAType atype,
+ RecordDescriptor rDesc, IGenericDatasetAdapterFactory dataSourceAdapterFactory) {
+ super(spec, 0, 1);
+ recordDescriptors[0] = rDesc;
+ this.adapterConfiguration = arguments;
+ this.atype = atype;
+ this.datasourceAdapterFactory = dataSourceAdapterFactory;
+ }
+
+ @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);
+ 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
new file mode 100644
index 0000000..fd846f2
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAccessByRIDAdapter.java
@@ -0,0 +1,672 @@
+/*
+ * 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.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 org.apache.log4j.Logger;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+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.BuiltinType;
+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.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.LongPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+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 transient AlgebricksPartitionConstraint clusterLocations;
+ private boolean newFrame;
+ private transient ByteBuffer frameBuffer;
+ private String inputFormat;
+ private Configuration conf;
+ private transient FileSystem fs;
+ private RecordDescriptor inRecDesc;
+
+ public HDFSAccessByRIDAdapter(IAType atype, String inputFormat, AlgebricksPartitionConstraint clusterLocations, RecordDescriptor inRecDesc, Configuration conf) {
+ super(atype);
+ this.clusterLocations = clusterLocations;
+ this.inputFormat = inputFormat;
+ this.conf = conf;
+ this.inRecDesc = inRecDesc;
+ }
+
+ @Override
+ public void configure(Map<String, Object> arguments) throws Exception {
+ this.configuration = arguments;
+ fs = FileSystem.get(conf);
+ //set up the parser factory here for now -> when everything works, make it professional
+ //The one below doesn't work for this one
+ //configureFormat();
+ 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;
+ }
+
+ //modefy the initialize function
+ //add the initial set up for the adapter
+ @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));
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public InputStream getInputStream(int partition) throws IOException {
+
+ //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");
+ }
+
+ @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 f8b381b..2b355ff 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,6 +18,7 @@
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;
@@ -26,6 +27,8 @@
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;
@@ -34,202 +37,348 @@
/**
* 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 {
- return new InputStream() {
+ if(conf.getInputFormat() instanceof RCFileInputFormat)
+ {
+ //if hdfs input format is rc-input-format, we return a different InputStream
+ 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 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;
- @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 = (Text) reader.createValue();
- return true;
- }
- }
- return false;
- }
+ /**
+ * read the split
+ */
+ reader = getRecordReader(currentSplitIndex);
+ key = reader.createKey();
+ value = 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 != null) {
- System.arraycopy(pendingValue.getBytes(), 0, buffer, offset + numBytes, pendingValue.getLength());
- buffer[offset + numBytes + pendingValue.getLength()] = (byte) EOL;
- numBytes += pendingValue.getLength() + 1;
- pendingValue = null;
- }
+ 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;
+ }
- 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;
- }
+ 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;
+ }
- @Override
- public int read() throws IOException {
- throw new NotImplementedException("Use read(byte[], int, int");
- }
+ 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 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;
- }
- }
+ 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;
+ }
- @Override
- public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
- return clusterLocations;
- }
+ };
+ }
+ 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;
+ }
}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSIndexingAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSIndexingAdapter.java
new file mode 100644
index 0000000..240621b
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSIndexingAdapter.java
@@ -0,0 +1,625 @@
+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 org.apache.log4j.Logger;
+
+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;
+
+
+/**
+ * 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;
+ public static final Logger LOGGER = Logger.getLogger(HDFSIndexingAdapter.class.getName());
+ private transient String[] readSchedule;
+ private transient boolean executed[];
+ private transient InputSplit[] inputSplits;
+ private transient JobConf conf;
+ private transient AlgebricksPartitionConstraint clusterLocations;
+
+ private transient String nodeName;
+
+ public static final byte[] fileNameFieldNameWithRecOpeningBraces = "{\"_file-name\":\"".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) {
+ 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 {
+ LOGGER.info("Configuring the adapter, why does it disappear");
+ this.configuration = arguments;
+ LOGGER.info("Configuring format");
+ 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 {
+ LOGGER.info("Creating the input stream in node: "+ nodeName);
+ //LOGGER.info("Number of input splits found = "+ inputSplits.length);
+ 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;
+ //fileName.length + byteLocation.length + rowNumberBytes.length;
+
+ //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))
+ {
+ LOGGER.info("Creating the indexing input stream with delimiter = "+ configuration.get(KEY_DELIMITER));
+ //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) {
+ 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("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 + 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 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+1, buffer, offset + numBytes,admValueSize - firstFieldLocation - 1);
+ buffer[offset + numBytes + admValueSize - firstFieldLocation] = (byte) EOL;
+ numBytes += admValueSize - firstFieldLocation;
+ }
+ }
+ }
+ 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
new file mode 100644
index 0000000..552e5ab
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HiveIndexingAdapter.java
@@ -0,0 +1,62 @@
+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) {
+ this.hdfsIndexingAdapter = new HDFSIndexingAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations);
+ 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
new file mode 100644
index 0000000..253f675
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IControlledAdapter.java
@@ -0,0 +1,17 @@
+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/ExternalIndexHashPartitionComputerFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/util/ExternalIndexHashPartitionComputerFactory.java
new file mode 100644
index 0000000..85814f5
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/util/ExternalIndexHashPartitionComputerFactory.java
@@ -0,0 +1,95 @@
+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 transient ByteBuffer serializedLong;
+ 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();
+ }
+ serializedLong = ByteBuffer.allocate(8);
+ return new ITuplePartitionComputer() {
+ 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)
+ {
+ //clear 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 dae1fb1..ac97e0b 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,215 +19,227 @@
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]
- };
-
- 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;
+ // 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]
+ };
-// ================================================================================
-// Auxiliary functions. Can parse the tokens used in the grammar as partial/auxiliary
-// ================================================================================
+ 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;
- [LEXER_AUXFUNCTIONS]
+ // ================================================================================
+ // Auxiliary functions. Can parse the tokens used in the grammar as partial/auxiliary
+ // ================================================================================
-// ================================================================================
-// 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;
+ [LEXER_AUXFUNCTIONS]
- [LEXER_LOGIC]
- }
+ // ================================================================================
+ // Main method. Return a TOKEN_CONSTANT
+ // ================================================================================
-// ================================================================================
-// Public interface
-// ================================================================================
-
- public [LEXER_NAME](java.io.Reader stream) throws IOException{
- reInit(stream);
- }
+ 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;
- 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();
- }
+ [LEXER_LOGIC]
+ }
- 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 when done with stream, must be called exiplicitly now.
+ public void close()throws IOException
+ {
+ inputStream.close();
+ }
- public void done(){
- buffer = null;
- }
+ //used before processing a new patch in the inputStream
+ public void reset(){
+ bufpos = endOf_USED_Buffer = 0; // -- -- -- reuse the buffer
+ }
-// ================================================================================
-// 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 interface
+ // ================================================================================
- 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 [LEXER_NAME](java.io.Reader stream) throws IOException{
+ reInit(stream);
+ }
- protected char readNextChar() throws IOException {
- if (++bufpos >= endOf_USED_Buffer)
- fillBuff();
- char c = buffer[bufpos];
- updateLineColumn(c);
- return c;
- }
+ 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();
+ }
- 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;
- }
+ 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 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/MetadataNode.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
index b980337..14a0695 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
@@ -42,6 +42,7 @@
import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
import edu.uci.ics.asterix.metadata.entities.Datatype;
import edu.uci.ics.asterix.metadata.entities.Dataverse;
+import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
import edu.uci.ics.asterix.metadata.entities.Function;
import edu.uci.ics.asterix.metadata.entities.Index;
import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
@@ -89,1051 +90,1072 @@
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
public class MetadataNode implements IMetadataNode {
- private static final long serialVersionUID = 1L;
+ private static final long serialVersionUID = 1L;
- private static final DatasetId METADATA_DATASET_ID = new DatasetId(MetadataPrimaryIndexes.METADATA_DATASET_ID);
+ private static final DatasetId METADATA_DATASET_ID = new DatasetId(MetadataPrimaryIndexes.METADATA_DATASET_ID);
- private IIndexLifecycleManager indexLifecycleManager;
- private ITransactionSubsystem transactionSubsystem;
+ private IIndexLifecycleManager indexLifecycleManager;
+ private ITransactionSubsystem transactionSubsystem;
- public static final MetadataNode INSTANCE = new MetadataNode();
+ public static final MetadataNode INSTANCE = new MetadataNode();
- private MetadataNode() {
- super();
- }
+ private MetadataNode() {
+ super();
+ }
- public void initialize(IAsterixAppRuntimeContext runtimeContext) {
- this.transactionSubsystem = runtimeContext.getTransactionSubsystem();
- this.indexLifecycleManager = runtimeContext.getIndexLifecycleManager();
- }
+ public void initialize(IAsterixAppRuntimeContext runtimeContext) {
+ this.transactionSubsystem = runtimeContext.getTransactionSubsystem();
+ this.indexLifecycleManager = runtimeContext.getIndexLifecycleManager();
+ }
- @Override
- public void beginTransaction(JobId transactionId) throws ACIDException, RemoteException {
- transactionSubsystem.getTransactionManager().beginTransaction(transactionId);
- }
+ @Override
+ public void beginTransaction(JobId transactionId) throws ACIDException, RemoteException {
+ transactionSubsystem.getTransactionManager().beginTransaction(transactionId);
+ }
- @Override
- public void commitTransaction(JobId jobId) throws RemoteException, ACIDException {
- ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
- transactionSubsystem.getTransactionManager().commitTransaction(txnCtx, new DatasetId(-1), -1);
- }
+ @Override
+ public void commitTransaction(JobId jobId) throws RemoteException, ACIDException {
+ ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+ transactionSubsystem.getTransactionManager().commitTransaction(txnCtx, new DatasetId(-1), -1);
+ }
- @Override
- public void abortTransaction(JobId jobId) throws RemoteException, ACIDException {
- try {
- ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
- transactionSubsystem.getTransactionManager().abortTransaction(txnCtx, new DatasetId(-1), -1);
- } catch (ACIDException e) {
- e.printStackTrace();
- throw e;
- }
- }
+ @Override
+ public void abortTransaction(JobId jobId) throws RemoteException, ACIDException {
+ try {
+ ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+ transactionSubsystem.getTransactionManager().abortTransaction(txnCtx, new DatasetId(-1), -1);
+ } catch (ACIDException e) {
+ e.printStackTrace();
+ throw e;
+ }
+ }
- @Override
- public void lock(JobId jobId, byte lockMode) throws ACIDException, RemoteException {
- ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
- transactionSubsystem.getLockManager().lock(METADATA_DATASET_ID, -1, lockMode, txnCtx);
- }
+ @Override
+ public void lock(JobId jobId, byte lockMode) throws ACIDException, RemoteException {
+ ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+ transactionSubsystem.getLockManager().lock(METADATA_DATASET_ID, -1, lockMode, txnCtx);
+ }
- @Override
- public void unlock(JobId jobId) throws ACIDException, RemoteException {
- ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
- transactionSubsystem.getLockManager().unlock(METADATA_DATASET_ID, -1, txnCtx);
- }
+ @Override
+ public void unlock(JobId jobId) throws ACIDException, RemoteException {
+ ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+ transactionSubsystem.getLockManager().unlock(METADATA_DATASET_ID, -1, txnCtx);
+ }
- @Override
- public void addDataverse(JobId jobId, Dataverse dataverse) throws MetadataException, RemoteException {
- try {
- DataverseTupleTranslator tupleReaderWriter = new DataverseTupleTranslator(true);
- ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(dataverse);
- insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, tuple);
- } catch (TreeIndexDuplicateKeyException e) {
- throw new MetadataException("A dataverse with this name " + dataverse.getDataverseName()
- + " already exists.", e);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public void addDataverse(JobId jobId, Dataverse dataverse) throws MetadataException, RemoteException {
+ try {
+ DataverseTupleTranslator tupleReaderWriter = new DataverseTupleTranslator(true);
+ ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(dataverse);
+ insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, tuple);
+ } catch (TreeIndexDuplicateKeyException e) {
+ throw new MetadataException("A dataverse with this name " + dataverse.getDataverseName()
+ + " already exists.", e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public void addDataset(JobId jobId, Dataset dataset) throws MetadataException, RemoteException {
- try {
- // Insert into the 'dataset' dataset.
- DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(true);
- ITupleReference datasetTuple = tupleReaderWriter.getTupleFromMetadataEntity(dataset);
- insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
- if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
- // Add the primary index for the dataset.
- InternalDatasetDetails id = (InternalDatasetDetails) dataset.getDatasetDetails();
- Index primaryIndex = new Index(dataset.getDataverseName(), dataset.getDatasetName(),
- dataset.getDatasetName(), IndexType.BTREE, id.getPrimaryKey(), true, dataset.getPendingOp());
+ @Override
+ public void addDataset(JobId jobId, Dataset dataset) throws MetadataException, RemoteException {
+ try {
+ // Insert into the 'dataset' dataset.
+ DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(true);
+ ITupleReference datasetTuple = tupleReaderWriter.getTupleFromMetadataEntity(dataset);
+ insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
+ if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
+ // Add the primary index for the dataset.
+ InternalDatasetDetails id = (InternalDatasetDetails) dataset.getDatasetDetails();
+ 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);
- }
- // 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);
+ }
+ 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);
+ }
+ }
- @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 {
- try {
- NodeGroupTupleTranslator tupleReaderWriter = new NodeGroupTupleTranslator(true);
- ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(nodeGroup);
- insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, tuple);
- } catch (TreeIndexDuplicateKeyException e) {
- throw new MetadataException("A nodegroup with name '" + nodeGroup.getNodeGroupName() + "' already exists.",
- e);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public void addNodeGroup(JobId jobId, NodeGroup nodeGroup) throws MetadataException, RemoteException {
+ try {
+ NodeGroupTupleTranslator tupleReaderWriter = new NodeGroupTupleTranslator(true);
+ ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(nodeGroup);
+ insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, tuple);
+ } catch (TreeIndexDuplicateKeyException e) {
+ throw new MetadataException("A nodegroup with name '" + nodeGroup.getNodeGroupName() + "' already exists.",
+ e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public void addDatatype(JobId jobId, Datatype datatype) throws MetadataException, RemoteException {
- try {
- DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(jobId, this, true);
- ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(datatype);
- insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
- } catch (TreeIndexDuplicateKeyException e) {
- throw new MetadataException("A datatype with name '" + datatype.getDatatypeName() + "' already exists.", e);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public void addDatatype(JobId jobId, Datatype datatype) throws MetadataException, RemoteException {
+ try {
+ DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(jobId, this, true);
+ ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(datatype);
+ insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
+ } catch (TreeIndexDuplicateKeyException e) {
+ throw new MetadataException("A datatype with name '" + datatype.getDatatypeName() + "' already exists.", e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public void addFunction(JobId jobId, Function function) throws MetadataException, RemoteException {
- try {
- // Insert into the 'function' dataset.
- FunctionTupleTranslator tupleReaderWriter = new FunctionTupleTranslator(true);
- ITupleReference functionTuple = tupleReaderWriter.getTupleFromMetadataEntity(function);
- insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, functionTuple);
+ @Override
+ public void addFunction(JobId jobId, Function function) throws MetadataException, RemoteException {
+ try {
+ // Insert into the 'function' dataset.
+ FunctionTupleTranslator tupleReaderWriter = new FunctionTupleTranslator(true);
+ ITupleReference functionTuple = tupleReaderWriter.getTupleFromMetadataEntity(function);
+ insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, functionTuple);
- } catch (TreeIndexDuplicateKeyException e) {
- throw new MetadataException("A function with this name " + function.getName() + " and arity "
- + function.getArity() + " already exists in dataverse '" + function.getDataverseName() + "'.", e);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ } catch (TreeIndexDuplicateKeyException e) {
+ throw new MetadataException("A function with this name " + function.getName() + " and arity "
+ + function.getArity() + " already exists in dataverse '" + function.getDataverseName() + "'.", e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- public void insertIntoDatatypeSecondaryIndex(JobId jobId, String dataverseName, String nestedTypeName,
- String topTypeName) throws Exception {
- ITupleReference tuple = createTuple(dataverseName, nestedTypeName, topTypeName);
- insertTupleIntoIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, tuple);
- }
+ public void insertIntoDatatypeSecondaryIndex(JobId jobId, String dataverseName, String nestedTypeName,
+ String topTypeName) throws Exception {
+ ITupleReference tuple = createTuple(dataverseName, nestedTypeName, topTypeName);
+ insertTupleIntoIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, tuple);
+ }
- private void insertTupleIntoIndex(JobId jobId, IMetadataIndex metadataIndex, ITupleReference tuple)
- throws Exception {
- long resourceID = metadataIndex.getResourceID();
- ILSMIndex lsmIndex = (ILSMIndex) indexLifecycleManager.getIndex(resourceID);
- indexLifecycleManager.open(resourceID);
+ private void insertTupleIntoIndex(JobId jobId, IMetadataIndex metadataIndex, ITupleReference tuple)
+ throws Exception {
+ long resourceID = metadataIndex.getResourceID();
+ ILSMIndex lsmIndex = (ILSMIndex) indexLifecycleManager.getIndex(resourceID);
+ indexLifecycleManager.open(resourceID);
- // prepare a Callback for logging
- IModificationOperationCallback modCallback = createIndexModificationCallback(jobId, resourceID, metadataIndex,
- lsmIndex, IndexOperation.INSERT);
+ // prepare a Callback for logging
+ IModificationOperationCallback modCallback = createIndexModificationCallback(jobId, resourceID, metadataIndex,
+ lsmIndex, IndexOperation.INSERT);
- IIndexAccessor indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
+ IIndexAccessor indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
- ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
- txnCtx.setTransactionType(TransactionType.READ_WRITE);
- txnCtx.registerIndexAndCallback(lsmIndex, (AbstractOperationCallback) modCallback);
+ ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+ txnCtx.setTransactionType(TransactionType.READ_WRITE);
+ txnCtx.registerIndexAndCallback(lsmIndex, (AbstractOperationCallback) modCallback);
- // TODO: fix exceptions once new BTree exception model is in hyracks.
- indexAccessor.insert(tuple);
+ // TODO: fix exceptions once new BTree exception model is in hyracks.
+ indexAccessor.insert(tuple);
- indexLifecycleManager.close(resourceID);
- }
+ indexLifecycleManager.close(resourceID);
+ }
- private IModificationOperationCallback createIndexModificationCallback(JobId jobId, long resourceId,
- IMetadataIndex metadataIndex, ILSMIndex lsmIndex, IndexOperation indexOp) throws Exception {
- ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+ private IModificationOperationCallback createIndexModificationCallback(JobId jobId, long resourceId,
+ IMetadataIndex metadataIndex, ILSMIndex lsmIndex, IndexOperation indexOp) throws Exception {
+ ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
- if (metadataIndex.isPrimaryIndex()) {
- return new PrimaryIndexModificationOperationCallback(metadataIndex.getDatasetId().getId(),
- metadataIndex.getPrimaryKeyIndexes(), txnCtx, transactionSubsystem.getLockManager(),
- transactionSubsystem, resourceId, ResourceType.LSM_BTREE, indexOp);
- } else {
- return new SecondaryIndexModificationOperationCallback(metadataIndex.getDatasetId().getId(),
- metadataIndex.getPrimaryKeyIndexes(), txnCtx, transactionSubsystem.getLockManager(),
- transactionSubsystem, resourceId, ResourceType.LSM_BTREE, indexOp);
- }
- }
+ if (metadataIndex.isPrimaryIndex()) {
+ return new PrimaryIndexModificationOperationCallback(metadataIndex.getDatasetId().getId(),
+ metadataIndex.getPrimaryKeyIndexes(), txnCtx, transactionSubsystem.getLockManager(),
+ transactionSubsystem, resourceId, ResourceType.LSM_BTREE, indexOp);
+ } else {
+ return new SecondaryIndexModificationOperationCallback(metadataIndex.getDatasetId().getId(),
+ metadataIndex.getPrimaryKeyIndexes(), txnCtx, transactionSubsystem.getLockManager(),
+ transactionSubsystem, resourceId, ResourceType.LSM_BTREE, indexOp);
+ }
+ }
- @Override
- public void dropDataverse(JobId jobId, String dataverseName) throws MetadataException, RemoteException {
- try {
- List<Dataset> dataverseDatasets;
+ @Override
+ public void dropDataverse(JobId jobId, String dataverseName) throws MetadataException, RemoteException {
+ try {
+ List<Dataset> dataverseDatasets;
- dataverseDatasets = getDataverseDatasets(jobId, dataverseName);
- if (dataverseDatasets != null && dataverseDatasets.size() > 0) {
- // Drop all datasets in this dataverse.
- for (int i = 0; i < dataverseDatasets.size(); i++) {
- dropDataset(jobId, dataverseName, dataverseDatasets.get(i).getDatasetName());
- }
- }
- List<Datatype> dataverseDatatypes;
- // As a side effect, acquires an S lock on the 'datatype' dataset
- // on behalf of txnId.
- dataverseDatatypes = getDataverseDatatypes(jobId, dataverseName);
- if (dataverseDatatypes != null && dataverseDatatypes.size() > 0) {
- // Drop all types in this dataverse.
- for (int i = 0; i < dataverseDatatypes.size(); i++) {
- forceDropDatatype(jobId, dataverseName, dataverseDatatypes.get(i).getDatatypeName());
- }
- }
+ dataverseDatasets = getDataverseDatasets(jobId, dataverseName);
+ if (dataverseDatasets != null && dataverseDatasets.size() > 0) {
+ // Drop all datasets in this dataverse.
+ for (int i = 0; i < dataverseDatasets.size(); i++) {
+ dropDataset(jobId, dataverseName, dataverseDatasets.get(i).getDatasetName());
+ }
+ }
+ List<Datatype> dataverseDatatypes;
+ // As a side effect, acquires an S lock on the 'datatype' dataset
+ // on behalf of txnId.
+ dataverseDatatypes = getDataverseDatatypes(jobId, dataverseName);
+ if (dataverseDatatypes != null && dataverseDatatypes.size() > 0) {
+ // Drop all types in this dataverse.
+ for (int i = 0; i < dataverseDatatypes.size(); i++) {
+ forceDropDatatype(jobId, dataverseName, dataverseDatatypes.get(i).getDatatypeName());
+ }
+ }
- // As a side effect, acquires an S lock on the 'Function' dataset
- // on behalf of txnId.
- List<Function> dataverseFunctions = getDataverseFunctions(jobId, dataverseName);
- if (dataverseFunctions != null && dataverseFunctions.size() > 0) {
- // Drop all functions in this dataverse.
- for (Function function : dataverseFunctions) {
- dropFunction(jobId, new FunctionSignature(dataverseName, function.getName(), function.getArity()));
- }
- }
+ // As a side effect, acquires an S lock on the 'Function' dataset
+ // on behalf of txnId.
+ List<Function> dataverseFunctions = getDataverseFunctions(jobId, dataverseName);
+ if (dataverseFunctions != null && dataverseFunctions.size() > 0) {
+ // Drop all functions in this dataverse.
+ for (Function function : dataverseFunctions) {
+ dropFunction(jobId, new FunctionSignature(dataverseName, function.getName(), function.getArity()));
+ }
+ }
- // As a side effect, acquires an S lock on the 'Adapter' dataset
- // on behalf of txnId.
- List<DatasourceAdapter> dataverseAdapters = getDataverseAdapters(jobId, dataverseName);
- if (dataverseAdapters != null && dataverseAdapters.size() > 0) {
- // Drop all functions in this dataverse.
- for (DatasourceAdapter adapter : dataverseAdapters) {
- dropAdapter(jobId, dataverseName, adapter.getAdapterIdentifier().getAdapterName());
- }
- }
+ // As a side effect, acquires an S lock on the 'Adapter' dataset
+ // on behalf of txnId.
+ List<DatasourceAdapter> dataverseAdapters = getDataverseAdapters(jobId, dataverseName);
+ if (dataverseAdapters != null && dataverseAdapters.size() > 0) {
+ // Drop all functions in this dataverse.
+ for (DatasourceAdapter adapter : dataverseAdapters) {
+ dropAdapter(jobId, dataverseName, adapter.getAdapterIdentifier().getAdapterName());
+ }
+ }
- // Delete the dataverse entry from the 'dataverse' dataset.
- ITupleReference searchKey = createTuple(dataverseName);
- // As a side effect, acquires an S lock on the 'dataverse' dataset
- // on behalf of txnId.
- ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, searchKey);
- deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, tuple);
+ // Delete the dataverse entry from the 'dataverse' dataset.
+ ITupleReference searchKey = createTuple(dataverseName);
+ // As a side effect, acquires an S lock on the 'dataverse' dataset
+ // on behalf of txnId.
+ ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, searchKey);
+ deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, tuple);
- // TODO: Change this to be a BTree specific exception, e.g.,
- // BTreeKeyDoesNotExistException.
- } catch (TreeIndexException e) {
- throw new MetadataException("Cannot drop dataverse '" + dataverseName + "' because it doesn't exist.", e);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ // TODO: Change this to be a BTree specific exception, e.g.,
+ // BTreeKeyDoesNotExistException.
+ } catch (TreeIndexException e) {
+ throw new MetadataException("Cannot drop dataverse '" + dataverseName + "' because it doesn't exist.", e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public void dropDataset(JobId jobId, String dataverseName, String datasetName) throws MetadataException,
- RemoteException {
- Dataset dataset;
- try {
- dataset = getDataset(jobId, dataverseName, datasetName);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- if (dataset == null) {
- throw new MetadataException("Cannot drop dataset '" + datasetName + "' because it doesn't exist.");
- }
- try {
- // Delete entry from the 'datasets' dataset.
- ITupleReference searchKey = createTuple(dataverseName, datasetName);
- // Searches the index for the tuple to be deleted. Acquires an S
- // lock on the 'dataset' dataset.
- try {
- ITupleReference datasetTuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATASET_DATASET,
- searchKey);
- deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
- } catch (TreeIndexException tie) {
- // ignore this exception and continue deleting all relevant
- // artifacts.
- }
+ @Override
+ public void dropDataset(JobId jobId, String dataverseName, String datasetName) throws MetadataException,
+ RemoteException {
+ Dataset dataset;
+ try {
+ dataset = getDataset(jobId, dataverseName, datasetName);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ if (dataset == null) {
+ throw new MetadataException("Cannot drop dataset '" + datasetName + "' because it doesn't exist.");
+ }
+ try {
+ // Delete entry from the 'datasets' dataset.
+ ITupleReference searchKey = createTuple(dataverseName, datasetName);
+ // Searches the index for the tuple to be deleted. Acquires an S
+ // lock on the 'dataset' dataset.
+ try {
+ ITupleReference datasetTuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATASET_DATASET,
+ searchKey);
+ deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
+ } 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 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(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);
- }
- }
+ // 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);
+ }
+ }
- @Override
- public void dropIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
- throws MetadataException, RemoteException {
- try {
- ITupleReference searchKey = createTuple(dataverseName, datasetName, indexName);
- // Searches the index for the tuple to be deleted. Acquires an S
- // lock on the 'index' dataset.
- ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey);
- deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, tuple);
- // TODO: Change this to be a BTree specific exception, e.g.,
- // BTreeKeyDoesNotExistException.
- } catch (TreeIndexException e) {
- throw new MetadataException("Cannot drop index '" + datasetName + "." + indexName
- + "' because it doesn't exist.", e);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public void dropIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
+ throws MetadataException, RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataverseName, datasetName, indexName);
+ // Searches the index for the tuple to be deleted. Acquires an S
+ // lock on the 'index' dataset.
+ ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey);
+ deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, tuple);
+ // TODO: Change this to be a BTree specific exception, e.g.,
+ // BTreeKeyDoesNotExistException.
+ } catch (TreeIndexException e) {
+ throw new MetadataException("Cannot drop index '" + datasetName + "." + indexName
+ + "' because it doesn't exist.", e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public void dropNodegroup(JobId jobId, String nodeGroupName) throws MetadataException, RemoteException {
- List<String> datasetNames;
- try {
- datasetNames = getDatasetNamesPartitionedOnThisNodeGroup(jobId, nodeGroupName);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- if (!datasetNames.isEmpty()) {
- StringBuilder sb = new StringBuilder();
- sb.append("Nodegroup '" + nodeGroupName
- + "' cannot be dropped; it was used for partitioning these datasets:");
- for (int i = 0; i < datasetNames.size(); i++)
- sb.append("\n" + (i + 1) + "- " + datasetNames.get(i) + ".");
- throw new MetadataException(sb.toString());
- }
- try {
- ITupleReference searchKey = createTuple(nodeGroupName);
- // Searches the index for the tuple to be deleted. Acquires an S
- // lock on the 'nodegroup' dataset.
- ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, searchKey);
- deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, tuple);
- // TODO: Change this to be a BTree specific exception, e.g.,
- // BTreeKeyDoesNotExistException.
- } catch (TreeIndexException e) {
- throw new MetadataException("Cannot drop nodegroup '" + nodeGroupName + "' because it doesn't exist", e);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public void dropNodegroup(JobId jobId, String nodeGroupName) throws MetadataException, RemoteException {
+ List<String> datasetNames;
+ try {
+ datasetNames = getDatasetNamesPartitionedOnThisNodeGroup(jobId, nodeGroupName);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ if (!datasetNames.isEmpty()) {
+ StringBuilder sb = new StringBuilder();
+ sb.append("Nodegroup '" + nodeGroupName
+ + "' cannot be dropped; it was used for partitioning these datasets:");
+ for (int i = 0; i < datasetNames.size(); i++)
+ sb.append("\n" + (i + 1) + "- " + datasetNames.get(i) + ".");
+ throw new MetadataException(sb.toString());
+ }
+ try {
+ ITupleReference searchKey = createTuple(nodeGroupName);
+ // Searches the index for the tuple to be deleted. Acquires an S
+ // lock on the 'nodegroup' dataset.
+ ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, searchKey);
+ deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, tuple);
+ // TODO: Change this to be a BTree specific exception, e.g.,
+ // BTreeKeyDoesNotExistException.
+ } catch (TreeIndexException e) {
+ throw new MetadataException("Cannot drop nodegroup '" + nodeGroupName + "' because it doesn't exist", e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public void dropDatatype(JobId jobId, String dataverseName, String datatypeName) throws MetadataException,
- RemoteException {
- List<String> datasetNames;
- List<String> usedDatatypes;
- try {
- datasetNames = getDatasetNamesDeclaredByThisDatatype(jobId, dataverseName, datatypeName);
- usedDatatypes = getDatatypeNamesUsingThisDatatype(jobId, dataverseName, datatypeName);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- // Check whether type is being used by datasets.
- if (!datasetNames.isEmpty()) {
- StringBuilder sb = new StringBuilder();
- sb.append("Cannot drop type '" + datatypeName + "'; it was used when creating these datasets:");
- for (int i = 0; i < datasetNames.size(); i++)
- sb.append("\n" + (i + 1) + "- " + datasetNames.get(i) + ".");
- throw new MetadataException(sb.toString());
- }
- // Check whether type is being used by other types.
- if (!usedDatatypes.isEmpty()) {
- StringBuilder sb = new StringBuilder();
- sb.append("Cannot drop type '" + datatypeName + "'; it is used in these datatypes:");
- for (int i = 0; i < usedDatatypes.size(); i++)
- sb.append("\n" + (i + 1) + "- " + usedDatatypes.get(i) + ".");
- throw new MetadataException(sb.toString());
- }
- // Delete the datatype entry, including all it's nested types.
- try {
- ITupleReference searchKey = createTuple(dataverseName, datatypeName);
- // Searches the index for the tuple to be deleted. Acquires an S
- // lock on the 'datatype' dataset.
- ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey);
- // This call uses the secondary index on datatype. Get nested types
- // before deleting entry from secondary index.
- List<String> nestedTypes = getNestedDatatypeNames(jobId, dataverseName, datatypeName);
- deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
- deleteFromDatatypeSecondaryIndex(jobId, dataverseName, datatypeName);
- for (String nestedType : nestedTypes) {
- Datatype dt = getDatatype(jobId, dataverseName, nestedType);
- if (dt != null && dt.getIsAnonymous()) {
- dropDatatype(jobId, dataverseName, dt.getDatatypeName());
- }
- }
- // TODO: Change this to be a BTree specific exception, e.g.,
- // BTreeKeyDoesNotExistException.
- } catch (TreeIndexException e) {
- throw new MetadataException("Cannot drop type '" + datatypeName + "' because it doesn't exist", e);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public void dropDatatype(JobId jobId, String dataverseName, String datatypeName) throws MetadataException,
+ RemoteException {
+ List<String> datasetNames;
+ List<String> usedDatatypes;
+ try {
+ datasetNames = getDatasetNamesDeclaredByThisDatatype(jobId, dataverseName, datatypeName);
+ usedDatatypes = getDatatypeNamesUsingThisDatatype(jobId, dataverseName, datatypeName);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ // Check whether type is being used by datasets.
+ if (!datasetNames.isEmpty()) {
+ StringBuilder sb = new StringBuilder();
+ sb.append("Cannot drop type '" + datatypeName + "'; it was used when creating these datasets:");
+ for (int i = 0; i < datasetNames.size(); i++)
+ sb.append("\n" + (i + 1) + "- " + datasetNames.get(i) + ".");
+ throw new MetadataException(sb.toString());
+ }
+ // Check whether type is being used by other types.
+ if (!usedDatatypes.isEmpty()) {
+ StringBuilder sb = new StringBuilder();
+ sb.append("Cannot drop type '" + datatypeName + "'; it is used in these datatypes:");
+ for (int i = 0; i < usedDatatypes.size(); i++)
+ sb.append("\n" + (i + 1) + "- " + usedDatatypes.get(i) + ".");
+ throw new MetadataException(sb.toString());
+ }
+ // Delete the datatype entry, including all it's nested types.
+ try {
+ ITupleReference searchKey = createTuple(dataverseName, datatypeName);
+ // Searches the index for the tuple to be deleted. Acquires an S
+ // lock on the 'datatype' dataset.
+ ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey);
+ // This call uses the secondary index on datatype. Get nested types
+ // before deleting entry from secondary index.
+ List<String> nestedTypes = getNestedDatatypeNames(jobId, dataverseName, datatypeName);
+ deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
+ deleteFromDatatypeSecondaryIndex(jobId, dataverseName, datatypeName);
+ for (String nestedType : nestedTypes) {
+ Datatype dt = getDatatype(jobId, dataverseName, nestedType);
+ if (dt != null && dt.getIsAnonymous()) {
+ dropDatatype(jobId, dataverseName, dt.getDatatypeName());
+ }
+ }
+ // TODO: Change this to be a BTree specific exception, e.g.,
+ // BTreeKeyDoesNotExistException.
+ } catch (TreeIndexException e) {
+ throw new MetadataException("Cannot drop type '" + datatypeName + "' because it doesn't exist", e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- private void forceDropDatatype(JobId jobId, String dataverseName, String datatypeName) throws AsterixException {
- try {
- ITupleReference searchKey = createTuple(dataverseName, datatypeName);
- // Searches the index for the tuple to be deleted. Acquires an S
- // lock on the 'datatype' dataset.
- ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey);
- deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
- deleteFromDatatypeSecondaryIndex(jobId, dataverseName, datatypeName);
- // TODO: Change this to be a BTree specific exception, e.g.,
- // BTreeKeyDoesNotExistException.
- } catch (TreeIndexException e) {
- throw new AsterixException("Cannot drop type '" + datatypeName + "' because it doesn't exist", e);
- } catch (AsterixException e) {
- throw e;
- } catch (Exception e) {
- throw new AsterixException(e);
- }
- }
+ private void forceDropDatatype(JobId jobId, String dataverseName, String datatypeName) throws AsterixException {
+ try {
+ ITupleReference searchKey = createTuple(dataverseName, datatypeName);
+ // Searches the index for the tuple to be deleted. Acquires an S
+ // lock on the 'datatype' dataset.
+ ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey);
+ deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
+ deleteFromDatatypeSecondaryIndex(jobId, dataverseName, datatypeName);
+ // TODO: Change this to be a BTree specific exception, e.g.,
+ // BTreeKeyDoesNotExistException.
+ } catch (TreeIndexException e) {
+ throw new AsterixException("Cannot drop type '" + datatypeName + "' because it doesn't exist", e);
+ } catch (AsterixException e) {
+ throw e;
+ } catch (Exception e) {
+ throw new AsterixException(e);
+ }
+ }
- private void deleteFromDatatypeSecondaryIndex(JobId jobId, String dataverseName, String datatypeName)
- throws AsterixException {
- try {
- List<String> nestedTypes = getNestedDatatypeNames(jobId, dataverseName, datatypeName);
- for (String nestedType : nestedTypes) {
- ITupleReference searchKey = createTuple(dataverseName, nestedType, datatypeName);
- // Searches the index for the tuple to be deleted. Acquires an S
- // lock on the DATATYPENAME_ON_DATATYPE_INDEX index.
- ITupleReference tuple = getTupleToBeDeleted(jobId,
- MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, searchKey);
- deleteTupleFromIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, tuple);
- }
- // TODO: Change this to be a BTree specific exception, e.g.,
- // BTreeKeyDoesNotExistException.
- } catch (TreeIndexException e) {
- throw new AsterixException("Cannot drop type '" + datatypeName + "' because it doesn't exist", e);
- } catch (AsterixException e) {
- throw e;
- } catch (Exception e) {
- throw new AsterixException(e);
- }
- }
+ private void deleteFromDatatypeSecondaryIndex(JobId jobId, String dataverseName, String datatypeName)
+ throws AsterixException {
+ try {
+ List<String> nestedTypes = getNestedDatatypeNames(jobId, dataverseName, datatypeName);
+ for (String nestedType : nestedTypes) {
+ ITupleReference searchKey = createTuple(dataverseName, nestedType, datatypeName);
+ // Searches the index for the tuple to be deleted. Acquires an S
+ // lock on the DATATYPENAME_ON_DATATYPE_INDEX index.
+ ITupleReference tuple = getTupleToBeDeleted(jobId,
+ MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, searchKey);
+ deleteTupleFromIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, tuple);
+ }
+ // TODO: Change this to be a BTree specific exception, e.g.,
+ // BTreeKeyDoesNotExistException.
+ } catch (TreeIndexException e) {
+ throw new AsterixException("Cannot drop type '" + datatypeName + "' because it doesn't exist", e);
+ } catch (AsterixException e) {
+ throw e;
+ } catch (Exception e) {
+ throw new AsterixException(e);
+ }
+ }
- private void deleteTupleFromIndex(JobId jobId, IMetadataIndex metadataIndex, ITupleReference tuple)
- throws Exception {
- long resourceID = metadataIndex.getResourceID();
- ILSMIndex lsmIndex = (ILSMIndex) indexLifecycleManager.getIndex(resourceID);
- indexLifecycleManager.open(resourceID);
- // prepare a Callback for logging
- IModificationOperationCallback modCallback = createIndexModificationCallback(jobId, resourceID, metadataIndex,
- lsmIndex, IndexOperation.DELETE);
- IIndexAccessor indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
+ private void deleteTupleFromIndex(JobId jobId, IMetadataIndex metadataIndex, ITupleReference tuple)
+ throws Exception {
+ long resourceID = metadataIndex.getResourceID();
+ ILSMIndex lsmIndex = (ILSMIndex) indexLifecycleManager.getIndex(resourceID);
+ indexLifecycleManager.open(resourceID);
+ // prepare a Callback for logging
+ IModificationOperationCallback modCallback = createIndexModificationCallback(jobId, resourceID, metadataIndex,
+ lsmIndex, IndexOperation.DELETE);
+ IIndexAccessor indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
- ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
- txnCtx.setTransactionType(TransactionType.READ_WRITE);
- txnCtx.registerIndexAndCallback(lsmIndex, (AbstractOperationCallback) modCallback);
+ ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+ txnCtx.setTransactionType(TransactionType.READ_WRITE);
+ txnCtx.registerIndexAndCallback(lsmIndex, (AbstractOperationCallback) modCallback);
- indexAccessor.delete(tuple);
- indexLifecycleManager.close(resourceID);
- }
+ indexAccessor.delete(tuple);
+ indexLifecycleManager.close(resourceID);
+ }
- @Override
- public List<Dataverse> getDataverses(JobId jobId) throws MetadataException, RemoteException {
- try {
- DataverseTupleTranslator tupleReaderWriter = new DataverseTupleTranslator(false);
- IValueExtractor<Dataverse> valueExtractor = new MetadataEntityValueExtractor<Dataverse>(tupleReaderWriter);
- List<Dataverse> results = new ArrayList<Dataverse>();
- searchIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, null, valueExtractor, results);
- if (results.isEmpty()) {
- return null;
- }
- return results;
- } catch (Exception e) {
- throw new MetadataException(e);
- }
+ @Override
+ public List<Dataverse> getDataverses(JobId jobId) throws MetadataException, RemoteException {
+ try {
+ DataverseTupleTranslator tupleReaderWriter = new DataverseTupleTranslator(false);
+ IValueExtractor<Dataverse> valueExtractor = new MetadataEntityValueExtractor<Dataverse>(tupleReaderWriter);
+ List<Dataverse> results = new ArrayList<Dataverse>();
+ searchIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, null, valueExtractor, results);
+ if (results.isEmpty()) {
+ return null;
+ }
+ return results;
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
- }
+ }
- @Override
- public Dataverse getDataverse(JobId jobId, String dataverseName) throws MetadataException, RemoteException {
+ @Override
+ public Dataverse getDataverse(JobId jobId, String dataverseName) throws MetadataException, RemoteException {
- try {
- ITupleReference searchKey = createTuple(dataverseName);
- DataverseTupleTranslator tupleReaderWriter = new DataverseTupleTranslator(false);
- IValueExtractor<Dataverse> valueExtractor = new MetadataEntityValueExtractor<Dataverse>(tupleReaderWriter);
- List<Dataverse> results = new ArrayList<Dataverse>();
- searchIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, searchKey, valueExtractor, results);
- if (results.isEmpty()) {
- return null;
- }
- return results.get(0);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
+ try {
+ ITupleReference searchKey = createTuple(dataverseName);
+ DataverseTupleTranslator tupleReaderWriter = new DataverseTupleTranslator(false);
+ IValueExtractor<Dataverse> valueExtractor = new MetadataEntityValueExtractor<Dataverse>(tupleReaderWriter);
+ List<Dataverse> results = new ArrayList<Dataverse>();
+ searchIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, searchKey, valueExtractor, results);
+ if (results.isEmpty()) {
+ return null;
+ }
+ return results.get(0);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
- }
+ }
- @Override
- public List<Dataset> getDataverseDatasets(JobId jobId, String dataverseName) throws MetadataException,
- RemoteException {
- try {
- ITupleReference searchKey = createTuple(dataverseName);
- DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(false);
- IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<Dataset>(tupleReaderWriter);
- List<Dataset> results = new ArrayList<Dataset>();
- searchIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey, valueExtractor, results);
- return results;
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public List<Dataset> getDataverseDatasets(JobId jobId, String dataverseName) throws MetadataException,
+ RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataverseName);
+ DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(false);
+ IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<Dataset>(tupleReaderWriter);
+ List<Dataset> results = new ArrayList<Dataset>();
+ searchIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey, valueExtractor, results);
+ return results;
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- private List<Datatype> getDataverseDatatypes(JobId jobId, String dataverseName) throws MetadataException,
- RemoteException {
- try {
- ITupleReference searchKey = createTuple(dataverseName);
- DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(jobId, this, false);
- IValueExtractor<Datatype> valueExtractor = new MetadataEntityValueExtractor<Datatype>(tupleReaderWriter);
- List<Datatype> results = new ArrayList<Datatype>();
- searchIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey, valueExtractor, results);
- return results;
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ private List<Datatype> getDataverseDatatypes(JobId jobId, String dataverseName) throws MetadataException,
+ RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataverseName);
+ DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(jobId, this, false);
+ IValueExtractor<Datatype> valueExtractor = new MetadataEntityValueExtractor<Datatype>(tupleReaderWriter);
+ List<Datatype> results = new ArrayList<Datatype>();
+ searchIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey, valueExtractor, results);
+ return results;
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public Dataset getDataset(JobId jobId, String dataverseName, String datasetName) throws MetadataException,
- RemoteException {
- try {
- ITupleReference searchKey = createTuple(dataverseName, datasetName);
- DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(false);
- List<Dataset> results = new ArrayList<Dataset>();
- IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<Dataset>(tupleReaderWriter);
- searchIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey, valueExtractor, results);
- if (results.isEmpty()) {
- return null;
- }
- return results.get(0);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public Dataset getDataset(JobId jobId, String dataverseName, String datasetName) throws MetadataException,
+ RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataverseName, datasetName);
+ DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(false);
+ List<Dataset> results = new ArrayList<Dataset>();
+ IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<Dataset>(tupleReaderWriter);
+ searchIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey, valueExtractor, results);
+ if (results.isEmpty()) {
+ return null;
+ }
+ return results.get(0);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- private List<String> getDatasetNamesDeclaredByThisDatatype(JobId jobId, String dataverseName, String datatypeName)
- throws MetadataException, RemoteException {
- try {
- ITupleReference searchKey = createTuple(dataverseName, datatypeName);
- List<String> results = new ArrayList<String>();
- IValueExtractor<String> valueExtractor = new DatasetNameValueExtractor();
- searchIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, searchKey, valueExtractor,
- results);
- return results;
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ private List<String> getDatasetNamesDeclaredByThisDatatype(JobId jobId, String dataverseName, String datatypeName)
+ throws MetadataException, RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataverseName, datatypeName);
+ List<String> results = new ArrayList<String>();
+ IValueExtractor<String> valueExtractor = new DatasetNameValueExtractor();
+ searchIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, searchKey, valueExtractor,
+ results);
+ return results;
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- public List<String> getDatatypeNamesUsingThisDatatype(JobId jobId, String dataverseName, String datatypeName)
- throws MetadataException, RemoteException {
- try {
- ITupleReference searchKey = createTuple(dataverseName, datatypeName);
- List<String> results = new ArrayList<String>();
- IValueExtractor<String> valueExtractor = new DatatypeNameValueExtractor(dataverseName, this);
- searchIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, searchKey, valueExtractor,
- results);
- return results;
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ public List<String> getDatatypeNamesUsingThisDatatype(JobId jobId, String dataverseName, String datatypeName)
+ throws MetadataException, RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataverseName, datatypeName);
+ List<String> results = new ArrayList<String>();
+ IValueExtractor<String> valueExtractor = new DatatypeNameValueExtractor(dataverseName, this);
+ searchIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, searchKey, valueExtractor,
+ results);
+ return results;
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- private List<String> getNestedDatatypeNames(JobId jobId, String dataverseName, String datatypeName)
- throws MetadataException, RemoteException {
- try {
- ITupleReference searchKey = createTuple(dataverseName);
- List<String> results = new ArrayList<String>();
- IValueExtractor<String> valueExtractor = new NestedDatatypeNameValueExtractor(datatypeName);
- searchIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, searchKey, valueExtractor,
- results);
- return results;
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ private List<String> getNestedDatatypeNames(JobId jobId, String dataverseName, String datatypeName)
+ throws MetadataException, RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataverseName);
+ List<String> results = new ArrayList<String>();
+ IValueExtractor<String> valueExtractor = new NestedDatatypeNameValueExtractor(datatypeName);
+ searchIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, searchKey, valueExtractor,
+ results);
+ return results;
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- public List<String> getDatasetNamesPartitionedOnThisNodeGroup(JobId jobId, String nodegroup)
- throws MetadataException, RemoteException {
- try {
- ITupleReference searchKey = createTuple(nodegroup);
- List<String> results = new ArrayList<String>();
- IValueExtractor<String> valueExtractor = new DatasetNameValueExtractor();
- searchIndex(jobId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, searchKey, valueExtractor, results);
- return results;
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ public List<String> getDatasetNamesPartitionedOnThisNodeGroup(JobId jobId, String nodegroup)
+ throws MetadataException, RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(nodegroup);
+ List<String> results = new ArrayList<String>();
+ IValueExtractor<String> valueExtractor = new DatasetNameValueExtractor();
+ searchIndex(jobId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, searchKey, valueExtractor, results);
+ return results;
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public Index getIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
- throws MetadataException, RemoteException {
- try {
- ITupleReference searchKey = createTuple(dataverseName, datasetName, indexName);
- IndexTupleTranslator tupleReaderWriter = new IndexTupleTranslator(false);
- IValueExtractor<Index> valueExtractor = new MetadataEntityValueExtractor<Index>(tupleReaderWriter);
- List<Index> results = new ArrayList<Index>();
- searchIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey, valueExtractor, results);
- if (results.isEmpty()) {
- return null;
- }
- return results.get(0);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public Index getIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
+ throws MetadataException, RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataverseName, datasetName, indexName);
+ IndexTupleTranslator tupleReaderWriter = new IndexTupleTranslator(false);
+ IValueExtractor<Index> valueExtractor = new MetadataEntityValueExtractor<Index>(tupleReaderWriter);
+ List<Index> results = new ArrayList<Index>();
+ searchIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey, valueExtractor, results);
+ if (results.isEmpty()) {
+ return null;
+ }
+ return results.get(0);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public List<Index> getDatasetIndexes(JobId jobId, String dataverseName, String datasetName)
- throws MetadataException, RemoteException {
- try {
- ITupleReference searchKey = createTuple(dataverseName, datasetName);
- IndexTupleTranslator tupleReaderWriter = new IndexTupleTranslator(false);
- IValueExtractor<Index> valueExtractor = new MetadataEntityValueExtractor<Index>(tupleReaderWriter);
- List<Index> results = new ArrayList<Index>();
- searchIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey, valueExtractor, results);
- return results;
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public List<Index> getDatasetIndexes(JobId jobId, String dataverseName, String datasetName)
+ throws MetadataException, RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataverseName, datasetName);
+ IndexTupleTranslator tupleReaderWriter = new IndexTupleTranslator(false);
+ IValueExtractor<Index> valueExtractor = new MetadataEntityValueExtractor<Index>(tupleReaderWriter);
+ List<Index> results = new ArrayList<Index>();
+ searchIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey, valueExtractor, results);
+ return results;
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public Datatype getDatatype(JobId jobId, String dataverseName, String datatypeName) throws MetadataException,
- RemoteException {
- try {
- ITupleReference searchKey = createTuple(dataverseName, datatypeName);
- DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(jobId, this, false);
- IValueExtractor<Datatype> valueExtractor = new MetadataEntityValueExtractor<Datatype>(tupleReaderWriter);
- List<Datatype> results = new ArrayList<Datatype>();
- searchIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey, valueExtractor, results);
- if (results.isEmpty()) {
- return null;
- }
- return results.get(0);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public Datatype getDatatype(JobId jobId, String dataverseName, String datatypeName) throws MetadataException,
+ RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataverseName, datatypeName);
+ DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(jobId, this, false);
+ IValueExtractor<Datatype> valueExtractor = new MetadataEntityValueExtractor<Datatype>(tupleReaderWriter);
+ List<Datatype> results = new ArrayList<Datatype>();
+ searchIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey, valueExtractor, results);
+ if (results.isEmpty()) {
+ return null;
+ }
+ return results.get(0);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public NodeGroup getNodeGroup(JobId jobId, String nodeGroupName) throws MetadataException, RemoteException {
- try {
- ITupleReference searchKey = createTuple(nodeGroupName);
- NodeGroupTupleTranslator tupleReaderWriter = new NodeGroupTupleTranslator(false);
- IValueExtractor<NodeGroup> valueExtractor = new MetadataEntityValueExtractor<NodeGroup>(tupleReaderWriter);
- List<NodeGroup> results = new ArrayList<NodeGroup>();
- searchIndex(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, searchKey, valueExtractor, results);
- if (results.isEmpty()) {
- return null;
- }
- return results.get(0);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public NodeGroup getNodeGroup(JobId jobId, String nodeGroupName) throws MetadataException, RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(nodeGroupName);
+ NodeGroupTupleTranslator tupleReaderWriter = new NodeGroupTupleTranslator(false);
+ IValueExtractor<NodeGroup> valueExtractor = new MetadataEntityValueExtractor<NodeGroup>(tupleReaderWriter);
+ List<NodeGroup> results = new ArrayList<NodeGroup>();
+ searchIndex(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, searchKey, valueExtractor, results);
+ if (results.isEmpty()) {
+ return null;
+ }
+ return results.get(0);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public Function getFunction(JobId jobId, FunctionSignature functionSignature) throws MetadataException,
- RemoteException {
- try {
- ITupleReference searchKey = createTuple(functionSignature.getNamespace(), functionSignature.getName(), ""
- + functionSignature.getArity());
- FunctionTupleTranslator tupleReaderWriter = new FunctionTupleTranslator(false);
- List<Function> results = new ArrayList<Function>();
- IValueExtractor<Function> valueExtractor = new MetadataEntityValueExtractor<Function>(tupleReaderWriter);
- searchIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey, valueExtractor, results);
- if (results.isEmpty()) {
- return null;
- }
- return results.get(0);
- } catch (Exception e) {
- e.printStackTrace();
- throw new MetadataException(e);
- }
- }
+ @Override
+ public Function getFunction(JobId jobId, FunctionSignature functionSignature) throws MetadataException,
+ RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(functionSignature.getNamespace(), functionSignature.getName(), ""
+ + functionSignature.getArity());
+ FunctionTupleTranslator tupleReaderWriter = new FunctionTupleTranslator(false);
+ List<Function> results = new ArrayList<Function>();
+ IValueExtractor<Function> valueExtractor = new MetadataEntityValueExtractor<Function>(tupleReaderWriter);
+ searchIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey, valueExtractor, results);
+ if (results.isEmpty()) {
+ return null;
+ }
+ return results.get(0);
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public void dropFunction(JobId jobId, FunctionSignature functionSignature) throws MetadataException,
- RemoteException {
+ @Override
+ public void dropFunction(JobId jobId, FunctionSignature functionSignature) throws MetadataException,
+ RemoteException {
- Function function;
- try {
- function = getFunction(jobId, functionSignature);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- if (function == null) {
- throw new MetadataException("Cannot drop function '" + functionSignature.toString()
- + "' because it doesn't exist.");
- }
- try {
- // Delete entry from the 'function' dataset.
- ITupleReference searchKey = createTuple(functionSignature.getNamespace(), functionSignature.getName(), ""
- + functionSignature.getArity());
- // Searches the index for the tuple to be deleted. Acquires an S
- // lock on the 'function' dataset.
- ITupleReference datasetTuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET,
- searchKey);
- deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, datasetTuple);
+ Function function;
+ try {
+ function = getFunction(jobId, functionSignature);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ if (function == null) {
+ throw new MetadataException("Cannot drop function '" + functionSignature.toString()
+ + "' because it doesn't exist.");
+ }
+ try {
+ // Delete entry from the 'function' dataset.
+ ITupleReference searchKey = createTuple(functionSignature.getNamespace(), functionSignature.getName(), ""
+ + functionSignature.getArity());
+ // Searches the index for the tuple to be deleted. Acquires an S
+ // lock on the 'function' dataset.
+ ITupleReference datasetTuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET,
+ searchKey);
+ deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, datasetTuple);
- // TODO: Change this to be a BTree specific exception, e.g.,
- // BTreeKeyDoesNotExistException.
- } catch (TreeIndexException e) {
- throw new MetadataException("There is no function with the name " + functionSignature.getName()
- + " and arity " + functionSignature.getArity(), e);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ // TODO: Change this to be a BTree specific exception, e.g.,
+ // BTreeKeyDoesNotExistException.
+ } catch (TreeIndexException e) {
+ throw new MetadataException("There is no function with the name " + functionSignature.getName()
+ + " and arity " + functionSignature.getArity(), e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- private ITupleReference getTupleToBeDeleted(JobId jobId, IMetadataIndex metadataIndex, ITupleReference searchKey)
- throws Exception {
- IValueExtractor<ITupleReference> valueExtractor = new TupleCopyValueExtractor(metadataIndex.getTypeTraits());
- List<ITupleReference> results = new ArrayList<ITupleReference>();
- searchIndex(jobId, metadataIndex, searchKey, valueExtractor, results);
- if (results.isEmpty()) {
- // TODO: Temporarily a TreeIndexException to make it get caught by
- // caller in the appropriate catch block.
- throw new TreeIndexException("Could not find entry to be deleted.");
- }
- // There should be exactly one result returned from the search.
- return results.get(0);
- }
+ private ITupleReference getTupleToBeDeleted(JobId jobId, IMetadataIndex metadataIndex, ITupleReference searchKey)
+ throws Exception {
+ IValueExtractor<ITupleReference> valueExtractor = new TupleCopyValueExtractor(metadataIndex.getTypeTraits());
+ List<ITupleReference> results = new ArrayList<ITupleReference>();
+ searchIndex(jobId, metadataIndex, searchKey, valueExtractor, results);
+ if (results.isEmpty()) {
+ // TODO: Temporarily a TreeIndexException to make it get caught by
+ // caller in the appropriate catch block.
+ throw new TreeIndexException("Could not find entry to be deleted.");
+ }
+ // There should be exactly one result returned from the search.
+ return results.get(0);
+ }
- // Debugging Method
- public String printMetadata() {
+ // Debugging Method
+ public String printMetadata() {
- StringBuilder sb = new StringBuilder();
- try {
- IMetadataIndex index = MetadataPrimaryIndexes.DATAVERSE_DATASET;
- long resourceID = index.getResourceID();
- IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
- indexLifecycleManager.open(resourceID);
- IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
- NoOpOperationCallback.INSTANCE);
- ITreeIndexCursor rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor();
+ StringBuilder sb = new StringBuilder();
+ try {
+ IMetadataIndex index = MetadataPrimaryIndexes.DATAVERSE_DATASET;
+ long resourceID = index.getResourceID();
+ IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
+ indexLifecycleManager.open(resourceID);
+ IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
+ NoOpOperationCallback.INSTANCE);
+ ITreeIndexCursor rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor();
- RangePredicate rangePred = null;
- rangePred = new RangePredicate(null, null, true, true, null, null);
- indexAccessor.search(rangeCursor, rangePred);
- try {
- while (rangeCursor.hasNext()) {
- rangeCursor.next();
- sb.append(TupleUtils.printTuple(rangeCursor.getTuple(),
- new ISerializerDeserializer[] { AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ASTRING) }));
- }
- } finally {
- rangeCursor.close();
- }
- indexLifecycleManager.close(resourceID);
+ RangePredicate rangePred = null;
+ rangePred = new RangePredicate(null, null, true, true, null, null);
+ indexAccessor.search(rangeCursor, rangePred);
+ try {
+ while (rangeCursor.hasNext()) {
+ rangeCursor.next();
+ sb.append(TupleUtils.printTuple(rangeCursor.getTuple(),
+ new ISerializerDeserializer[] { AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ASTRING) }));
+ }
+ } finally {
+ rangeCursor.close();
+ }
+ indexLifecycleManager.close(resourceID);
- index = MetadataPrimaryIndexes.DATASET_DATASET;
- resourceID = index.getResourceID();
- indexInstance = indexLifecycleManager.getIndex(resourceID);
- indexLifecycleManager.open(resourceID);
- indexAccessor = indexInstance
- .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
- rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor();
+ index = MetadataPrimaryIndexes.DATASET_DATASET;
+ resourceID = index.getResourceID();
+ indexInstance = indexLifecycleManager.getIndex(resourceID);
+ indexLifecycleManager.open(resourceID);
+ indexAccessor = indexInstance
+ .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+ rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor();
- rangePred = null;
- rangePred = new RangePredicate(null, null, true, true, null, null);
- indexAccessor.search(rangeCursor, rangePred);
- try {
- while (rangeCursor.hasNext()) {
- rangeCursor.next();
- sb.append(TupleUtils.printTuple(rangeCursor.getTuple(), new ISerializerDeserializer[] {
- AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING),
- AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING) }));
- }
- } finally {
- rangeCursor.close();
- }
- indexLifecycleManager.close(resourceID);
+ rangePred = null;
+ rangePred = new RangePredicate(null, null, true, true, null, null);
+ indexAccessor.search(rangeCursor, rangePred);
+ try {
+ while (rangeCursor.hasNext()) {
+ rangeCursor.next();
+ sb.append(TupleUtils.printTuple(rangeCursor.getTuple(), new ISerializerDeserializer[] {
+ AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING),
+ AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING) }));
+ }
+ } finally {
+ rangeCursor.close();
+ }
+ indexLifecycleManager.close(resourceID);
- index = MetadataPrimaryIndexes.INDEX_DATASET;
- resourceID = index.getResourceID();
- indexInstance = indexLifecycleManager.getIndex(resourceID);
- indexLifecycleManager.open(resourceID);
- indexAccessor = indexInstance
- .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
- rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor();
+ index = MetadataPrimaryIndexes.INDEX_DATASET;
+ resourceID = index.getResourceID();
+ indexInstance = indexLifecycleManager.getIndex(resourceID);
+ indexLifecycleManager.open(resourceID);
+ indexAccessor = indexInstance
+ .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+ rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor();
- rangePred = null;
- rangePred = new RangePredicate(null, null, true, true, null, null);
- indexAccessor.search(rangeCursor, rangePred);
- try {
- while (rangeCursor.hasNext()) {
- rangeCursor.next();
- sb.append(TupleUtils.printTuple(rangeCursor.getTuple(), new ISerializerDeserializer[] {
- AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING),
- AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING),
- AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING) }));
- }
- } finally {
- rangeCursor.close();
- }
- indexLifecycleManager.close(resourceID);
- } catch (Exception e) {
- e.printStackTrace();
- }
- return sb.toString();
- }
+ rangePred = null;
+ rangePred = new RangePredicate(null, null, true, true, null, null);
+ indexAccessor.search(rangeCursor, rangePred);
+ try {
+ while (rangeCursor.hasNext()) {
+ rangeCursor.next();
+ sb.append(TupleUtils.printTuple(rangeCursor.getTuple(), new ISerializerDeserializer[] {
+ AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING),
+ AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING),
+ AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING) }));
+ }
+ } finally {
+ rangeCursor.close();
+ }
+ indexLifecycleManager.close(resourceID);
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ return sb.toString();
+ }
- private <ResultType> void searchIndex(JobId jobId, IMetadataIndex index, ITupleReference searchKey,
- IValueExtractor<ResultType> valueExtractor, List<ResultType> results) throws Exception {
- IBinaryComparatorFactory[] comparatorFactories = index.getKeyBinaryComparatorFactory();
- long resourceID = index.getResourceID();
- IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
- indexLifecycleManager.open(resourceID);
- IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
- NoOpOperationCallback.INSTANCE);
- ITreeIndexCursor rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor();
+ private <ResultType> void searchIndex(JobId jobId, IMetadataIndex index, ITupleReference searchKey,
+ IValueExtractor<ResultType> valueExtractor, List<ResultType> results) throws Exception {
+ IBinaryComparatorFactory[] comparatorFactories = index.getKeyBinaryComparatorFactory();
+ long resourceID = index.getResourceID();
+ IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
+ indexLifecycleManager.open(resourceID);
+ IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
+ NoOpOperationCallback.INSTANCE);
+ ITreeIndexCursor rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor();
- IBinaryComparator[] searchCmps = null;
- MultiComparator searchCmp = null;
- RangePredicate rangePred = null;
- if (searchKey != null) {
- searchCmps = new IBinaryComparator[searchKey.getFieldCount()];
- for (int i = 0; i < searchKey.getFieldCount(); i++) {
- searchCmps[i] = comparatorFactories[i].createBinaryComparator();
- }
- searchCmp = new MultiComparator(searchCmps);
- }
- rangePred = new RangePredicate(searchKey, searchKey, true, true, searchCmp, searchCmp);
- indexAccessor.search(rangeCursor, rangePred);
+ IBinaryComparator[] searchCmps = null;
+ MultiComparator searchCmp = null;
+ RangePredicate rangePred = null;
+ if (searchKey != null) {
+ searchCmps = new IBinaryComparator[searchKey.getFieldCount()];
+ for (int i = 0; i < searchKey.getFieldCount(); i++) {
+ searchCmps[i] = comparatorFactories[i].createBinaryComparator();
+ }
+ searchCmp = new MultiComparator(searchCmps);
+ }
+ rangePred = new RangePredicate(searchKey, searchKey, true, true, searchCmp, searchCmp);
+ indexAccessor.search(rangeCursor, rangePred);
- try {
- while (rangeCursor.hasNext()) {
- rangeCursor.next();
- ResultType result = valueExtractor.getValue(jobId, rangeCursor.getTuple());
- if (result != null) {
- results.add(result);
- }
- }
- } finally {
- rangeCursor.close();
- }
- indexLifecycleManager.close(resourceID);
- }
+ try {
+ while (rangeCursor.hasNext()) {
+ rangeCursor.next();
+ ResultType result = valueExtractor.getValue(jobId, rangeCursor.getTuple());
+ if (result != null) {
+ results.add(result);
+ }
+ }
+ } finally {
+ rangeCursor.close();
+ }
+ indexLifecycleManager.close(resourceID);
+ }
- @Override
- public void initializeDatasetIdFactory(JobId jobId) throws MetadataException, RemoteException {
- int mostRecentDatasetId = MetadataPrimaryIndexes.FIRST_AVAILABLE_USER_DATASET_ID;
- long resourceID = MetadataPrimaryIndexes.DATASET_DATASET.getResourceID();
- try {
- IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
- indexLifecycleManager.open(resourceID);
- IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
- NoOpOperationCallback.INSTANCE);
- IIndexCursor rangeCursor = indexAccessor.createSearchCursor();
+ @Override
+ public void initializeDatasetIdFactory(JobId jobId) throws MetadataException, RemoteException {
+ int mostRecentDatasetId = MetadataPrimaryIndexes.FIRST_AVAILABLE_USER_DATASET_ID;
+ long resourceID = MetadataPrimaryIndexes.DATASET_DATASET.getResourceID();
+ try {
+ IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
+ indexLifecycleManager.open(resourceID);
+ IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
+ NoOpOperationCallback.INSTANCE);
+ IIndexCursor rangeCursor = indexAccessor.createSearchCursor();
- DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(false);
- IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<Dataset>(tupleReaderWriter);
- RangePredicate rangePred = new RangePredicate(null, null, true, true, null, null);
+ DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(false);
+ IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<Dataset>(tupleReaderWriter);
+ RangePredicate rangePred = new RangePredicate(null, null, true, true, null, null);
- indexAccessor.search(rangeCursor, rangePred);
- int datasetId;
+ indexAccessor.search(rangeCursor, rangePred);
+ int datasetId;
- try {
- while (rangeCursor.hasNext()) {
- rangeCursor.next();
- ITupleReference ref = rangeCursor.getTuple();
- Dataset ds = valueExtractor.getValue(jobId, rangeCursor.getTuple());
- datasetId = ((Dataset) valueExtractor.getValue(jobId, rangeCursor.getTuple())).getDatasetId();
- if (mostRecentDatasetId < datasetId) {
- mostRecentDatasetId = datasetId;
- }
- }
- } finally {
- rangeCursor.close();
- }
+ try {
+ while (rangeCursor.hasNext()) {
+ rangeCursor.next();
+ ITupleReference ref = rangeCursor.getTuple();
+ Dataset ds = valueExtractor.getValue(jobId, rangeCursor.getTuple());
+ datasetId = ((Dataset) valueExtractor.getValue(jobId, rangeCursor.getTuple())).getDatasetId();
+ if (mostRecentDatasetId < datasetId) {
+ mostRecentDatasetId = datasetId;
+ }
+ }
+ } finally {
+ rangeCursor.close();
+ }
- } catch (Exception e) {
- throw new MetadataException(e);
- }
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
- DatasetIdFactory.initialize(mostRecentDatasetId);
- }
+ DatasetIdFactory.initialize(mostRecentDatasetId);
+ }
- // TODO: Can use Hyrack's TupleUtils for this, once we switch to a newer
- // Hyracks version.
- public ITupleReference createTuple(String... fields) throws HyracksDataException {
- ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ASTRING);
- AMutableString aString = new AMutableString("");
- ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fields.length);
- for (String s : fields) {
- aString.setValue(s);
- stringSerde.serialize(aString, tupleBuilder.getDataOutput());
- tupleBuilder.addFieldEndOffset();
- }
- ArrayTupleReference tuple = new ArrayTupleReference();
- tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
- return tuple;
- }
+ // TODO: Can use Hyrack's TupleUtils for this, once we switch to a newer
+ // Hyracks version.
+ public ITupleReference createTuple(String... fields) throws HyracksDataException {
+ ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ASTRING);
+ AMutableString aString = new AMutableString("");
+ ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fields.length);
+ for (String s : fields) {
+ aString.setValue(s);
+ stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+ tupleBuilder.addFieldEndOffset();
+ }
+ ArrayTupleReference tuple = new ArrayTupleReference();
+ tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+ return tuple;
+ }
- @Override
- public List<Function> getDataverseFunctions(JobId jobId, String dataverseName) throws MetadataException,
- RemoteException {
- try {
- ITupleReference searchKey = createTuple(dataverseName);
- FunctionTupleTranslator tupleReaderWriter = new FunctionTupleTranslator(false);
- IValueExtractor<Function> valueExtractor = new MetadataEntityValueExtractor<Function>(tupleReaderWriter);
- List<Function> results = new ArrayList<Function>();
- searchIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey, valueExtractor, results);
- return results;
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public List<Function> getDataverseFunctions(JobId jobId, String dataverseName) throws MetadataException,
+ RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataverseName);
+ FunctionTupleTranslator tupleReaderWriter = new FunctionTupleTranslator(false);
+ IValueExtractor<Function> valueExtractor = new MetadataEntityValueExtractor<Function>(tupleReaderWriter);
+ List<Function> results = new ArrayList<Function>();
+ searchIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey, valueExtractor, results);
+ return results;
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public void addAdapter(JobId jobId, DatasourceAdapter adapter) throws MetadataException, RemoteException {
- try {
- // Insert into the 'Adapter' dataset.
- DatasourceAdapterTupleTranslator tupleReaderWriter = new DatasourceAdapterTupleTranslator(true);
- ITupleReference adapterTuple = tupleReaderWriter.getTupleFromMetadataEntity(adapter);
- insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, adapterTuple);
+ @Override
+ public void addAdapter(JobId jobId, DatasourceAdapter adapter) throws MetadataException, RemoteException {
+ try {
+ // Insert into the 'Adapter' dataset.
+ DatasourceAdapterTupleTranslator tupleReaderWriter = new DatasourceAdapterTupleTranslator(true);
+ ITupleReference adapterTuple = tupleReaderWriter.getTupleFromMetadataEntity(adapter);
+ insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, adapterTuple);
- } catch (TreeIndexDuplicateKeyException e) {
- throw new MetadataException("A adapter with this name " + adapter.getAdapterIdentifier().getAdapterName()
- + " already exists in dataverse '" + adapter.getAdapterIdentifier().getNamespace() + "'.", e);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
+ } catch (TreeIndexDuplicateKeyException e) {
+ throw new MetadataException("A adapter with this name " + adapter.getAdapterIdentifier().getAdapterName()
+ + " already exists in dataverse '" + adapter.getAdapterIdentifier().getNamespace() + "'.", e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
- }
+ }
- @Override
- public void dropAdapter(JobId jobId, String dataverseName, String adapterName) throws MetadataException,
- RemoteException {
- DatasourceAdapter adapter;
- try {
- adapter = getAdapter(jobId, dataverseName, adapterName);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- if (adapter == null) {
- throw new MetadataException("Cannot drop adapter '" + adapter + "' because it doesn't exist.");
- }
- try {
- // Delete entry from the 'Adapter' dataset.
- ITupleReference searchKey = createTuple(dataverseName, adapterName);
- // Searches the index for the tuple to be deleted. Acquires an S
- // lock on the 'Adapter' dataset.
- ITupleReference datasetTuple = getTupleToBeDeleted(jobId,
- MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey);
- deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, datasetTuple);
+ @Override
+ public void dropAdapter(JobId jobId, String dataverseName, String adapterName) throws MetadataException,
+ RemoteException {
+ DatasourceAdapter adapter;
+ try {
+ adapter = getAdapter(jobId, dataverseName, adapterName);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ if (adapter == null) {
+ throw new MetadataException("Cannot drop adapter '" + adapter + "' because it doesn't exist.");
+ }
+ try {
+ // Delete entry from the 'Adapter' dataset.
+ ITupleReference searchKey = createTuple(dataverseName, adapterName);
+ // Searches the index for the tuple to be deleted. Acquires an S
+ // lock on the 'Adapter' dataset.
+ ITupleReference datasetTuple = getTupleToBeDeleted(jobId,
+ MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey);
+ deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, datasetTuple);
- // TODO: Change this to be a BTree specific exception, e.g.,
- // BTreeKeyDoesNotExistException.
- } catch (TreeIndexException e) {
- throw new MetadataException("Cannot drop adapter '" + adapterName, e);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
+ // TODO: Change this to be a BTree specific exception, e.g.,
+ // BTreeKeyDoesNotExistException.
+ } catch (TreeIndexException e) {
+ throw new MetadataException("Cannot drop adapter '" + adapterName, e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
- }
+ }
- @Override
- public DatasourceAdapter getAdapter(JobId jobId, String dataverseName, String adapterName)
- throws MetadataException, RemoteException {
- try {
- ITupleReference searchKey = createTuple(dataverseName, adapterName);
- DatasourceAdapterTupleTranslator tupleReaderWriter = new DatasourceAdapterTupleTranslator(false);
- List<DatasourceAdapter> results = new ArrayList<DatasourceAdapter>();
- IValueExtractor<DatasourceAdapter> valueExtractor = new MetadataEntityValueExtractor<DatasourceAdapter>(
- tupleReaderWriter);
- searchIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey, valueExtractor, results);
- if (results.isEmpty()) {
- return null;
- }
- return results.get(0);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public DatasourceAdapter getAdapter(JobId jobId, String dataverseName, String adapterName)
+ throws MetadataException, RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataverseName, adapterName);
+ DatasourceAdapterTupleTranslator tupleReaderWriter = new DatasourceAdapterTupleTranslator(false);
+ List<DatasourceAdapter> results = new ArrayList<DatasourceAdapter>();
+ IValueExtractor<DatasourceAdapter> valueExtractor = new MetadataEntityValueExtractor<DatasourceAdapter>(
+ tupleReaderWriter);
+ searchIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey, valueExtractor, results);
+ if (results.isEmpty()) {
+ return null;
+ }
+ return results.get(0);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public List<DatasourceAdapter> getDataverseAdapters(JobId jobId, String dataverseName) throws MetadataException,
- RemoteException {
- try {
- ITupleReference searchKey = createTuple(dataverseName);
- DatasourceAdapterTupleTranslator tupleReaderWriter = new DatasourceAdapterTupleTranslator(false);
- IValueExtractor<DatasourceAdapter> valueExtractor = new MetadataEntityValueExtractor<DatasourceAdapter>(
- tupleReaderWriter);
- List<DatasourceAdapter> results = new ArrayList<DatasourceAdapter>();
- searchIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey, valueExtractor, results);
- return results;
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public List<DatasourceAdapter> getDataverseAdapters(JobId jobId, String dataverseName) throws MetadataException,
+ RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataverseName);
+ DatasourceAdapterTupleTranslator tupleReaderWriter = new DatasourceAdapterTupleTranslator(false);
+ IValueExtractor<DatasourceAdapter> valueExtractor = new MetadataEntityValueExtractor<DatasourceAdapter>(
+ tupleReaderWriter);
+ List<DatasourceAdapter> results = new ArrayList<DatasourceAdapter>();
+ searchIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey, valueExtractor, results);
+ return results;
+ } 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/bootstrap/MetadataRecordTypes.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataRecordTypes.java
index 50681ee..d3eee76 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
@@ -125,12 +125,13 @@
// 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" };
- IAType[] fieldTypes = { BuiltinType.ASTRING, orderedPropertyListType };
+ String[] fieldNames = { "DatasourceAdapter", "Properties", "GroupName" };
+ IAType[] fieldTypes = { BuiltinType.ASTRING, orderedPropertyListType, BuiltinType.ASTRING };
return new ARecordType(null, fieldNames, fieldTypes, true);
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
index 1ade7ef..98320ce 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
@@ -41,6 +41,8 @@
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;
@@ -398,6 +400,114 @@
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(dataScanner, constraint);
}
+
+ @SuppressWarnings("rawtypes")
+ public Pair<ExternalDataIndexingOperatorDescriptor, AlgebricksPartitionConstraint> buildExternalDataIndexingRuntime(
+ JobSpecification jobSpec, IAType itemType, ExternalDatasetDetails datasetDetails, IDataFormat format)
+ throws AlgebricksException {
+ 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).createIndexingAdapter(
+ 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");
+ }
+ ARecordType rt = (ARecordType) itemType;
+ ISerializerDeserializer payloadSerde = format.getSerdeProvider().getSerializerDeserializer(itemType);
+ RecordDescriptor recordDesc = new RecordDescriptor(new ISerializerDeserializer[] { payloadSerde });
+ ExternalDataIndexingOperatorDescriptor dataIndexScanner = new ExternalDataIndexingOperatorDescriptor(jobSpec,
+ wrapPropertiesEmpty(datasetDetails.getProperties()), rt, recordDesc, adapterFactory);
+ AlgebricksPartitionConstraint constraint;
+ try {
+ constraint = adapter.getPartitionConstraint();
+ } catch (Exception e) {
+ throw new AlgebricksException(e);
+ }
+ return new Pair<ExternalDataIndexingOperatorDescriptor, AlgebricksPartitionConstraint>(dataIndexScanner, constraint);
+ }
+
+ @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 dataAccessor = new ExternalDataAccessByRIDOperatorDescriptor(jobSpec, wrapPropertiesEmpty(datasetDetails.getProperties()),
+ itemType, outRecDesc, adapterFactory);
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraints = splitProviderAndPartitionConstraintsForExternalDataset(dataset.getDataverseName(),dataset.getDatasetName(),secondaryIndex.getIndexName());
+ return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(dataAccessor, splitsAndConstraints.second);
+ }
@SuppressWarnings("rawtypes")
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildScannerRuntime(JobSpecification jobSpec,
@@ -500,94 +610,140 @@
}
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildBtreeRuntime(JobSpecification jobSpec,
- List<LogicalVariable> outputVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
- JobGenContext context, boolean retainInput, Dataset dataset, String indexName, int[] lowKeyFields,
- int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive, Object implConfig)
- throws AlgebricksException {
- boolean isSecondary = true;
- 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);
+ List<LogicalVariable> outputVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
+ JobGenContext context, boolean retainInput, Dataset dataset, String indexName, int[] lowKeyFields,
+ 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;
+ Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
+ dataset.getDatasetName(), indexName);
+ int numSecondaryKeys = secondaryIndex.getKeyFieldNames().size();
+ numKeys += numSecondaryKeys;
+ int 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);
+ }
+ ISearchOperationCallbackFactory searchCallbackFactory = null;
+ searchCallbackFactory = new SecondaryIndexSearchOperationCallbackFactory();
+ AsterixRuntimeComponentsProvider rtcProvider = AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_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,
+ AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER, rtcProvider,
+ rtcProvider, storageProperties.getBloomFilterFalsePositiveRate()), retainInput,
+ searchCallbackFactory);
+ return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeSearchOp, spPc.second);
+ } catch (MetadataException me) {
+ throw new AlgebricksException(me);
+ }
+ }
+ else
+ {
+ 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);
- }
+ 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;
- }
+ 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 = isSecondary ? AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER
- : AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_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 ? AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER
- : new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()), rtcProvider,
- rtcProvider, storageProperties.getBloomFilterFalsePositiveRate()), retainInput,
- searchCallbackFactory);
+ 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 = isSecondary ? AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER
+ : AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_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 ? AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER
+ : new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()), rtcProvider,
+ rtcProvider, storageProperties.getBloomFilterFalsePositiveRate()), retainInput,
+ searchCallbackFactory);
- return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeSearchOp, spPc.second);
+ return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeSearchOp, spPc.second);
- } catch (MetadataException me) {
- throw new AlgebricksException(me);
- }
- }
-
+ } catch (MetadataException me) {
+ throw new AlgebricksException(me);
+ }
+ }
+ }
+
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildRtreeRuntime(JobSpecification jobSpec,
List<LogicalVariable> outputVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
JobGenContext context, boolean retainInput, Dataset dataset, String indexName, int[] keyFields)
@@ -698,9 +854,6 @@
throws AlgebricksException {
AqlDataSource ads = findDataSource(dataSourceId);
Dataset dataset = ads.getDataset();
- if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
- throw new AlgebricksException("No index for external dataset " + dataSourceId);
- }
try {
String indexName = (String) indexId;
Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
@@ -1362,6 +1515,12 @@
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);
@@ -1452,6 +1611,56 @@
}
}
+ 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",
@@ -1464,6 +1673,8 @@
"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 18cef340..87fc5a5 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
@@ -41,13 +41,21 @@
private static final long serialVersionUID = 1L;
private final String adapter;
private final Map<String, String> properties;
+ private final String nodeGroupName;
private final static ARecordType externalRecordType = MetadataRecordTypes.EXTERNAL_DETAILS_RECORDTYPE;
private final static ARecordType propertyRecordType = MetadataRecordTypes.DATASOURCE_ADAPTER_PROPERTIES_RECORDTYPE;
- public ExternalDatasetDetails(String adapter, Map<String, String> properties) {
+ public ExternalDatasetDetails(String adapter, Map<String, String> properties, String nodeGroupName) {
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() {
@@ -63,6 +71,10 @@
return DatasetType.EXTERNAL;
}
+ public String getNodeGroupName() {
+ return nodeGroupName;
+ }
+
@Override
public void writeDatasetDetailsRecordType(DataOutput out) throws HyracksDataException {
IARecordBuilder externalRecordBuilder = new RecordBuilder();
@@ -90,6 +102,13 @@
writePropertyTypeRecord(name, value, itemValue.getDataOutput());
listBuilder.addItem(itemValue);
}
+
+ //write field 2
+ fieldValue.reset();
+ aString.setValue(getNodeGroupName());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ externalRecordBuilder.addField(MetadataRecordTypes.EXTERNAL_DETAILS_ARECORD_GROUPNAME_FIELD_INDEX, fieldValue);
+
fieldValue.reset();
listBuilder.write(fieldValue.getDataOutput(), true);
externalRecordBuilder.addField(MetadataRecordTypes.EXTERNAL_DETAILS_ARECORD_PROPERTIES_FIELD_INDEX, fieldValue);
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 61f856a..d346a6c 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,7 +219,10 @@
.getStringValue();
properties.put(key, value);
}
- datasetDetails = new ExternalDatasetDetails(adapter, properties);
+ String groupName = ((AString) datasetDetailsRecord
+ .getValueByPos(MetadataRecordTypes.EXTERNAL_DETAILS_ARECORD_GROUPNAME_FIELD_INDEX ))
+ .getStringValue();
+ datasetDetails = new ExternalDatasetDetails(adapter, properties,groupName);
}
Map<String, String> hints = getDatasetHints(datasetRecord);
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 820f277..534c183 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
@@ -21,8 +21,10 @@
import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
import edu.uci.ics.asterix.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;
@@ -32,6 +34,8 @@
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 {
@@ -82,6 +86,32 @@
}
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), Int64(Block byte location), Int32(row number)>
+ IBinaryHashFunctionFactory[] bhffs = new IBinaryHashFunctionFactory[3];
+ 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), Int64(Record byte location)>
+ IBinaryHashFunctionFactory[] bhffs = new IBinaryHashFunctionFactory[2];
+ bhffs[0] = hashFunProvider.getBinaryHashFunctionFactory(BuiltinType.ASTRING);
+ bhffs[1] = hashFunProvider.getBinaryHashFunctionFactory(BuiltinType.AINT64);
+ return bhffs;
+ }
+ }
public static ITypeTraits[] computeTupleTypeTraits(Dataset dataset, ARecordType itemType)
throws AlgebricksException {
@@ -112,6 +142,17 @@
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 cc3fd04..00b7459 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,6 +33,10 @@
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 41cf3c1..2aa765c 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,6 +243,8 @@
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);
@@ -660,6 +662,14 @@
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/operators/file/ADMDataParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ADMDataParser.java
index b8c8659..e08c509 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,6 +92,16 @@
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
new file mode 100644
index 0000000..a4f1691
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractControlledTupleParser.java
@@ -0,0 +1,123 @@
+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 2322338..78159f5 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,6 +62,7 @@
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 7cb9bb0..34f75e6 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 delimited files.
+ * parsing adm formatted input 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
new file mode 100644
index 0000000..aa2d9ba
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledADMTupleParser.java
@@ -0,0 +1,23 @@
+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
new file mode 100644
index 0000000..d72366c
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledADMTupleParserFactory.java
@@ -0,0 +1,28 @@
+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
new file mode 100644
index 0000000..15643d4
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledDelimitedDataTupleParser.java
@@ -0,0 +1,23 @@
+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
new file mode 100644
index 0000000..ced33ef
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledDelimitedDataTupleParserFactory.java
@@ -0,0 +1,29 @@
+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
new file mode 100644
index 0000000..10b09f5
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledTupleParser.java
@@ -0,0 +1,29 @@
+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 921dbf3..59eb908 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
@@ -96,6 +96,17 @@
}
@Override
+ public void close() {
+ cursor.close();
+ }
+
+ @Override
+ public void reset()
+ {
+ cursor.reset();
+ }
+
+ @Override
public boolean parse(DataOutput out) throws AsterixException, IOException {
if (cursor.nextRecord()) {
@@ -146,7 +157,7 @@
private static final int INITIAL_BUFFER_SIZE = 4096;
private static final int INCREMENT = 4096;
- private final Reader in;
+ private Reader in;
private char[] buffer;
private int start;
@@ -163,6 +174,21 @@
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 46d281b..cd9ae2e 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,4 +52,17 @@
* @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 e037ec6..ffd9edd 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,7 +224,8 @@
while (continueIngestion) {
tb.reset();
if (!parser.parse(tb.getDataOutput())) {
- break;
+ parser.close();
+ 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 bf3c086..23d4ef4 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
@@ -19,9 +19,11 @@
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
@@ -67,6 +69,12 @@
public String getName() {
return "file_feed";
}
+
+ @Override
+ public IDatasourceAdapter createIndexingAdapter(
+ Map<String, Object> configuration, IAType atype) 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) {
@@ -83,4 +91,10 @@
}
}
+ @Override
+ public IControlledAdapter createAccessByRIDAdapter(
+ Map<String, Object> configuration, IAType atype) throws Exception {
+ throw new NotImplementedException("Rate Controlled Access by RID Adapter is not implemented for feeds");
+ }
+
}
\ No newline at end of file