Fixed issue 96. Got rid of a bunch of AqlCompiledSomethings that have been replaced by the corresponding persistent metadata entities.
git-svn-id: https://asterixdb.googlecode.com/svn/branches/asterix_stabilization@454 eaa15691-b419-025a-1212-ee371bd00084
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/BTreeSearchPOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/BTreeSearchPOperator.java
index 6b305e2..dad8d0a 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/BTreeSearchPOperator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/BTreeSearchPOperator.java
@@ -3,17 +3,14 @@
import java.util.ArrayList;
import java.util.List;
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl.IndexKind;
import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
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.om.functions.AsterixBuiltinFunctions;
import edu.uci.ics.asterix.optimizer.rules.am.BTreeJobGenParams;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -61,21 +58,11 @@
}
BTreeJobGenParams jobGenParams = new BTreeJobGenParams();
jobGenParams.readFromFuncArgs(unnestFuncExpr.getArguments());
- if (jobGenParams.getIndexKind() != IndexKind.BTREE) {
- throw new NotImplementedException(jobGenParams.getIndexKind() + " indexes are not implemented.");
- }
int[] lowKeyIndexes = getKeyIndexes(jobGenParams.getLowKeyVarList(), inputSchemas);
int[] highKeyIndexes = getKeyIndexes(jobGenParams.getHighKeyVarList(), inputSchemas);
AqlMetadataProvider metadataProvider = (AqlMetadataProvider) context.getMetadataProvider();
AqlCompiledMetadataDeclarations metadata = metadataProvider.getMetadataDeclarations();
- AqlCompiledDatasetDecl datasetDecl = metadata.findDataset(jobGenParams.getDatasetName());
- if (datasetDecl == null) {
- throw new AlgebricksException("Unknown dataset " + jobGenParams.getDatasetName());
- }
- if (datasetDecl.getDatasetType() == DatasetType.EXTERNAL) {
- throw new AlgebricksException("Trying to run btree search over external dataset ("
- + jobGenParams.getDatasetName() + ").");
- }
+ Dataset dataset = metadata.findDataset(jobGenParams.getDatasetName());
IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(op);
List<LogicalVariable> outputVars = unnestMap.getVariables();
if (jobGenParams.getRetainInput()) {
@@ -84,7 +71,7 @@
}
Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> btreeSearch = AqlMetadataProvider.buildBtreeRuntime(
builder.getJobSpec(), outputVars, opSchema, typeEnv, metadata, context, jobGenParams.getRetainInput(),
- jobGenParams.getDatasetName(), datasetDecl, jobGenParams.getIndexName(), lowKeyIndexes, highKeyIndexes,
+ jobGenParams.getDatasetName(), dataset, jobGenParams.getIndexName(), lowKeyIndexes, highKeyIndexes,
jobGenParams.isLowKeyInclusive(), jobGenParams.isHighKeyInclusive());
builder.contributeHyracksOperator(unnestMap, btreeSearch.first);
builder.contributeAlgebricksPartitionConstraint(btreeSearch.first, btreeSearch.second);
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
index f6c49ab..ae2559b 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
@@ -3,14 +3,12 @@
import java.util.ArrayList;
import java.util.List;
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
import edu.uci.ics.asterix.common.dataflow.IAsterixApplicationContextInfo;
-import edu.uci.ics.asterix.metadata.MetadataException;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
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;
import edu.uci.ics.asterix.om.base.IAObject;
import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
@@ -52,7 +50,7 @@
import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
/**
- * Contributes the runtime operator for an unnest-map representing an inverted-index search.
+ * Contributes the runtime operator for an unnest-map representing an inverted-index search.
*/
public class InvertedIndexPOperator extends IndexSearchPOperator {
public InvertedIndexPOperator(IDataSourceIndex<String, AqlSourceId> idx, boolean requiresBroadcast) {
@@ -61,15 +59,13 @@
@Override
public PhysicalOperatorTag getOperatorTag() {
- // TODO: Currently, I'm relying on my own version of Algebricks, not the released one.
- // Need to add this tag in Algebricks.
return PhysicalOperatorTag.INVERTED_INDEX_SEARCH;
}
@Override
public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
- throws AlgebricksException {
+ throws AlgebricksException {
UnnestMapOperator unnestMapOp = (UnnestMapOperator) op;
ILogicalExpression unnestExpr = unnestMapOp.getExpressionRef().getValue();
if (unnestExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
@@ -81,48 +77,40 @@
}
InvertedIndexJobGenParams jobGenParams = new InvertedIndexJobGenParams();
jobGenParams.readFromFuncArgs(unnestFuncExpr.getArguments());
-
+
AqlMetadataProvider metadataProvider = (AqlMetadataProvider) context.getMetadataProvider();
AqlCompiledMetadataDeclarations metadata = metadataProvider.getMetadataDeclarations();
- AqlCompiledDatasetDecl datasetDecl = metadata.findDataset(jobGenParams.getDatasetName());
- if (datasetDecl == null) {
- throw new AlgebricksException("Unknown dataset " + jobGenParams.getDatasetName());
- }
- if (datasetDecl.getDatasetType() == DatasetType.EXTERNAL) {
- throw new AlgebricksException("Trying to run inverted index search over external dataset (" + jobGenParams.getDatasetName() + ").");
- }
+ Dataset dataset = metadata.findDataset(jobGenParams.getDatasetName());
int[] keyIndexes = getKeyIndexes(jobGenParams.getKeyVarList(), inputSchemas);
-
+
// Build runtime.
Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> invIndexSearch = buildInvertedIndexRuntime(metadata,
context, builder.getJobSpec(), unnestMapOp, opSchema, jobGenParams.getRetainInput(),
- jobGenParams.getDatasetName(), datasetDecl, jobGenParams.getIndexName(),
- jobGenParams.getSearchKeyType(), keyIndexes, jobGenParams.getSearchModifierType(),
- jobGenParams.getSimilarityThreshold());
+ jobGenParams.getDatasetName(), dataset, jobGenParams.getIndexName(), jobGenParams.getSearchKeyType(),
+ keyIndexes, jobGenParams.getSearchModifierType(), jobGenParams.getSimilarityThreshold());
// Contribute operator in hyracks job.
builder.contributeHyracksOperator(unnestMapOp, invIndexSearch.first);
- builder.contributeAlgebricksPartitionConstraint(invIndexSearch.first, invIndexSearch.second);
+ builder.contributeAlgebricksPartitionConstraint(invIndexSearch.first, invIndexSearch.second);
ILogicalOperator srcExchange = unnestMapOp.getInputs().get(0).getValue();
builder.contributeGraphEdge(srcExchange, 0, unnestMapOp, 0);
}
-
+
public static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildInvertedIndexRuntime(
- AqlCompiledMetadataDeclarations metadata, JobGenContext context,
- JobSpecification jobSpec, UnnestMapOperator unnestMap,
- IOperatorSchema opSchema, boolean retainInput, String datasetName,
- AqlCompiledDatasetDecl datasetDecl, String indexName,
- ATypeTag searchKeyType, int[] keyFields, SearchModifierType searchModifierType,
- IAlgebricksConstantValue similarityThreshold) throws AlgebricksException {
- IAObject simThresh = ((AsterixConstantValue)similarityThreshold).getObject();
- String itemTypeName = datasetDecl.getItemTypeName();
+ AqlCompiledMetadataDeclarations metadata, JobGenContext context, JobSpecification jobSpec,
+ UnnestMapOperator unnestMap, IOperatorSchema opSchema, boolean retainInput, String datasetName,
+ Dataset dataset, String indexName, ATypeTag searchKeyType, int[] keyFields,
+ SearchModifierType searchModifierType, IAlgebricksConstantValue similarityThreshold)
+ throws AlgebricksException {
+ IAObject simThresh = ((AsterixConstantValue) similarityThreshold).getObject();
+ String itemTypeName = dataset.getItemTypeName();
IAType itemType = metadata.findType(itemTypeName);
- int numPrimaryKeys = DatasetUtils.getPartitioningFunctions(datasetDecl).size();
- AqlCompiledIndexDecl index = DatasetUtils.findSecondaryIndexByName(datasetDecl, indexName);
- if (index == null) {
+ int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
+ Index secondaryIndex = metadata.getIndex(dataset.getDataverseName(), dataset.getDatasetName(), indexName);
+ if (secondaryIndex == null) {
throw new AlgebricksException("Code generation error: no index " + indexName + " for dataset "
+ datasetName);
}
- List<String> secondaryKeyFields = index.getFieldExprs();
+ List<String> secondaryKeyFields = secondaryIndex.getKeyFieldNames();
int numSecondaryKeys = secondaryKeyFields.size();
if (numSecondaryKeys != 1) {
throw new AlgebricksException(
@@ -134,7 +122,7 @@
throw new AlgebricksException("Only record types can be indexed.");
}
ARecordType recordType = (ARecordType) itemType;
- Pair<IAType, Boolean> keyPairType = AqlCompiledIndexDecl.getNonNullableKeyFieldType(secondaryKeyFields.get(0), recordType);
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(secondaryKeyFields.get(0), recordType);
IAType secondaryKeyType = keyPairType.first;
if (secondaryKeyType == null) {
throw new AlgebricksException("Could not find field " + secondaryKeyFields.get(0) + " in the schema.");
@@ -148,7 +136,7 @@
tokenComparatorFactories[i] = InvertedIndexAccessMethod.getTokenBinaryComparatorFactory(secondaryKeyType);
tokenTypeTraits[i] = InvertedIndexAccessMethod.getTokenTypeTrait(secondaryKeyType);
}
-
+
IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(unnestMap);
List<LogicalVariable> outputVars = unnestMap.getVariables();
if (retainInput) {
@@ -156,37 +144,33 @@
VariableUtilities.getLiveVariables(unnestMap, outputVars);
}
RecordDescriptor outputRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
-
+
int start = outputRecDesc.getFieldCount() - numPrimaryKeys;
- IBinaryComparatorFactory[] invListsComparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(outputVars, start, numPrimaryKeys, typeEnv, context);
- ITypeTraits[] invListsTypeTraits = JobGenHelper.variablesToTypeTraits(outputVars, start, numPrimaryKeys, typeEnv, context);
-
- IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint;
- try {
- secondarySplitsAndConstraint = metadata
- .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
- datasetName, indexName);
- } catch (MetadataException e) {
- throw new AlgebricksException(e);
- }
-
+ IBinaryComparatorFactory[] invListsComparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(
+ outputVars, start, numPrimaryKeys, typeEnv, context);
+ ITypeTraits[] invListsTypeTraits = JobGenHelper.variablesToTypeTraits(outputVars, start, numPrimaryKeys,
+ typeEnv, context);
+
+ IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadata
+ .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
Pair<IFileSplitProvider, IFileSplitProvider> fileSplitProviders = metadata
.getInvertedIndexFileSplitProviders(secondarySplitsAndConstraint.first);
-
+
// TODO: Here we assume there is only one search key field.
int queryField = keyFields[0];
// Get tokenizer and search modifier factories.
- IInvertedIndexSearchModifierFactory searchModifierFactory = InvertedIndexAccessMethod.getSearchModifierFactory(searchModifierType, simThresh, index);
- IBinaryTokenizerFactory queryTokenizerFactory = InvertedIndexAccessMethod.getBinaryTokenizerFactory(searchModifierType, searchKeyType, index);
- InvertedIndexSearchOperatorDescriptor invIndexSearchOp = new InvertedIndexSearchOperatorDescriptor(
- jobSpec, queryField, appContext.getStorageManagerInterface(),
- fileSplitProviders.first, fileSplitProviders.second,
- appContext.getIndexRegistryProvider(), tokenTypeTraits,
- tokenComparatorFactories, invListsTypeTraits,
- invListsComparatorFactories, new BTreeDataflowHelperFactory(),
- queryTokenizerFactory, searchModifierFactory, outputRecDesc,
- retainInput, NoOpOperationCallbackProvider.INSTANCE);
- return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(invIndexSearchOp, secondarySplitsAndConstraint.second);
+ IInvertedIndexSearchModifierFactory searchModifierFactory = InvertedIndexAccessMethod.getSearchModifierFactory(
+ searchModifierType, simThresh, secondaryIndex);
+ IBinaryTokenizerFactory queryTokenizerFactory = InvertedIndexAccessMethod.getBinaryTokenizerFactory(
+ searchModifierType, searchKeyType, secondaryIndex);
+ InvertedIndexSearchOperatorDescriptor invIndexSearchOp = new InvertedIndexSearchOperatorDescriptor(jobSpec,
+ queryField, appContext.getStorageManagerInterface(), fileSplitProviders.first,
+ fileSplitProviders.second, appContext.getIndexRegistryProvider(), tokenTypeTraits,
+ tokenComparatorFactories, invListsTypeTraits, invListsComparatorFactories,
+ new BTreeDataflowHelperFactory(), queryTokenizerFactory, searchModifierFactory, outputRecDesc,
+ retainInput, NoOpOperationCallbackProvider.INSTANCE);
+ return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(invIndexSearchOp,
+ secondarySplitsAndConstraint.second);
}
}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/RTreeSearchPOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/RTreeSearchPOperator.java
index 1832343..27a477c 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/RTreeSearchPOperator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/RTreeSearchPOperator.java
@@ -1,16 +1,13 @@
package edu.uci.ics.asterix.algebra.operators.physical;
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl.IndexKind;
import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
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.om.functions.AsterixBuiltinFunctions;
import edu.uci.ics.asterix.optimizer.rules.am.RTreeJobGenParams;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -56,23 +53,16 @@
RTreeJobGenParams jobGenParams = new RTreeJobGenParams();
jobGenParams.readFromFuncArgs(unnestFuncExpr.getArguments());
- if (jobGenParams.getIndexKind() != IndexKind.RTREE) {
- throw new NotImplementedException(jobGenParams.getIndexKind() + " indexes are not implemented.");
- }
int[] keyIndexes = getKeyIndexes(jobGenParams.getKeyVarList(), inputSchemas);
AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
AqlCompiledMetadataDeclarations metadata = mp.getMetadataDeclarations();
- AqlCompiledDatasetDecl adecl = metadata.findDataset(jobGenParams.getDatasetName());
- if (adecl == null) {
+ Dataset dataset = metadata.findDataset(jobGenParams.getDatasetName());
+ if (dataset == null) {
throw new AlgebricksException("Unknown dataset " + jobGenParams.getDatasetName());
}
- if (adecl.getDatasetType() == DatasetType.EXTERNAL) {
- throw new AlgebricksException("Trying to run rtree search over external dataset ("
- + jobGenParams.getDatasetName() + ").");
- }
Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> rtreeSearch = AqlMetadataProvider.buildRtreeRuntime(
- metadata, context, builder.getJobSpec(), jobGenParams.getDatasetName(), adecl,
+ metadata, context, builder.getJobSpec(), jobGenParams.getDatasetName(), dataset,
jobGenParams.getIndexName(), keyIndexes);
builder.contributeHyracksOperator(unnestMap, rtreeSearch.first);
builder.contributeAlgebricksPartitionConstraint(rtreeSearch.first, rtreeSearch.second);
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
index dc5d659..a9de14f 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
@@ -8,14 +8,13 @@
import edu.uci.ics.asterix.aql.util.FunctionUtils;
import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl.IndexKind;
+import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
import edu.uci.ics.asterix.metadata.declared.AqlIndex;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
-import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.Index;
import edu.uci.ics.asterix.om.base.AInt32;
import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
@@ -80,8 +79,8 @@
AssignOperator assignOp = (AssignOperator) op2;
ILogicalExpression assignExpr = assignOp.getExpressions().get(0).getValue();
if (assignExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
- ScalarFunctionCallExpression funcExpr = (ScalarFunctionCallExpression) assignOp.getExpressions()
- .get(0).getValue();
+ ScalarFunctionCallExpression funcExpr = (ScalarFunctionCallExpression) assignOp.getExpressions().get(0)
+ .getValue();
fid = funcExpr.getFunctionIdentifier();
}
}
@@ -92,30 +91,32 @@
AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
AqlCompiledMetadataDeclarations metadata = mp.getMetadataDeclarations();
String datasetName = datasetSource.getId().getDatasetName();
- AqlCompiledDatasetDecl adecl = metadata.findDataset(datasetName);
- if (adecl == null) {
+ Dataset dataset = metadata.findDataset(datasetName);
+ if (dataset == null) {
throw new AlgebricksException("Unknown dataset " + datasetName);
}
- if (adecl.getDatasetType() == DatasetType.EXTERNAL) {
+ if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
return false;
}
List<LogicalVariable> projectVars = new ArrayList<LogicalVariable>();
VariableUtilities.getUsedVariables(op1, projectVars);
// Create operators for secondary index insert/delete.
- String itemTypeName = adecl.getItemTypeName();
+ String itemTypeName = dataset.getItemTypeName();
IAType itemType = metadata.findType(itemTypeName);
if (itemType.getTypeTag() != ATypeTag.RECORD) {
throw new AlgebricksException("Only record types can be indexed.");
}
ARecordType recType = (ARecordType) itemType;
- List<AqlCompiledIndexDecl> secondaryIndexes = DatasetUtils.getSecondaryIndexes(adecl);
- if (secondaryIndexes.isEmpty()) {
- return false;
- }
+ List<Index> indexes = metadata.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
ILogicalOperator currentTop = op1;
- for (AqlCompiledIndexDecl index : secondaryIndexes) {
- List<String> secondaryKeyFields = index.getFieldExprs();
+ boolean hasSecondaryIndex = false;
+ for (Index index : indexes) {
+ if (!index.isSecondaryIndex()) {
+ continue;
+ }
+ hasSecondaryIndex = true;
+ List<String> secondaryKeyFields = index.getKeyFieldNames();
List<LogicalVariable> secondaryKeyVars = new ArrayList<LogicalVariable>();
List<Mutable<ILogicalExpression>> expressions = new ArrayList<Mutable<ILogicalExpression>>();
List<Mutable<ILogicalExpression>> secondaryExpressions = new ArrayList<Mutable<ILogicalExpression>>();
@@ -146,7 +147,7 @@
project.getInputs().add(new MutableObject<ILogicalOperator>(currentTop));
context.computeAndSetTypeEnvironmentForOperator(project);
context.computeAndSetTypeEnvironmentForOperator(assign);
- if (index.getKind() == IndexKind.BTREE) {
+ if (index.getIndexType() == IndexType.BTREE) {
for (LogicalVariable secondaryKeyVar : secondaryKeyVars) {
secondaryExpressions.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
secondaryKeyVar)));
@@ -160,9 +161,9 @@
indexUpdate.getInputs().add(new MutableObject<ILogicalOperator>(assign));
currentTop = indexUpdate;
context.computeAndSetTypeEnvironmentForOperator(indexUpdate);
- } else if (index.getKind() == IndexKind.RTREE) {
- Pair<IAType, Boolean> keyPairType = AqlCompiledIndexDecl.getNonNullableKeyFieldType(
- secondaryKeyFields.get(0), recType);
+ } else if (index.getIndexType() == IndexType.RTREE) {
+ Pair<IAType, Boolean> keyPairType = Index
+ .getNonNullableKeyFieldType(secondaryKeyFields.get(0), recType);
IAType spatialType = keyPairType.first;
int dimension = NonTaggedFormatUtil.getNumDimensions(spatialType.getTypeTag());
int numKeys = dimension * 2;
@@ -199,11 +200,13 @@
IndexInsertDeleteOperator indexUpdate = new IndexInsertDeleteOperator(dataSourceIndex,
insertOp.getPrimaryKeyExpressions(), secondaryExpressions, filterExpression,
insertOp.getOperation());
- indexUpdate.getInputs().add(new MutableObject<ILogicalOperator>(assignCoordinates));
- currentTop = indexUpdate;
+ indexUpdate.getInputs().add(new MutableObject<ILogicalOperator>(assignCoordinates));
+ currentTop = indexUpdate;
context.computeAndSetTypeEnvironmentForOperator(indexUpdate);
}
-
+ }
+ if (!hasSecondaryIndex) {
+ return false;
}
op0.getInputs().clear();
op0.getInputs().add(new MutableObject<ILogicalOperator>(currentTop));
@@ -219,7 +222,7 @@
IAType secondaryKeyType = (IAType) typeEnv.getVarType(secondaryKeyVar);
if (!isNullableType(secondaryKeyType) && !forceFilter) {
continue;
- }
+ }
ScalarFunctionCallExpression isNullFuncExpr = new ScalarFunctionCallExpression(
FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.IS_NULL),
new MutableObject<ILogicalExpression>(new VariableReferenceExpression(secondaryKeyVar)));
@@ -242,10 +245,10 @@
}
return filterExpression;
}
-
+
private boolean isNullableType(IAType type) {
if (type.getTypeTag() == ATypeTag.UNION) {
- return ((AUnionType)type).isNullableType();
+ return ((AUnionType) type).isNullableType();
}
return false;
}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushFieldAccessRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushFieldAccessRule.java
index 5a3d3b8..de0ee38 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushFieldAccessRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushFieldAccessRule.java
@@ -11,11 +11,11 @@
import edu.uci.ics.asterix.algebra.base.AsterixOperatorAnnotations;
import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
import edu.uci.ics.asterix.common.exceptions.AsterixRuntimeException;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
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;
import edu.uci.ics.asterix.om.base.AInt32;
import edu.uci.ics.asterix.om.base.AString;
@@ -55,7 +55,8 @@
}
@Override
- public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
if (context.checkIfInDontApplySet(this, op)) {
return false;
@@ -116,11 +117,11 @@
AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
AqlCompiledMetadataDeclarations metadata = mp.getMetadataDeclarations();
AqlSourceId asid = ((IDataSource<AqlSourceId>) scan.getDataSource()).getId();
- AqlCompiledDatasetDecl adecl = metadata.findDataset(asid.getDatasetName());
- if (adecl == null) {
+ Dataset dataset = metadata.findDataset(asid.getDatasetName());
+ if (dataset == null) {
throw new AlgebricksException("Dataset " + asid.getDatasetName() + " not found.");
}
- if (adecl.getDatasetType() != DatasetType.INTERNAL && adecl.getDatasetType() != DatasetType.FEED) {
+ if (dataset.getDatasetType() != DatasetType.INTERNAL && dataset.getDatasetType() != DatasetType.FEED) {
return false;
}
ILogicalExpression e1 = accessFun.getArguments().get(1).getValue();
@@ -134,7 +135,7 @@
fldName = ((AString) obj).getStringValue();
} else {
int pos = ((AInt32) obj).getIntegerValue();
- String tName = adecl.getItemTypeName();
+ String tName = dataset.getItemTypeName();
IAType t = metadata.findType(tName);
if (t.getTypeTag() != ATypeTag.RECORD) {
return false;
@@ -146,9 +147,16 @@
fldName = rt.getFieldNames()[pos];
}
- List<AqlCompiledIndexDecl> idxList = DatasetUtils.findSecondaryIndexesByOneOfTheKeys(adecl, fldName);
+ List<Index> datasetIndexes = metadata.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
+ boolean hasSecondaryIndex = false;
+ for (Index index : datasetIndexes) {
+ if (index.isSecondaryIndex()) {
+ hasSecondaryIndex = true;
+ break;
+ }
+ }
- return idxList != null && !idxList.isEmpty();
+ return hasSecondaryIndex;
}
private boolean tryingToPushThroughSelectionWithSameDataSource(AssignOperator access, AbstractLogicalOperator op2) {
@@ -285,12 +293,12 @@
AqlSourceId asid = dataSource.getId();
AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
AqlCompiledMetadataDeclarations metadata = mp.getMetadataDeclarations();
- AqlCompiledDatasetDecl adecl = metadata.findDataset(asid.getDatasetName());
- if (adecl == null) {
+ Dataset dataset = metadata.findDataset(asid.getDatasetName());
+ if (dataset == null) {
throw new AlgebricksException("Dataset " + asid.getDatasetName() + " not found.");
}
- if (adecl.getDatasetType() != DatasetType.INTERNAL
- && adecl.getDatasetType() != DatasetType.FEED) {
+ if (dataset.getDatasetType() != DatasetType.INTERNAL
+ && dataset.getDatasetType() != DatasetType.FEED) {
setAsFinal(access, context, finalAnnot);
return false;
}
@@ -301,7 +309,7 @@
fldName = ((AString) obj).getStringValue();
} else {
int pos = ((AInt32) obj).getIntegerValue();
- String tName = adecl.getItemTypeName();
+ String tName = dataset.getItemTypeName();
IAType t = metadata.findType(tName);
if (t.getTypeTag() != ATypeTag.RECORD) {
return false;
@@ -313,7 +321,7 @@
}
fldName = rt.getFieldNames()[pos];
}
- int p = DatasetUtils.getPositionOfPartitioningKeyField(adecl, fldName);
+ int p = DatasetUtils.getPositionOfPartitioningKeyField(dataset, fldName);
if (p < 0) { // not one of the partitioning fields
setAsFinal(access, context, finalAnnot);
return false;
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
index d7ebdc4..74f790f 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
@@ -9,7 +9,7 @@
import edu.uci.ics.asterix.algebra.operators.physical.BTreeSearchPOperator;
import edu.uci.ics.asterix.algebra.operators.physical.InvertedIndexPOperator;
import edu.uci.ics.asterix.algebra.operators.physical.RTreeSearchPOperator;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl.IndexKind;
+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.om.functions.AsterixBuiltinFunctions;
@@ -157,9 +157,9 @@
throw new AlgebricksException("Could not find index " + jobGenParams.getIndexName()
+ " for dataset " + dataSourceId);
}
- IndexKind indexKind = jobGenParams.getIndexKind();
+ IndexType indexType = jobGenParams.getIndexType();
boolean requiresBroadcast = jobGenParams.getRequiresBroadcast();
- switch (indexKind) {
+ switch (indexType) {
case BTREE: {
op.setPhysicalOperator(new BTreeSearchPOperator(dsi, requiresBroadcast));
break;
@@ -177,7 +177,7 @@
break;
}
default: {
- throw new NotImplementedException(indexKind + " indexes are not implemented.");
+ throw new NotImplementedException(indexType + " indexes are not implemented.");
}
}
}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/UnnestToDataScanRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/UnnestToDataScanRule.java
index fb0c656..f25a671 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/UnnestToDataScanRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/UnnestToDataScanRule.java
@@ -6,13 +6,12 @@
import org.apache.commons.lang3.mutable.Mutable;
import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledFeedDatasetDetails;
import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
import edu.uci.ics.asterix.metadata.declared.AqlSourceId;
import edu.uci.ics.asterix.metadata.declared.ExternalFeedDataSource;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
import edu.uci.ics.asterix.om.base.AString;
import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
@@ -78,8 +77,8 @@
AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
AqlCompiledMetadataDeclarations metadata = mp.getMetadataDeclarations();
- AqlCompiledDatasetDecl acdd = metadata.findDataset(datasetName);
- if (acdd == null) {
+ Dataset dataset = metadata.findDataset(datasetName);
+ if (dataset == null) {
throw new AlgebricksException("Could not find dataset " + datasetName);
}
@@ -87,9 +86,8 @@
ArrayList<LogicalVariable> v = new ArrayList<LogicalVariable>();
- if (acdd.getDatasetType() == DatasetType.INTERNAL || acdd.getDatasetType() == DatasetType.FEED) {
-
- int numPrimaryKeys = DatasetUtils.getPartitioningFunctions(acdd).size();
+ if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
+ int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
for (int i = 0; i < numPrimaryKeys; i++) {
v.add(context.newVar());
}
@@ -105,7 +103,7 @@
return true;
}
-
+
if (fid.equals(AsterixBuiltinFunctions.FEED_INGEST)) {
if (unnest.getPositionalVariable() != null) {
throw new AlgebricksException("No positional variables are allowed over datasets.");
@@ -127,14 +125,14 @@
AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
AqlCompiledMetadataDeclarations metadata = mp.getMetadataDeclarations();
- AqlCompiledDatasetDecl acdd = metadata.findDataset(datasetName);
+ Dataset dataset = metadata.findDataset(datasetName);
- if (acdd == null) {
+ if (dataset == null) {
throw new AlgebricksException("Could not find dataset " + datasetName);
}
- if (acdd.getDatasetType() != DatasetType.FEED) {
- throw new IllegalArgumentException("invalid dataset type:" + acdd.getDatasetType());
+ if (dataset.getDatasetType() != DatasetType.FEED) {
+ throw new IllegalArgumentException("invalid dataset type:" + dataset.getDatasetType());
}
AqlSourceId asid = new AqlSourceId(metadata.getDataverseName(), datasetName);
@@ -149,7 +147,7 @@
v.add(unnest.getVariable());
- DataSourceScanOperator scan = new DataSourceScanOperator(v, createDummyFeedDataSource(asid, acdd,
+ DataSourceScanOperator scan = new DataSourceScanOperator(v, createDummyFeedDataSource(asid, dataset,
metadata));
List<Mutable<ILogicalOperator>> scanInpList = scan.getInputs();
@@ -161,36 +159,22 @@
return true;
}
}
-
return false;
}
- private AqlDataSource createDummyFeedDataSource(AqlSourceId aqlId, AqlCompiledDatasetDecl acdd,
+ private AqlDataSource createDummyFeedDataSource(AqlSourceId aqlId, Dataset dataset,
AqlCompiledMetadataDeclarations metadata) throws AlgebricksException {
-
- AqlCompiledFeedDatasetDetails feedDetails = (AqlCompiledFeedDatasetDetails) acdd.getAqlCompiledDatasetDetails();
-
if (!aqlId.getDataverseName().equals(metadata.getDataverseName())) {
return null;
}
-
- String tName = acdd.getItemTypeName();
- IAType itemType;
- try {
- itemType = metadata.findType(tName);
- } catch (Exception e) {
- throw new AlgebricksException(e);
- }
-
- AqlCompiledDatasetDecl dummySourceDecl = new AqlCompiledDatasetDecl(acdd.getName(), tName,
- DatasetType.EXTERNAL, feedDetails);
-
- ExternalFeedDataSource extDataSource = new ExternalFeedDataSource(aqlId, dummySourceDecl, itemType,
+ String tName = dataset.getItemTypeName();
+ IAType itemType = metadata.findType(tName);
+ ExternalFeedDataSource extDataSource = new ExternalFeedDataSource(aqlId, dataset, itemType,
AqlDataSource.AqlDataSourceType.EXTERNAL_FEED);
return extDataSource;
}
-
+
public void addPrimaryKey(List<LogicalVariable> scanVariables, IOptimizationContext context) {
int n = scanVariables.size();
List<LogicalVariable> head = new ArrayList<LogicalVariable>(scanVariables.subList(0, n - 1));
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
index e4f795b..b378ed7 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
@@ -7,14 +7,17 @@
import org.apache.commons.lang3.mutable.Mutable;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
+import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
+import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.Index;
import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
import edu.uci.ics.asterix.om.base.AInt32;
import edu.uci.ics.asterix.om.base.AString;
import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
@@ -33,10 +36,13 @@
* Class that embodies the commonalities between rewrite rules for access methods.
*/
public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRewriteRule {
-
+
+ private AqlCompiledMetadataDeclarations metadata;
+
public abstract Map<FunctionIdentifier, List<IAccessMethod>> getAccessMethods();
-
- protected static void registerAccessMethod(IAccessMethod accessMethod, Map<FunctionIdentifier, List<IAccessMethod>> accessMethods) {
+
+ protected static void registerAccessMethod(IAccessMethod accessMethod,
+ Map<FunctionIdentifier, List<IAccessMethod>> accessMethods) {
List<FunctionIdentifier> funcs = accessMethod.getOptimizableFunctions();
for (FunctionIdentifier funcIdent : funcs) {
List<IAccessMethod> l = accessMethods.get(funcIdent);
@@ -47,16 +53,23 @@
l.add(accessMethod);
}
}
-
+
@Override
public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
return false;
}
-
- protected void fillSubTreeIndexExprs(OptimizableOperatorSubTree subTree, Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
+
+ protected void setMetadataDeclarations(IOptimizationContext context) {
+ AqlMetadataProvider metadataProvider = (AqlMetadataProvider) context.getMetadataProvider();
+ metadata = metadataProvider.getMetadataDeclarations();
+ }
+
+ protected void fillSubTreeIndexExprs(OptimizableOperatorSubTree subTree,
+ Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) throws AlgebricksException {
// The assign may be null if there is only a filter on the primary index key.
// Match variables from lowest assign which comes directly after the dataset scan.
- List<LogicalVariable> varList = (!subTree.assigns.isEmpty()) ? subTree.assigns.get(subTree.assigns.size() - 1).getVariables() : subTree.dataSourceScan.getVariables();
+ List<LogicalVariable> varList = (!subTree.assigns.isEmpty()) ? subTree.assigns.get(subTree.assigns.size() - 1)
+ .getVariables() : subTree.dataSourceScan.getVariables();
Iterator<Map.Entry<IAccessMethod, AccessMethodAnalysisContext>> amIt = analyzedAMs.entrySet().iterator();
// Check applicability of indexes by access method type.
while (amIt.hasNext()) {
@@ -80,26 +93,25 @@
}
}
}
-
+
/**
* Simply picks the first index that it finds.
* TODO: Improve this decision process by making it more systematic.
*/
- protected Pair<IAccessMethod, AqlCompiledIndexDecl> chooseIndex(
- Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
+ protected Pair<IAccessMethod, Index> chooseIndex(Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
Iterator<Map.Entry<IAccessMethod, AccessMethodAnalysisContext>> amIt = analyzedAMs.entrySet().iterator();
while (amIt.hasNext()) {
Map.Entry<IAccessMethod, AccessMethodAnalysisContext> amEntry = amIt.next();
AccessMethodAnalysisContext analysisCtx = amEntry.getValue();
- Iterator<Map.Entry<AqlCompiledIndexDecl, List<Integer>>> indexIt = analysisCtx.indexExprs.entrySet().iterator();
+ Iterator<Map.Entry<Index, List<Integer>>> indexIt = analysisCtx.indexExprs.entrySet().iterator();
if (indexIt.hasNext()) {
- Map.Entry<AqlCompiledIndexDecl, List<Integer>> indexEntry = indexIt.next();
- return new Pair<IAccessMethod, AqlCompiledIndexDecl>(amEntry.getKey(), indexEntry.getKey());
+ Map.Entry<Index, List<Integer>> indexEntry = indexIt.next();
+ return new Pair<IAccessMethod, Index>(amEntry.getKey(), indexEntry.getKey());
}
}
return null;
}
-
+
/**
* Removes irrelevant access methods candidates, based on whether the
* expressions in the query match those in the index. For example, some
@@ -107,18 +119,17 @@
* only require a match on a prefix of fields to be applicable. This methods
* removes all index candidates indexExprs that are definitely not
* applicable according to the expressions involved.
- *
*/
public void pruneIndexCandidates(IAccessMethod accessMethod, AccessMethodAnalysisContext analysisCtx) {
- Iterator<Map.Entry<AqlCompiledIndexDecl, List<Integer>>> it = analysisCtx.indexExprs.entrySet().iterator();
+ Iterator<Map.Entry<Index, List<Integer>>> it = analysisCtx.indexExprs.entrySet().iterator();
while (it.hasNext()) {
- Map.Entry<AqlCompiledIndexDecl, List<Integer>> entry = it.next();
- AqlCompiledIndexDecl index = entry.getKey();
+ Map.Entry<Index, List<Integer>> entry = it.next();
+ Index index = entry.getKey();
Iterator<Integer> exprsIter = entry.getValue().iterator();
boolean allUsed = true;
int lastFieldMatched = -1;
- for (int i = 0; i < index.getFieldExprs().size(); i++) {
- String keyField = index.getFieldExprs().get(i);
+ for (int i = 0; i < index.getKeyFieldNames().size(); i++) {
+ String keyField = index.getKeyFieldNames().get(i);
boolean foundKeyField = false;
while (exprsIter.hasNext()) {
Integer ix = exprsIter.next();
@@ -140,7 +151,7 @@
if (!foundKeyField) {
allUsed = false;
break;
- }
+ }
}
// If the access method requires all exprs to be matched but they are not, remove this candidate.
if (!allUsed && accessMethod.matchAllIndexExprs()) {
@@ -154,37 +165,39 @@
}
}
}
-
+
/**
* Analyzes the given selection condition, filling analyzedAMs with applicable access method types.
* At this point we are not yet consulting the metadata whether an actual index exists or not.
*/
- protected boolean analyzeCondition(ILogicalExpression cond, List<AssignOperator> assigns, Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
+ protected boolean analyzeCondition(ILogicalExpression cond, List<AssignOperator> assigns,
+ Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) cond;
FunctionIdentifier funcIdent = funcExpr.getFunctionIdentifier();
// Don't consider optimizing a disjunctive condition with an index (too complicated for now).
if (funcIdent == AlgebricksBuiltinFunctions.OR) {
return false;
}
- boolean found = analyzeFunctionExpr(funcExpr, assigns, analyzedAMs);
+ boolean found = analyzeFunctionExpr(funcExpr, assigns, analyzedAMs);
for (Mutable<ILogicalExpression> arg : funcExpr.getArguments()) {
ILogicalExpression argExpr = arg.getValue();
if (argExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
continue;
}
AbstractFunctionCallExpression argFuncExpr = (AbstractFunctionCallExpression) argExpr;
- boolean matchFound = analyzeFunctionExpr(argFuncExpr, assigns, analyzedAMs);
+ boolean matchFound = analyzeFunctionExpr(argFuncExpr, assigns, analyzedAMs);
found = found || matchFound;
}
return found;
}
-
+
/**
* Finds applicable access methods for the given function expression based
* on the function identifier, and an analysis of the function's arguments.
* Updates the analyzedAMs accordingly.
*/
- protected boolean analyzeFunctionExpr(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns, Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
+ protected boolean analyzeFunctionExpr(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns,
+ Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
FunctionIdentifier funcIdent = funcExpr.getFunctionIdentifier();
if (funcIdent == AlgebricksBuiltinFunctions.AND) {
return false;
@@ -197,7 +210,7 @@
boolean atLeastOneMatchFound = false;
// Place holder for a new analysis context in case we need one.
AccessMethodAnalysisContext newAnalysisCtx = new AccessMethodAnalysisContext();
- for(IAccessMethod accessMethod : relevantAMs) {
+ for (IAccessMethod accessMethod : relevantAMs) {
AccessMethodAnalysisContext analysisCtx = analyzedAMs.get(accessMethod);
// Use the current place holder.
if (analysisCtx == null) {
@@ -216,38 +229,38 @@
}
return atLeastOneMatchFound;
}
-
+
/**
- * Finds secondary indexes whose keys include fieldName, and adds a mapping in analysisCtx.indexEsprs
+ * Finds secondary indexes whose keys include fieldName, and adds a mapping in analysisCtx.indexEsprs
* from that index to the a corresponding optimizable function expression.
*
* @return true if a candidate index was added to foundIndexExprs, false
* otherwise
+ * @throws AlgebricksException
*/
- protected boolean fillIndexExprs(String fieldName, int matchedFuncExprIndex,
- AqlCompiledDatasetDecl datasetDecl, AccessMethodAnalysisContext analysisCtx) {
- AqlCompiledIndexDecl primaryIndexDecl = DatasetUtils.getPrimaryIndex(datasetDecl);
- List<String> primaryIndexFields = primaryIndexDecl.getFieldExprs();
- List<AqlCompiledIndexDecl> indexCandidates = DatasetUtils.findSecondaryIndexesByOneOfTheKeys(datasetDecl, fieldName);
- // Check whether the primary index is a candidate. If so, add it to the list.
- if (primaryIndexFields.contains(fieldName)) {
- if (indexCandidates == null) {
- indexCandidates = new ArrayList<AqlCompiledIndexDecl>(1);
+ protected boolean fillIndexExprs(String fieldName, int matchedFuncExprIndex, Dataset dataset,
+ AccessMethodAnalysisContext analysisCtx) throws AlgebricksException {
+ List<Index> datasetIndexes = metadata.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
+ List<Index> indexCandidates = new ArrayList<Index>();
+ // Add an index to the candidates if one of the indexed fields is fieldName.
+ for (Index index : datasetIndexes) {
+ if (index.getKeyFieldNames().contains(fieldName)) {
+ indexCandidates.add(index);
}
- indexCandidates.add(primaryIndexDecl);
}
// No index candidates for fieldName.
- if (indexCandidates == null) {
+ if (indexCandidates.isEmpty()) {
return false;
}
// Go through the candidates and fill indexExprs.
- for (AqlCompiledIndexDecl index : indexCandidates) {
- analysisCtx.addIndexExpr(datasetDecl, index, matchedFuncExprIndex);
+ for (Index index : indexCandidates) {
+ analysisCtx.addIndexExpr(dataset, index, matchedFuncExprIndex);
}
return true;
}
- protected void fillAllIndexExprs(List<LogicalVariable> varList, OptimizableOperatorSubTree subTree, AccessMethodAnalysisContext analysisCtx) {
+ protected void fillAllIndexExprs(List<LogicalVariable> varList, OptimizableOperatorSubTree subTree,
+ AccessMethodAnalysisContext analysisCtx) throws AlgebricksException {
for (int optFuncExprIndex = 0; optFuncExprIndex < analysisCtx.matchedFuncExprs.size(); optFuncExprIndex++) {
for (int varIndex = 0; varIndex < varList.size(); varIndex++) {
LogicalVariable var = varList.get(varIndex);
@@ -263,7 +276,8 @@
// Get the fieldName corresponding to the assigned variable at varIndex
// from the assign operator right above the datasource scan.
// If the expr at varIndex is not a fieldAccess we get back null.
- fieldName = getFieldNameOfFieldAccess(subTree.assigns.get(subTree.assigns.size() - 1), subTree.recordType, varIndex);
+ fieldName = getFieldNameOfFieldAccess(subTree.assigns.get(subTree.assigns.size() - 1),
+ subTree.recordType, varIndex);
if (fieldName == null) {
continue;
}
@@ -274,24 +288,24 @@
break;
}
// The variable value is one of the partitioning fields.
- fieldName = DatasetUtils.getPartitioningExpressions(subTree.datasetDecl).get(varIndex);
+ fieldName = DatasetUtils.getPartitioningKeys(subTree.dataset).get(varIndex);
}
// Set the fieldName in the corresponding matched function expression, and remember matching subtree.
optFuncExpr.setFieldName(funcVarIndex, fieldName);
optFuncExpr.setOptimizableSubTree(funcVarIndex, subTree);
- fillIndexExprs(fieldName, optFuncExprIndex, subTree.datasetDecl, analysisCtx);
+ fillIndexExprs(fieldName, optFuncExprIndex, subTree.dataset, analysisCtx);
}
}
}
-
+
/**
* Returns the field name corresponding to the assigned variable at varIndex.
* Returns null if the expr at varIndex is not a field access function.
*/
protected String getFieldNameOfFieldAccess(AssignOperator assign, ARecordType recordType, int varIndex) {
// Get expression corresponding to var at varIndex.
- AbstractLogicalExpression assignExpr = (AbstractLogicalExpression) assign.getExpressions()
- .get(varIndex).getValue();
+ AbstractLogicalExpression assignExpr = (AbstractLogicalExpression) assign.getExpressions().get(varIndex)
+ .getValue();
if (assignExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
return null;
}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java
index b977e29..fc45063 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java
@@ -4,35 +4,35 @@
import java.util.HashMap;
import java.util.List;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.Index;
/**
* Context for analyzing the applicability of a single access method.
*/
public class AccessMethodAnalysisContext {
-
+
public List<IOptimizableFuncExpr> matchedFuncExprs = new ArrayList<IOptimizableFuncExpr>();
-
+
// Contains candidate indexes and a list of integers that index into matchedFuncExprs.
// We are mapping from candidate indexes to a list of function expressions
// that match one of the index's expressions.
- public HashMap<AqlCompiledIndexDecl, List<Integer>> indexExprs = new HashMap<AqlCompiledIndexDecl, List<Integer>>();
-
+ public HashMap<Index, List<Integer>> indexExprs = new HashMap<Index, List<Integer>>();
+
// Maps from index to the dataset it is indexing.
- public HashMap<AqlCompiledIndexDecl, AqlCompiledDatasetDecl> indexDatasetMap = new HashMap<AqlCompiledIndexDecl, AqlCompiledDatasetDecl>();
-
- public void addIndexExpr(AqlCompiledDatasetDecl dataset, AqlCompiledIndexDecl index, Integer exprIndex) {
- List<Integer> exprs = indexExprs.get(index);
- if (exprs == null) {
- exprs = new ArrayList<Integer>();
- indexExprs.put(index, exprs);
- }
- exprs.add(exprIndex);
- indexDatasetMap.put(index, dataset);
+ public HashMap<Index, Dataset> indexDatasetMap = new HashMap<Index, Dataset>();
+
+ public void addIndexExpr(Dataset dataset, Index index, Integer exprIndex) {
+ List<Integer> exprs = indexExprs.get(index);
+ if (exprs == null) {
+ exprs = new ArrayList<Integer>();
+ indexExprs.put(index, exprs);
+ }
+ exprs.add(exprIndex);
+ indexDatasetMap.put(index, dataset);
}
-
- public List<Integer> getIndexExprs(AqlCompiledIndexDecl index) {
+
+ public List<Integer> getIndexExprs(Index index) {
return indexExprs.get(index);
}
}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodJobGenParams.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodJobGenParams.java
index 22ed47a..af30163 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodJobGenParams.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodJobGenParams.java
@@ -5,7 +5,7 @@
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.commons.lang3.mutable.MutableObject;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl.IndexKind;
+import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
import edu.uci.ics.asterix.om.base.AInt32;
import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -19,27 +19,28 @@
*/
public class AccessMethodJobGenParams {
protected String indexName;
- protected IndexKind indexKind;
+ protected IndexType indexType;
protected String datasetName;
protected boolean retainInput;
protected boolean requiresBroadcast;
-
+
private final int NUM_PARAMS = 5;
-
+
public AccessMethodJobGenParams() {
}
-
- public AccessMethodJobGenParams(String indexName, IndexKind indexKind, String datasetName, boolean retainInput, boolean requiresBroadcast) {
+
+ public AccessMethodJobGenParams(String indexName, IndexType indexType, String datasetName, boolean retainInput,
+ boolean requiresBroadcast) {
this.indexName = indexName;
- this.indexKind = indexKind;
+ this.indexType = indexType;
this.datasetName = datasetName;
this.retainInput = retainInput;
this.requiresBroadcast = requiresBroadcast;
}
-
+
public void writeToFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createStringConstant(indexName)));
- funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createInt32Constant(indexKind.ordinal())));
+ funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createInt32Constant(indexType.ordinal())));
funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createStringConstant(datasetName)));
funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createBooleanConstant(retainInput)));
funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createBooleanConstant(requiresBroadcast)));
@@ -47,18 +48,18 @@
public void readFromFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
indexName = AccessMethodUtils.getStringConstant(funcArgs.get(0));
- indexKind = IndexKind.values()[AccessMethodUtils.getInt32Constant(funcArgs.get(1))];
+ indexType = IndexType.values()[AccessMethodUtils.getInt32Constant(funcArgs.get(1))];
datasetName = AccessMethodUtils.getStringConstant(funcArgs.get(2));
retainInput = AccessMethodUtils.getBooleanConstant(funcArgs.get(3));
requiresBroadcast = AccessMethodUtils.getBooleanConstant(funcArgs.get(4));
}
-
+
public String getIndexName() {
return indexName;
}
- public IndexKind getIndexKind() {
- return indexKind;
+ public IndexType getIndexType() {
+ return indexType;
}
public String getDatasetName() {
@@ -72,7 +73,7 @@
public boolean getRequiresBroadcast() {
return requiresBroadcast;
}
-
+
protected void writeVarList(List<LogicalVariable> varList, List<Mutable<ILogicalExpression>> funcArgs) {
Mutable<ILogicalExpression> numKeysRef = new MutableObject<ILogicalExpression>(new ConstantExpression(
new AsterixConstantValue(new AInt32(varList.size()))));
@@ -83,7 +84,7 @@
funcArgs.add(keyVarRef);
}
}
-
+
protected int readVarList(List<Mutable<ILogicalExpression>> funcArgs, int index, List<LogicalVariable> varList) {
int numLowKeys = AccessMethodUtils.getInt32Constant(funcArgs.get(index));
if (numLowKeys > 0) {
@@ -95,7 +96,7 @@
}
return index + numLowKeys + 1;
}
-
+
protected int getNumParams() {
return NUM_PARAMS;
}
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 305b858..cd3712d 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
@@ -7,9 +7,9 @@
import org.apache.commons.lang3.mutable.MutableObject;
import edu.uci.ics.asterix.aql.util.FunctionUtils;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl.IndexKind;
+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;
import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
import edu.uci.ics.asterix.om.base.ABoolean;
import edu.uci.ics.asterix.om.base.AInt32;
@@ -22,7 +22,6 @@
import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
-import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
@@ -40,55 +39,51 @@
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;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
/**
- * Static helper functions for rewriting plans using indexes.
+ * Static helper functions for rewriting plans using indexes.
*/
public class AccessMethodUtils {
- public static void appendPrimaryIndexTypes(AqlCompiledDatasetDecl datasetDecl, IAType itemType, List<Object> target) {
- List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions = DatasetUtils
- .getPartitioningFunctions(datasetDecl);
- for (Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> t : partitioningFunctions) {
- target.add(t.third);
+ public static void appendPrimaryIndexTypes(Dataset dataset, IAType itemType, List<Object> target) {
+ 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) {
+
+ public static ConstantExpression createStringConstant(String str) {
return new ConstantExpression(new AsterixConstantValue(new AString(str)));
}
-
- public static ConstantExpression createInt32Constant(int i) {
+
+ public static ConstantExpression createInt32Constant(int i) {
return new ConstantExpression(new AsterixConstantValue(new AInt32(i)));
}
-
- public static ConstantExpression createBooleanConstant(boolean b) {
+
+ 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 String getStringConstant(Mutable<ILogicalExpression> expr) {
- IAObject obj = ((AsterixConstantValue)((ConstantExpression) expr.getValue())
- .getValue()).getObject();
- return ((AString)obj).getStringValue();
+
+ public static String getStringConstant(Mutable<ILogicalExpression> expr) {
+ IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
+ return ((AString) obj).getStringValue();
}
-
+
public static int getInt32Constant(Mutable<ILogicalExpression> expr) {
- IAObject obj = ((AsterixConstantValue)((ConstantExpression) expr.getValue())
- .getValue()).getObject();
- return ((AInt32)obj).getIntegerValue();
+ IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
+ return ((AInt32) obj).getIntegerValue();
}
-
+
public static boolean getBooleanConstant(Mutable<ILogicalExpression> expr) {
- IAObject obj = ((AsterixConstantValue)((ConstantExpression) expr.getValue())
- .getValue()).getObject();
- return ((ABoolean)obj).getBoolean();
+ IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
+ return ((ABoolean) obj).getBoolean();
}
-
+
public static boolean analyzeFuncExprArgsForOneConstAndVar(AbstractFunctionCallExpression funcExpr,
AccessMethodAnalysisContext analysisCtx) {
IAlgebricksConstantValue constFilterVal = null;
@@ -114,48 +109,49 @@
analysisCtx.matchedFuncExprs.add(new OptimizableFuncExpr(funcExpr, fieldVar, constFilterVal));
return true;
}
-
- public static int getNumSecondaryKeys(AqlCompiledDatasetDecl datasetDecl, AqlCompiledIndexDecl indexDecl,
- ARecordType recordType) throws AlgebricksException {
- switch (indexDecl.getKind()) {
+
+ public static int getNumSecondaryKeys(Index index, ARecordType recordType) throws AlgebricksException {
+ switch (index.getIndexType()) {
case BTREE:
case WORD_INVIX:
case NGRAM_INVIX: {
- return indexDecl.getFieldExprs().size();
+ return index.getKeyFieldNames().size();
}
case RTREE: {
- Pair<IAType, Boolean> keyPairType = AqlCompiledIndexDecl.getNonNullableKeyFieldType(indexDecl.getFieldExprs().get(0), recordType);
+ 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: " + indexDecl.getKind());
+ throw new AlgebricksException("Unknown index kind: " + index.getIndexType());
}
}
}
-
+
/**
* Appends the types of the fields produced by the given secondary index to dest.
*/
- public static void appendSecondaryIndexTypes(AqlCompiledDatasetDecl datasetDecl, ARecordType recordType,
- AqlCompiledIndexDecl indexDecl, boolean primaryKeysOnly, List<Object> dest) throws AlgebricksException {
+ public static void appendSecondaryIndexTypes(Dataset dataset, ARecordType recordType, Index index,
+ boolean primaryKeysOnly, List<Object> dest) throws AlgebricksException {
if (!primaryKeysOnly) {
- switch (indexDecl.getKind()) {
+ switch (index.getIndexType()) {
case BTREE:
case WORD_INVIX:
case NGRAM_INVIX: {
- for (String sk : indexDecl.getFieldExprs()) {
- Pair<IAType, Boolean> keyPairType = AqlCompiledIndexDecl.getNonNullableKeyFieldType(sk, recordType);
+ for (String sk : index.getKeyFieldNames()) {
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(sk, recordType);
dest.add(keyPairType.first);
}
break;
}
case RTREE: {
- Pair<IAType, Boolean> keyPairType = AqlCompiledIndexDecl.getNonNullableKeyFieldType(indexDecl.getFieldExprs().get(0), recordType);
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(index.getKeyFieldNames()
+ .get(0), recordType);
IAType keyType = keyPairType.first;
IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(keyType.getTypeTag());
- int numKeys = getNumSecondaryKeys(datasetDecl, indexDecl, recordType);
+ int numKeys = getNumSecondaryKeys(index, recordType);
for (int i = 0; i < numKeys; i++) {
dest.add(nestedKeyType);
}
@@ -164,26 +160,25 @@
}
}
// Primary keys.
- for (Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> t : DatasetUtils
- .getPartitioningFunctions(datasetDecl)) {
- dest.add(t.third);
+ List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+ for (String partitioningKey : partitioningKeys) {
+ dest.add(recordType.getFieldType(partitioningKey));
}
}
-
- public static void appendSecondaryIndexOutputVars(AqlCompiledDatasetDecl datasetDecl, ARecordType recordType,
- AqlCompiledIndexDecl indexDecl, boolean primaryKeysOnly, IOptimizationContext context,
- List<LogicalVariable> dest) throws AlgebricksException {
- int numPrimaryKeys = DatasetUtils.getPartitioningFunctions(datasetDecl).size();
- int numSecondaryKeys = getNumSecondaryKeys(datasetDecl, indexDecl, recordType);
+
+ 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 List<LogicalVariable> getPrimaryKeyVarsFromUnnestMap(AqlCompiledDatasetDecl datasetDecl,
- ILogicalOperator unnestMapOp) {
- int numPrimaryKeys = DatasetUtils.getPartitioningFunctions(datasetDecl).size();
+
+ 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.
@@ -194,11 +189,10 @@
}
return primaryKeyVars;
}
-
- public static UnnestMapOperator createSecondaryIndexUnnestMap(AqlCompiledDatasetDecl datasetDecl,
- ARecordType recordType, AqlCompiledIndexDecl indexDecl, ILogicalOperator inputOp,
- AccessMethodJobGenParams jobGenParams, IOptimizationContext context, boolean outputPrimaryKeysOnly,
- boolean retainInput) throws AlgebricksException {
+
+ 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);
@@ -206,32 +200,35 @@
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(datasetDecl, recordType, indexDecl, outputPrimaryKeysOnly, context, secondaryIndexUnnestVars);
- appendSecondaryIndexTypes(datasetDecl, recordType, indexDecl, outputPrimaryKeysOnly, secondaryIndexOutputTypes);
+ 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);
+ 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));
+ 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, AqlCompiledDatasetDecl datasetDecl,
- ARecordType recordType, ILogicalOperator inputOp,
- IOptimizationContext context, boolean sortPrimaryKeys, boolean retainInput, boolean requiresBroadcast) throws AlgebricksException {
- List<LogicalVariable> primaryKeyVars = AccessMethodUtils.getPrimaryKeyVarsFromUnnestMap(datasetDecl, inputOp);
+
+ 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));
+ Mutable<ILogicalExpression> vRef = new MutableObject<ILogicalExpression>(
+ new VariableReferenceExpression(pkVar));
order.getOrderExpressions().add(
new Pair<IOrder, Mutable<ILogicalExpression>>(OrderOperator.ASC_ORDER, vRef));
}
@@ -242,7 +239,8 @@
}
// 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(datasetDecl.getName(), IndexKind.BTREE, datasetDecl.getName(), retainInput, requiresBroadcast);
+ BTreeJobGenParams jobGenParams = new BTreeJobGenParams(dataset.getDatasetName(), IndexType.BTREE,
+ dataset.getDatasetName(), retainInput, requiresBroadcast);
// Set low/high inclusive to true for a point lookup.
jobGenParams.setLowKeyInclusive(true);
jobGenParams.setHighKeyInclusive(true);
@@ -254,14 +252,15 @@
List<Object> primaryIndexOutputTypes = new ArrayList<Object>();
// Append output variables/types generated by the primary-index search (not forwarded from input).
primaryIndexUnnestVars.addAll(dataSourceScan.getVariables());
- appendPrimaryIndexTypes(datasetDecl, recordType, primaryIndexOutputTypes);
+ appendPrimaryIndexTypes(dataset, recordType, primaryIndexOutputTypes);
// 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);
+ 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);
+ 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));
@@ -272,5 +271,5 @@
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 25902bc..414dca3 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
@@ -9,10 +9,9 @@
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.commons.lang3.mutable.MutableObject;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl.IndexKind;
-import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
+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;
import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -37,15 +36,19 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
/**
- * Class for helping rewrite rules to choose and apply BTree indexes.
+ * Class for helping rewrite rules to choose and apply BTree indexes.
*/
public class BTreeAccessMethod implements IAccessMethod {
// Describes whether a search predicate is an open/closed interval.
private enum LimitType {
- LOW_INCLUSIVE, LOW_EXCLUSIVE, HIGH_INCLUSIVE, HIGH_EXCLUSIVE, EQUAL
+ LOW_INCLUSIVE,
+ LOW_EXCLUSIVE,
+ HIGH_INCLUSIVE,
+ HIGH_EXCLUSIVE,
+ EQUAL
}
-
+
// TODO: There is some redundancy here, since these are listed in AlgebricksBuiltinFunctions as well.
private static List<FunctionIdentifier> funcIdents = new ArrayList<FunctionIdentifier>();
static {
@@ -56,16 +59,17 @@
funcIdents.add(AlgebricksBuiltinFunctions.GT);
funcIdents.add(AlgebricksBuiltinFunctions.NEQ);
}
-
+
public static BTreeAccessMethod INSTANCE = new BTreeAccessMethod();
-
+
@Override
public List<FunctionIdentifier> getOptimizableFunctions() {
return funcIdents;
}
-
+
@Override
- public boolean analyzeFuncExprArgs(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns, AccessMethodAnalysisContext analysisCtx) {
+ public boolean analyzeFuncExprArgs(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns,
+ AccessMethodAnalysisContext analysisCtx) {
return AccessMethodUtils.analyzeFuncExprArgsForOneConstAndVar(funcExpr, analysisCtx);
}
@@ -81,10 +85,10 @@
}
@Override
- public boolean applySelectPlanTransformation(Mutable<ILogicalOperator> selectRef, OptimizableOperatorSubTree subTree,
- AqlCompiledIndexDecl chosenIndex, AccessMethodAnalysisContext analysisCtx, IOptimizationContext context)
- throws AlgebricksException {
- AqlCompiledDatasetDecl datasetDecl = subTree.datasetDecl;
+ public boolean applySelectPlanTransformation(Mutable<ILogicalOperator> selectRef,
+ OptimizableOperatorSubTree subTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
+ IOptimizationContext context) throws AlgebricksException {
+ Dataset dataset = subTree.dataset;
ARecordType recordType = subTree.recordType;
SelectOperator select = (SelectOperator) selectRef.getValue();
DataSourceScanOperator dataSourceScan = subTree.dataSourceScan;
@@ -92,9 +96,9 @@
AssignOperator assign = null;
if (assignRef != null) {
assign = (AssignOperator) assignRef.getValue();
- }
- int numSecondaryKeys = chosenIndex.getFieldExprs().size();
-
+ }
+ int numSecondaryKeys = chosenIndex.getKeyFieldNames().size();
+
// Info on high and low keys for the BTree search predicate.
IAlgebricksConstantValue[] lowKeyConstants = new IAlgebricksConstantValue[numSecondaryKeys];
IAlgebricksConstantValue[] highKeyConstants = new IAlgebricksConstantValue[numSecondaryKeys];
@@ -102,7 +106,7 @@
LimitType[] highKeyLimits = new LimitType[numSecondaryKeys];
boolean[] lowKeyInclusive = new boolean[numSecondaryKeys];
boolean[] highKeyInclusive = new boolean[numSecondaryKeys];
-
+
List<Integer> exprList = analysisCtx.indexExprs.get(chosenIndex);
List<IOptimizableFuncExpr> matchedFuncExprs = analysisCtx.matchedFuncExprs;
// List of function expressions that will be replaced by the secondary-index search.
@@ -117,7 +121,7 @@
for (Integer exprIndex : exprList) {
// Position of the field of matchedFuncExprs.get(exprIndex) in the chosen index's indexed exprs.
IOptimizableFuncExpr optFuncExpr = matchedFuncExprs.get(exprIndex);
- int keyPos = indexOf(optFuncExpr.getFieldName(0), chosenIndex.getFieldExprs());
+ int keyPos = indexOf(optFuncExpr.getFieldName(0), chosenIndex.getKeyFieldNames());
if (keyPos < 0) {
throw new InternalError();
}
@@ -201,10 +205,12 @@
if (lowKeyInclusive[i] != lowKeyInclusive[0] || highKeyInclusive[i] != highKeyInclusive[0]) {
return false;
}
- if (lowKeyLimits[0] == null && lowKeyLimits[i] != null || lowKeyLimits[0] != null && lowKeyLimits[i] == null) {
+ if (lowKeyLimits[0] == null && lowKeyLimits[i] != null || lowKeyLimits[0] != null
+ && lowKeyLimits[i] == null) {
return false;
}
- if (highKeyLimits[0] == null && highKeyLimits[i] != null || highKeyLimits[0] != null && highKeyLimits[i] == null) {
+ if (highKeyLimits[0] == null && highKeyLimits[i] != null || highKeyLimits[0] != null
+ && highKeyLimits[i] == null) {
return false;
}
}
@@ -214,7 +220,7 @@
if (highKeyLimits[0] == null) {
highKeyInclusive[0] = true;
}
-
+
// Here we generate vars and funcs for assigning the secondary-index keys to be fed into the secondary-index search.
// List of variables for the assign.
ArrayList<LogicalVariable> keyVarList = new ArrayList<LogicalVariable>();
@@ -222,30 +228,32 @@
ArrayList<Mutable<ILogicalExpression>> keyExprList = new ArrayList<Mutable<ILogicalExpression>>();
int numLowKeys = createKeyVarsAndExprs(lowKeyLimits, lowKeyConstants, keyExprList, keyVarList, context);
int numHighKeys = createKeyVarsAndExprs(highKeyLimits, highKeyConstants, keyExprList, keyVarList, context);
-
- BTreeJobGenParams jobGenParams = new BTreeJobGenParams(chosenIndex.getIndexName(), IndexKind.BTREE, datasetDecl.getName(), false, false);
+
+ BTreeJobGenParams jobGenParams = new BTreeJobGenParams(chosenIndex.getIndexName(), IndexType.BTREE,
+ dataset.getDatasetName(), false, false);
jobGenParams.setLowKeyInclusive(lowKeyInclusive[0]);
jobGenParams.setHighKeyInclusive(highKeyInclusive[0]);
jobGenParams.setLowKeyVarList(keyVarList, 0, numLowKeys);
jobGenParams.setHighKeyVarList(keyVarList, numLowKeys, numHighKeys);
-
+
// Assign operator that sets the secondary-index search-key fields.
AssignOperator assignSearchKeys = new AssignOperator(keyVarList, keyExprList);
// Input to this assign is the EmptyTupleSource (which the dataSourceScan also must have had as input).
assignSearchKeys.getInputs().add(dataSourceScan.getInputs().get(0));
assignSearchKeys.setExecutionMode(dataSourceScan.getExecutionMode());
-
- UnnestMapOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(datasetDecl,
- recordType, chosenIndex, assignSearchKeys, jobGenParams, context, false, false);
-
+
+ UnnestMapOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
+ chosenIndex, assignSearchKeys, jobGenParams, context, false, false);
+
// Generate the rest of the upstream plan which feeds the search results into the primary index.
UnnestMapOperator primaryIndexUnnestOp;
- boolean isPrimaryIndex = chosenIndex == DatasetUtils.getPrimaryIndex(datasetDecl);
+ boolean isPrimaryIndex = chosenIndex.getIndexName().equals(dataset.getDatasetName());
if (!isPrimaryIndex) {
- primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceScan, datasetDecl, recordType, secondaryIndexUnnestOp, context, true, false, false);
+ primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceScan, dataset, recordType,
+ secondaryIndexUnnestOp, context, true, false, false);
} else {
List<Object> primaryIndexOutputTypes = new ArrayList<Object>();
- AccessMethodUtils.appendPrimaryIndexTypes(datasetDecl, recordType, primaryIndexOutputTypes);
+ AccessMethodUtils.appendPrimaryIndexTypes(dataset, recordType, primaryIndexOutputTypes);
primaryIndexUnnestOp = new UnnestMapOperator(dataSourceScan.getVariables(),
secondaryIndexUnnestOp.getExpressionRef(), primaryIndexOutputTypes, false);
primaryIndexUnnestOp.getInputs().add(new MutableObject<ILogicalOperator>(assignSearchKeys));
@@ -276,19 +284,18 @@
}
return true;
}
-
+
@Override
public boolean applyJoinPlanTransformation(Mutable<ILogicalOperator> joinRef,
- OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree,
- AqlCompiledIndexDecl chosenIndex, AccessMethodAnalysisContext analysisCtx, IOptimizationContext context)
- throws AlgebricksException {
+ OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree, Index chosenIndex,
+ AccessMethodAnalysisContext analysisCtx, IOptimizationContext context) throws AlgebricksException {
// TODO: Implement this.
return false;
}
private int createKeyVarsAndExprs(LimitType[] keyLimits, IAlgebricksConstantValue[] keyConstants,
ArrayList<Mutable<ILogicalExpression>> keyExprList, ArrayList<LogicalVariable> keyVarList,
- IOptimizationContext context) {
+ IOptimizationContext context) {
if (keyLimits[0] == null) {
return 0;
}
@@ -300,8 +307,9 @@
}
return numKeys;
}
-
- private void getNewSelectExprs(SelectOperator select, Set<ILogicalExpression> replacedFuncExprs, List<Mutable<ILogicalExpression>> remainingFuncExprs) {
+
+ private void getNewSelectExprs(SelectOperator select, Set<ILogicalExpression> replacedFuncExprs,
+ List<Mutable<ILogicalExpression>> remainingFuncExprs) {
remainingFuncExprs.clear();
if (replacedFuncExprs.isEmpty()) {
return;
@@ -334,7 +342,7 @@
}
}
}
-
+
private <T> int indexOf(T value, List<T> coll) {
int i = 0;
for (T member : coll) {
@@ -345,9 +353,10 @@
}
return -1;
}
-
+
private LimitType getLimitType(IOptimizableFuncExpr optFuncExpr) {
- ComparisonKind ck = AlgebricksBuiltinFunctions.getComparisonType(optFuncExpr.getFuncExpr().getFunctionIdentifier());
+ ComparisonKind ck = AlgebricksBuiltinFunctions.getComparisonType(optFuncExpr.getFuncExpr()
+ .getFunctionIdentifier());
LimitType limit = null;
switch (ck) {
case EQ: {
@@ -380,7 +389,7 @@
}
return limit;
}
-
+
// Returns true if there is a constant value on the left-hand side if the given optimizable function (assuming a binary function).
public boolean constantIsOnLhs(IOptimizableFuncExpr optFuncExpr) {
return optFuncExpr.getFuncExpr().getArguments().get(0) == optFuncExpr.getConstantVal(0);
@@ -395,7 +404,7 @@
}
@Override
- public boolean exprIsOptimizable(AqlCompiledIndexDecl index, IOptimizableFuncExpr optFuncExpr) {
+ public boolean exprIsOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) {
// No additional analysis required for BTrees.
return true;
}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeJobGenParams.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeJobGenParams.java
index 5dea91f..c377a34 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeJobGenParams.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeJobGenParams.java
@@ -6,7 +6,7 @@
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.commons.lang3.mutable.MutableObject;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl.IndexKind;
+import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
@@ -19,43 +19,43 @@
protected List<LogicalVariable> lowKeyVarList;
protected List<LogicalVariable> highKeyVarList;
-
+
protected boolean lowKeyInclusive;
protected boolean highKeyInclusive;
-
+
public BTreeJobGenParams() {
super();
}
-
- public BTreeJobGenParams(String indexName, IndexKind indexKind, String datasetName, boolean retainInput,
+
+ public BTreeJobGenParams(String indexName, IndexType indexType, String datasetName, boolean retainInput,
boolean requiresBroadcast) {
- super(indexName, indexKind, datasetName, retainInput, requiresBroadcast);
+ super(indexName, indexType, datasetName, retainInput, requiresBroadcast);
}
public void setLowKeyVarList(List<LogicalVariable> keyVarList, int startIndex, int numKeys) {
lowKeyVarList = new ArrayList<LogicalVariable>(numKeys);
setKeyVarList(keyVarList, lowKeyVarList, startIndex, numKeys);
}
-
+
public void setHighKeyVarList(List<LogicalVariable> keyVarList, int startIndex, int numKeys) {
highKeyVarList = new ArrayList<LogicalVariable>(numKeys);
setKeyVarList(keyVarList, highKeyVarList, startIndex, numKeys);
}
-
+
private void setKeyVarList(List<LogicalVariable> src, List<LogicalVariable> dest, int startIndex, int numKeys) {
for (int i = 0; i < numKeys; i++) {
dest.add(src.get(startIndex + i));
}
}
-
+
public void setLowKeyInclusive(boolean lowKeyInclusive) {
this.lowKeyInclusive = lowKeyInclusive;
}
-
+
public void setHighKeyInclusive(boolean highKeyInclusive) {
this.highKeyInclusive = highKeyInclusive;
}
-
+
public void writeToFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
super.writeToFuncArgs(funcArgs);
writeVarList(lowKeyVarList, funcArgs);
@@ -63,7 +63,7 @@
writeKeyInclusive(lowKeyInclusive, funcArgs);
writeKeyInclusive(highKeyInclusive, funcArgs);
}
-
+
public void readFromFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
super.readFromFuncArgs(funcArgs);
int index = super.getNumParams();
@@ -71,19 +71,19 @@
highKeyVarList = new ArrayList<LogicalVariable>();
int nextIndex = readVarList(funcArgs, index, lowKeyVarList);
nextIndex = readVarList(funcArgs, nextIndex, highKeyVarList);
- readKeyInclusives(funcArgs, nextIndex);
+ readKeyInclusives(funcArgs, nextIndex);
}
-
+
private void readKeyInclusives(List<Mutable<ILogicalExpression>> funcArgs, int index) {
lowKeyInclusive = ((ConstantExpression) funcArgs.get(index).getValue()).getValue().isTrue();
highKeyInclusive = ((ConstantExpression) funcArgs.get(index + 1).getValue()).getValue().isTrue();
}
-
+
private void writeKeyInclusive(boolean keyInclusive, List<Mutable<ILogicalExpression>> funcArgs) {
ILogicalExpression keyExpr = keyInclusive ? ConstantExpression.TRUE : ConstantExpression.FALSE;
funcArgs.add(new MutableObject<ILogicalExpression>(keyExpr));
}
-
+
public List<LogicalVariable> getLowKeyVarList() {
return lowKeyVarList;
}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IAccessMethod.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IAccessMethod.java
index 8dc8b8e..eadbc69 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IAccessMethod.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IAccessMethod.java
@@ -4,7 +4,7 @@
import org.apache.commons.lang3.mutable.Mutable;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
+import edu.uci.ics.asterix.metadata.entities.Index;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
@@ -19,13 +19,13 @@
* with a given index.
*/
public interface IAccessMethod {
-
+
/**
* @return A list of function identifiers that are optimizable by this
* access method.
*/
public List<FunctionIdentifier> getOptimizableFunctions();
-
+
/**
* Analyzes the arguments of a given optimizable funcExpr to see if this
* access method is applicable (e.g., one arg is a constant and one is a
@@ -37,8 +37,9 @@
* @return true if funcExpr is optimizable by this access method, false
* otherwise
*/
- public boolean analyzeFuncExprArgs(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns, AccessMethodAnalysisContext analysisCtx);
-
+ public boolean analyzeFuncExprArgs(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns,
+ AccessMethodAnalysisContext analysisCtx);
+
/**
* Indicates whether all index expressions must be matched in order for this
* index to be applicable.
@@ -54,23 +55,23 @@
* @return boolean
*/
public boolean matchPrefixIndexExprs();
-
+
/**
* Applies the plan transformation to use chosenIndex to optimize a selection query.
*/
- public boolean applySelectPlanTransformation(Mutable<ILogicalOperator> selectRef, OptimizableOperatorSubTree subTree,
- AqlCompiledIndexDecl chosenIndex, AccessMethodAnalysisContext analysisCtx, IOptimizationContext context)
- throws AlgebricksException;
-
+ public boolean applySelectPlanTransformation(Mutable<ILogicalOperator> selectRef,
+ OptimizableOperatorSubTree subTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
+ IOptimizationContext context) throws AlgebricksException;
+
/**
* Applies the plan transformation to use chosenIndex to optimize a join query.
*/
- public boolean applyJoinPlanTransformation(Mutable<ILogicalOperator> joinRef, OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree,
- AqlCompiledIndexDecl chosenIndex, AccessMethodAnalysisContext analysisCtx, IOptimizationContext context)
- throws AlgebricksException;
-
+ public boolean applyJoinPlanTransformation(Mutable<ILogicalOperator> joinRef,
+ OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree, Index chosenIndex,
+ AccessMethodAnalysisContext analysisCtx, IOptimizationContext context) throws AlgebricksException;
+
/**
* Analyzes expr to see whether it is optimizable by the given concrete index.
*/
- public boolean exprIsOptimizable(AqlCompiledIndexDecl index, IOptimizableFuncExpr optFuncExpr);
+ public boolean exprIsOptimizable(Index index, IOptimizableFuncExpr optFuncExpr);
}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
index 72acc1d..e4555aa 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
@@ -6,8 +6,8 @@
import org.apache.commons.lang3.mutable.Mutable;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.entities.Index;
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.ILogicalExpression;
@@ -47,26 +47,28 @@
* to accept any subtree on one side, as long as the other side has a datasource scan.
*/
public class IntroduceJoinAccessMethodRule extends AbstractIntroduceAccessMethodRule {
-
+
protected Mutable<ILogicalOperator> joinRef = null;
protected InnerJoinOperator join = null;
- protected AbstractFunctionCallExpression joinCond = null;
- protected final OptimizableOperatorSubTree leftSubTree = new OptimizableOperatorSubTree();
- protected final OptimizableOperatorSubTree rightSubTree = new OptimizableOperatorSubTree();
-
- // Register access methods.
- protected static Map<FunctionIdentifier, List<IAccessMethod>> accessMethods = new HashMap<FunctionIdentifier, List<IAccessMethod>>();
- static {
- registerAccessMethod(InvertedIndexAccessMethod.INSTANCE, accessMethods);
- }
-
+ protected AbstractFunctionCallExpression joinCond = null;
+ protected final OptimizableOperatorSubTree leftSubTree = new OptimizableOperatorSubTree();
+ protected final OptimizableOperatorSubTree rightSubTree = new OptimizableOperatorSubTree();
+
+ // Register access methods.
+ protected static Map<FunctionIdentifier, List<IAccessMethod>> accessMethods = new HashMap<FunctionIdentifier, List<IAccessMethod>>();
+ static {
+ registerAccessMethod(InvertedIndexAccessMethod.INSTANCE, accessMethods);
+ }
+
@Override
- public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
- // Match operator pattern and initialize optimizable sub trees.
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ setMetadataDeclarations(context);
+
+ // Match operator pattern and initialize optimizable sub trees.
if (!matchesOperatorPattern(opRef, context)) {
return false;
}
-
// Analyze condition on those optimizable subtrees that have a datasource scan.
Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs = new HashMap<IAccessMethod, AccessMethodAnalysisContext>();
boolean matchInLeftSubTree = false;
@@ -101,14 +103,14 @@
fillSubTreeIndexExprs(rightSubTree, analyzedAMs);
}
pruneIndexCandidates(analyzedAMs);
-
+
// Choose index to be applied.
- Pair<IAccessMethod, AqlCompiledIndexDecl> chosenIndex = chooseIndex(analyzedAMs);
+ Pair<IAccessMethod, Index> chosenIndex = chooseIndex(analyzedAMs);
if (chosenIndex == null) {
context.addToDontApplySet(this, join);
return false;
}
-
+
// Apply plan transformation using chosen index.
AccessMethodAnalysisContext analysisCtx = analyzedAMs.get(chosenIndex.first);
boolean res = chosenIndex.first.applyJoinPlanTransformation(joinRef, leftSubTree, rightSubTree,
@@ -119,7 +121,7 @@
context.addToDontApplySet(this, join);
return res;
}
-
+
protected boolean matchesOperatorPattern(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
// First check that the operator is a join and its condition is a function call.
AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef.getValue();
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
index 1d2cc0e..59b11fc 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
@@ -6,8 +6,8 @@
import org.apache.commons.lang3.mutable.Mutable;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.entities.Index;
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.ILogicalExpression;
@@ -51,29 +51,32 @@
*
*/
public class IntroduceSelectAccessMethodRule extends AbstractIntroduceAccessMethodRule {
-
- // Operators representing the patterns to be matched:
+
+ // Operators representing the patterns to be matched:
// These ops are set in matchesPattern()
protected Mutable<ILogicalOperator> selectRef = null;
protected SelectOperator select = null;
- protected AbstractFunctionCallExpression selectCond = null;
- protected final OptimizableOperatorSubTree subTree = new OptimizableOperatorSubTree();
+ protected AbstractFunctionCallExpression selectCond = null;
+ protected final OptimizableOperatorSubTree subTree = new OptimizableOperatorSubTree();
- // Register access methods.
- protected static Map<FunctionIdentifier, List<IAccessMethod>> accessMethods = new HashMap<FunctionIdentifier, List<IAccessMethod>>();
- static {
- registerAccessMethod(BTreeAccessMethod.INSTANCE, accessMethods);
- registerAccessMethod(RTreeAccessMethod.INSTANCE, accessMethods);
- registerAccessMethod(InvertedIndexAccessMethod.INSTANCE, accessMethods);
- }
-
+ // Register access methods.
+ protected static Map<FunctionIdentifier, List<IAccessMethod>> accessMethods = new HashMap<FunctionIdentifier, List<IAccessMethod>>();
+ static {
+ registerAccessMethod(BTreeAccessMethod.INSTANCE, accessMethods);
+ registerAccessMethod(RTreeAccessMethod.INSTANCE, accessMethods);
+ registerAccessMethod(InvertedIndexAccessMethod.INSTANCE, accessMethods);
+ }
+
@Override
- public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
- // Match operator pattern and initialize operator members.
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ setMetadataDeclarations(context);
+
+ // Match operator pattern and initialize operator members.
if (!matchesOperatorPattern(opRef, context)) {
return false;
}
-
+
// Analyze select condition.
Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs = new HashMap<IAccessMethod, AccessMethodAnalysisContext>();
if (!analyzeCondition(selectCond, subTree.assigns, analyzedAMs)) {
@@ -84,28 +87,28 @@
if (!subTree.setDatasetAndTypeMetadata((AqlMetadataProvider) context.getMetadataProvider())) {
return false;
}
-
+
fillSubTreeIndexExprs(subTree, analyzedAMs);
pruneIndexCandidates(analyzedAMs);
-
+
// Choose index to be applied.
- Pair<IAccessMethod, AqlCompiledIndexDecl> chosenIndex = chooseIndex(analyzedAMs);
+ Pair<IAccessMethod, Index> chosenIndex = chooseIndex(analyzedAMs);
if (chosenIndex == null) {
context.addToDontApplySet(this, select);
return false;
}
-
+
// Apply plan transformation using chosen index.
AccessMethodAnalysisContext analysisCtx = analyzedAMs.get(chosenIndex.first);
- boolean res = chosenIndex.first.applySelectPlanTransformation(selectRef, subTree,
- chosenIndex.second, analysisCtx, context);
+ boolean res = chosenIndex.first.applySelectPlanTransformation(selectRef, subTree, chosenIndex.second,
+ analysisCtx, context);
if (res) {
OperatorPropertiesUtil.typeOpRec(opRef, context);
}
context.addToDontApplySet(this, select);
return res;
}
-
+
protected boolean matchesOperatorPattern(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
// First check that the operator is a select and its condition is a function call.
AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef.getValue();
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
index fe3a539..dbd92c0 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
@@ -14,9 +14,8 @@
import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
import edu.uci.ics.asterix.formats.nontagged.AqlBinaryTokenizerFactoryProvider;
import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl.IndexKind;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.Index;
import edu.uci.ics.asterix.om.base.AFloat;
import edu.uci.ics.asterix.om.base.AInt32;
import edu.uci.ics.asterix.om.base.ANull;
@@ -66,7 +65,7 @@
import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
/**
- * Class for helping rewrite rules to choose and apply inverted indexes.
+ * Class for helping rewrite rules to choose and apply inverted indexes.
*/
public class InvertedIndexAccessMethod implements IAccessMethod {
@@ -77,15 +76,15 @@
EDIT_DISTANCE,
INVALID
}
-
+
private static List<FunctionIdentifier> funcIdents = new ArrayList<FunctionIdentifier>();
static {
- funcIdents.add(AsterixBuiltinFunctions.CONTAINS);
+ funcIdents.add(AsterixBuiltinFunctions.CONTAINS);
// For matching similarity-check functions. For example, similarity-jaccard-check returns a list of two items,
// and the select condition will get the first list-item and check whether it evaluates to true.
funcIdents.add(AsterixBuiltinFunctions.GET_ITEM);
}
-
+
// These function identifiers are matched in this AM's analyzeFuncExprArgs(),
// and are not visible to the outside driver.
private static HashSet<FunctionIdentifier> secondLevelFuncIdents = new HashSet<FunctionIdentifier>();
@@ -93,23 +92,25 @@
secondLevelFuncIdents.add(AsterixBuiltinFunctions.SIMILARITY_JACCARD_CHECK);
secondLevelFuncIdents.add(AsterixBuiltinFunctions.EDIT_DISTANCE_CHECK);
}
-
+
public static InvertedIndexAccessMethod INSTANCE = new InvertedIndexAccessMethod();
-
+
@Override
public List<FunctionIdentifier> getOptimizableFunctions() {
- return funcIdents;
+ return funcIdents;
}
@Override
- public boolean analyzeFuncExprArgs(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns, AccessMethodAnalysisContext analysisCtx) {
+ public boolean analyzeFuncExprArgs(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns,
+ AccessMethodAnalysisContext analysisCtx) {
if (funcExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.CONTAINS) {
return AccessMethodUtils.analyzeFuncExprArgsForOneConstAndVar(funcExpr, analysisCtx);
}
return analyzeGetItemFuncExpr(funcExpr, assigns, analysisCtx);
}
-
- public boolean analyzeGetItemFuncExpr(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns, AccessMethodAnalysisContext analysisCtx) {
+
+ public boolean analyzeGetItemFuncExpr(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns,
+ AccessMethodAnalysisContext analysisCtx) {
if (funcExpr.getFunctionIdentifier() != AsterixBuiltinFunctions.GET_ITEM) {
return false;
}
@@ -121,8 +122,8 @@
}
// The first arg must be a variable or a function expr.
// If it is a variable we must track its origin in the assigns to get the original function expr.
- if (arg1.getExpressionTag() != LogicalExpressionTag.VARIABLE &&
- arg1.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ if (arg1.getExpressionTag() != LogicalExpressionTag.VARIABLE
+ && arg1.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
return false;
}
AbstractFunctionCallExpression matchedFuncExpr = null;
@@ -132,7 +133,7 @@
}
// The get-item arg is a variable. Search the assigns for its origination function.
int matchedAssignIndex = -1;
- if (arg1.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ if (arg1.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
VariableReferenceExpression varRefExpr = (VariableReferenceExpression) arg1;
// Try to find variable ref expr in all assigns.
for (int i = 0; i < assigns.size(); i++) {
@@ -163,8 +164,10 @@
if (!secondLevelFuncIdents.contains(matchedFuncExpr.getFunctionIdentifier())) {
return false;
}
- boolean selectMatchFound = analyzeSelectSimilarityCheckFuncExprArgs(matchedFuncExpr, assigns, matchedAssignIndex, analysisCtx);
- boolean joinMatchFound = analyzeJoinSimilarityCheckFuncExprArgs(matchedFuncExpr, assigns, matchedAssignIndex, analysisCtx);
+ boolean selectMatchFound = analyzeSelectSimilarityCheckFuncExprArgs(matchedFuncExpr, assigns,
+ matchedAssignIndex, analysisCtx);
+ boolean joinMatchFound = analyzeJoinSimilarityCheckFuncExprArgs(matchedFuncExpr, assigns, matchedAssignIndex,
+ analysisCtx);
if (selectMatchFound || joinMatchFound) {
return true;
}
@@ -184,7 +187,7 @@
ILogicalExpression arg1 = funcExpr.getArguments().get(0).getValue();
ILogicalExpression arg2 = funcExpr.getArguments().get(1).getValue();
// We expect arg1 and arg2 to be non-constants for a join.
- if (arg1.getExpressionTag() == LogicalExpressionTag.CONSTANT
+ if (arg1.getExpressionTag() == LogicalExpressionTag.CONSTANT
|| arg2.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
return false;
}
@@ -196,10 +199,11 @@
if (fieldVar2 == null) {
return false;
}
- analysisCtx.matchedFuncExprs.add(new OptimizableFuncExpr(funcExpr, new LogicalVariable[] { fieldVar1, fieldVar2 }, new IAlgebricksConstantValue[] { constThreshVal }));
+ analysisCtx.matchedFuncExprs.add(new OptimizableFuncExpr(funcExpr,
+ new LogicalVariable[] { fieldVar1, fieldVar2 }, new IAlgebricksConstantValue[] { constThreshVal }));
return true;
}
-
+
private boolean analyzeSelectSimilarityCheckFuncExprArgs(AbstractFunctionCallExpression funcExpr,
List<AssignOperator> assigns, int matchedAssignIndex, AccessMethodAnalysisContext analysisCtx) {
// There should be exactly three arguments.
@@ -215,11 +219,11 @@
// Determine whether one arg is constant, and the other is non-constant.
ILogicalExpression constArg = null;
ILogicalExpression nonConstArg = null;
- if (arg1.getExpressionTag() == LogicalExpressionTag.CONSTANT
- && arg2.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+ if (arg1.getExpressionTag() == LogicalExpressionTag.CONSTANT
+ && arg2.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
constArg = arg1;
nonConstArg = arg2;
- } else if(arg2.getExpressionTag() == LogicalExpressionTag.CONSTANT
+ } else if (arg2.getExpressionTag() == LogicalExpressionTag.CONSTANT
&& arg1.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
constArg = arg2;
nonConstArg = arg1;
@@ -232,21 +236,22 @@
if (fieldVar == null) {
return false;
}
- analysisCtx.matchedFuncExprs.add(new OptimizableFuncExpr(funcExpr, new LogicalVariable[] { fieldVar }, new IAlgebricksConstantValue[] { constFilterVal, constThreshVal }));
+ analysisCtx.matchedFuncExprs.add(new OptimizableFuncExpr(funcExpr, new LogicalVariable[] { fieldVar },
+ new IAlgebricksConstantValue[] { constFilterVal, constThreshVal }));
return true;
}
-
- private LogicalVariable getNonConstArgFieldVar(ILogicalExpression nonConstArg, AbstractFunctionCallExpression funcExpr,
- List<AssignOperator> assigns, int matchedAssignIndex) {
+
+ private LogicalVariable getNonConstArgFieldVar(ILogicalExpression nonConstArg,
+ AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns, int matchedAssignIndex) {
LogicalVariable fieldVar = null;
// Analyze nonConstArg depending on similarity function.
- if (funcExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.SIMILARITY_JACCARD_CHECK) {
+ if (funcExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.SIMILARITY_JACCARD_CHECK) {
AbstractFunctionCallExpression nonConstFuncExpr = funcExpr;
if (nonConstArg.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
nonConstFuncExpr = (AbstractFunctionCallExpression) nonConstArg;
// TODO: Currently, we're only looking for word and gram tokens (non hashed).
- if (nonConstFuncExpr.getFunctionIdentifier() != AsterixBuiltinFunctions.WORD_TOKENS &&
- nonConstFuncExpr.getFunctionIdentifier() != AsterixBuiltinFunctions.GRAM_TOKENS) {
+ if (nonConstFuncExpr.getFunctionIdentifier() != AsterixBuiltinFunctions.WORD_TOKENS
+ && nonConstFuncExpr.getFunctionIdentifier() != AsterixBuiltinFunctions.GRAM_TOKENS) {
return null;
}
// Find the variable that is being tokenized.
@@ -269,8 +274,8 @@
}
AbstractFunctionCallExpression childFuncExpr = (AbstractFunctionCallExpression) childExpr;
// If fieldVar references the result of a tokenization, then we should remember the variable being tokenized.
- if (childFuncExpr.getFunctionIdentifier() != AsterixBuiltinFunctions.WORD_TOKENS &&
- childFuncExpr.getFunctionIdentifier() != AsterixBuiltinFunctions.GRAM_TOKENS) {
+ if (childFuncExpr.getFunctionIdentifier() != AsterixBuiltinFunctions.WORD_TOKENS
+ && childFuncExpr.getFunctionIdentifier() != AsterixBuiltinFunctions.GRAM_TOKENS) {
break;
}
// We expect the tokenizer's argument to be a variable, otherwise we cannot apply an index.
@@ -292,12 +297,12 @@
}
if (funcExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.EDIT_DISTANCE_CHECK) {
if (nonConstArg.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
- fieldVar = ((VariableReferenceExpression) nonConstArg).getVariableReference();
+ fieldVar = ((VariableReferenceExpression) nonConstArg).getVariableReference();
}
}
return fieldVar;
}
-
+
@Override
public boolean matchAllIndexExprs() {
return true;
@@ -308,18 +313,20 @@
return false;
}
- private ILogicalOperator createSecondaryToPrimaryPlan(OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree,
- AqlCompiledIndexDecl chosenIndex, IOptimizableFuncExpr optFuncExpr, boolean retainInput, boolean requiresBroadcast, IOptimizationContext context) throws AlgebricksException {
- AqlCompiledDatasetDecl datasetDecl = indexSubTree.datasetDecl;
+ private ILogicalOperator createSecondaryToPrimaryPlan(OptimizableOperatorSubTree indexSubTree,
+ OptimizableOperatorSubTree probeSubTree, Index chosenIndex, IOptimizableFuncExpr optFuncExpr,
+ boolean retainInput, boolean requiresBroadcast, IOptimizationContext context) throws AlgebricksException {
+ Dataset dataset = indexSubTree.dataset;
ARecordType recordType = indexSubTree.recordType;
DataSourceScanOperator dataSourceScan = indexSubTree.dataSourceScan;
-
- InvertedIndexJobGenParams jobGenParams = new InvertedIndexJobGenParams(chosenIndex.getIndexName(), chosenIndex.getKind(), datasetDecl.getName(), retainInput, requiresBroadcast);
+
+ InvertedIndexJobGenParams jobGenParams = new InvertedIndexJobGenParams(chosenIndex.getIndexName(),
+ chosenIndex.getIndexType(), dataset.getDatasetName(), retainInput, requiresBroadcast);
// Add function-specific args such as search modifier, and possibly a similarity threshold.
addFunctionSpecificArgs(optFuncExpr, jobGenParams);
// Add the type of search key from the optFuncExpr.
addSearchKeyType(optFuncExpr, indexSubTree, context, jobGenParams);
-
+
// Operator that feeds the secondary-index search.
AbstractLogicalOperator inputOp = null;
// Here we generate vars and funcs for assigning the secondary-index keys to be fed into the secondary-index search.
@@ -332,7 +339,7 @@
// Add key vars and exprs to argument list.
addKeyVarsAndExprs(optFuncExpr, keyVarList, keyExprList, context);
// Assign operator that sets the secondary-index search-key fields.
- inputOp = new AssignOperator(keyVarList, keyExprList);
+ inputOp = new AssignOperator(keyVarList, keyExprList);
// Input to this assign is the EmptyTupleSource (which the dataSourceScan also must have had as input).
inputOp.getInputs().add(dataSourceScan.getInputs().get(0));
inputOp.setExecutionMode(dataSourceScan.getExecutionMode());
@@ -343,14 +350,14 @@
inputOp = (AbstractLogicalOperator) probeSubTree.root;
}
jobGenParams.setKeyVarList(keyVarList);
- UnnestMapOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(datasetDecl,
- recordType, chosenIndex, inputOp, jobGenParams, context, true, retainInput);
+ UnnestMapOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
+ chosenIndex, inputOp, jobGenParams, context, true, retainInput);
// Generate the rest of the upstream plan which feeds the search results into the primary index.
- UnnestMapOperator primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceScan,
- datasetDecl, recordType, secondaryIndexUnnestOp, context, true, retainInput, false);
+ UnnestMapOperator primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceScan, dataset,
+ recordType, secondaryIndexUnnestOp, context, true, retainInput, false);
return primaryIndexUnnestOp;
}
-
+
/**
* Returns the variable which acts as the input search key to a secondary
* index that optimizes optFuncExpr by replacing rewriting indexSubTree
@@ -365,96 +372,99 @@
return optFuncExpr.getLogicalVar(0);
}
}
-
+
@Override
- public boolean applySelectPlanTransformation(Mutable<ILogicalOperator> selectRef, OptimizableOperatorSubTree subTree,
- AqlCompiledIndexDecl chosenIndex, AccessMethodAnalysisContext analysisCtx, IOptimizationContext context)
- throws AlgebricksException {
+ public boolean applySelectPlanTransformation(Mutable<ILogicalOperator> selectRef,
+ OptimizableOperatorSubTree subTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
+ IOptimizationContext context) throws AlgebricksException {
IOptimizableFuncExpr optFuncExpr = chooseOptFuncExpr(chosenIndex, analysisCtx);
- ILogicalOperator indexPlanRootOp = createSecondaryToPrimaryPlan(subTree, null, chosenIndex, optFuncExpr, false, false, context);
+ ILogicalOperator indexPlanRootOp = createSecondaryToPrimaryPlan(subTree, null, chosenIndex, optFuncExpr, false,
+ false, context);
// Replace the datasource scan with the new plan rooted at primaryIndexUnnestMap.
subTree.dataSourceScanRef.setValue(indexPlanRootOp);
return true;
}
-
+
@Override
public boolean applyJoinPlanTransformation(Mutable<ILogicalOperator> joinRef,
- OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree,
- AqlCompiledIndexDecl chosenIndex, AccessMethodAnalysisContext analysisCtx, IOptimizationContext context)
- throws AlgebricksException {
+ OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree, Index chosenIndex,
+ AccessMethodAnalysisContext analysisCtx, IOptimizationContext context) throws AlgebricksException {
// Figure out if the index is applicable on the left or right side (if both, we arbitrarily prefer the left side).
- AqlCompiledDatasetDecl dataset = analysisCtx.indexDatasetMap.get(chosenIndex);
+ Dataset dataset = analysisCtx.indexDatasetMap.get(chosenIndex);
// Determine probe and index subtrees based on chosen index.
OptimizableOperatorSubTree indexSubTree = null;
OptimizableOperatorSubTree probeSubTree = null;
- if (dataset.getName().equals(leftSubTree.datasetDecl.getName())) {
+ if (dataset.getDatasetName().equals(leftSubTree.dataset.getDatasetName())) {
indexSubTree = leftSubTree;
probeSubTree = rightSubTree;
- } else if (dataset.getName().equals(rightSubTree.datasetDecl.getName())) {
+ } else if (dataset.getDatasetName().equals(rightSubTree.dataset.getDatasetName())) {
indexSubTree = rightSubTree;
probeSubTree = leftSubTree;
}
IOptimizableFuncExpr optFuncExpr = chooseOptFuncExpr(chosenIndex, analysisCtx);
-
+
// Clone the original join condition because we may have to modify it (and we also need the original).
InnerJoinOperator join = (InnerJoinOperator) joinRef.getValue();
ILogicalExpression joinCond = join.getCondition().getValue().cloneExpression();
-
+
// Remember original live variables to make sure our new index-based plan returns exactly those vars as well.
List<LogicalVariable> originalLiveVars = new ArrayList<LogicalVariable>();
VariableUtilities.getLiveVariables(join, originalLiveVars);
-
+
// Create "panic" (non indexed) nested-loop join path if necessary.
Mutable<ILogicalOperator> panicJoinRef = null;
if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == AsterixBuiltinFunctions.EDIT_DISTANCE_CHECK) {
panicJoinRef = new MutableObject<ILogicalOperator>(joinRef.getValue());
- Mutable<ILogicalOperator> newProbeRootRef = createPanicNestedLoopJoinPlan(panicJoinRef, indexSubTree, probeSubTree, optFuncExpr, chosenIndex, context);
+ Mutable<ILogicalOperator> newProbeRootRef = createPanicNestedLoopJoinPlan(panicJoinRef, indexSubTree,
+ probeSubTree, optFuncExpr, chosenIndex, context);
probeSubTree.rootRef.setValue(newProbeRootRef.getValue());
probeSubTree.root = newProbeRootRef.getValue();
}
// Create regular indexed-nested loop join path.
- ILogicalOperator indexPlanRootOp = createSecondaryToPrimaryPlan(indexSubTree, probeSubTree, chosenIndex, optFuncExpr, true, true, context);
+ ILogicalOperator indexPlanRootOp = createSecondaryToPrimaryPlan(indexSubTree, probeSubTree, chosenIndex,
+ optFuncExpr, true, true, context);
indexSubTree.dataSourceScanRef.setValue(indexPlanRootOp);
-
+
// Change join into a select with the same condition.
SelectOperator topSelect = new SelectOperator(new MutableObject<ILogicalExpression>(joinCond));
topSelect.getInputs().add(indexSubTree.rootRef);
topSelect.setExecutionMode(ExecutionMode.LOCAL);
context.computeAndSetTypeEnvironmentForOperator(topSelect);
-
+
// Add a project operator on top to guarantee that our new index-based plan returns exactly the same variables as the original plan.
ProjectOperator projectOp = new ProjectOperator(originalLiveVars);
projectOp.getInputs().add(new MutableObject<ILogicalOperator>(topSelect));
projectOp.setExecutionMode(ExecutionMode.LOCAL);
context.computeAndSetTypeEnvironmentForOperator(projectOp);
joinRef.setValue(projectOp);
-
+
// Hook up the indexed-nested loop join path with the "panic" (non indexed) nested-loop join path by putting a union all on top.
if (panicJoinRef != null) {
// Gather live variables from the index plan and the panic plan.
List<LogicalVariable> indexPlanLiveVars = new ArrayList<LogicalVariable>();
- VariableUtilities.getLiveVariables(joinRef.getValue(), indexPlanLiveVars);
- List<LogicalVariable> panicPlanLiveVars = new ArrayList<LogicalVariable>();
- VariableUtilities.getLiveVariables(panicJoinRef.getValue(), panicPlanLiveVars);
- if (indexPlanLiveVars.size() != panicPlanLiveVars.size()) {
- throw new AlgebricksException("Unequal number of variables returned from index plan and panic plan.");
- }
- // Create variable mapping for union all operator.
- List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap = new ArrayList<Triple<LogicalVariable, LogicalVariable, LogicalVariable>>();
- for (int i = 0; i < indexPlanLiveVars.size(); i++) {
- varMap.add(new Triple<LogicalVariable, LogicalVariable, LogicalVariable>(indexPlanLiveVars.get(i), panicPlanLiveVars.get(i), indexPlanLiveVars.get(i)));
- }
- UnionAllOperator unionAllOp = new UnionAllOperator(varMap);
- unionAllOp.getInputs().add(new MutableObject<ILogicalOperator>(joinRef.getValue()));
- unionAllOp.getInputs().add(panicJoinRef);
- unionAllOp.setExecutionMode(ExecutionMode.PARTITIONED);
- context.computeAndSetTypeEnvironmentForOperator(unionAllOp);
- joinRef.setValue(unionAllOp);
+ VariableUtilities.getLiveVariables(joinRef.getValue(), indexPlanLiveVars);
+ List<LogicalVariable> panicPlanLiveVars = new ArrayList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(panicJoinRef.getValue(), panicPlanLiveVars);
+ if (indexPlanLiveVars.size() != panicPlanLiveVars.size()) {
+ throw new AlgebricksException("Unequal number of variables returned from index plan and panic plan.");
+ }
+ // Create variable mapping for union all operator.
+ List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap = new ArrayList<Triple<LogicalVariable, LogicalVariable, LogicalVariable>>();
+ for (int i = 0; i < indexPlanLiveVars.size(); i++) {
+ varMap.add(new Triple<LogicalVariable, LogicalVariable, LogicalVariable>(indexPlanLiveVars.get(i),
+ panicPlanLiveVars.get(i), indexPlanLiveVars.get(i)));
+ }
+ UnionAllOperator unionAllOp = new UnionAllOperator(varMap);
+ unionAllOp.getInputs().add(new MutableObject<ILogicalOperator>(joinRef.getValue()));
+ unionAllOp.getInputs().add(panicJoinRef);
+ unionAllOp.setExecutionMode(ExecutionMode.PARTITIONED);
+ context.computeAndSetTypeEnvironmentForOperator(unionAllOp);
+ joinRef.setValue(unionAllOp);
}
return true;
}
-
- private IOptimizableFuncExpr chooseOptFuncExpr(AqlCompiledIndexDecl chosenIndex, AccessMethodAnalysisContext analysisCtx) {
+
+ private IOptimizableFuncExpr chooseOptFuncExpr(Index chosenIndex, AccessMethodAnalysisContext analysisCtx) {
// TODO: We can probably do something smarter here.
// Pick the first expr optimizable by this index.
List<Integer> indexExprs = analysisCtx.getIndexExprs(chosenIndex);
@@ -462,15 +472,18 @@
return analysisCtx.matchedFuncExprs.get(firstExprIndex);
}
- private Mutable<ILogicalOperator> createPanicNestedLoopJoinPlan(Mutable<ILogicalOperator> joinRef, OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree, IOptimizableFuncExpr optFuncExpr, AqlCompiledIndexDecl chosenIndex, IOptimizationContext context) throws AlgebricksException {
+ private Mutable<ILogicalOperator> createPanicNestedLoopJoinPlan(Mutable<ILogicalOperator> joinRef,
+ OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree,
+ IOptimizableFuncExpr optFuncExpr, Index chosenIndex, IOptimizationContext context)
+ throws AlgebricksException {
LogicalVariable inputSearchVar = getInputSearchVar(optFuncExpr, indexSubTree);
-
+
// We split the plan into two "branches", and add selections on each side.
AbstractLogicalOperator replicateOp = new ReplicateOperator(2);
replicateOp.getInputs().add(new MutableObject<ILogicalOperator>(probeSubTree.root));
replicateOp.setExecutionMode(ExecutionMode.PARTITIONED);
- context.computeAndSetTypeEnvironmentForOperator(replicateOp);
-
+ context.computeAndSetTypeEnvironmentForOperator(replicateOp);
+
// Create select ops for removing tuples that are filterable and not filterable, respectively.
IVariableTypeEnvironment topTypeEnv = context.getOutputTypeEnvironment(joinRef.getValue());
IAType inputSearchVarType = (IAType) topTypeEnv.getVarType(inputSearchVar);
@@ -478,23 +491,22 @@
Mutable<ILogicalOperator> isNotFilterableSelectOpRef = new MutableObject<ILogicalOperator>();
createIsFilterableSelectOps(replicateOp, inputSearchVar, inputSearchVarType, optFuncExpr, chosenIndex, context,
isFilterableSelectOpRef, isNotFilterableSelectOpRef);
-
+
List<LogicalVariable> originalLiveVars = new ArrayList<LogicalVariable>();
VariableUtilities.getLiveVariables(indexSubTree.root, originalLiveVars);
-
+
// Copy the scan subtree in indexSubTree.
Counter counter = new Counter(context.getVarCounter());
LogicalOperatorDeepCopyVisitor deepCopyVisitor = new LogicalOperatorDeepCopyVisitor(counter);
- ILogicalOperator scanSubTree = deepCopyVisitor.deepCopy(indexSubTree.root, null);
+ ILogicalOperator scanSubTree = deepCopyVisitor.deepCopy(indexSubTree.root, null);
context.setVarCounter(counter.get());
- //context.computeAndSetTypeEnvironmentForOperator(scanSubTree);
-
+
List<LogicalVariable> copyLiveVars = new ArrayList<LogicalVariable>();
VariableUtilities.getLiveVariables(scanSubTree, copyLiveVars);
-
- // Replace the inputs of the given join op, and replace variables in its
- // condition since we deep-copied one of the scanner subtrees which
- // changed variables.
+
+ // Replace the inputs of the given join op, and replace variables in its
+ // condition since we deep-copied one of the scanner subtrees which
+ // changed variables.
InnerJoinOperator joinOp = (InnerJoinOperator) joinRef.getValue();
// Substitute vars in the join condition due to copying of the scanSubTree.
List<LogicalVariable> joinCondUsedVars = new ArrayList<LogicalVariable>();
@@ -503,80 +515,99 @@
int ix = originalLiveVars.indexOf(joinCondUsedVars.get(i));
if (ix >= 0) {
joinOp.getCondition().getValue().substituteVar(originalLiveVars.get(ix), copyLiveVars.get(ix));
- }
- }
+ }
+ }
joinOp.getInputs().clear();
joinOp.getInputs().add(new MutableObject<ILogicalOperator>(scanSubTree));
// Make sure that the build input (which may be materialized causing blocking) comes from
// the split+select, otherwise the plan will have a deadlock.
joinOp.getInputs().add(isNotFilterableSelectOpRef);
context.computeAndSetTypeEnvironmentForOperator(joinOp);
-
+
// Return the new root of the probeSubTree.
return isFilterableSelectOpRef;
}
-
- private void createIsFilterableSelectOps(ILogicalOperator inputOp, LogicalVariable inputSearchVar, IAType inputSearchVarType, IOptimizableFuncExpr optFuncExpr, AqlCompiledIndexDecl chosenIndex, IOptimizationContext context, Mutable<ILogicalOperator> isFilterableSelectOpRef, Mutable<ILogicalOperator> isNotFilterableSelectOpRef) throws AlgebricksException {
+
+ private void createIsFilterableSelectOps(ILogicalOperator inputOp, LogicalVariable inputSearchVar,
+ IAType inputSearchVarType, IOptimizableFuncExpr optFuncExpr, Index chosenIndex,
+ IOptimizationContext context, Mutable<ILogicalOperator> isFilterableSelectOpRef,
+ Mutable<ILogicalOperator> isNotFilterableSelectOpRef) throws AlgebricksException {
// Create select operator for removing tuples that are not filterable.
// First determine the proper filter function and args based on the type of the input search var.
ILogicalExpression isFilterableExpr = null;
switch (inputSearchVarType.getTypeTag()) {
case STRING: {
List<Mutable<ILogicalExpression>> isFilterableArgs = new ArrayList<Mutable<ILogicalExpression>>(4);
- isFilterableArgs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(inputSearchVar)));
+ isFilterableArgs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
+ inputSearchVar)));
// Since we are optimizing a join, the similarity threshold should be the only constant in the optimizable function expression.
- isFilterableArgs.add(new MutableObject<ILogicalExpression>(new ConstantExpression(optFuncExpr.getConstantVal(0))));
- isFilterableArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createInt32Constant(chosenIndex.getGramLength())));
+ isFilterableArgs.add(new MutableObject<ILogicalExpression>(new ConstantExpression(optFuncExpr
+ .getConstantVal(0))));
+ isFilterableArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils
+ .createInt32Constant(chosenIndex.getGramLength())));
// TODO: Currently usePrePost is hardcoded to be true.
- isFilterableArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createBooleanConstant(true)));
- isFilterableExpr = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.EDIT_DISTANCE_STRING_IS_FILTERABLE), isFilterableArgs);
+ isFilterableArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils
+ .createBooleanConstant(true)));
+ isFilterableExpr = new ScalarFunctionCallExpression(
+ FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.EDIT_DISTANCE_STRING_IS_FILTERABLE),
+ isFilterableArgs);
break;
}
case UNORDEREDLIST:
case ORDEREDLIST: {
List<Mutable<ILogicalExpression>> isFilterableArgs = new ArrayList<Mutable<ILogicalExpression>>(2);
- isFilterableArgs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(inputSearchVar)));
+ isFilterableArgs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
+ inputSearchVar)));
// Since we are optimizing a join, the similarity threshold should be the only constant in the optimizable function expression.
- isFilterableArgs.add(new MutableObject<ILogicalExpression>(new ConstantExpression(optFuncExpr.getConstantVal(0))));
- isFilterableExpr = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.EDIT_DISTANCE_LIST_IS_FILTERABLE), isFilterableArgs);
+ isFilterableArgs.add(new MutableObject<ILogicalExpression>(new ConstantExpression(optFuncExpr
+ .getConstantVal(0))));
+ isFilterableExpr = new ScalarFunctionCallExpression(
+ FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.EDIT_DISTANCE_LIST_IS_FILTERABLE),
+ isFilterableArgs);
break;
}
default: {
}
}
- SelectOperator isFilterableSelectOp = new SelectOperator(new MutableObject<ILogicalExpression>(isFilterableExpr));
+ SelectOperator isFilterableSelectOp = new SelectOperator(
+ new MutableObject<ILogicalExpression>(isFilterableExpr));
isFilterableSelectOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
isFilterableSelectOp.setExecutionMode(ExecutionMode.LOCAL);
context.computeAndSetTypeEnvironmentForOperator(isFilterableSelectOp);
-
+
// Select operator for removing tuples that are filterable.
List<Mutable<ILogicalExpression>> isNotFilterableArgs = new ArrayList<Mutable<ILogicalExpression>>();
isNotFilterableArgs.add(new MutableObject<ILogicalExpression>(isFilterableExpr));
- ILogicalExpression isNotFilterableExpr = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.NOT), isNotFilterableArgs);
- SelectOperator isNotFilterableSelectOp = new SelectOperator(new MutableObject<ILogicalExpression>(isNotFilterableExpr));
+ ILogicalExpression isNotFilterableExpr = new ScalarFunctionCallExpression(
+ FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.NOT), isNotFilterableArgs);
+ SelectOperator isNotFilterableSelectOp = new SelectOperator(new MutableObject<ILogicalExpression>(
+ isNotFilterableExpr));
isNotFilterableSelectOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
isNotFilterableSelectOp.setExecutionMode(ExecutionMode.LOCAL);
context.computeAndSetTypeEnvironmentForOperator(isNotFilterableSelectOp);
-
+
isFilterableSelectOpRef.setValue(isFilterableSelectOp);
isNotFilterableSelectOpRef.setValue(isNotFilterableSelectOp);
}
-
- private void addSearchKeyType(IOptimizableFuncExpr optFuncExpr, OptimizableOperatorSubTree indexSubTree, IOptimizationContext context, InvertedIndexJobGenParams jobGenParams) throws AlgebricksException {
+
+ private void addSearchKeyType(IOptimizableFuncExpr optFuncExpr, OptimizableOperatorSubTree indexSubTree,
+ IOptimizationContext context, InvertedIndexJobGenParams jobGenParams) throws AlgebricksException {
// If we have two variables in the optFunxExpr, then we are optimizing a join.
IAType type = null;
ATypeTag typeTag = null;
- if (optFuncExpr.getNumLogicalVars() == 2) {
+ if (optFuncExpr.getNumLogicalVars() == 2) {
// Find the type of the variable that is going to feed into the index search.
if (optFuncExpr.getOperatorSubTree(0) == indexSubTree) {
// If the index is on a dataset in subtree 0, then subtree 1 will feed.
- type = (IAType) context.getOutputTypeEnvironment(optFuncExpr.getOperatorSubTree(1).root).getVarType(optFuncExpr.getLogicalVar(1));
+ type = (IAType) context.getOutputTypeEnvironment(optFuncExpr.getOperatorSubTree(1).root).getVarType(
+ optFuncExpr.getLogicalVar(1));
} else {
// If the index is on a dataset in subtree 1, then subtree 0 will feed.
- type = (IAType) context.getOutputTypeEnvironment(optFuncExpr.getOperatorSubTree(0).root).getVarType(optFuncExpr.getLogicalVar(0));
+ type = (IAType) context.getOutputTypeEnvironment(optFuncExpr.getOperatorSubTree(0).root).getVarType(
+ optFuncExpr.getLogicalVar(0));
}
typeTag = type.getTypeTag();
- } else {
+ } else {
// We are optimizing a selection query. Add the type of the search key constant.
AsterixConstantValue constVal = (AsterixConstantValue) optFuncExpr.getConstantVal(0);
IAObject obj = constVal.getObject();
@@ -588,7 +619,7 @@
}
jobGenParams.setSearchKeyType(typeTag);
}
-
+
private void addFunctionSpecificArgs(IOptimizableFuncExpr optFuncExpr, InvertedIndexJobGenParams jobGenParams) {
if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == AsterixBuiltinFunctions.CONTAINS) {
jobGenParams.setSearchModifierType(SearchModifierType.CONJUNCTIVE);
@@ -606,17 +637,19 @@
}
}
- private void addKeyVarsAndExprs(IOptimizableFuncExpr optFuncExpr, ArrayList<LogicalVariable> keyVarList, ArrayList<Mutable<ILogicalExpression>> keyExprList, IOptimizationContext context) throws AlgebricksException {
+ private void addKeyVarsAndExprs(IOptimizableFuncExpr optFuncExpr, ArrayList<LogicalVariable> keyVarList,
+ ArrayList<Mutable<ILogicalExpression>> keyExprList, IOptimizationContext context)
+ throws AlgebricksException {
// For now we are assuming a single secondary index key.
// Add a variable and its expr to the lists which will be passed into an assign op.
LogicalVariable keyVar = context.newVar();
keyVarList.add(keyVar);
- keyExprList.add(new MutableObject<ILogicalExpression>(new ConstantExpression(optFuncExpr.getConstantVal(0))));
+ keyExprList.add(new MutableObject<ILogicalExpression>(new ConstantExpression(optFuncExpr.getConstantVal(0))));
return;
}
@Override
- public boolean exprIsOptimizable(AqlCompiledIndexDecl index, IOptimizableFuncExpr optFuncExpr) {
+ public boolean exprIsOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) {
if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == AsterixBuiltinFunctions.EDIT_DISTANCE_CHECK) {
// Must be for a join query.
if (optFuncExpr.getNumConstantVals() == 1) {
@@ -631,16 +664,16 @@
AInt32 edThresh = (AInt32) intObj;
int mergeThreshold = 0;
// We can only optimize edit distance on strings using an ngram index.
- if (listOrStrObj.getType().getTypeTag() == ATypeTag.STRING && index.getKind() == IndexKind.NGRAM_INVIX) {
- AString astr = (AString) listOrStrObj;
+ if (listOrStrObj.getType().getTypeTag() == ATypeTag.STRING && index.getIndexType() == IndexType.NGRAM_INVIX) {
+ AString astr = (AString) listOrStrObj;
// Compute merge threshold.
mergeThreshold = (astr.getStringValue().length() + index.getGramLength() - 1)
- edThresh.getIntegerValue() * index.getGramLength();
}
// We can only optimize edit distance on lists using a word index.
if ((listOrStrObj.getType().getTypeTag() == ATypeTag.ORDEREDLIST || listOrStrObj.getType().getTypeTag() == ATypeTag.UNORDEREDLIST)
- && index.getKind() == IndexKind.WORD_INVIX) {
- IACollection alist = (IACollection) listOrStrObj;
+ && index.getIndexType() == IndexType.WORD_INVIX) {
+ IACollection alist = (IACollection) listOrStrObj;
// Compute merge threshold.
mergeThreshold = alist.size() - edThresh.getIntegerValue();
}
@@ -664,12 +697,12 @@
if (nonConstArg.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
AbstractFunctionCallExpression nonConstfuncExpr = (AbstractFunctionCallExpression) nonConstArg;
// We can use this index if the tokenization function matches the index type.
- if (nonConstfuncExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.WORD_TOKENS &&
- index.getKind() == IndexKind.WORD_INVIX) {
+ if (nonConstfuncExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.WORD_TOKENS
+ && index.getIndexType() == IndexType.WORD_INVIX) {
return true;
}
- if (nonConstfuncExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.GRAM_TOKENS &&
- index.getKind() == IndexKind.NGRAM_INVIX) {
+ if (nonConstfuncExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.GRAM_TOKENS
+ && index.getIndexType() == IndexType.NGRAM_INVIX) {
return true;
}
}
@@ -682,12 +715,12 @@
}
// We can only optimize contains with ngram indexes.
if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == AsterixBuiltinFunctions.CONTAINS
- && index.getKind() == IndexKind.NGRAM_INVIX) {
+ && index.getIndexType() == IndexType.NGRAM_INVIX) {
// Check that the constant search string has at least gramLength characters.
AsterixConstantValue strConstVal = (AsterixConstantValue) optFuncExpr.getConstantVal(0);
IAObject strObj = strConstVal.getObject();
if (strObj.getType().getTypeTag() == ATypeTag.STRING) {
- AString astr = (AString) strObj;
+ AString astr = (AString) strObj;
if (astr.getStringValue().length() >= index.getGramLength()) {
return true;
}
@@ -695,10 +728,10 @@
}
return false;
}
-
+
public static IBinaryComparatorFactory getTokenBinaryComparatorFactory(IAType keyType) throws AlgebricksException {
IAType type = keyType;
- ATypeTag typeTag = keyType.getTypeTag();
+ ATypeTag typeTag = keyType.getTypeTag();
// Extract item type from list.
if (typeTag == ATypeTag.UNORDEREDLIST || typeTag == ATypeTag.ORDEREDLIST) {
AbstractCollectionType listType = (AbstractCollectionType) keyType;
@@ -708,10 +741,9 @@
type = listType.getItemType();
}
// Ignore case for string types.
- return AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(
- type, true, true);
+ return AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(type, true, true);
}
-
+
public static ITypeTraits getTokenTypeTrait(IAType keyType) throws AlgebricksException {
IAType type = keyType;
ATypeTag typeTag = keyType.getTypeTag();
@@ -732,10 +764,10 @@
}
return AqlTypeTraitProvider.INSTANCE.getTypeTrait(type);
}
-
- public static IBinaryTokenizerFactory getBinaryTokenizerFactory(SearchModifierType searchModifierType, ATypeTag searchKeyType, AqlCompiledIndexDecl index)
- throws AlgebricksException {
- switch (index.getKind()) {
+
+ public static IBinaryTokenizerFactory getBinaryTokenizerFactory(SearchModifierType searchModifierType,
+ ATypeTag searchKeyType, Index index) throws AlgebricksException {
+ switch (index.getIndexType()) {
case WORD_INVIX: {
return AqlBinaryTokenizerFactoryProvider.INSTANCE.getWordTokenizerFactory(searchKeyType, false);
}
@@ -746,29 +778,30 @@
index.getGramLength(), prePost, false);
}
default: {
- throw new AlgebricksException("Tokenizer not applicable to index kind '" + index.getKind() + "'.");
+ throw new AlgebricksException("Tokenizer not applicable to index kind '" + index.getIndexType() + "'.");
}
}
}
-
- public static IBinaryTokenizerFactory getBinaryTokenizerFactory(ATypeTag keyType, IndexType indexType, int gramLength)
- throws AlgebricksException {
+
+ public static IBinaryTokenizerFactory getBinaryTokenizerFactory(ATypeTag keyType, IndexType indexType,
+ int gramLength) throws AlgebricksException {
switch (indexType) {
case WORD_INVIX: {
return AqlBinaryTokenizerFactoryProvider.INSTANCE.getWordTokenizerFactory(keyType, false);
}
case NGRAM_INVIX: {
- return AqlBinaryTokenizerFactoryProvider.INSTANCE.getNGramTokenizerFactory(keyType,
- gramLength, true, false);
+ return AqlBinaryTokenizerFactoryProvider.INSTANCE.getNGramTokenizerFactory(keyType, gramLength, true,
+ false);
}
default: {
throw new AlgebricksException("Tokenizer not applicable to index type '" + indexType + "'.");
}
}
}
-
- public static IInvertedIndexSearchModifierFactory getSearchModifierFactory(SearchModifierType searchModifierType, IAObject simThresh, AqlCompiledIndexDecl index) throws AlgebricksException {
- switch(searchModifierType) {
+
+ public static IInvertedIndexSearchModifierFactory getSearchModifierFactory(SearchModifierType searchModifierType,
+ IAObject simThresh, Index index) throws AlgebricksException {
+ switch (searchModifierType) {
case CONJUNCTIVE: {
return new ConjunctiveSearchModifierFactory();
}
@@ -778,7 +811,7 @@
}
case EDIT_DISTANCE: {
int edThresh = ((AInt32) simThresh).getIntegerValue();
- switch (index.getKind()) {
+ switch (index.getIndexType()) {
case NGRAM_INVIX: {
// Edit distance on strings, filtered with overlapping grams.
return new EditDistanceSearchModifierFactory(index.getGramLength(), edThresh);
@@ -788,7 +821,8 @@
return new ListEditDistanceSearchModifierFactory(edThresh);
}
default: {
- throw new AlgebricksException("Incompatible search modifier '" + searchModifierType + "' for index type '" + index.getKind() + "'");
+ throw new AlgebricksException("Incompatible search modifier '" + searchModifierType
+ + "' for index type '" + index.getIndexType() + "'");
}
}
}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/InvertedIndexJobGenParams.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/InvertedIndexJobGenParams.java
index fa5b5e1..530606e 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/InvertedIndexJobGenParams.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/InvertedIndexJobGenParams.java
@@ -6,7 +6,7 @@
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.commons.lang3.mutable.MutableObject;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl.IndexKind;
+import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
import edu.uci.ics.asterix.om.types.ATypeTag;
import edu.uci.ics.asterix.optimizer.rules.am.InvertedIndexAccessMethod.SearchModifierType;
@@ -20,44 +20,47 @@
* and from a list of function arguments, typically of an unnest-map.
*/
public class InvertedIndexJobGenParams extends AccessMethodJobGenParams {
-
+
protected SearchModifierType searchModifierType;
protected IAlgebricksConstantValue similarityThreshold;
protected ATypeTag searchKeyType;
protected List<LogicalVariable> keyVarList;
protected List<LogicalVariable> nonKeyVarList;
-
+
public InvertedIndexJobGenParams() {
}
-
- public InvertedIndexJobGenParams(String indexName, IndexKind indexKind, String datasetName, boolean retainInput, boolean requiresBroadcast) {
- super(indexName, indexKind, datasetName, retainInput, requiresBroadcast);
+
+ public InvertedIndexJobGenParams(String indexName, IndexType indexType, String datasetName, boolean retainInput,
+ boolean requiresBroadcast) {
+ super(indexName, indexType, datasetName, retainInput, requiresBroadcast);
}
-
+
public void setSearchModifierType(SearchModifierType searchModifierType) {
this.searchModifierType = searchModifierType;
}
-
+
public void setSimilarityThreshold(IAlgebricksConstantValue similarityThreshold) {
this.similarityThreshold = similarityThreshold;
}
-
+
public void setSearchKeyType(ATypeTag searchKeyType) {
this.searchKeyType = searchKeyType;
}
-
+
public void setKeyVarList(List<LogicalVariable> keyVarList) {
this.keyVarList = keyVarList;
}
-
+
public void writeToFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
super.writeToFuncArgs(funcArgs);
// Write search modifier type.
- funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createInt32Constant(searchModifierType.ordinal())));
+ funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createInt32Constant(searchModifierType
+ .ordinal())));
// Write similarity threshold.
funcArgs.add(new MutableObject<ILogicalExpression>(new ConstantExpression(similarityThreshold)));
// Write search key type.
- funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createInt32Constant(searchKeyType.ordinal())));
+ funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createInt32Constant(searchKeyType
+ .ordinal())));
// Write key var list.
writeVarList(keyVarList, funcArgs);
// Write non-key var list.
@@ -65,7 +68,7 @@
writeVarList(nonKeyVarList, funcArgs);
}
}
-
+
public void readFromFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
super.readFromFuncArgs(funcArgs);
int index = super.getNumParams();
@@ -73,7 +76,8 @@
int searchModifierOrdinal = AccessMethodUtils.getInt32Constant(funcArgs.get(index));
searchModifierType = SearchModifierType.values()[searchModifierOrdinal];
// Read similarity threshold. Concrete type depends on search modifier.
- similarityThreshold = ((AsterixConstantValue) ((ConstantExpression) funcArgs.get(index + 1).getValue()).getValue());
+ similarityThreshold = ((AsterixConstantValue) ((ConstantExpression) funcArgs.get(index + 1).getValue())
+ .getValue());
// Read type of search key.
int typeTagOrdinal = AccessMethodUtils.getInt32Constant(funcArgs.get(index + 2));
searchKeyType = ATypeTag.values()[typeTagOrdinal];
@@ -84,23 +88,23 @@
// We don't need to read the non-key var list.
nonKeyVarList = null;
}
-
+
public SearchModifierType getSearchModifierType() {
return searchModifierType;
}
-
+
public IAlgebricksConstantValue getSimilarityThreshold() {
return similarityThreshold;
}
-
+
public ATypeTag getSearchKeyType() {
return searchKeyType;
}
-
+
public List<LogicalVariable> getKeyVarList() {
return keyVarList;
}
-
+
public List<LogicalVariable> getNonKeyVarList() {
return nonKeyVarList;
}
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 dc4f8fc..80f8cc1 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
@@ -6,9 +6,9 @@
import org.apache.commons.lang3.mutable.Mutable;
import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.ATypeTag;
import edu.uci.ics.asterix.om.types.IAType;
@@ -22,7 +22,7 @@
/**
* Operator subtree that matches the following patterns, and provides convenient access to its nodes:
- * (select)? <-- (assign)+ <-- (datasource scan)
+ * (select)? <-- (assign)+ <-- (datasource scan)
* and
* (select)? <-- (datasource scan)
*/
@@ -34,9 +34,9 @@
public Mutable<ILogicalOperator> dataSourceScanRef = null;
public DataSourceScanOperator dataSourceScan = null;
// Dataset and type metadata. Set in setDatasetAndTypeMetadata().
- public AqlCompiledDatasetDecl datasetDecl = null;
- public ARecordType recordType = null;
-
+ public Dataset dataset = null;
+ public ARecordType recordType = null;
+
public boolean initFromSubTree(Mutable<ILogicalOperator> subTreeOpRef) {
rootRef = subTreeOpRef;
root = subTreeOpRef.getValue();
@@ -62,7 +62,7 @@
assignRefs.add(subTreeOpRef);
assigns.add((AssignOperator) subTreeOp);
subTreeOpRef = subTreeOp.getInputs().get(0);
- subTreeOp = (AbstractLogicalOperator) subTreeOpRef.getValue();
+ subTreeOp = (AbstractLogicalOperator) subTreeOpRef.getValue();
} while (subTreeOp.getOperatorTag() == LogicalOperatorTag.ASSIGN);
// Set to last valid assigns.
subTreeOpRef = assignRefs.get(assignRefs.size() - 1);
@@ -77,11 +77,11 @@
dataSourceScan = (DataSourceScanOperator) op3;
return true;
}
-
+
/**
* Find the dataset corresponding to the datasource scan in the metadata.
* Also sets recordType to be the type of that dataset.
- */
+ */
public boolean setDatasetAndTypeMetadata(AqlMetadataProvider metadataProvider) throws AlgebricksException {
if (dataSourceScan == null) {
return false;
@@ -92,22 +92,22 @@
return false;
}
AqlCompiledMetadataDeclarations metadata = metadataProvider.getMetadataDeclarations();
- datasetDecl = metadata.findDataset(datasetName);
- if (datasetDecl == null) {
+ dataset = metadata.findDataset(datasetName);
+ if (dataset == null) {
throw new AlgebricksException("No metadata for dataset " + datasetName);
}
- if (datasetDecl.getDatasetType() != DatasetType.INTERNAL && datasetDecl.getDatasetType() != DatasetType.FEED) {
+ if (dataset.getDatasetType() != DatasetType.INTERNAL && dataset.getDatasetType() != DatasetType.FEED) {
return false;
}
// Get the record type for that dataset.
- IAType itemType = metadata.findType(datasetDecl.getItemTypeName());
+ IAType itemType = metadata.findType(dataset.getItemTypeName());
if (itemType.getTypeTag() != ATypeTag.RECORD) {
return false;
}
recordType = (ARecordType) itemType;
return true;
}
-
+
public boolean hasDataSourceScan() {
return dataSourceScan != null;
}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/RTreeAccessMethod.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/RTreeAccessMethod.java
index ac22095..dfd3ff7 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/RTreeAccessMethod.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/RTreeAccessMethod.java
@@ -7,9 +7,9 @@
import org.apache.commons.lang3.mutable.MutableObject;
import edu.uci.ics.asterix.aql.util.FunctionUtils;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl.IndexKind;
+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;
import edu.uci.ics.asterix.om.base.AInt32;
import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
@@ -31,7 +31,7 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
/**
- * Class for helping rewrite rules to choose and apply RTree indexes.
+ * Class for helping rewrite rules to choose and apply RTree indexes.
*/
public class RTreeAccessMethod implements IAccessMethod {
@@ -39,19 +39,20 @@
static {
funcIdents.add(AsterixBuiltinFunctions.SPATIAL_INTERSECT);
}
-
+
public static RTreeAccessMethod INSTANCE = new RTreeAccessMethod();
-
+
@Override
public List<FunctionIdentifier> getOptimizableFunctions() {
return funcIdents;
}
@Override
- public boolean analyzeFuncExprArgs(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns, AccessMethodAnalysisContext analysisCtx) {
+ public boolean analyzeFuncExprArgs(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns,
+ AccessMethodAnalysisContext analysisCtx) {
return AccessMethodUtils.analyzeFuncExprArgsForOneConstAndVar(funcExpr, analysisCtx);
}
-
+
@Override
public boolean matchAllIndexExprs() {
return true;
@@ -63,27 +64,28 @@
}
@Override
- public boolean applySelectPlanTransformation(Mutable<ILogicalOperator> selectRef, OptimizableOperatorSubTree subTree,
- AqlCompiledIndexDecl chosenIndex, AccessMethodAnalysisContext analysisCtx, IOptimizationContext context)
- throws AlgebricksException {
- AqlCompiledDatasetDecl datasetDecl = subTree.datasetDecl;
+ public boolean applySelectPlanTransformation(Mutable<ILogicalOperator> selectRef,
+ OptimizableOperatorSubTree subTree, Index index, AccessMethodAnalysisContext analysisCtx,
+ IOptimizationContext context) throws AlgebricksException {
+ Dataset dataset = subTree.dataset;
ARecordType recordType = subTree.recordType;
// TODO: We can probably do something smarter here based on selectivity or MBR area.
// Pick the first expr optimizable by this index.
- List<Integer> indexExprs = analysisCtx.getIndexExprs(chosenIndex);
+ List<Integer> indexExprs = analysisCtx.getIndexExprs(index);
int firstExprIndex = indexExprs.get(0);
IOptimizableFuncExpr optFuncExpr = analysisCtx.matchedFuncExprs.get(firstExprIndex);
-
+
// Get the number of dimensions corresponding to the field indexed by
// chosenIndex.
- Pair<IAType, Boolean> keyPairType = AqlCompiledIndexDecl.getNonNullableKeyFieldType(optFuncExpr.getFieldName(0), recordType);
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(optFuncExpr.getFieldName(0), recordType);
IAType spatialType = keyPairType.first;
int numDimensions = NonTaggedFormatUtil.getNumDimensions(spatialType.getTypeTag());
int numSecondaryKeys = numDimensions * 2;
-
+
DataSourceScanOperator dataSourceScan = subTree.dataSourceScan;
// TODO: For now retainInput and requiresBroadcast are always false.
- RTreeJobGenParams jobGenParams = new RTreeJobGenParams(chosenIndex.getIndexName(), IndexKind.RTREE, datasetDecl.getName(), false, false);
+ RTreeJobGenParams jobGenParams = new RTreeJobGenParams(index.getIndexName(), IndexType.RTREE,
+ dataset.getDatasetName(), false, false);
// A spatial object is serialized in the constant of the func expr we are optimizing.
// The R-Tree expects as input an MBR represented with 1 field per dimension.
// Here we generate vars and funcs for extracting MBR fields from the constant into fields of a tuple (as the R-Tree expects them).
@@ -96,14 +98,16 @@
AbstractFunctionCallExpression createMBR = new ScalarFunctionCallExpression(
FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.CREATE_MBR));
// Spatial object is the constant from the func expr we are optimizing.
- createMBR.getArguments().add(new MutableObject<ILogicalExpression>(new ConstantExpression(optFuncExpr.getConstantVal(0))));
+ createMBR.getArguments().add(
+ new MutableObject<ILogicalExpression>(new ConstantExpression(optFuncExpr.getConstantVal(0))));
// The number of dimensions.
createMBR.getArguments().add(
- new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(
- new AInt32(numDimensions)))));
+ new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(new AInt32(
+ numDimensions)))));
// Which part of the MBR to extract.
createMBR.getArguments().add(
- new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(new AInt32(i)))));
+ new MutableObject<ILogicalExpression>(new ConstantExpression(
+ new AsterixConstantValue(new AInt32(i)))));
// Add a variable and its expr to the lists which will be passed into an assign op.
LogicalVariable keyVar = context.newVar();
keyVarList.add(keyVar);
@@ -117,27 +121,26 @@
assignSearchKeys.getInputs().add(dataSourceScan.getInputs().get(0));
assignSearchKeys.setExecutionMode(dataSourceScan.getExecutionMode());
- UnnestMapOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(datasetDecl,
- recordType, chosenIndex, assignSearchKeys, jobGenParams, context, false, false);
+ UnnestMapOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
+ index, assignSearchKeys, jobGenParams, context, false, false);
// Generate the rest of the upstream plan which feeds the search results into the primary index.
- UnnestMapOperator primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceScan,
- datasetDecl, recordType, secondaryIndexUnnestOp, context, true, false, false);
+ UnnestMapOperator primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceScan, dataset,
+ recordType, secondaryIndexUnnestOp, context, true, false, false);
// Replace the datasource scan with the new plan rooted at primaryIndexUnnestMap.
subTree.dataSourceScanRef.setValue(primaryIndexUnnestOp);
return true;
}
-
+
@Override
public boolean applyJoinPlanTransformation(Mutable<ILogicalOperator> joinRef,
- OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree,
- AqlCompiledIndexDecl chosenIndex, AccessMethodAnalysisContext analysisCtx, IOptimizationContext context)
- throws AlgebricksException {
+ OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree, Index chosenIndex,
+ AccessMethodAnalysisContext analysisCtx, IOptimizationContext context) throws AlgebricksException {
// TODO Implement this.
return false;
}
-
+
@Override
- public boolean exprIsOptimizable(AqlCompiledIndexDecl index, IOptimizableFuncExpr optFuncExpr) {
+ public boolean exprIsOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) {
// No additional analysis required.
return true;
}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/RTreeJobGenParams.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/RTreeJobGenParams.java
index 2d850b2..b3153f9 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/RTreeJobGenParams.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/RTreeJobGenParams.java
@@ -5,7 +5,7 @@
import org.apache.commons.lang3.mutable.Mutable;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl.IndexKind;
+import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
@@ -14,36 +14,33 @@
* and from a list of function arguments, typically of an unnest-map.
*/
public class RTreeJobGenParams extends AccessMethodJobGenParams {
-
+
protected List<LogicalVariable> keyVarList;
-
+
public RTreeJobGenParams() {
}
-
- public RTreeJobGenParams(String indexName, IndexKind indexKind, String datasetName, boolean retainInput, boolean requiresBroadcast) {
- this.indexName = indexName;
- this.indexKind = indexKind;
- this.datasetName = datasetName;
- this.retainInput = retainInput;
- this.requiresBroadcast = requiresBroadcast;
+
+ public RTreeJobGenParams(String indexName, IndexType indexType, String datasetName, boolean retainInput,
+ boolean requiresBroadcast) {
+ super(indexName, indexType, datasetName, retainInput, requiresBroadcast);
}
-
+
public void writeToFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
super.writeToFuncArgs(funcArgs);
writeVarList(keyVarList, funcArgs);
}
-
+
public void readFromFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
super.readFromFuncArgs(funcArgs);
int index = super.getNumParams();
keyVarList = new ArrayList<LogicalVariable>();
readVarList(funcArgs, index, keyVarList);
}
-
+
public void setKeyVarList(List<LogicalVariable> keyVarList) {
this.keyVarList = keyVarList;
}
-
+
public List<LogicalVariable> getKeyVarList() {
return keyVarList;
}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java
index b1970ac..e004a7d 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java
@@ -80,8 +80,6 @@
import edu.uci.ics.asterix.metadata.MetadataException;
import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
import edu.uci.ics.asterix.metadata.bootstrap.AsterixProperties;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledInternalDatasetDetails;
import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
import edu.uci.ics.asterix.metadata.declared.AqlLogicalPlanAndMetadataImpl;
@@ -89,6 +87,8 @@
import edu.uci.ics.asterix.metadata.declared.AqlSourceId;
import edu.uci.ics.asterix.metadata.declared.FileSplitDataSink;
import edu.uci.ics.asterix.metadata.declared.FileSplitSinkId;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
import edu.uci.ics.asterix.om.base.AInt32;
import edu.uci.ics.asterix.om.base.AString;
import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
@@ -218,34 +218,31 @@
topOp.getInputs().add(new MutableObject<ILogicalOperator>(project));
} else {
String dataVerseName = compiledDeclarations.getDataverseName();
- AqlCompiledDatasetDecl adecl = compiledDeclarations.findDataset(outputDatasetName);
- if (adecl == null) {
+ Dataset dataset = compiledDeclarations.findDataset(outputDatasetName);
+ if (dataset == null) {
throw new AlgebricksException("Cannot find dataset " + outputDatasetName);
}
AqlSourceId sourceId = new AqlSourceId(dataVerseName, outputDatasetName);
- String itemTypeName = adecl.getItemTypeName();
+ String itemTypeName = dataset.getItemTypeName();
IAType itemType = compiledDeclarations.findType(itemTypeName);
- AqlDataSource dataSource = new AqlDataSource(sourceId, adecl, itemType);
-
- if (adecl.getDatasetType() == DatasetType.EXTERNAL) {
+ AqlDataSource dataSource = new AqlDataSource(sourceId, dataset, itemType);
+ if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
throw new AlgebricksException("Cannot write output to an external dataset.");
}
ArrayList<LogicalVariable> vars = new ArrayList<LogicalVariable>();
ArrayList<Mutable<ILogicalExpression>> exprs = new ArrayList<Mutable<ILogicalExpression>>();
List<Mutable<ILogicalExpression>> varRefsForLoading = new ArrayList<Mutable<ILogicalExpression>>();
- AqlCompiledInternalDatasetDetails datasetDetails = (AqlCompiledInternalDatasetDetails) adecl
- .getAqlCompiledDatasetDetails();
- List<String> partitionKeys = datasetDetails.getPartitioningExprs();
+ List<String> partitionKeys = DatasetUtils.getPartitioningKeys(dataset);
for (String keyFieldName : partitionKeys) {
IFunctionInfo finfoAccess = AsterixBuiltinFunctions
.getAsterixFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME);
@SuppressWarnings("unchecked")
ScalarFunctionCallExpression f = new ScalarFunctionCallExpression(finfoAccess,
new MutableObject<ILogicalExpression>(new VariableReferenceExpression(METADATA_DUMMY_VAR)),
- new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(new AString(
- keyFieldName)))));
+ new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(
+ new AString(keyFieldName)))));
f.substituteVar(METADATA_DUMMY_VAR, resVar);
exprs.add(new MutableObject<ILogicalExpression>(f));
LogicalVariable v = context.newVar();
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlPlusExpressionToPlanTranslator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlPlusExpressionToPlanTranslator.java
index e6aaaad..d8f294d 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlPlusExpressionToPlanTranslator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlPlusExpressionToPlanTranslator.java
@@ -81,12 +81,12 @@
import edu.uci.ics.asterix.common.functions.FunctionConstants;
import edu.uci.ics.asterix.formats.base.IDataFormat;
import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
import edu.uci.ics.asterix.metadata.declared.AqlLogicalPlanAndMetadataImpl;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
import edu.uci.ics.asterix.metadata.declared.FileSplitDataSink;
import edu.uci.ics.asterix.metadata.declared.FileSplitSinkId;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
import edu.uci.ics.asterix.om.base.AInt32;
import edu.uci.ics.asterix.om.base.AString;
@@ -94,6 +94,7 @@
import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
import edu.uci.ics.asterix.om.functions.AsterixFunction;
import edu.uci.ics.asterix.om.functions.AsterixFunctionInfo;
+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;
@@ -144,17 +145,13 @@
import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
/**
- *
* Each visit returns a pair of an operator and a variable. The variable
* corresponds to the new column, if any, added to the tuple flow. E.g., for
* Unnest, the column is the variable bound to the elements in the list, for
* Subplan it is null.
- *
* The first argument of a visit method is the expression which is translated.
- *
* The second argument of a visit method is the tuple source for the current
* subtree.
- *
*/
public class AqlPlusExpressionToPlanTranslator extends AbstractAqlTranslator implements
@@ -254,19 +251,21 @@
topOp = new WriteOperator(writeExprList, sink);
topOp.getInputs().add(new MutableObject<ILogicalOperator>(project));
} else {
- AqlCompiledDatasetDecl adecl = compiledDeclarations.findDataset(outputDatasetName);
- if (adecl == null) {
+ Dataset dataset = compiledDeclarations.findDataset(outputDatasetName);
+ if (dataset == null) {
throw new AlgebricksException("Cannot find dataset " + outputDatasetName);
}
-
- if (adecl.getDatasetType() == DatasetType.EXTERNAL) {
+ if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
throw new AlgebricksException("Cannot write output to an external dataset.");
}
+ ARecordType itemType = (ARecordType) compiledDeclarations.findType(dataset.getItemTypeName());
+ List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
ArrayList<LogicalVariable> vars = new ArrayList<LogicalVariable>();
ArrayList<Mutable<ILogicalExpression>> exprs = new ArrayList<Mutable<ILogicalExpression>>();
List<Mutable<ILogicalExpression>> varRefsForLoading = new ArrayList<Mutable<ILogicalExpression>>();
- for (Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> partitioner : DatasetUtils
- .getPartitioningFunctions(adecl)) {
+ for (String partitioningKey : partitioningKeys) {
+ Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> partitioner = format
+ .partitioningEvaluatorFactory(itemType, partitioningKey);
AbstractFunctionCallExpression f = partitioner.second.cloneExpression();
f.substituteVar(METADATA_DUMMY_VAR, resVar);
exprs.add(new MutableObject<ILogicalExpression>(f));
@@ -321,8 +320,8 @@
returnedOp = new UnnestOperator(v, new MutableObject<ILogicalExpression>(makeUnnestExpression(eo.first)));
} else {
LogicalVariable pVar = context.newVar(fc.getPosVarExpr());
- returnedOp = new UnnestOperator(v, new MutableObject<ILogicalExpression>(makeUnnestExpression(eo.first)), pVar,
- BuiltinType.AINT32);
+ returnedOp = new UnnestOperator(v, new MutableObject<ILogicalExpression>(makeUnnestExpression(eo.first)),
+ pVar, BuiltinType.AINT32);
}
returnedOp.getInputs().add(eo.second);
@@ -339,8 +338,8 @@
case VARIABLE_EXPRESSION: {
v = context.newVar(lc.getVarExpr());
LogicalVariable prev = context.getVar(((VariableExpr) lc.getBindingExpr()).getVar().getId());
- returnedOp = new AssignOperator(v,
- new MutableObject<ILogicalExpression>(new VariableReferenceExpression(prev)));
+ returnedOp = new AssignOperator(v, new MutableObject<ILogicalExpression>(
+ new VariableReferenceExpression(prev)));
returnedOp.getInputs().add(tupSource);
break;
}
@@ -429,15 +428,16 @@
f = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.GET_ITEM));
f.getArguments().add(new MutableObject<ILogicalExpression>(p.first));
f.getArguments().add(
- new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(new AInt32(i)))));
+ new MutableObject<ILogicalExpression>(new ConstantExpression(
+ new AsterixConstantValue(new AInt32(i)))));
}
- AssignOperator a = new AssignOperator(v, new MutableObject<ILogicalExpression>(f));
+ AssignOperator a = new AssignOperator(v, new MutableObject<ILogicalExpression>(f));
a.getInputs().add(p.second);
return new Pair<ILogicalOperator, LogicalVariable>(a, v);
}
@Override
- public Pair<ILogicalOperator, LogicalVariable> visitCallExpr(CallExpr fcall, Mutable<ILogicalOperator> tupSource)
+ public Pair<ILogicalOperator, LogicalVariable> visitCallExpr(CallExpr fcall, Mutable<ILogicalOperator> tupSource)
throws AsterixException {
LogicalVariable v = context.newVar();
AsterixFunction fid = fcall.getIdent();
@@ -469,11 +469,11 @@
}
}
- FunctionIdentifier fi = new FunctionIdentifier(AlgebricksBuiltinFunctions.ALGEBRICKS_NS, fid.getFunctionName(), false);
+ FunctionIdentifier fi = new FunctionIdentifier(AlgebricksBuiltinFunctions.ALGEBRICKS_NS, fid.getFunctionName(),
+ false);
AsterixFunctionInfo afi = AsterixBuiltinFunctions.lookupFunction(fi);
FunctionIdentifier builtinAquafi = afi == null ? null : afi.getFunctionIdentifier();
-
if (builtinAquafi != null) {
fi = builtinAquafi;
} else {
@@ -503,7 +503,8 @@
}
@Override
- public Pair<ILogicalOperator, LogicalVariable> visitFunctionDecl(FunctionDecl fd, Mutable<ILogicalOperator> tupSource) {
+ public Pair<ILogicalOperator, LogicalVariable> visitFunctionDecl(FunctionDecl fd,
+ Mutable<ILogicalOperator> tupSource) {
// TODO Auto-generated method stub
throw new NotImplementedException();
}
@@ -547,9 +548,10 @@
AggregateFunctionCallExpression fListify = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
AsterixBuiltinFunctions.LISTIFY, flArgs);
AggregateOperator agg = new AggregateOperator(mkSingletonArrayList(aggVar),
- (List)mkSingletonArrayList(new MutableObject<ILogicalExpression>(fListify)));
+ (List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(fListify)));
agg.getInputs().add(
- new MutableObject<ILogicalOperator>(new NestedTupleSourceOperator(new MutableObject<ILogicalOperator>(gOp))));
+ new MutableObject<ILogicalOperator>(new NestedTupleSourceOperator(
+ new MutableObject<ILogicalOperator>(gOp))));
ILogicalPlan plan = new ALogicalPlanImpl(new MutableObject<ILogicalOperator>(agg));
gOp.getNestedPlans().add(plan);
// Hide the variable that was part of the "with", replacing it with
@@ -583,8 +585,8 @@
new MutableObject<ILogicalOperator>(sp)));
Pair<ILogicalOperator, LogicalVariable> pThen = ifexpr.getThenExpr().accept(this, nestedSource);
- SelectOperator sel1 = new SelectOperator(new MutableObject<ILogicalExpression>(
- new VariableReferenceExpression(varCond)));
+ SelectOperator sel1 = new SelectOperator(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
+ varCond)));
sel1.getInputs().add(new MutableObject<ILogicalOperator>(pThen.first));
Pair<ILogicalOperator, LogicalVariable> pElse = ifexpr.getElseExpr().accept(this, nestedSource);
@@ -604,9 +606,9 @@
LogicalVariable resV = context.newVar();
AbstractFunctionCallExpression concatNonNull = new ScalarFunctionCallExpression(
- FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.CONCAT_NON_NULL), new MutableObject<ILogicalExpression>(
- new VariableReferenceExpression(pThen.second)), new MutableObject<ILogicalExpression>(
- new VariableReferenceExpression(pElse.second)));
+ FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.CONCAT_NON_NULL),
+ new MutableObject<ILogicalExpression>(new VariableReferenceExpression(pThen.second)),
+ new MutableObject<ILogicalExpression>(new VariableReferenceExpression(pElse.second)));
AssignOperator a = new AssignOperator(resV, new MutableObject<ILogicalExpression>(concatNonNull));
a.getInputs().add(new MutableObject<ILogicalOperator>(sp));
@@ -625,8 +627,8 @@
}
@Override
- public Pair<ILogicalOperator, LogicalVariable> visitOperatorExpr(OperatorExpr op, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
+ public Pair<ILogicalOperator, LogicalVariable> visitOperatorExpr(OperatorExpr op,
+ Mutable<ILogicalOperator> tupSource) throws AsterixException {
ArrayList<OperatorType> ops = op.getOpList();
int nOps = ops.size();
@@ -683,7 +685,8 @@
}
}
} else { // don't forget the last expression...
- ((AbstractFunctionCallExpression) currExpr).getArguments().add(new MutableObject<ILogicalExpression>(e));
+ ((AbstractFunctionCallExpression) currExpr).getArguments()
+ .add(new MutableObject<ILogicalExpression>(e));
if (i == 1 && op.isBroadcastOperand(i)) {
BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
bcast.setObject(BroadcastSide.RIGHT);
@@ -712,8 +715,9 @@
Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(e, topOp);
OrderModifier m = modifIter.next();
OrderOperator.IOrder comp = (m == OrderModifier.ASC) ? OrderOperator.ASC_ORDER : OrderOperator.DESC_ORDER;
- ord.getOrderExpressions().add(
- new Pair<IOrder, Mutable<ILogicalExpression>>(comp, new MutableObject<ILogicalExpression>(p.first)));
+ ord.getOrderExpressions()
+ .add(new Pair<IOrder, Mutable<ILogicalExpression>>(comp, new MutableObject<ILogicalExpression>(
+ p.first)));
topOp = p.second;
}
ord.getInputs().add(topOp);
@@ -775,7 +779,7 @@
}
LogicalVariable qeVar = context.newVar();
AggregateOperator a = new AggregateOperator(mkSingletonArrayList(qeVar),
- (List)mkSingletonArrayList(new MutableObject<ILogicalExpression>(fAgg)));
+ (List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(fAgg)));
a.getInputs().add(new MutableObject<ILogicalOperator>(s));
return new Pair<ILogicalOperator, LogicalVariable>(a, qeVar);
}
@@ -848,8 +852,8 @@
// Should we ever get to this method?
LogicalVariable var = context.newVar();
LogicalVariable oldV = context.getVar(v.getVar().getId());
- AssignOperator a = new AssignOperator(var,
- new MutableObject<ILogicalExpression>(new VariableReferenceExpression(oldV)));
+ AssignOperator a = new AssignOperator(var, new MutableObject<ILogicalExpression>(
+ new VariableReferenceExpression(oldV)));
a.getInputs().add(tupSource);
return new Pair<ILogicalOperator, LogicalVariable>(a, var);
}
@@ -1122,7 +1126,7 @@
funAgg.getArguments().add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(var)));
LogicalVariable varListified = context.newVar();
AggregateOperator agg = new AggregateOperator(mkSingletonArrayList(varListified),
- (List)mkSingletonArrayList(new MutableObject<ILogicalExpression>(funAgg)));
+ (List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(funAgg)));
agg.getInputs().add(opRef);
ILogicalOperator res;
if (bProject) {
@@ -1238,7 +1242,6 @@
return null;
}
-
@Override
public Pair<ILogicalOperator, LogicalVariable> visitOrderedListTypeDefiniton(OrderedListTypeDefinition olte,
Mutable<ILogicalOperator> arg) throws AsterixException {
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/DmlTranslator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/DmlTranslator.java
index d0e9626..c8b1079 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/DmlTranslator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/DmlTranslator.java
@@ -35,7 +35,6 @@
import edu.uci.ics.asterix.metadata.MetadataException;
import edu.uci.ics.asterix.metadata.MetadataManager;
import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
import edu.uci.ics.asterix.metadata.entities.Dataset;
import edu.uci.ics.asterix.metadata.entities.FeedDatasetDetails;
@@ -461,11 +460,11 @@
clauseList.add(dieClause);
}
- AqlCompiledDatasetDecl aqlDataset = compiledDeclarations.findDataset(datasetName);
- if (aqlDataset == null) {
+ Dataset dataset = compiledDeclarations.findDataset(datasetName);
+ if (dataset == null) {
throw new AlgebricksException("Unknown dataset " + datasetName);
}
- String itemTypeName = aqlDataset.getItemTypeName();
+ String itemTypeName = dataset.getItemTypeName();
IAType itemType = compiledDeclarations.findType(itemTypeName);
ARecordType recType = (ARecordType) itemType;
String[] fieldNames = recType.getFieldNames();
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/DdlTranslator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/DdlTranslator.java
index 8c32522..dc3fb25 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/DdlTranslator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/DdlTranslator.java
@@ -308,7 +308,7 @@
List<Dataset> datasets = MetadataManager.INSTANCE.getDataverseDatasets(mdTxnCtx, dvName);
for (int j = 0; j < datasets.size(); j++) {
String datasetName = datasets.get(j).getDatasetName();
- DatasetType dsType = datasets.get(j).getType();
+ DatasetType dsType = datasets.get(j).getDatasetType();
if (dsType == DatasetType.INTERNAL || dsType == DatasetType.FEED) {
List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dvName,
datasetName);
@@ -341,7 +341,7 @@
if (!stmtDelete.getIfExists())
throw new AlgebricksException("There is no dataset with this name " + datasetName + ".");
} else {
- if (ds.getType() == DatasetType.INTERNAL || ds.getType() == DatasetType.FEED) {
+ if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED) {
List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx,
compiledDeclarations.getDataverseName(), datasetName);
for (int j = 0; j < indexes.size(); j++) {
@@ -362,7 +362,7 @@
datasetName);
if (ds == null)
throw new AlgebricksException("There is no dataset with this name " + datasetName + ".");
- if (ds.getType() == DatasetType.INTERNAL || ds.getType() == DatasetType.FEED) {
+ if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED) {
String indexName = stmtDelete.getIndexName().getValue();
Index idx = MetadataManager.INSTANCE.getIndex(mdTxnCtx,
compiledDeclarations.getDataverseName(), datasetName, indexName);
@@ -502,7 +502,7 @@
CompiledDatasetDropStatement cds = new CompiledDatasetDropStatement(datasetName);
Dataset ds = MetadataManager.INSTANCE
.getDataset(mdTxnCtx, compiledDeclarations.getDataverseName(), datasetName);
- if (ds.getType() == DatasetType.INTERNAL || ds.getType() == DatasetType.FEED) {
+ if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED) {
JobSpecification[] jobs = DatasetOperations.createDropDatasetJobSpec(cds, compiledDeclarations);
for (JobSpecification job : jobs)
runJob(hcc, job);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
index 5aa0f6e..b028e06 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
@@ -27,11 +27,11 @@
import edu.uci.ics.asterix.common.context.AsterixStorageManagerInterface;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.formats.base.IDataFormat;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledExternalDatasetDetails;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
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.IAType;
@@ -95,32 +95,38 @@
IIndexRegistryProvider<IIndex> indexRegistryProvider = AsterixIndexRegistryProvider.INSTANCE;
IStorageManagerInterface storageManager = AsterixStorageManagerInterface.INSTANCE;
- AqlCompiledDatasetDecl adecl = metadata.findDataset(datasetName);
- if (adecl == null) {
+ Dataset dataset = metadata.findDataset(datasetName);
+ if (dataset == null) {
throw new AlgebricksException("DROP DATASET: No metadata for dataset " + datasetName);
}
- if (adecl.getDatasetType() == DatasetType.EXTERNAL) {
+ if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
return new JobSpecification[0];
}
- List<AqlCompiledIndexDecl> secondaryIndexes = DatasetUtils.getSecondaryIndexes(adecl);
-
+ List<Index> datasetIndexes = metadata.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
+ int numSecondaryIndexes = 0;
+ for (Index index : datasetIndexes) {
+ if (index.isSecondaryIndex()) {
+ numSecondaryIndexes++;
+ }
+ }
JobSpecification[] specs;
-
- if (secondaryIndexes != null && !secondaryIndexes.isEmpty()) {
- int n = secondaryIndexes.size();
- specs = new JobSpecification[n + 1];
+ if (numSecondaryIndexes > 0) {
+ specs = new JobSpecification[numSecondaryIndexes + 1];
int i = 0;
// First, drop secondary indexes.
- for (AqlCompiledIndexDecl acid : secondaryIndexes) {
- specs[i] = new JobSpecification();
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> idxSplitsAndConstraint = metadata
- .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, acid.getIndexName());
- TreeIndexDropOperatorDescriptor secondaryBtreeDrop = new TreeIndexDropOperatorDescriptor(specs[i],
- storageManager, indexRegistryProvider, idxSplitsAndConstraint.first);
- AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(specs[i], secondaryBtreeDrop,
- idxSplitsAndConstraint.second);
- i++;
+ for (Index index : datasetIndexes) {
+ if (index.isSecondaryIndex()) {
+ specs[i] = new JobSpecification();
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> idxSplitsAndConstraint = metadata
+ .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName,
+ index.getIndexName());
+ TreeIndexDropOperatorDescriptor secondaryBtreeDrop = new TreeIndexDropOperatorDescriptor(specs[i],
+ storageManager, indexRegistryProvider, idxSplitsAndConstraint.first);
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(specs[i], secondaryBtreeDrop,
+ idxSplitsAndConstraint.second);
+ i++;
+ }
}
} else {
specs = new JobSpecification[1];
@@ -140,17 +146,17 @@
return specs;
}
- // TODO: Lots of common code in this file. Refactor everything after merging in asterix-fix-issue-9.
public static JobSpecification createDatasetJobSpec(String datasetName, AqlCompiledMetadataDeclarations metadata)
throws AsterixException, AlgebricksException {
- AqlCompiledDatasetDecl compiledDatasetDecl = metadata.findDataset(datasetName);
- if (compiledDatasetDecl == null) {
+ Dataset dataset = metadata.findDataset(datasetName);
+ if (dataset == null) {
throw new AsterixException("Could not find dataset " + datasetName);
}
+ ARecordType itemType = (ARecordType) metadata.findType(dataset.getItemTypeName());
JobSpecification spec = new JobSpecification();
- IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
- compiledDatasetDecl, metadata.getFormat().getBinaryComparatorFactoryProvider());
- ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(compiledDatasetDecl, metadata);
+ IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
+ itemType, metadata.getFormat().getBinaryComparatorFactoryProvider());
+ ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(dataset, itemType);
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, datasetName);
FileSplit[] fs = splitsAndConstraint.first.getFileSplits();
@@ -170,49 +176,47 @@
return spec;
}
+ @SuppressWarnings("rawtypes")
public static Job createLoadDatasetJobSpec(CompiledLoadFromFileStatement loadStmt,
AqlCompiledMetadataDeclarations metadata) throws AsterixException, AlgebricksException {
String datasetName = loadStmt.getDatasetName();
- AqlCompiledDatasetDecl compiledDatasetDecl = metadata.findDataset(datasetName);
- if (compiledDatasetDecl == null) {
+ Dataset dataset = metadata.findDataset(datasetName);
+ if (dataset == null) {
throw new AsterixException("Could not find dataset " + datasetName);
}
- if (compiledDatasetDecl.getDatasetType() != DatasetType.INTERNAL
- && compiledDatasetDecl.getDatasetType() != DatasetType.FEED) {
+ if (dataset.getDatasetType() != DatasetType.INTERNAL && dataset.getDatasetType() != DatasetType.FEED) {
throw new AsterixException("Cannot load data into dataset (" + datasetName + ")" + "of type "
- + compiledDatasetDecl.getDatasetType());
+ + dataset.getDatasetType());
}
JobSpecification spec = new JobSpecification();
- ARecordType itemType = (ARecordType) metadata.findType(compiledDatasetDecl.getItemTypeName());
+ ARecordType itemType = (ARecordType) metadata.findType(dataset.getItemTypeName());
IDataFormat format = metadata.getFormat();
ISerializerDeserializer payloadSerde = format.getSerdeProvider().getSerializerDeserializer(itemType);
- IBinaryHashFunctionFactory[] hashFactories = DatasetUtils.computeKeysBinaryHashFunFactories(
- compiledDatasetDecl, metadata.getFormat().getBinaryHashFunctionFactoryProvider());
- IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
- compiledDatasetDecl, metadata.getFormat().getBinaryComparatorFactoryProvider());
- ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(compiledDatasetDecl, metadata);
+ IBinaryHashFunctionFactory[] hashFactories = DatasetUtils.computeKeysBinaryHashFunFactories(dataset, itemType,
+ metadata.getFormat().getBinaryHashFunctionFactoryProvider());
+ IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
+ itemType, metadata.getFormat().getBinaryComparatorFactoryProvider());
+ ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(dataset, itemType);
- AqlCompiledExternalDatasetDetails externalDatasetDetails = new AqlCompiledExternalDatasetDetails(
- loadStmt.getAdapter(), loadStmt.getProperties());
+ ExternalDatasetDetails externalDatasetDetails = new ExternalDatasetDetails(loadStmt.getAdapter(),
+ loadStmt.getProperties());
Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = AqlMetadataProvider
.buildExternalDataScannerRuntime(spec, itemType, externalDatasetDetails, format);
IOperatorDescriptor scanner = p.first;
AlgebricksPartitionConstraint scannerPc = p.second;
- RecordDescriptor recDesc = computePayloadKeyRecordDescriptor(compiledDatasetDecl, payloadSerde,
+ RecordDescriptor recDesc = computePayloadKeyRecordDescriptor(dataset, itemType, payloadSerde,
metadata.getFormat());
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, scanner, scannerPc);
- AssignRuntimeFactory assign = makeAssignRuntimeFactory(compiledDatasetDecl);
+ AssignRuntimeFactory assign = makeAssignRuntimeFactory(dataset, itemType, metadata.getFormat());
AlgebricksMetaOperatorDescriptor asterixOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
new IPushRuntimeFactory[] { assign }, new RecordDescriptor[] { recDesc });
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, asterixOp, scannerPc);
- List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions = DatasetUtils
- .getPartitioningFunctions(compiledDatasetDecl);
- int numKeys = partitioningFunctions.size();
+ int numKeys = DatasetUtils.getPartitioningKeys(dataset).size();
int[] keys = new int[numKeys];
for (int i = 0; i < numKeys; i++) {
keys[i] = i + 1;
@@ -270,14 +274,14 @@
return fs.getNodeName() + ":" + fs.getLocalFile().toString();
}
- private static AssignRuntimeFactory makeAssignRuntimeFactory(AqlCompiledDatasetDecl compiledDatasetDecl) {
- List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions = DatasetUtils
- .getPartitioningFunctions(compiledDatasetDecl);
- int numKeys = partitioningFunctions.size();
+ private static AssignRuntimeFactory makeAssignRuntimeFactory(Dataset dataset, ARecordType itemType,
+ IDataFormat format) throws AlgebricksException {
+ List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+ int numKeys = partitioningKeys.size();
ICopyEvaluatorFactory[] evalFactories = new ICopyEvaluatorFactory[numKeys];
for (int i = 0; i < numKeys; i++) {
- Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType = partitioningFunctions
- .get(i);
+ Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType = format
+ .partitioningEvaluatorFactory(itemType, partitioningKeys.get(i));
evalFactories[i] = evalFactoryAndType.first;
}
int[] outColumns = new int[numKeys];
@@ -296,21 +300,18 @@
return new AssignRuntimeFactory(outColumns, sefs, projectionList);
}
- private static RecordDescriptor computePayloadKeyRecordDescriptor(AqlCompiledDatasetDecl compiledDatasetDecl,
+ @SuppressWarnings("rawtypes")
+ private static RecordDescriptor computePayloadKeyRecordDescriptor(Dataset dataset, ARecordType itemType,
ISerializerDeserializer payloadSerde, IDataFormat dataFormat) throws AlgebricksException {
- List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions = DatasetUtils
- .getPartitioningFunctions(compiledDatasetDecl);
- int numKeys = partitioningFunctions.size();
+ List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+ int numKeys = partitioningKeys.size();
ISerializerDeserializer[] recordFields = new ISerializerDeserializer[1 + numKeys];
recordFields[0] = payloadSerde;
for (int i = 0; i < numKeys; i++) {
- Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType = partitioningFunctions
- .get(i);
- IAType keyType = evalFactoryAndType.third;
+ IAType keyType = itemType.getFieldType(partitioningKeys.get(i));
ISerializerDeserializer keySerde = dataFormat.getSerdeProvider().getSerializerDeserializer(keyType);
recordFields[i + 1] = keySerde;
}
return new RecordDescriptor(recordFields);
}
-
}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/FeedOperations.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/FeedOperations.java
index 4d7cbe2..66a5d56 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/FeedOperations.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/FeedOperations.java
@@ -24,30 +24,18 @@
import edu.uci.ics.asterix.feed.comm.FeedMessage;
import edu.uci.ics.asterix.feed.comm.IFeedMessage;
import edu.uci.ics.asterix.feed.comm.IFeedMessage.MessageType;
-import edu.uci.ics.asterix.formats.base.IDataFormat;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledFeedDatasetDetails;
import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
-import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
-import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.FeedDatasetDetails;
import edu.uci.ics.asterix.translator.DmlTranslator.CompiledControlFeedStatement;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
-import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.LogicalExpressionJobGenToExpressionRuntimeProviderAdapter;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
-import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import edu.uci.ics.hyracks.algebricks.runtime.operators.std.AssignRuntimeFactory;
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.misc.NullSinkOperatorDescriptor;
public class FeedOperations {
@@ -77,12 +65,17 @@
LOGGER.info(" DATASETPATH: " + datasetPath);
- AqlCompiledDatasetDecl adecl = metadata.findDataset(datasetName);
- if (adecl == null) {
+ Dataset dataset;
+ try {
+ dataset = metadata.findDataset(datasetName);
+ } catch (AlgebricksException e) {
+ throw new AsterixException(e);
+ }
+ if (dataset == null) {
throw new AsterixException("FEED DATASET: No metadata for dataset " + datasetName);
}
- if (adecl.getDatasetType() != DatasetType.FEED) {
- throw new AsterixException("Operation not support for dataset type " + adecl.getDatasetType());
+ if (dataset.getDatasetType() != DatasetType.FEED) {
+ throw new AsterixException("Operation not support for dataset type " + dataset.getDatasetType());
}
JobSpecification spec = new JobSpecification();
@@ -107,8 +100,8 @@
try {
Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = AqlMetadataProvider.buildFeedMessengerRuntime(
- spec, metadata, (AqlCompiledFeedDatasetDetails) adecl.getAqlCompiledDatasetDetails(),
- metadata.getDataverseName(), datasetName, feedMessages);
+ spec, metadata, (FeedDatasetDetails) dataset.getDatasetDetails(), metadata.getDataverseName(),
+ datasetName, feedMessages);
feedMessenger = p.first;
messengerPc = p.second;
} catch (AlgebricksException e) {
@@ -126,54 +119,4 @@
return spec;
}
-
- private static AssignRuntimeFactory makeAssignRuntimeFactory(AqlCompiledDatasetDecl compiledDatasetDecl) {
- List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions = DatasetUtils
- .getPartitioningFunctions(compiledDatasetDecl);
- int numKeys = partitioningFunctions.size();
- ICopyEvaluatorFactory[] evalFactories = new ICopyEvaluatorFactory[numKeys];
-
- int index = 0;
- for (Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType : partitioningFunctions) {
- evalFactories[index++] = evalFactoryAndType.first;
- }
-
- int[] outColumns = new int[numKeys];
- int[] projectionList = new int[numKeys + 1];
- projectionList[0] = 0;
-
- for (int i = 0; i < numKeys; i++) {
- outColumns[i] = i + 1;
- projectionList[i + 1] = i + 1;
- }
- IScalarEvaluatorFactory[] sefs = new IScalarEvaluatorFactory[evalFactories.length];
- for (int i = 0; i < evalFactories.length; ++i) {
- sefs[i] = new LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.ScalarEvaluatorFactoryAdapter(
- evalFactories[i]);
- }
- return new AssignRuntimeFactory(outColumns, sefs, projectionList);
- }
-
- @SuppressWarnings("unchecked")
- private static RecordDescriptor computePayloadKeyRecordDescriptor(AqlCompiledDatasetDecl compiledDatasetDecl,
- ISerializerDeserializer payloadSerde, IDataFormat dataFormat) throws AlgebricksException {
-
- List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions = DatasetUtils
- .getPartitioningFunctions(compiledDatasetDecl);
- int numKeys = partitioningFunctions.size();
- ISerializerDeserializer[] recordFields = new ISerializerDeserializer[1 + numKeys];
- recordFields[0] = payloadSerde;
- int index = 0;
- for (Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType : partitioningFunctions) {
- IAType keyType = evalFactoryAndType.third;
- ISerializerDeserializer keySerde = dataFormat.getSerdeProvider().getSerializerDeserializer(keyType);
- recordFields[index + 1] = keySerde;
- index++;
- }
- return new RecordDescriptor(recordFields);
- }
-
- private static String stringOf(FileSplit fs) {
- return fs.getNodeName() + ":" + fs.getLocalFile().toString();
- }
}
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 c1be9f9..65f6f5e 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
@@ -12,9 +12,9 @@
import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
import edu.uci.ics.asterix.metadata.MetadataException;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.Index;
import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.IAType;
@@ -26,9 +26,7 @@
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
-import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.LogicalExpressionJobGenToExpressionRuntimeProviderAdapter;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
import edu.uci.ics.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
import edu.uci.ics.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
import edu.uci.ics.hyracks.algebricks.data.ISerializerDeserializerProvider;
@@ -68,7 +66,7 @@
protected int numSecondaryKeys;
protected AqlCompiledMetadataDeclarations metadata;
protected String datasetName;
- protected AqlCompiledDatasetDecl datasetDecl;
+ protected Dataset dataset;
protected ARecordType itemType;
protected ISerializerDeserializer payloadSerde;
protected IFileSplitProvider primaryFileSplitProvider;
@@ -124,16 +122,16 @@
this.metadata = metadata;
datasetName = createIndexStmt.getDatasetName();
secondaryIndexName = createIndexStmt.getIndexName();
- datasetDecl = metadata.findDataset(datasetName);
- if (datasetDecl == null) {
+ dataset = metadata.findDataset(datasetName);
+ if (dataset == null) {
throw new AsterixException("Unknown dataset " + datasetName);
}
- if (datasetDecl.getDatasetType() == DatasetType.EXTERNAL) {
+ if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
throw new AsterixException("Cannot index an external dataset (" + datasetName + ").");
}
- itemType = (ARecordType) metadata.findType(datasetDecl.getItemTypeName());
+ itemType = (ARecordType) metadata.findType(dataset.getItemTypeName());
payloadSerde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType);
- numPrimaryKeys = DatasetUtils.getPartitioningFunctions(datasetDecl).size();
+ numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
numSecondaryKeys = createIndexStmt.getKeyFields().size();
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> primarySplitsAndConstraint = metadata
.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, datasetName);
@@ -149,22 +147,18 @@
}
protected void setPrimaryRecDescAndComparators() throws AlgebricksException {
- int numPrimaryKeys = DatasetUtils.getPartitioningFunctions(datasetDecl).size();
+ 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];
ISerializerDeserializerProvider serdeProvider = metadata.getFormat().getSerdeProvider();
- List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions = DatasetUtils
- .getPartitioningFunctions(datasetDecl);
- int i = 0;
- for (Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType : partitioningFunctions) {
- IAType keyType = evalFactoryAndType.third;
- ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(keyType);
- primaryRecFields[i] = keySerde;
+ for (int i = 0; i < numPrimaryKeys; i++) {
+ IAType keyType = itemType.getFieldType(partitioningKeys.get(i));
+ primaryRecFields[i] = serdeProvider.getSerializerDeserializer(keyType);
primaryComparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(
keyType, true);
primaryTypeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
- ++i;
}
primaryRecFields[numPrimaryKeys] = payloadSerde;
primaryTypeTraits[numPrimaryKeys] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(itemType);
@@ -185,8 +179,7 @@
for (int i = 0; i < numSecondaryKeys; i++) {
secondaryFieldAccessEvalFactories[i] = metadata.getFormat().getFieldAccessEvaluatorFactory(itemType,
secondaryKeyFields.get(i), numPrimaryKeys);
- Pair<IAType, Boolean> keyTypePair = AqlCompiledIndexDecl.getNonNullableKeyFieldType(
- secondaryKeyFields.get(i), itemType);
+ Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(secondaryKeyFields.get(i), itemType);
IAType keyType = keyTypePair.first;
anySecondaryKeyIsNullable = anySecondaryKeyIsNullable || keyTypePair.second;
ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(keyType);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java
index 224401f..d5e8222 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java
@@ -5,7 +5,7 @@
import edu.uci.ics.asterix.common.context.AsterixIndexRegistryProvider;
import edu.uci.ics.asterix.common.context.AsterixStorageManagerInterface;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
+import edu.uci.ics.asterix.metadata.entities.Index;
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.asterix.optimizer.rules.am.InvertedIndexAccessMethod;
import edu.uci.ics.asterix.translator.DmlTranslator.CompiledCreateIndexStatement;
@@ -73,8 +73,7 @@
for (int i = 0; i < numSecondaryKeys; i++) {
secondaryFieldAccessEvalFactories[i] = metadata.getFormat().getFieldAccessEvaluatorFactory(itemType,
secondaryKeyFields.get(i), numPrimaryKeys);
- Pair<IAType, Boolean> keyTypePair = AqlCompiledIndexDecl.getNonNullableKeyFieldType(
- secondaryKeyFields.get(i), itemType);
+ Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(secondaryKeyFields.get(i), itemType);
secondaryKeyType = keyTypePair.first;
anySecondaryKeyIsNullable = anySecondaryKeyIsNullable || keyTypePair.second;
ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(secondaryKeyType);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java
index 6399ae3..3127573 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java
@@ -9,7 +9,7 @@
import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
+import edu.uci.ics.asterix.metadata.entities.Index;
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
import edu.uci.ics.asterix.translator.DmlTranslator.CompiledCreateIndexStatement;
@@ -69,8 +69,7 @@
+ numSecondaryKeys
+ " fields as a key for the R-tree index. There can be only one field as a key for the R-tree index.");
}
- Pair<IAType, Boolean> spatialTypePair = AqlCompiledIndexDecl.getNonNullableKeyFieldType(
- secondaryKeyFields.get(0), itemType);
+ Pair<IAType, Boolean> spatialTypePair = Index.getNonNullableKeyFieldType(secondaryKeyFields.get(0), itemType);
IAType spatialType = spatialTypePair.first;
anySecondaryKeyIsNullable = spatialTypePair.second;
if (spatialType == null) {
diff --git a/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan b/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan
index ede1063..03fd04e 100644
--- a/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan
@@ -15,9 +15,8 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
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 6ac37c3..591154a 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
@@ -91,7 +91,7 @@
private MetadataNode() {
super();
}
-
+
public void initialize(AsterixAppRuntimeContext runtimeContext) {
this.transactionProvider = runtimeContext.getTransactionProvider();
this.indexRegistry = runtimeContext.getIndexRegistry();
@@ -152,7 +152,7 @@
DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(true);
ITupleReference datasetTuple = tupleReaderWriter.getTupleFromMetadataEntity(dataset);
insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
- if (dataset.getType() == DatasetType.INTERNAL || dataset.getType() == DatasetType.FEED) {
+ 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(),
@@ -163,7 +163,7 @@
insertTupleIntoIndex(txnId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, nodeGroupTuple);
}
// Add entry in datatype secondary index.
- ITupleReference dataTypeTuple = createTuple(dataset.getDataverseName(), dataset.getDatatypeName(),
+ ITupleReference dataTypeTuple = createTuple(dataset.getDataverseName(), dataset.getItemTypeName(),
dataset.getDatasetName());
insertTupleIntoIndex(txnId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeTuple);
} catch (BTreeDuplicateKeyException e) {
@@ -320,7 +320,7 @@
ITupleReference datasetTuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey);
deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
// Delete entry from secondary index 'group'.
- if (dataset.getType() == DatasetType.INTERNAL || dataset.getType() == DatasetType.FEED) {
+ 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
@@ -330,14 +330,14 @@
deleteTupleFromIndex(txnId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, groupNameTuple);
}
// Delete entry from secondary index 'type'.
- ITupleReference dataTypeSearchKey = createTuple(dataverseName, dataset.getDatatypeName(), datasetName);
+ 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.
ITupleReference dataTypeTuple = getTupleToBeDeleted(txnId,
MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeSearchKey);
deleteTupleFromIndex(txnId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeTuple);
// Delete entry(s) from the 'indexes' dataset.
- if (dataset.getType() == DatasetType.INTERNAL || dataset.getType() == DatasetType.FEED) {
+ if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
List<Index> datasetIndexes = getDatasetIndexes(txnId, dataverseName, datasetName);
for (Index index : datasetIndexes) {
dropIndex(txnId, dataverseName, datasetName, index.getIndexName());
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledDatasetDecl.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledDatasetDecl.java
deleted file mode 100644
index 1d806b1..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledDatasetDecl.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.asterix.metadata.declared;
-
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-
-public class AqlCompiledDatasetDecl {
-
- private String name;
- private String itemTypeName;
- private DatasetType datasetType;
- private IAqlCompiledDatasetDetails acdd;
-
- public AqlCompiledDatasetDecl(String name, String itemTypeName, DatasetType datasetType,
- IAqlCompiledDatasetDetails acdd) {
- this.name = name;
- this.itemTypeName = itemTypeName;
- this.datasetType = datasetType;
- this.acdd = acdd;
- }
-
- public String getName() {
- return name;
- }
-
- public String getItemTypeName() {
- return itemTypeName;
- }
-
- public DatasetType getDatasetType() {
- return datasetType;
- }
-
- public IAqlCompiledDatasetDetails getAqlCompiledDatasetDetails() {
- return acdd;
- }
-
-}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledExternalDatasetDetails.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledExternalDatasetDetails.java
deleted file mode 100644
index 2e5eb3a..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledExternalDatasetDetails.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Copyright 2009-2011 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.metadata.declared;
-
-import java.util.Map;
-
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-
-public class AqlCompiledExternalDatasetDetails implements IAqlCompiledDatasetDetails {
-
- private final String adapter;
- private final Map<String, String> properties;
-
- public AqlCompiledExternalDatasetDetails(String adapter, Map<String, String> properties) {
- this.adapter = adapter;
- this.properties = properties;
- }
-
- public String getAdapter() {
- return adapter;
- }
-
- public Map<String, String> getProperties() {
- return properties;
- }
-
- @Override
- public DatasetType getDatasetType() {
- return DatasetType.EXTERNAL;
- }
-}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledFeedDatasetDetails.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledFeedDatasetDetails.java
deleted file mode 100644
index 5cfc8be..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledFeedDatasetDetails.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Copyright 2009-2011 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.metadata.declared;
-
-import java.util.List;
-import java.util.Map;
-
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
-
-public class AqlCompiledFeedDatasetDetails extends
- AqlCompiledInternalDatasetDetails {
- private final String adapter;
- private final Map<String, String> properties;
- private final String functionIdentifier;
- private final String feedState;
-
- public AqlCompiledFeedDatasetDetails(
- List<String> partitioningExprs,
- List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitionFuns,
- String nodegroupName, AqlCompiledIndexDecl primaryIndex,
- List<AqlCompiledIndexDecl> secondaryIndexes, String adapter,
- Map<String, String> properties, String functionIdentifier,
- String feedState) {
- super(partitioningExprs, partitionFuns, nodegroupName, primaryIndex,
- secondaryIndexes);
- this.adapter = adapter;
- this.properties = properties;
- this.functionIdentifier = functionIdentifier;
- this.feedState = feedState;
- }
-
- public String getAdapter() {
- return adapter;
- }
-
- public Map<String, String> getProperties() {
- return properties;
- }
-
- public String getFunctionIdentifier() {
- return functionIdentifier;
- }
-
- public DatasetType getDatasetType() {
- return DatasetType.FEED;
- }
-
- public String getFeedState() {
- return feedState;
- }
-
-}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledIndexDecl.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledIndexDecl.java
deleted file mode 100644
index 5755331..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledIndexDecl.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.asterix.metadata.declared;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.asterix.om.types.AUnionType;
-import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
-
-public class AqlCompiledIndexDecl {
-
- public enum IndexKind {
- BTREE,
- RTREE,
- WORD_INVIX,
- NGRAM_INVIX
- }
-
- private String indexName;
- private IndexKind kind;
- private List<String> fieldExprs = new ArrayList<String>();
- // Only for NGRAM indexes.
- private int gramLength;
-
- public AqlCompiledIndexDecl(String indexName, IndexKind kind, List<String> fieldExprs, int gramLength) {
- this.indexName = indexName;
- this.kind = kind;
- this.fieldExprs = fieldExprs;
- this.gramLength = gramLength;
- }
-
- public AqlCompiledIndexDecl(String indexName, IndexKind kind, List<String> fieldExprs) {
- this.indexName = indexName;
- this.kind = kind;
- this.fieldExprs = fieldExprs;
- this.gramLength = -1;
- }
-
- @Override
- public String toString() {
- return "INDEX " + indexName + " (" + kind + ") " + fieldExprs;
- }
-
- public IndexKind getKind() {
- return kind;
- }
-
- public String getIndexName() {
- return indexName;
- }
-
- public List<String> getFieldExprs() {
- return fieldExprs;
- }
-
- public int getGramLength() {
- return gramLength;
- }
-
- public static Pair<IAType, Boolean> getNonNullableKeyFieldType(String expr, ARecordType recType)
- throws AlgebricksException {
- IAType keyType = AqlCompiledIndexDecl.keyFieldType(expr, recType);
- boolean nullable = false;
- if (keyType.getTypeTag() == ATypeTag.UNION) {
- AUnionType unionType = (AUnionType) keyType;
- if (unionType.isNullableType()) {
- // The non-null type is always at index 1.
- keyType = unionType.getUnionList().get(1);
- nullable = true;
- }
- }
- return new Pair<IAType, Boolean>(keyType, nullable);
- }
-
- private static IAType keyFieldType(String expr, ARecordType recType) throws AlgebricksException {
- String[] names = recType.getFieldNames();
- int n = names.length;
- for (int i = 0; i < n; i++) {
- if (names[i].equals(expr)) {
- return recType.getFieldTypes()[i];
- }
- }
- throw new AlgebricksException("Could not find field " + expr + " in the schema.");
- }
-}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledInternalDatasetDetails.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledInternalDatasetDetails.java
deleted file mode 100644
index 912b766..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledInternalDatasetDetails.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/*
- * Copyright 2009-2011 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.metadata.declared;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
-
-public class AqlCompiledInternalDatasetDetails implements IAqlCompiledDatasetDetails {
- private final List<String> partitioningExprs;
- private final List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitionFuns;
- private final String nodegroupName;
- private final List<AqlCompiledIndexDecl> secondaryIndexes;
- private final AqlCompiledIndexDecl primaryIndex;
- private HashMap<String, List<AqlCompiledIndexDecl>> secondaryIndexInvertedList;
-
- public AqlCompiledInternalDatasetDetails(List<String> partitioningExprs,
- List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitionFuns, String nodegroupName,
- AqlCompiledIndexDecl primaryIndex, List<AqlCompiledIndexDecl> secondaryIndexes) {
- this.partitioningExprs = partitioningExprs;
- this.partitionFuns = partitionFuns;
- this.nodegroupName = nodegroupName;
- this.primaryIndex = primaryIndex;
- this.secondaryIndexes = secondaryIndexes;
- invertSecondaryIndexExprs();
- }
-
- public String toString() {
- StringBuilder sb = new StringBuilder();
- sb.append("dataset partitioned-by " + partitionFuns + " on " + nodegroupName);
- if (secondaryIndexes != null && !secondaryIndexes.isEmpty()) {
- sb.append(System.getProperty("line.separator") + " with indexes: " + secondaryIndexes);
- }
- return sb.toString();
- }
-
- public List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> getPartitioningFunctions() {
- return partitionFuns;
- }
-
- public List<String> getPartitioningExprs() {
- return partitioningExprs;
- }
-
- public int getPositionOfPartitioningKeyField(String fieldName) {
- int pos = 0;
- for (String pe : partitioningExprs) {
- if (pe.equals(fieldName)) {
- return pos;
- }
- ++pos;
- }
- return -1;
- }
-
- public String getNodegroupName() {
- return nodegroupName;
- }
-
- public List<AqlCompiledIndexDecl> getSecondaryIndexes() {
- return secondaryIndexes;
- }
-
- public AqlCompiledIndexDecl getPrimaryIndex() {
- return primaryIndex;
- }
-
- public List<AqlCompiledIndexDecl> findSecondaryIndexesByOneOfTheKeys(String fieldExpr) {
- return secondaryIndexInvertedList.get(fieldExpr);
- }
-
- public AqlCompiledIndexDecl findSecondaryIndexByExactKeyList(List<String> fieldExprs) {
- if (secondaryIndexes == null) {
- return null;
- }
- for (AqlCompiledIndexDecl acid : secondaryIndexes) {
- if (acid.getFieldExprs().equals(fieldExprs)) {
- return acid;
- }
- }
- return null;
- }
-
- public AqlCompiledIndexDecl findSecondaryIndexByName(String idxName) {
- if (secondaryIndexes == null) {
- return null;
- }
- for (AqlCompiledIndexDecl acid : secondaryIndexes) {
- if (acid.getIndexName().equals(idxName)) {
- return acid;
- }
- }
- return null;
- }
-
- private void invertSecondaryIndexExprs() {
- secondaryIndexInvertedList = new HashMap<String, List<AqlCompiledIndexDecl>>();
- if (secondaryIndexes == null) {
- return;
- }
- for (AqlCompiledIndexDecl idx : secondaryIndexes) {
- for (String s : idx.getFieldExprs()) {
- List<AqlCompiledIndexDecl> idxList = secondaryIndexInvertedList.get(s);
- if (idxList == null) {
- idxList = new ArrayList<AqlCompiledIndexDecl>();
- secondaryIndexInvertedList.put(s, idxList);
- }
- idxList.add(idx);
- }
- }
- }
-
- @Override
- public DatasetType getDatasetType() {
- return DatasetType.INTERNAL;
- }
-}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java
index 756fab4..ba7c797 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java
@@ -23,7 +23,6 @@
import edu.uci.ics.asterix.common.annotations.TypeDataGen;
import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.formats.base.IDataFormat;
import edu.uci.ics.asterix.metadata.MetadataException;
@@ -31,25 +30,18 @@
import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
import edu.uci.ics.asterix.metadata.api.IMetadataManager;
import edu.uci.ics.asterix.metadata.bootstrap.AsterixProperties;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl.IndexKind;
import edu.uci.ics.asterix.metadata.entities.Dataset;
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.FeedDatasetDetails;
import edu.uci.ics.asterix.metadata.entities.Index;
import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
import edu.uci.ics.asterix.metadata.entities.NodeGroup;
-import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
-import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
import edu.uci.ics.hyracks.algebricks.data.IAWriterFactory;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
@@ -81,10 +73,11 @@
this.dataverseName = dataverseName;
this.outputFile = outputFile;
this.config = config;
- if (stores == null && online)
+ if (stores == null && online) {
this.stores = AsterixProperties.INSTANCE.getStores();
- else
+ } else {
this.stores = stores;
+ }
this.types = types;
this.typeDataGenMap = typeDataGenMap;
this.writerFactory = writerFactory;
@@ -113,17 +106,16 @@
}
public void disconnectFromDataverse() throws AlgebricksException {
- if (!isConnected)
+ if (!isConnected) {
throw new AlgebricksException("You are not connected to any dataverse");
- else {
- dataverseName = null;
- format = null;
- isConnected = false;
}
+ dataverseName = null;
+ format = null;
+ isConnected = false;
}
public boolean isConnectedToDataverse() {
- return this.isConnected;
+ return isConnected;
}
public String getDataverseName() {
@@ -135,8 +127,9 @@
}
public IDataFormat getFormat() throws AlgebricksException {
- if (!isConnected)
+ if (!isConnected) {
throw new AlgebricksException("You need first to connect to a dataverse.");
+ }
return format;
}
@@ -157,8 +150,13 @@
return type.getDatatype();
}
- public List<String> findNodeGroupNodeNames(String nodeGroupName) throws AlgebricksException, MetadataException {
- NodeGroup ng = metadataManager.getNodegroup(mdTxnCtx, nodeGroupName);
+ public List<String> findNodeGroupNodeNames(String nodeGroupName) throws AlgebricksException {
+ NodeGroup ng;
+ try {
+ ng = metadataManager.getNodegroup(mdTxnCtx, nodeGroupName);
+ } catch (MetadataException e) {
+ throw new AlgebricksException(e);
+ }
if (ng == null) {
throw new AlgebricksException("No node group with this name " + nodeGroupName);
}
@@ -173,103 +171,44 @@
return stores;
}
- public AqlCompiledDatasetDecl findDataset(String datasetName) {
+ public Dataset findDataset(String datasetName) throws AlgebricksException {
try {
- Dataset datasetRecord = this.metadataManager.getDataset(mdTxnCtx, dataverseName, datasetName);
- if (datasetRecord == null) {
- return null;
- }
-
- IAqlCompiledDatasetDetails acdd = null;
- switch (datasetRecord.getType()) {
- case FEED:
- case INTERNAL: {
- String typeName = datasetRecord.getDatatypeName();
- InternalDatasetDetails id = (InternalDatasetDetails) datasetRecord.getDatasetDetails();
- ARecordType recType = (ARecordType) findType(typeName);
- List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningEvalFactories = computePartitioningEvaluatorFactories(
- id.getPartitioningKey(), recType);
- List<Index> indexRecord = this.metadataManager.getDatasetIndexes(mdTxnCtx, dataverseName,
- datasetName);
- AqlCompiledIndexDecl primaryIndex = null;
- List<AqlCompiledIndexDecl> secondaryIndexes = new ArrayList<AqlCompiledIndexDecl>();
- for (int i = 0; i < indexRecord.size(); i++) {
- Index rec = indexRecord.get(i);
- if (rec.isPrimaryIndex()) {
- primaryIndex = new AqlCompiledIndexDecl(rec.getIndexName(), IndexKind.BTREE,
- rec.getKeyFieldNames());
- } else {
- secondaryIndexes.add(new AqlCompiledIndexDecl(rec.getIndexName(), getIndexKindFromType(rec
- .getIndexType()), rec.getKeyFieldNames(), rec.getGramLength()));
- }
- }
-
- if (datasetRecord.getType() == DatasetType.INTERNAL) {
- acdd = new AqlCompiledInternalDatasetDetails(id.getPartitioningKey(),
- partitioningEvalFactories, id.getNodeGroupName(), primaryIndex, secondaryIndexes);
- } else {
- acdd = new AqlCompiledFeedDatasetDetails(id.getPartitioningKey(), partitioningEvalFactories,
- id.getNodeGroupName(), primaryIndex, secondaryIndexes,
- ((FeedDatasetDetails) id).getAdapter(), ((FeedDatasetDetails) id).getProperties(),
- ((FeedDatasetDetails) id).getFunctionIdentifier(), ((FeedDatasetDetails) id)
- .getFeedState().toString());
- }
- break;
- }
-
- case EXTERNAL: {
- acdd = new AqlCompiledExternalDatasetDetails(
- ((ExternalDatasetDetails) datasetRecord.getDatasetDetails()).getAdapter(),
- ((ExternalDatasetDetails) datasetRecord.getDatasetDetails()).getProperties());
- break;
- }
-
- }
- AqlCompiledDatasetDecl dataset = new AqlCompiledDatasetDecl(datasetRecord.getDatasetName(),
- datasetRecord.getDatatypeName(), datasetRecord.getType(), acdd);
- return dataset;
-
- } catch (Exception e) {
- throw new IllegalStateException(e);
+ return metadataManager.getDataset(mdTxnCtx, dataverseName, datasetName);
+ } catch (MetadataException e) {
+ throw new AlgebricksException(e);
}
}
- private IndexKind getIndexKindFromType(IndexType type) {
- switch (type) {
- case BTREE:
- return IndexKind.BTREE;
- case RTREE:
- return IndexKind.RTREE;
- case WORD_INVIX:
- return IndexKind.WORD_INVIX;
- case NGRAM_INVIX:
- return IndexKind.NGRAM_INVIX;
+ public List<Index> getDatasetIndexes(String dataverseName, String datasetName) throws AlgebricksException {
+ try {
+ return metadataManager.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
+ } catch (MetadataException e) {
+ throw new AlgebricksException(e);
}
- return null;
+ }
+
+ public Index getDatasetPrimaryIndex(String dataverseName, String datasetName) throws AlgebricksException {
+ try {
+ return metadataManager.getIndex(mdTxnCtx, dataverseName, datasetName, datasetName);
+ } catch (MetadataException e) {
+ throw new AlgebricksException(e);
+ }
}
+ public Index getIndex(String dataverseName, String datasetName, String indexName) throws AlgebricksException {
+ try {
+ return metadataManager.getIndex(mdTxnCtx, dataverseName, datasetName, indexName);
+ } catch (MetadataException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
public void setOutputFile(FileSplit outputFile) {
this.outputFile = outputFile;
}
- public List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> computePartitioningEvaluatorFactories(
- List<String> partitioningExprs, ARecordType recType) {
- List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> evalFactories = new ArrayList<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>>(
- partitioningExprs.size());
- for (String expr : partitioningExprs) {
- Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFact = null;
- try {
- evalFact = format.partitioningEvaluatorFactory(recType, expr);
- } catch (AlgebricksException e) {
- throw new IllegalStateException(e);
- }
- evalFactories.add(evalFact);
- }
- return evalFactories;
- }
-
public Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
- String datasetName, String targetIdxName) throws AlgebricksException, MetadataException {
+ String datasetName, String targetIdxName) throws AlgebricksException {
FileSplit[] splits = splitsForInternalOrFeedDataset(datasetName, targetIdxName);
IFileSplitProvider splitProvider = new ConstantFileSplitProvider(splits);
String[] loc = new String[splits.length];
@@ -296,23 +235,20 @@
}
private FileSplit[] splitsForInternalOrFeedDataset(String datasetName, String targetIdxName)
- throws AlgebricksException, MetadataException {
+ throws AlgebricksException {
File relPathFile = new File(getRelativePath(datasetName + "_idx_" + targetIdxName));
- AqlCompiledDatasetDecl adecl = findDataset(datasetName);
- if (adecl.getDatasetType() != DatasetType.INTERNAL & adecl.getDatasetType() != DatasetType.FEED) {
+ Dataset dataset = findDataset(datasetName);
+ if (dataset.getDatasetType() != DatasetType.INTERNAL & dataset.getDatasetType() != DatasetType.FEED) {
throw new AlgebricksException("Not an internal or feed dataset");
}
- AqlCompiledInternalDatasetDetails compiledDatasetDetails = (AqlCompiledInternalDatasetDetails) adecl
- .getAqlCompiledDatasetDetails();
- List<String> nodeGroup = findNodeGroupNodeNames(compiledDatasetDetails.getNodegroupName());
-
+ InternalDatasetDetails datasetDetails = (InternalDatasetDetails) dataset.getDatasetDetails();
+ List<String> nodeGroup = findNodeGroupNodeNames(datasetDetails.getNodeGroupName());
if (nodeGroup == null) {
- throw new AlgebricksException("Couldn't find node group " + compiledDatasetDetails.getNodegroupName());
+ 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) {
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java
index 4ae38eb..b7ba1a1 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java
@@ -21,12 +21,13 @@
import java.util.Set;
import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
+import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSource;
import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSourcePropertiesProvider;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
@@ -41,12 +42,11 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.properties.RandomPartitioningProperty;
import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
import edu.uci.ics.hyracks.algebricks.core.algebra.properties.UnorderedPartitionedProperty;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
public class AqlDataSource implements IDataSource<AqlSourceId> {
private AqlSourceId id;
- private AqlCompiledDatasetDecl adecl;
+ private Dataset dataset;
private IAType[] schemaTypes;
private INodeDomain domain;
private AqlDataSourceType datasourceType;
@@ -58,14 +58,14 @@
EXTERNAL_FEED
}
- public AqlDataSource(AqlSourceId id, AqlCompiledDatasetDecl adecl, IAType itemType, AqlDataSourceType datasourceType)
+ public AqlDataSource(AqlSourceId id, Dataset dataset, IAType itemType, AqlDataSourceType datasourceType)
throws AlgebricksException {
this.id = id;
- this.adecl = adecl;
+ this.dataset = dataset;
this.datasourceType = datasourceType;
switch (datasourceType) {
case FEED:
- initFeedDataset(itemType, adecl);
+ initFeedDataset(itemType, dataset);
case INTERNAL: {
initInternalDataset(itemType);
break;
@@ -81,12 +81,12 @@
}
}
- public AqlDataSource(AqlSourceId id, AqlCompiledDatasetDecl adecl, IAType itemType) throws AlgebricksException {
+ public AqlDataSource(AqlSourceId id, Dataset dataset, IAType itemType) throws AlgebricksException {
this.id = id;
- this.adecl = adecl;
- switch (adecl.getDatasetType()) {
+ this.dataset = dataset;
+ switch (dataset.getDatasetType()) {
case FEED:
- initFeedDataset(itemType, adecl);
+ initFeedDataset(itemType, dataset);
break;
case INTERNAL:
initInternalDataset(itemType);
@@ -101,32 +101,32 @@
}
}
+ // TODO: Seems like initFeedDataset() could simply call this method.
private void initInternalDataset(IAType itemType) {
- List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions = DatasetUtils
- .getPartitioningFunctions(adecl);
- int n = partitioningFunctions.size();
+ List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+ ARecordType recordType = (ARecordType) itemType;
+ int n = partitioningKeys.size();
schemaTypes = new IAType[n + 1];
for (int i = 0; i < n; i++) {
- schemaTypes[i] = partitioningFunctions.get(i).third;
+ schemaTypes[i] = recordType.getFieldType(partitioningKeys.get(i));
}
schemaTypes[n] = itemType;
- domain = new AsterixNodeGroupDomain(DatasetUtils.getNodegroupName(adecl));
+ domain = new AsterixNodeGroupDomain(DatasetUtils.getNodegroupName(dataset));
}
- private void initFeedDataset(IAType itemType, AqlCompiledDatasetDecl decl) {
-
- if (decl.getAqlCompiledDatasetDetails() instanceof AqlCompiledExternalDatasetDetails) {
+ private void initFeedDataset(IAType itemType, Dataset dataset) {
+ if (dataset.getDatasetDetails() instanceof ExternalDatasetDetails) {
initExternalDataset(itemType);
} else {
- List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions = DatasetUtils
- .getPartitioningFunctions(adecl);
- int n = partitioningFunctions.size();
+ List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+ int n = partitioningKeys.size();
schemaTypes = new IAType[n + 1];
+ ARecordType recordType = (ARecordType) itemType;
for (int i = 0; i < n; i++) {
- schemaTypes[i] = partitioningFunctions.get(i).third;
+ schemaTypes[i] = recordType.getFieldType(partitioningKeys.get(i));
}
schemaTypes[n] = itemType;
- domain = new AsterixNodeGroupDomain(DatasetUtils.getNodegroupName(adecl));
+ domain = new AsterixNodeGroupDomain(DatasetUtils.getNodegroupName(dataset));
}
}
@@ -152,8 +152,8 @@
return id;
}
- public AqlCompiledDatasetDecl getCompiledDatasetDecl() {
- return adecl;
+ public Dataset getDataset() {
+ return dataset;
}
@Override
@@ -170,7 +170,7 @@
@Override
public IDataSourcePropertiesProvider getPropertiesProvider() {
- return new AqlDataSourcePartitioningProvider(adecl.getDatasetType(), domain);
+ return new AqlDataSourcePartitioningProvider(dataset.getDatasetType(), domain);
}
@Override
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlIndex.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlIndex.java
index 4a6d6e4..bac7733 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlIndex.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlIndex.java
@@ -15,24 +15,26 @@
package edu.uci.ics.asterix.metadata.declared;
+import edu.uci.ics.asterix.metadata.entities.Index;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSource;
import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSourceIndex;
public class AqlIndex implements IDataSourceIndex<String, AqlSourceId> {
- private final AqlCompiledIndexDecl acid;
+ private final Index index;
private final AqlCompiledMetadataDeclarations acmd;
private final String datasetName;
// Every transactions needs to work with its own instance of an
// AqlMetadataProvider.
- public AqlIndex(AqlCompiledIndexDecl acid, AqlCompiledMetadataDeclarations acmd, String datasetName) {
- this.acid = acid;
+ public AqlIndex(Index index, AqlCompiledMetadataDeclarations acmd, String datasetName) {
+ this.index = index;
this.acmd = acmd;
this.datasetName = datasetName;
}
+ // TODO: Maybe Index can directly implement IDataSourceIndex<String, AqlSourceId>
@Override
public IDataSource<AqlSourceId> getDataSource() {
try {
@@ -45,7 +47,7 @@
@Override
public String getId() {
- return acid.getIndexName();
+ return index.getIndexName();
}
}
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 57b2e72..a3d3472 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
@@ -34,7 +34,10 @@
import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl.IndexKind;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
+import edu.uci.ics.asterix.metadata.entities.FeedDatasetDetails;
+import edu.uci.ics.asterix.metadata.entities.Index;
import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
import edu.uci.ics.asterix.om.types.ARecordType;
@@ -47,12 +50,10 @@
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.common.utils.Triple;
import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionRuntimeProvider;
import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSink;
@@ -63,7 +64,6 @@
import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
import edu.uci.ics.hyracks.algebricks.data.IPrinterFactory;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import edu.uci.ics.hyracks.algebricks.runtime.operators.std.SinkWriterRuntimeFactory;
@@ -121,23 +121,23 @@
List<LogicalVariable> projectVariables, boolean projectPushed, IOperatorSchema opSchema,
IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec)
throws AlgebricksException {
- AqlCompiledDatasetDecl adecl = metadata.findDataset(dataSource.getId().getDatasetName());
- if (adecl == null) {
+ Dataset dataset = metadata.findDataset(dataSource.getId().getDatasetName());
+ if (dataset == null) {
throw new AlgebricksException("Unknown dataset " + dataSource.getId().getDatasetName());
}
- switch (adecl.getDatasetType()) {
+ switch (dataset.getDatasetType()) {
case FEED:
if (dataSource instanceof ExternalFeedDataSource) {
- return buildExternalDatasetScan(jobSpec, adecl, dataSource);
+ return buildExternalDatasetScan(jobSpec, dataset, dataSource);
} else {
- return buildInternalDatasetScan(jobSpec, scanVariables, opSchema, typeEnv, adecl, dataSource,
+ return buildInternalDatasetScan(jobSpec, scanVariables, opSchema, typeEnv, dataset, dataSource,
context);
}
case INTERNAL: {
- return buildInternalDatasetScan(jobSpec, scanVariables, opSchema, typeEnv, adecl, dataSource, context);
+ return buildInternalDatasetScan(jobSpec, scanVariables, opSchema, typeEnv, dataset, dataSource, context);
}
case EXTERNAL: {
- return buildExternalDatasetScan(jobSpec, adecl, dataSource);
+ return buildExternalDatasetScan(jobSpec, dataset, dataSource);
}
default: {
throw new IllegalArgumentException();
@@ -147,40 +147,32 @@
private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildInternalDatasetScan(JobSpecification jobSpec,
List<LogicalVariable> outputVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
- AqlCompiledDatasetDecl acedl, IDataSource<AqlSourceId> dataSource, JobGenContext context)
- throws AlgebricksException {
+ Dataset dataset, IDataSource<AqlSourceId> dataSource, JobGenContext context) throws AlgebricksException {
AqlSourceId asid = dataSource.getId();
+ String dataverseName = asid.getDataverseName();
String datasetName = asid.getDatasetName();
- String indexName = DatasetUtils.getPrimaryIndex(acedl).getIndexName();
- return buildBtreeRuntime(jobSpec, outputVars, opSchema, typeEnv, metadata, context, false, datasetName, acedl,
- indexName, null, null, true, true);
+ Index primaryIndex = metadata.getDatasetPrimaryIndex(dataverseName, datasetName);
+ return buildBtreeRuntime(jobSpec, outputVars, opSchema, typeEnv, metadata, context, false, datasetName,
+ dataset, primaryIndex.getIndexName(), null, null, true, true);
}
private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildExternalDatasetScan(JobSpecification jobSpec,
- AqlCompiledDatasetDecl acedl, IDataSource<AqlSourceId> dataSource) throws AlgebricksException {
- String itemTypeName = acedl.getItemTypeName();
- IAType itemType;
- try {
- itemType = metadata.findType(itemTypeName);
- } catch (Exception e) {
- throw new AlgebricksException(e);
- }
-
+ Dataset dataset, IDataSource<AqlSourceId> dataSource) throws AlgebricksException {
+ String itemTypeName = dataset.getItemTypeName();
+ IAType itemType = metadata.findType(itemTypeName);
if (dataSource instanceof ExternalFeedDataSource) {
- AqlCompiledFeedDatasetDetails acfdd = (AqlCompiledFeedDatasetDetails) ((ExternalFeedDataSource) dataSource)
- .getCompiledDatasetDecl().getAqlCompiledDatasetDetails();
-
- return buildFeedIntakeRuntime(jobSpec, metadata.getDataverseName(), acedl.getName(), itemType, acfdd,
- metadata.getFormat());
+ FeedDatasetDetails datasetDetails = (FeedDatasetDetails) dataset.getDatasetDetails();
+ return buildFeedIntakeRuntime(jobSpec, metadata.getDataverseName(), dataset.getDatasetName(), itemType,
+ datasetDetails, metadata.getFormat());
} else {
return buildExternalDataScannerRuntime(jobSpec, itemType,
- (AqlCompiledExternalDatasetDetails) acedl.getAqlCompiledDatasetDetails(), metadata.getFormat());
+ (ExternalDatasetDetails) dataset.getDatasetDetails(), metadata.getFormat());
}
}
@SuppressWarnings("rawtypes")
public static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildExternalDataScannerRuntime(
- JobSpecification jobSpec, IAType itemType, AqlCompiledExternalDatasetDetails decl, IDataFormat format)
+ JobSpecification jobSpec, IAType itemType, ExternalDatasetDetails datasetDetails, IDataFormat format)
throws AlgebricksException {
if (itemType.getTypeTag() != ATypeTag.RECORD) {
throw new AlgebricksException("Can only scan datasets of records.");
@@ -188,7 +180,7 @@
IDatasourceReadAdapter adapter;
try {
- adapter = (IDatasourceReadAdapter) Class.forName(decl.getAdapter()).newInstance();
+ adapter = (IDatasourceReadAdapter) Class.forName(datasetDetails.getAdapter()).newInstance();
} catch (Exception e) {
e.printStackTrace();
throw new AlgebricksException("unable to load the adapter class " + e);
@@ -201,7 +193,7 @@
ARecordType rt = (ARecordType) itemType;
try {
- adapter.configure(decl.getProperties(), itemType);
+ adapter.configure(datasetDetails.getProperties(), itemType);
} catch (Exception e) {
e.printStackTrace();
throw new AlgebricksException("unable to configure the datasource adapter " + e);
@@ -211,7 +203,7 @@
RecordDescriptor scannerDesc = new RecordDescriptor(new ISerializerDeserializer[] { payloadSerde });
ExternalDataScanOperatorDescriptor dataScanner = new ExternalDataScanOperatorDescriptor(jobSpec,
- decl.getAdapter(), decl.getProperties(), rt, scannerDesc);
+ datasetDetails.getAdapter(), datasetDetails.getProperties(), rt, scannerDesc);
dataScanner.setDatasourceAdapter(adapter);
AlgebricksPartitionConstraint constraint = adapter.getPartitionConstraint();
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(dataScanner, constraint);
@@ -243,13 +235,13 @@
@SuppressWarnings("rawtypes")
public static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildFeedIntakeRuntime(
JobSpecification jobSpec, String dataverse, String dataset, IAType itemType,
- AqlCompiledFeedDatasetDetails decl, IDataFormat format) throws AlgebricksException {
+ FeedDatasetDetails datasetDetails, IDataFormat format) throws AlgebricksException {
if (itemType.getTypeTag() != ATypeTag.RECORD) {
throw new AlgebricksException("Can only consume records.");
}
IDatasourceAdapter adapter;
try {
- adapter = (IDatasourceAdapter) Class.forName(decl.getAdapter()).newInstance();
+ adapter = (IDatasourceAdapter) Class.forName(datasetDetails.getAdapter()).newInstance();
} catch (Exception e) {
e.printStackTrace();
throw new AlgebricksException("unable to load the adapter class " + e);
@@ -257,7 +249,7 @@
ARecordType rt = (ARecordType) itemType;
try {
- adapter.configure(decl.getProperties(), itemType);
+ adapter.configure(datasetDetails.getProperties(), itemType);
} catch (Exception e) {
e.printStackTrace();
throw new AlgebricksException("unable to configure the datasource adapter " + e);
@@ -267,46 +259,39 @@
RecordDescriptor feedDesc = new RecordDescriptor(new ISerializerDeserializer[] { payloadSerde });
FeedIntakeOperatorDescriptor feedIngestor = new FeedIntakeOperatorDescriptor(jobSpec, new FeedId(dataverse,
- dataset), decl.getAdapter(), decl.getProperties(), rt, feedDesc);
+ dataset), datasetDetails.getAdapter(), datasetDetails.getProperties(), rt, feedDesc);
AlgebricksPartitionConstraint constraint = adapter.getPartitionConstraint();
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(feedIngestor, constraint);
}
public static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildFeedMessengerRuntime(
- JobSpecification jobSpec, AqlCompiledMetadataDeclarations metadata, AqlCompiledFeedDatasetDetails decl,
+ JobSpecification jobSpec, AqlCompiledMetadataDeclarations metadata, FeedDatasetDetails datasetDetails,
String dataverse, String dataset, List<IFeedMessage> feedMessages) throws AlgebricksException {
-
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc;
- try {
- spPc = metadata.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataset, dataset);
- } catch (Exception e) {
- throw new AlgebricksException(e);
- }
-
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc = metadata
+ .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataset, dataset);
FeedMessageOperatorDescriptor feedMessenger = new FeedMessageOperatorDescriptor(jobSpec, dataverse, dataset,
feedMessages);
-
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(feedMessenger, spPc.second);
}
public static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildBtreeRuntime(JobSpecification jobSpec,
List<LogicalVariable> outputVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
AqlCompiledMetadataDeclarations metadata, JobGenContext context, boolean retainInput, String datasetName,
- AqlCompiledDatasetDecl datasetDecl, String indexName, int[] lowKeyFields, int[] highKeyFields,
- boolean lowKeyInclusive, boolean highKeyInclusive) throws AlgebricksException {
+ Dataset dataset, String indexName, int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive,
+ boolean highKeyInclusive) throws AlgebricksException {
boolean isSecondary = true;
- AqlCompiledIndexDecl primaryIndexDecl = DatasetUtils.getPrimaryIndex(datasetDecl);
- if (primaryIndexDecl != null) {
- isSecondary = !indexName.equals(primaryIndexDecl.getIndexName());
+ Index primaryIndex = metadata.getDatasetPrimaryIndex(dataset.getDataverseName(), dataset.getDatasetName());
+ if (primaryIndex != null) {
+ isSecondary = !indexName.equals(primaryIndex.getIndexName());
}
- int numPrimaryKeys = DatasetUtils.getPartitioningFunctions(datasetDecl).size();
+ int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
RecordDescriptor outputRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
int numKeys = numPrimaryKeys;
int keysStartIndex = outputRecDesc.getFieldCount() - numKeys - 1;
if (isSecondary) {
- AqlCompiledIndexDecl secondaryIndexDecl = DatasetUtils.findSecondaryIndexByName(datasetDecl, indexName);
- int numSecondaryKeys = secondaryIndexDecl.getFieldExprs().size();
+ Index secondaryIndex = metadata.getIndex(dataset.getDataverseName(), dataset.getDatasetName(), indexName);
+ int numSecondaryKeys = secondaryIndex.getKeyFieldNames().size();
numKeys += numSecondaryKeys;
keysStartIndex = outputRecDesc.getFieldCount() - numKeys;
}
@@ -332,23 +317,15 @@
@SuppressWarnings("rawtypes")
public static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildRtreeRuntime(
AqlCompiledMetadataDeclarations metadata, JobGenContext context, JobSpecification jobSpec,
- String datasetName, AqlCompiledDatasetDecl ddecl, String indexName, int[] keyFields)
- throws AlgebricksException {
- String itemTypeName = ddecl.getItemTypeName();
- IAType itemType;
- try {
- itemType = metadata.findType(itemTypeName);
- } catch (Exception e) {
- throw new AlgebricksException(e);
- }
-
+ String datasetName, Dataset dataset, String indexName, int[] keyFields) throws AlgebricksException {
+ ARecordType recType = (ARecordType) metadata.findType(dataset.getItemTypeName());
boolean isSecondary = true;
- AqlCompiledIndexDecl primIdxDecl = DatasetUtils.getPrimaryIndex(ddecl);
- if (primIdxDecl != null) {
- isSecondary = !indexName.equals(primIdxDecl.getIndexName());
+ Index primaryIndex = metadata.getDatasetPrimaryIndex(dataset.getDataverseName(), dataset.getDatasetName());
+ if (primaryIndex != null) {
+ isSecondary = !indexName.equals(primaryIndex.getIndexName());
}
- int numPrimaryKeys = DatasetUtils.getPartitioningFunctions(ddecl).size();
+ int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
ISerializerDeserializer[] recordFields;
IBinaryComparatorFactory[] comparatorFactories;
ITypeTraits[] typeTraits;
@@ -356,77 +333,60 @@
int numSecondaryKeys = 0;
int numNestedSecondaryKeyFields = 0;
int i = 0;
- if (isSecondary) {
- AqlCompiledIndexDecl cid = DatasetUtils.findSecondaryIndexByName(ddecl, indexName);
- if (cid == null) {
- throw new AlgebricksException("Code generation error: no index " + indexName + " for dataset "
- + datasetName);
- }
- List<String> secondaryKeyFields = cid.getFieldExprs();
- numSecondaryKeys = secondaryKeyFields.size();
-
- if (numSecondaryKeys != 1) {
- throw new AlgebricksException(
- "Cannot use "
- + numSecondaryKeys
- + " fields as a key for the R-tree index. There can be only one field as a key for the R-tree index.");
- }
-
- if (itemType.getTypeTag() != ATypeTag.RECORD) {
- throw new AlgebricksException("Only record types can be indexed.");
- }
- ARecordType recType = (ARecordType) itemType;
-
- Pair<IAType, Boolean> keyTypePair = AqlCompiledIndexDecl.getNonNullableKeyFieldType(
- secondaryKeyFields.get(0), recType);
- IAType keyType = keyTypePair.first;
- if (keyType == null) {
- throw new AlgebricksException("Could not find field " + secondaryKeyFields.get(0) + " in the schema.");
- }
-
- int dimension = NonTaggedFormatUtil.getNumDimensions(keyType.getTypeTag());
- numNestedSecondaryKeyFields = dimension * 2;
-
- int numFields = numNestedSecondaryKeyFields + numPrimaryKeys;
- recordFields = new ISerializerDeserializer[numFields];
- typeTraits = new ITypeTraits[numFields];
- comparatorFactories = new IBinaryComparatorFactory[numNestedSecondaryKeyFields];
- valueProviderFactories = new IPrimitiveValueProviderFactory[numNestedSecondaryKeyFields];
-
- IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(keyType.getTypeTag());
- for (i = 0; i < numNestedSecondaryKeyFields; i++) {
- ISerializerDeserializer keySerde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(nestedKeyType);
- recordFields[i] = keySerde;
- comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(
- nestedKeyType, true);
- typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(nestedKeyType);
- valueProviderFactories[i] = AqlPrimitiveValueProviderFactory.INSTANCE;
- }
- } else {
+ if (!isSecondary) {
throw new AlgebricksException("R-tree can only be used as a secondary index");
}
+ Index secondaryIndex = metadata.getIndex(dataset.getDataverseName(), dataset.getDatasetName(), indexName);
+ if (secondaryIndex == null) {
+ throw new AlgebricksException("Code generation error: no index " + indexName + " for dataset "
+ + datasetName);
+ }
+ List<String> secondaryKeyFields = secondaryIndex.getKeyFieldNames();
+ numSecondaryKeys = secondaryKeyFields.size();
+ if (numSecondaryKeys != 1) {
+ throw new AlgebricksException(
+ "Cannot use "
+ + numSecondaryKeys
+ + " fields as a key for the R-tree index. There can be only one field as a key for the R-tree index.");
+ }
+ Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(secondaryKeyFields.get(0), recType);
+ IAType keyType = keyTypePair.first;
+ if (keyType == null) {
+ throw new AlgebricksException("Could not find field " + secondaryKeyFields.get(0) + " in the schema.");
+ }
+ int dimension = NonTaggedFormatUtil.getNumDimensions(keyType.getTypeTag());
+ numNestedSecondaryKeyFields = dimension * 2;
- for (Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType : DatasetUtils
- .getPartitioningFunctions(ddecl)) {
- IAType keyType = evalFactoryAndType.third;
+ int numFields = numNestedSecondaryKeyFields + numPrimaryKeys;
+ recordFields = new ISerializerDeserializer[numFields];
+ typeTraits = new ITypeTraits[numFields];
+ comparatorFactories = new IBinaryComparatorFactory[numNestedSecondaryKeyFields];
+ valueProviderFactories = new IPrimitiveValueProviderFactory[numNestedSecondaryKeyFields];
+
+ IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(keyType.getTypeTag());
+ for (i = 0; i < numNestedSecondaryKeyFields; i++) {
ISerializerDeserializer keySerde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(keyType);
+ .getSerializerDeserializer(nestedKeyType);
recordFields[i] = keySerde;
- typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
+ comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(
+ nestedKeyType, true);
+ typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(nestedKeyType);
+ valueProviderFactories[i] = AqlPrimitiveValueProviderFactory.INSTANCE;
+ }
+
+ List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+ for (String partitioningKey : partitioningKeys) {
+ IAType type = recType.getFieldType(partitioningKey);
+ ISerializerDeserializer keySerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(type);
+ recordFields[i] = keySerde;
+ typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(type);
++i;
}
-
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
RecordDescriptor recDesc = new RecordDescriptor(recordFields);
-
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc;
- try {
- spPc = metadata.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
- } catch (Exception e) {
- throw new AlgebricksException(e);
- }
-
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc = metadata
+ .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
RTreeSearchOperatorDescriptor rtreeSearchOp = new RTreeSearchOperatorDescriptor(jobSpec, recDesc,
appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(), spPc.first, typeTraits,
comparatorFactories, keyFields, new RTreeDataflowHelperFactory(valueProviderFactories), false,
@@ -453,20 +413,19 @@
public IDataSourceIndex<String, AqlSourceId> findDataSourceIndex(String indexId, AqlSourceId dataSourceId)
throws AlgebricksException {
AqlDataSource ads = findDataSource(dataSourceId);
- AqlCompiledDatasetDecl adecl = ads.getCompiledDatasetDecl();
- if (adecl.getDatasetType() == DatasetType.EXTERNAL) {
+ Dataset dataset = ads.getDataset();
+ if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
throw new AlgebricksException("No index for external dataset " + dataSourceId);
}
- String idxName = (String) indexId;
- AqlCompiledIndexDecl acid = DatasetUtils.findSecondaryIndexByName(adecl, idxName);
- AqlSourceId asid = (AqlSourceId) dataSourceId;
- if (acid != null) {
- return new AqlIndex(acid, metadata, asid.getDatasetName());
+ String indexName = (String) indexId;
+ Index secondaryIndex = metadata.getIndex(dataset.getDataverseName(), dataset.getDatasetName(), indexName);
+ if (secondaryIndex != null) {
+ return new AqlIndex(secondaryIndex, metadata, dataset.getDatasetName());
} else {
- AqlCompiledIndexDecl primIdx = DatasetUtils.getPrimaryIndex(adecl);
- if (primIdx.getIndexName().equals(indexId)) {
- return new AqlIndex(primIdx, metadata, asid.getDatasetName());
+ Index primaryIndex = metadata.getDatasetPrimaryIndex(dataset.getDataverseName(), dataset.getDatasetName());
+ if (primaryIndex.getIndexName().equals(indexId)) {
+ return new AqlIndex(primaryIndex, metadata, dataset.getDatasetName());
} else {
return null;
}
@@ -478,29 +437,29 @@
if (!aqlId.getDataverseName().equals(metadata.getDataverseName())) {
return null;
}
- AqlCompiledDatasetDecl acdd = metadata.findDataset(aqlId.getDatasetName());
- if (acdd == null) {
+ Dataset dataset = metadata.findDataset(aqlId.getDatasetName());
+ if (dataset == null) {
throw new AlgebricksException("Datasource with id " + aqlId + " was not found.");
}
- String tName = acdd.getItemTypeName();
- IAType itemType;
- try {
- itemType = metadata.findType(tName);
- } catch (Exception e) {
- throw new AlgebricksException(e);
- }
- return new AqlDataSource(aqlId, acdd, itemType);
+ String tName = dataset.getItemTypeName();
+ IAType itemType = metadata.findType(tName);
+ return new AqlDataSource(aqlId, dataset, itemType);
}
@Override
public boolean scannerOperatorIsLeaf(IDataSource<AqlSourceId> dataSource) {
AqlSourceId asid = dataSource.getId();
String datasetName = asid.getDatasetName();
- AqlCompiledDatasetDecl adecl = metadata.findDataset(datasetName);
- if (adecl == null) {
+ Dataset dataset = null;
+ try {
+ dataset = metadata.findDataset(datasetName);
+ } catch (AlgebricksException e) {
+ throw new IllegalStateException(e);
+ }
+ if (dataset == null) {
throw new IllegalArgumentException("Unknown dataset " + datasetName);
}
- return adecl.getDatasetType() == DatasetType.EXTERNAL;
+ return dataset.getDatasetType() == DatasetType.EXTERNAL;
}
@Override
@@ -520,27 +479,23 @@
}
fieldPermutation[numKeys] = propagatedSchema.findVariable(payload);
- AqlCompiledDatasetDecl compiledDatasetDecl = metadata.findDataset(datasetName);
- if (compiledDatasetDecl == null) {
+ Dataset dataset = metadata.findDataset(datasetName);
+ if (dataset == null) {
throw new AlgebricksException("Unknown dataset " + datasetName);
}
- String indexName = DatasetUtils.getPrimaryIndex(compiledDatasetDecl).getIndexName();
+ Index primaryIndex = metadata.getDatasetPrimaryIndex(dataset.getDataverseName(), dataset.getDatasetName());
+ String indexName = primaryIndex.getIndexName();
- ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(compiledDatasetDecl, metadata);
+ String itemTypeName = dataset.getItemTypeName();
+ ARecordType itemType = (ARecordType) metadata.findType(itemTypeName);
+ ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(dataset, itemType);
+ IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
+ itemType, context.getBinaryComparatorFactoryProvider());
+
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
+ .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
-
- IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
- compiledDatasetDecl, context.getBinaryComparatorFactoryProvider());
-
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint;
- try {
- splitsAndConstraint = metadata.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName,
- indexName);
- } catch (Exception e) {
- throw new AlgebricksException(e);
- }
-
TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(),
splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation,
@@ -549,16 +504,14 @@
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad, splitsAndConstraint.second);
}
- @Override
- public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertRuntime(
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertOrDeleteRuntime(IndexOp indexOp,
IDataSource<AqlSourceId> dataSource, IOperatorSchema propagatedSchema, List<LogicalVariable> keys,
LogicalVariable payload, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec)
throws AlgebricksException {
String datasetName = dataSource.getId().getDatasetName();
int numKeys = keys.size();
- // move key fields to front
+ // Move key fields to front.
int[] fieldPermutation = new int[numKeys + 1];
- // System.arraycopy(keys, 0, fieldPermutation, 0, numKeys);
int i = 0;
for (LogicalVariable varKey : keys) {
int idx = propagatedSchema.findVariable(varKey);
@@ -567,78 +520,75 @@
}
fieldPermutation[numKeys] = propagatedSchema.findVariable(payload);
- AqlCompiledDatasetDecl compiledDatasetDecl = metadata.findDataset(datasetName);
- if (compiledDatasetDecl == null) {
+ Dataset dataset = metadata.findDataset(datasetName);
+ if (dataset == null) {
throw new AlgebricksException("Unknown dataset " + datasetName);
}
- String indexName = DatasetUtils.getPrimaryIndex(compiledDatasetDecl).getIndexName();
+ Index primaryIndex = metadata.getDatasetPrimaryIndex(dataset.getDataverseName(), dataset.getDatasetName());
+ String indexName = primaryIndex.getIndexName();
- ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(compiledDatasetDecl, metadata);
+ String itemTypeName = dataset.getItemTypeName();
+ ARecordType itemType = (ARecordType) metadata.findType(itemTypeName);
+
+ ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(dataset, itemType);
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
-
- IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
- compiledDatasetDecl, context.getBinaryComparatorFactoryProvider());
-
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint;
- try {
- splitsAndConstraint = metadata.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName,
- indexName);
- } catch (Exception e) {
- throw new AlgebricksException(e);
- }
-
+ IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
+ itemType, context.getBinaryComparatorFactoryProvider());
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
+ .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
TreeIndexInsertUpdateDeleteOperatorDescriptor btreeBulkLoad = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, recordDesc, appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(),
- splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation, IndexOp.INSERT,
+ splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation, indexOp,
new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackProvider.INSTANCE, txnId);
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad, splitsAndConstraint.second);
}
@Override
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertRuntime(
+ IDataSource<AqlSourceId> dataSource, IOperatorSchema propagatedSchema, List<LogicalVariable> keys,
+ LogicalVariable payload, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec)
+ throws AlgebricksException {
+ return getInsertOrDeleteRuntime(IndexOp.INSERT, dataSource, propagatedSchema, keys, payload, recordDesc,
+ context, spec);
+ }
+
+ @Override
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getDeleteRuntime(
IDataSource<AqlSourceId> dataSource, IOperatorSchema propagatedSchema, List<LogicalVariable> keys,
LogicalVariable payload, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec)
throws AlgebricksException {
- String datasetName = dataSource.getId().getDatasetName();
- int numKeys = keys.size();
- // move key fields to front
- int[] fieldPermutation = new int[numKeys + 1];
- // System.arraycopy(keys, 0, fieldPermutation, 0, numKeys);
- int i = 0;
- for (LogicalVariable varKey : keys) {
- int idx = propagatedSchema.findVariable(varKey);
- fieldPermutation[i] = idx;
- i++;
- }
- fieldPermutation[numKeys] = propagatedSchema.findVariable(payload);
+ return getInsertOrDeleteRuntime(IndexOp.DELETE, dataSource, propagatedSchema, keys, payload, recordDesc,
+ context, spec);
+ }
- AqlCompiledDatasetDecl compiledDatasetDecl = metadata.findDataset(datasetName);
- if (compiledDatasetDecl == null) {
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexInsertOrDeleteRuntime(IndexOp indexOp,
+ IDataSourceIndex<String, AqlSourceId> dataSourceIndex, IOperatorSchema propagatedSchema,
+ IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys,
+ List<LogicalVariable> secondaryKeys, ILogicalExpression filterExpr, RecordDescriptor recordDesc,
+ JobGenContext context, JobSpecification spec) throws AlgebricksException {
+ String indexName = dataSourceIndex.getId();
+ String datasetName = dataSourceIndex.getDataSource().getId().getDatasetName();
+ Dataset dataset = metadata.findDataset(datasetName);
+ if (dataset == null) {
throw new AlgebricksException("Unknown dataset " + datasetName);
}
- String indexName = DatasetUtils.getPrimaryIndex(compiledDatasetDecl).getIndexName();
-
- ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(compiledDatasetDecl, metadata);
-
- IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
-
- IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
- compiledDatasetDecl, context.getBinaryComparatorFactoryProvider());
-
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint;
- try {
- splitsAndConstraint = metadata.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName,
- indexName);
- } catch (Exception e) {
- throw new AlgebricksException(e);
+ Index secondaryIndex = metadata.getIndex(dataset.getDataverseName(), dataset.getDatasetName(), indexName);
+ AsterixTupleFilterFactory filterFactory = createTupleFilterFactory(inputSchemas, typeEnv, filterExpr, context);
+ switch (secondaryIndex.getIndexType()) {
+ case BTREE: {
+ return getBTreeDmlRuntime(datasetName, indexName, propagatedSchema, primaryKeys, secondaryKeys,
+ filterFactory, recordDesc, context, spec, indexOp);
+ }
+ case RTREE: {
+ return getRTreeDmlRuntime(datasetName, indexName, propagatedSchema, primaryKeys, secondaryKeys,
+ filterFactory, recordDesc, context, spec, indexOp);
+ }
+ default: {
+ throw new AlgebricksException("Insert and delete not implemented for index type: "
+ + secondaryIndex.getIndexType());
+ }
}
-
- TreeIndexInsertUpdateDeleteOperatorDescriptor btreeBulkLoad = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
- spec, recordDesc, appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(),
- splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation, IndexOp.DELETE,
- new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackProvider.INSTANCE, txnId);
- return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad, splitsAndConstraint.second);
}
@Override
@@ -647,21 +597,8 @@
IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys,
List<LogicalVariable> secondaryKeys, ILogicalExpression filterExpr, RecordDescriptor recordDesc,
JobGenContext context, JobSpecification spec) throws AlgebricksException {
- String indexName = dataSourceIndex.getId();
- String datasetName = dataSourceIndex.getDataSource().getId().getDatasetName();
- AqlCompiledDatasetDecl compiledDatasetDecl = metadata.findDataset(datasetName);
- if (compiledDatasetDecl == null) {
- throw new AlgebricksException("Unknown dataset " + datasetName);
- }
- AqlCompiledIndexDecl cid = DatasetUtils.findSecondaryIndexByName(compiledDatasetDecl, indexName);
- AsterixTupleFilterFactory filterFactory = createTupleFilterFactory(inputSchemas, typeEnv, filterExpr, context);
- if (cid.getKind() == IndexKind.BTREE) {
- return getBTreeDmlRuntime(datasetName, indexName, propagatedSchema, primaryKeys, secondaryKeys,
- filterFactory, recordDesc, context, spec, IndexOp.INSERT);
- } else {
- return getRTreeDmlRuntime(datasetName, indexName, propagatedSchema, primaryKeys, secondaryKeys,
- filterFactory, recordDesc, context, spec, IndexOp.INSERT);
- }
+ return getIndexInsertOrDeleteRuntime(IndexOp.INSERT, dataSourceIndex, propagatedSchema, inputSchemas, typeEnv,
+ primaryKeys, secondaryKeys, filterExpr, recordDesc, context, spec);
}
@Override
@@ -670,21 +607,8 @@
IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys,
List<LogicalVariable> secondaryKeys, ILogicalExpression filterExpr, RecordDescriptor recordDesc,
JobGenContext context, JobSpecification spec) throws AlgebricksException {
- String indexName = dataSourceIndex.getId();
- String datasetName = dataSourceIndex.getDataSource().getId().getDatasetName();
- AqlCompiledDatasetDecl compiledDatasetDecl = metadata.findDataset(datasetName);
- if (compiledDatasetDecl == null) {
- throw new AlgebricksException("Unknown dataset " + datasetName);
- }
- AqlCompiledIndexDecl cid = DatasetUtils.findSecondaryIndexByName(compiledDatasetDecl, indexName);
- AsterixTupleFilterFactory filterFactory = createTupleFilterFactory(inputSchemas, typeEnv, filterExpr, context);
- if (cid.getKind() == IndexKind.BTREE) {
- return getBTreeDmlRuntime(datasetName, indexName, propagatedSchema, primaryKeys, secondaryKeys,
- filterFactory, recordDesc, context, spec, IndexOp.DELETE);
- } else {
- return getRTreeDmlRuntime(datasetName, indexName, propagatedSchema, primaryKeys, secondaryKeys,
- filterFactory, recordDesc, context, spec, IndexOp.DELETE);
- }
+ return getIndexInsertOrDeleteRuntime(IndexOp.DELETE, dataSourceIndex, propagatedSchema, inputSchemas, typeEnv,
+ primaryKeys, secondaryKeys, filterExpr, recordDesc, context, spec);
}
private AsterixTupleFilterFactory createTupleFilterFactory(IOperatorSchema[] inputSchemas,
@@ -719,39 +643,33 @@
i++;
}
- // dataset
- AqlCompiledDatasetDecl compiledDatasetDecl = metadata.findDataset(datasetName);
- if (compiledDatasetDecl == null) {
+ Dataset dataset = metadata.findDataset(datasetName);
+ if (dataset == null) {
throw new AlgebricksException("Unknown dataset " + datasetName);
}
- String itemTypeName = compiledDatasetDecl.getItemTypeName();
- IAType itemType;
- try {
- itemType = metadata.findType(itemTypeName);
- } catch (Exception e) {
- throw new AlgebricksException(e);
- }
+ String itemTypeName = dataset.getItemTypeName();
+ IAType itemType = metadata.findType(itemTypeName);
if (itemType.getTypeTag() != ATypeTag.RECORD) {
throw new AlgebricksException("Only record types can be indexed.");
}
ARecordType recType = (ARecordType) itemType;
- // index parameters
- AqlCompiledIndexDecl cid = DatasetUtils.findSecondaryIndexByName(compiledDatasetDecl, indexName);
- List<String> secondaryKeyExprs = cid.getFieldExprs();
+ // Index parameters.
+ Index secondaryIndex = metadata.getIndex(dataset.getDataverseName(), dataset.getDatasetName(), indexName);
+ List<String> secondaryKeyExprs = secondaryIndex.getKeyFieldNames();
ITypeTraits[] typeTraits = new ITypeTraits[numKeys];
IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[numKeys];
for (i = 0; i < secondaryKeys.size(); ++i) {
- Pair<IAType, Boolean> keyPairType = AqlCompiledIndexDecl.getNonNullableKeyFieldType(secondaryKeyExprs
- .get(i).toString(), recType);
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(secondaryKeyExprs.get(i).toString(),
+ recType);
IAType keyType = keyPairType.first;
comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType,
true);
typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
}
- for (Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType : DatasetUtils
- .getPartitioningFunctions(compiledDatasetDecl)) {
- IAType keyType = evalFactoryAndType.third;
+ List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+ for (String partitioningKey : partitioningKeys) {
+ IAType keyType = recType.getFieldType(partitioningKey);
comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType,
true);
typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
@@ -759,13 +677,8 @@
}
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint;
- try {
- splitsAndConstraint = metadata.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName,
- indexName);
- } catch (Exception e) {
- throw new AlgebricksException(e);
- }
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
+ .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
TreeIndexInsertUpdateDeleteOperatorDescriptor btreeBulkLoad = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, recordDesc, appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(),
splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation, indexOp,
@@ -777,22 +690,16 @@
String indexName, IOperatorSchema propagatedSchema, List<LogicalVariable> primaryKeys,
List<LogicalVariable> secondaryKeys, AsterixTupleFilterFactory filterFactory, RecordDescriptor recordDesc,
JobGenContext context, JobSpecification spec, IndexOp indexOp) throws AlgebricksException {
- AqlCompiledDatasetDecl compiledDatasetDecl = metadata.findDataset(datasetName);
- String itemTypeName = compiledDatasetDecl.getItemTypeName();
- IAType itemType;
- try {
- itemType = metadata.findType(itemTypeName);
- } catch (Exception e) {
- throw new AlgebricksException(e);
- }
+ Dataset dataset = metadata.findDataset(datasetName);
+ String itemTypeName = dataset.getItemTypeName();
+ IAType itemType = metadata.findType(itemTypeName);
if (itemType.getTypeTag() != ATypeTag.RECORD) {
throw new AlgebricksException("Only record types can be indexed.");
}
ARecordType recType = (ARecordType) itemType;
- AqlCompiledIndexDecl cid = DatasetUtils.findSecondaryIndexByName(compiledDatasetDecl, indexName);
- List<String> secondaryKeyExprs = cid.getFieldExprs();
- Pair<IAType, Boolean> keyPairType = AqlCompiledIndexDecl.getNonNullableKeyFieldType(secondaryKeyExprs.get(0),
- recType);
+ Index secondaryIndex = metadata.getIndex(dataset.getDataverseName(), dataset.getDatasetName(), indexName);
+ List<String> secondaryKeyExprs = secondaryIndex.getKeyFieldNames();
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(secondaryKeyExprs.get(0), recType);
IAType spatialType = keyPairType.first;
int dimension = NonTaggedFormatUtil.getNumDimensions(spatialType.getTypeTag());
int numSecondaryKeys = dimension * 2;
@@ -816,16 +723,14 @@
IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(spatialType.getTypeTag());
IPrimitiveValueProviderFactory[] valueProviderFactories = new IPrimitiveValueProviderFactory[numSecondaryKeys];
for (i = 0; i < numSecondaryKeys; i++) {
- ISerializerDeserializer keySerde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(nestedKeyType);
comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(
nestedKeyType, true);
typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(nestedKeyType);
valueProviderFactories[i] = AqlPrimitiveValueProviderFactory.INSTANCE;
}
- for (Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType : DatasetUtils
- .getPartitioningFunctions(compiledDatasetDecl)) {
- IAType keyType = evalFactoryAndType.third;
+ List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+ for (String partitioningKey : partitioningKeys) {
+ IAType keyType = recType.getFieldType(partitioningKey);
comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType,
true);
typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
@@ -833,13 +738,8 @@
}
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint;
- try {
- splitsAndConstraint = metadata.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName,
- indexName);
- } catch (Exception e) {
- throw new AlgebricksException(e);
- }
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
+ .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
TreeIndexInsertUpdateDeleteOperatorDescriptor rtreeUpdate = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, recordDesc, appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(),
splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation, indexOp,
@@ -860,5 +760,4 @@
public IFunctionInfo lookupFunction(FunctionIdentifier fid) {
return AsterixBuiltinFunctions.lookupFunction(fid);
}
-
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ExternalFeedDataSource.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ExternalFeedDataSource.java
index 16e8bae..566265f 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ExternalFeedDataSource.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ExternalFeedDataSource.java
@@ -1,17 +1,17 @@
package edu.uci.ics.asterix.metadata.declared;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
public class ExternalFeedDataSource extends AqlDataSource {
- public ExternalFeedDataSource(AqlSourceId id, AqlCompiledDatasetDecl adecl, IAType itemType) throws AlgebricksException {
- super(id,adecl, itemType);
+ public ExternalFeedDataSource(AqlSourceId id, Dataset dataset, IAType itemType) throws AlgebricksException {
+ super(id, dataset, itemType);
}
- public ExternalFeedDataSource(AqlSourceId id, AqlCompiledDatasetDecl adecl, IAType itemType, AqlDataSourceType dataSourceType) throws AlgebricksException {
- super(id,adecl, itemType, dataSourceType);
+ public ExternalFeedDataSource(AqlSourceId id, Dataset dataset, IAType itemType, AqlDataSourceType dataSourceType)
+ throws AlgebricksException {
+ super(id, dataset, itemType, dataSourceType);
}
-
-
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/IAqlCompiledDatasetDetails.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/IAqlCompiledDatasetDetails.java
deleted file mode 100644
index 2253db9..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/IAqlCompiledDatasetDetails.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Copyright 2009-2011 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.metadata.declared;
-
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-
-public interface IAqlCompiledDatasetDetails {
-
- public DatasetType getDatasetType();
-}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Dataset.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Dataset.java
index e810bc9..d383955 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Dataset.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Dataset.java
@@ -30,16 +30,16 @@
private final String dataverseName;
// Enforced to be unique within a dataverse.
private final String datasetName;
- // Type of values stored in this dataset.
- private final String datatypeName;
+ // Type of items stored in this dataset.
+ private final String itemTypeName;
private final DatasetType datasetType;
private IDatasetDetails datasetDetails;
- public Dataset(String dataverseName, String datasetName, String datatypeName, IDatasetDetails datasetDetails,
+ public Dataset(String dataverseName, String datasetName, String itemTypeName, IDatasetDetails datasetDetails,
DatasetType datasetType) {
this.dataverseName = dataverseName;
this.datasetName = datasetName;
- this.datatypeName = datatypeName;
+ this.itemTypeName = itemTypeName;
this.datasetType = datasetType;
this.datasetDetails = datasetDetails;
}
@@ -52,11 +52,11 @@
return datasetName;
}
- public String getDatatypeName() {
- return datatypeName;
+ public String getItemTypeName() {
+ return itemTypeName;
}
- public DatasetType getType() {
+ public DatasetType getDatasetType() {
return datasetType;
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedDatasetDetails.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedDatasetDetails.java
index 36db14a..9761c47 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedDatasetDetails.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedDatasetDetails.java
@@ -92,7 +92,7 @@
// write field 2
listBuilder.reset((AOrderedListType) MetadataRecordTypes.FEED_DETAILS_RECORDTYPE.getFieldTypes()[2]);
- for (String field : partitioningKey) {
+ for (String field : partitioningKeys) {
itemValue.reset();
aString.setValue(field);
stringSerde.serialize(aString, itemValue.getDataOutput());
@@ -104,7 +104,7 @@
// write field 3
listBuilder.reset((AOrderedListType) MetadataRecordTypes.FEED_DETAILS_RECORDTYPE.getFieldTypes()[3]);
- for (String field : primaryKey) {
+ for (String field : primaryKeys) {
itemValue.reset();
aString.setValue(field);
stringSerde.serialize(aString, itemValue.getDataOutput());
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Index.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Index.java
index bd3f8a7..7c6d9ed 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Index.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Index.java
@@ -19,6 +19,12 @@
import java.util.List;
import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.AUnionType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
/**
* Metadata describing an index.
@@ -91,4 +97,30 @@
public boolean isSecondaryIndex() {
return !isPrimaryIndex();
}
+
+ public static Pair<IAType, Boolean> getNonNullableKeyFieldType(String expr, ARecordType recType)
+ throws AlgebricksException {
+ IAType keyType = Index.keyFieldType(expr, recType);
+ boolean nullable = false;
+ if (keyType.getTypeTag() == ATypeTag.UNION) {
+ AUnionType unionType = (AUnionType) keyType;
+ if (unionType.isNullableType()) {
+ // The non-null type is always at index 1.
+ keyType = unionType.getUnionList().get(1);
+ nullable = true;
+ }
+ }
+ return new Pair<IAType, Boolean>(keyType, nullable);
+ }
+
+ private static IAType keyFieldType(String expr, ARecordType recType) throws AlgebricksException {
+ String[] names = recType.getFieldNames();
+ int n = names.length;
+ for (int i = 0; i < n; i++) {
+ if (names[i].equals(expr)) {
+ return recType.getFieldTypes()[i];
+ }
+ }
+ throw new AlgebricksException("Could not find field " + expr + " in the schema.");
+ }
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/InternalDatasetDetails.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/InternalDatasetDetails.java
index de51dc8..53d10e4 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/InternalDatasetDetails.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/InternalDatasetDetails.java
@@ -48,29 +48,29 @@
protected final FileStructure fileStructure;
protected final PartitioningStrategy partitioningStrategy;
- protected final List<String> partitioningKey;
- protected final List<String> primaryKey;
- protected final String groupName;
+ protected final List<String> partitioningKeys;
+ protected final List<String> primaryKeys;
+ protected final String nodeGroupName;
public InternalDatasetDetails(FileStructure fileStructure, PartitioningStrategy partitioningStrategy,
List<String> partitioningKey, List<String> primaryKey, String groupName) {
this.fileStructure = fileStructure;
this.partitioningStrategy = partitioningStrategy;
- this.partitioningKey = partitioningKey;
- this.primaryKey = primaryKey;
- this.groupName = groupName;
+ this.partitioningKeys = partitioningKey;
+ this.primaryKeys = primaryKey;
+ this.nodeGroupName = groupName;
}
public String getNodeGroupName() {
- return this.groupName;
+ return nodeGroupName;
}
public List<String> getPartitioningKey() {
- return this.partitioningKey;
+ return partitioningKeys;
}
public List<String> getPrimaryKey() {
- return primaryKey;
+ return primaryKeys;
}
public FileStructure getFileStructure() {
@@ -115,7 +115,7 @@
// write field 2
listBuilder
.reset((AOrderedListType) MetadataRecordTypes.INTERNAL_DETAILS_RECORDTYPE.getFieldTypes()[MetadataRecordTypes.INTERNAL_DETAILS_ARECORD_PARTITIONKEY_FIELD_INDEX]);
- for (String field : partitioningKey) {
+ for (String field : partitioningKeys) {
itemValue.reset();
aString.setValue(field);
stringSerde.serialize(aString, itemValue.getDataOutput());
@@ -129,7 +129,7 @@
// write field 3
listBuilder
.reset((AOrderedListType) MetadataRecordTypes.INTERNAL_DETAILS_RECORDTYPE.getFieldTypes()[MetadataRecordTypes.INTERNAL_DETAILS_ARECORD_PRIMARYKEY_FIELD_INDEX]);
- for (String field : primaryKey) {
+ for (String field : primaryKeys) {
itemValue.reset();
aString.setValue(field);
stringSerde.serialize(aString, itemValue.getDataOutput());
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 78703f3..2eda4a8 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
@@ -147,7 +147,7 @@
.getValueByPos(MetadataRecordTypes.FEED_DETAILS_ARECORD_STATE_FIELD_INDEX))
.getStringValue();
- datasetDetails = new FeedDatasetDetails(fileStructure, partitioningStrategy, partitioningKey,
+ datasetDetails = new FeedDatasetDetails(fileStructure, partitioningStrategy, partitioningKey,
partitioningKey, groupName, adapter, properties, functionIdentifier, feedState);
}
break;
@@ -209,13 +209,13 @@
// write field 2
fieldValue.reset();
- aString.setValue(dataset.getDatatypeName());
+ aString.setValue(dataset.getItemTypeName());
stringSerde.serialize(aString, fieldValue.getDataOutput());
recordBuilder.addField(MetadataRecordTypes.DATASET_ARECORD_DATATYPENAME_FIELD_INDEX, fieldValue);
// write field 3
fieldValue.reset();
- aString.setValue(dataset.getType().toString());
+ aString.setValue(dataset.getDatasetType().toString());
stringSerde.serialize(aString, fieldValue.getDataOutput());
recordBuilder.addField(MetadataRecordTypes.DATASET_ARECORD_DATASETTYPE_FIELD_INDEX, fieldValue);
@@ -241,7 +241,7 @@
throws HyracksDataException {
dataset.getDatasetDetails().writeDatasetDetailsRecordType(fieldValue.getDataOutput());
- switch (dataset.getType()) {
+ switch (dataset.getDatasetType()) {
case INTERNAL:
recordBuilder.addField(MetadataRecordTypes.DATASET_ARECORD_INTERNALDETAILS_FIELD_INDEX, fieldValue);
break;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/MetadataBuiltinFunctions.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/MetadataBuiltinFunctions.java
index dd05b00..edb3808 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/MetadataBuiltinFunctions.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/MetadataBuiltinFunctions.java
@@ -1,8 +1,8 @@
package edu.uci.ics.asterix.metadata.functions;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
import edu.uci.ics.asterix.om.base.AString;
import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
@@ -53,11 +53,11 @@
AsterixConstantValue acv = (AsterixConstantValue) ((ConstantExpression) a1).getValue();
String datasetName = ((AString) acv.getObject()).getStringValue();
AqlCompiledMetadataDeclarations metadata = ((AqlMetadataProvider) mp).getMetadataDeclarations();
- AqlCompiledDatasetDecl acdd = metadata.findDataset(datasetName);
- if (acdd == null) {
+ Dataset dataset = metadata.findDataset(datasetName);
+ if (dataset == null) {
throw new AlgebricksException("Could not find dataset " + datasetName);
}
- String tn = acdd.getItemTypeName();
+ String tn = dataset.getItemTypeName();
IAType t2 = metadata.findType(tn);
if (t2 == null) {
throw new AlgebricksException("No type for dataset " + datasetName);
@@ -89,11 +89,11 @@
AsterixConstantValue acv = (AsterixConstantValue) ((ConstantExpression) a1).getValue();
String datasetName = ((AString) acv.getObject()).getStringValue();
AqlCompiledMetadataDeclarations metadata = ((AqlMetadataProvider) mp).getMetadataDeclarations();
- AqlCompiledDatasetDecl acdd = metadata.findDataset(datasetName);
- if (acdd == null) {
+ Dataset dataset = metadata.findDataset(datasetName);
+ if (dataset == null) {
throw new AlgebricksException("Could not find dataset " + datasetName);
}
- String tn = acdd.getItemTypeName();
+ String tn = dataset.getItemTypeName();
IAType t2 = metadata.findType(tn);
if (t2 == null) {
throw new AlgebricksException("No type for dataset " + datasetName);
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 a34f14b..cead5f2 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
@@ -4,115 +4,78 @@
import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledInternalDatasetDetails;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
+import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
import edu.uci.ics.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
import edu.uci.ics.hyracks.algebricks.data.IBinaryHashFunctionFactoryProvider;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
public class DatasetUtils {
- public static IBinaryComparatorFactory[] computeKeysBinaryComparatorFactories(
- AqlCompiledDatasetDecl compiledDatasetDecl, IBinaryComparatorFactoryProvider comparatorFactoryProvider)
+ public static IBinaryComparatorFactory[] computeKeysBinaryComparatorFactories(Dataset dataset,
+ ARecordType itemType, IBinaryComparatorFactoryProvider comparatorFactoryProvider)
throws AlgebricksException {
- if (compiledDatasetDecl.getDatasetType() == DatasetType.EXTERNAL)
+ if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
throw new AlgebricksException("not implemented");
- List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions;
- partitioningFunctions = getPartitioningFunctions(compiledDatasetDecl);
- int numKeys = partitioningFunctions.size();
- IBinaryComparatorFactory[] bcfs = new IBinaryComparatorFactory[numKeys];
- for (int i = 0; i < numKeys; i++) {
- Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType = partitioningFunctions
- .get(i);
- IAType keyType = evalFactoryAndType.third;
+ }
+ List<String> partitioningKeys = getPartitioningKeys(dataset);
+ IBinaryComparatorFactory[] bcfs = new IBinaryComparatorFactory[partitioningKeys.size()];
+ for (int i = 0; i < partitioningKeys.size(); i++) {
+ IAType keyType = itemType.getFieldType(partitioningKeys.get(i));
bcfs[i] = comparatorFactoryProvider.getBinaryComparatorFactory(keyType, true);
}
return bcfs;
}
- public static IBinaryHashFunctionFactory[] computeKeysBinaryHashFunFactories(
- AqlCompiledDatasetDecl compiledDatasetDecl, IBinaryHashFunctionFactoryProvider hashFunProvider)
- throws AlgebricksException {
- if (compiledDatasetDecl.getDatasetType() == DatasetType.EXTERNAL)
+ public static IBinaryHashFunctionFactory[] computeKeysBinaryHashFunFactories(Dataset dataset, ARecordType itemType,
+ IBinaryHashFunctionFactoryProvider hashFunProvider) throws AlgebricksException {
+ if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
throw new AlgebricksException("not implemented");
- List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions;
- partitioningFunctions = ((AqlCompiledInternalDatasetDetails) compiledDatasetDecl.getAqlCompiledDatasetDetails())
- .getPartitioningFunctions();
- int numKeys = partitioningFunctions.size();
- IBinaryHashFunctionFactory[] bhffs = new IBinaryHashFunctionFactory[numKeys];
- for (int i = 0; i < numKeys; i++) {
- Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType = partitioningFunctions
- .get(i);
- IAType keyType = evalFactoryAndType.third;
+ }
+ List<String> partitioningKeys = getPartitioningKeys(dataset);
+ IBinaryHashFunctionFactory[] bhffs = new IBinaryHashFunctionFactory[partitioningKeys.size()];
+ for (int i = 0; i < partitioningKeys.size(); i++) {
+ IAType keyType = itemType.getFieldType(partitioningKeys.get(i));
bhffs[i] = hashFunProvider.getBinaryHashFunctionFactory(keyType);
}
return bhffs;
}
- public static ITypeTraits[] computeTupleTypeTraits(AqlCompiledDatasetDecl compiledDatasetDecl,
- AqlCompiledMetadataDeclarations datasetDecls) throws AlgebricksException {
- if (compiledDatasetDecl.getDatasetType() == DatasetType.EXTERNAL)
+ public static ITypeTraits[] computeTupleTypeTraits(Dataset dataset, ARecordType itemType)
+ throws AlgebricksException {
+ if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
throw new AlgebricksException("not implemented");
- List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions;
- partitioningFunctions = ((AqlCompiledInternalDatasetDetails) compiledDatasetDecl.getAqlCompiledDatasetDetails())
- .getPartitioningFunctions();
- int numKeys = partitioningFunctions.size();
+ }
+ List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+ int numKeys = partitioningKeys.size();
ITypeTraits[] typeTraits = new ITypeTraits[numKeys + 1];
for (int i = 0; i < numKeys; i++) {
- Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType = partitioningFunctions
- .get(i);
- IAType keyType = evalFactoryAndType.third;
+ IAType keyType = itemType.getFieldType(partitioningKeys.get(i));
typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
}
- IAType payloadType = datasetDecls.findType(compiledDatasetDecl.getItemTypeName());
- typeTraits[numKeys] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(payloadType);
+ typeTraits[numKeys] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(itemType);
return typeTraits;
}
- public static List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> getPartitioningFunctions(
- AqlCompiledDatasetDecl decl) {
- return ((AqlCompiledInternalDatasetDetails) decl.getAqlCompiledDatasetDetails()).getPartitioningFunctions();
+ public static List<String> getPartitioningKeys(Dataset dataset) {
+ return ((InternalDatasetDetails) dataset.getDatasetDetails()).getPartitioningKey();
}
- public static String getNodegroupName(AqlCompiledDatasetDecl decl) {
- return (((AqlCompiledInternalDatasetDetails) decl.getAqlCompiledDatasetDetails())).getNodegroupName();
+ public static String getNodegroupName(Dataset dataset) {
+ return (((InternalDatasetDetails) dataset.getDatasetDetails())).getNodeGroupName();
}
- public static AqlCompiledIndexDecl getPrimaryIndex(AqlCompiledDatasetDecl decl) {
- return (((AqlCompiledInternalDatasetDetails) decl.getAqlCompiledDatasetDetails())).getPrimaryIndex();
-
+ public static int getPositionOfPartitioningKeyField(Dataset dataset, String fieldExpr) {
+ List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+ for (int i = 0; i < partitioningKeys.size(); i++) {
+ if (partitioningKeys.get(i).equals(fieldExpr)) {
+ return i;
+ }
+ }
+ return -1;
}
-
- public static AqlCompiledIndexDecl findSecondaryIndexByName(AqlCompiledDatasetDecl decl, String indexName) {
- return (((AqlCompiledInternalDatasetDetails) decl.getAqlCompiledDatasetDetails())
- .findSecondaryIndexByName(indexName));
- }
-
- public static List<AqlCompiledIndexDecl> findSecondaryIndexesByOneOfTheKeys(AqlCompiledDatasetDecl decl,
- String fieldExpr) {
- return (((AqlCompiledInternalDatasetDetails) decl.getAqlCompiledDatasetDetails()))
- .findSecondaryIndexesByOneOfTheKeys(fieldExpr);
- }
-
- public static int getPositionOfPartitioningKeyField(AqlCompiledDatasetDecl decl, String fieldExpr) {
- return (((AqlCompiledInternalDatasetDetails) decl.getAqlCompiledDatasetDetails()))
- .getPositionOfPartitioningKeyField(fieldExpr);
- }
-
- public static List<String> getPartitioningExpressions(AqlCompiledDatasetDecl decl) {
- return (((AqlCompiledInternalDatasetDetails) decl.getAqlCompiledDatasetDetails())).getPartitioningExprs();
- }
-
- public static List<AqlCompiledIndexDecl> getSecondaryIndexes(AqlCompiledDatasetDecl decl) {
- return (((AqlCompiledInternalDatasetDetails) decl.getAqlCompiledDatasetDetails())).getSecondaryIndexes();
- }
-
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ARecordType.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ARecordType.java
index 2953966..0e2f825 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ARecordType.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ARecordType.java
@@ -2,7 +2,9 @@
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.HashMap;
import java.util.List;
+import java.util.Map;
import edu.uci.ics.asterix.common.annotations.IRecordTypeAnnotation;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
@@ -15,13 +17,17 @@
private String[] fieldNames;
private IAType[] fieldTypes;
private boolean isOpen;
- private transient List<IRecordTypeAnnotation> annotations = new ArrayList<IRecordTypeAnnotation>();
+ private transient final List<IRecordTypeAnnotation> annotations = new ArrayList<IRecordTypeAnnotation>();
+ private transient final Map<String, Integer> typeMap = new HashMap<String, Integer>();
public ARecordType(String typeName, String[] fieldNames, IAType[] fieldTypes, boolean isOpen) {
super(typeName);
this.fieldNames = fieldNames;
this.fieldTypes = fieldTypes;
this.isOpen = isOpen;
+ for (int i = 0; i < fieldNames.length; i++) {
+ typeMap.put(fieldNames[i], i);
+ }
}
public final String[] getFieldNames() {
@@ -76,6 +82,10 @@
return -1;
}
+ public IAType getFieldType(String fieldName) {
+ return fieldTypes[typeMap.get(fieldName)];
+ }
+
@Override
public String getDisplayName() {
return "ARecord";