merged asterix_stabilization -r722:751 to asterix_lsm_stabilization
git-svn-id: https://asterixdb.googlecode.com/svn/branches/asterix_lsm_stabilization@756 eaa15691-b419-025a-1212-ee371bd00084
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 ae2559b..c2bb117 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,6 +3,7 @@
import java.util.ArrayList;
import java.util.List;
+import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
import edu.uci.ics.asterix.common.dataflow.IAsterixApplicationContextInfo;
import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
@@ -43,11 +44,11 @@
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifierFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.InvertedIndexSearchOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifierFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexSearchOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
/**
* Contributes the runtime operator for an unnest-map representing an inverted-index search.
@@ -164,12 +165,14 @@
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);
+ LSMInvertedIndexSearchOperatorDescriptor invIndexSearchOp = new LSMInvertedIndexSearchOperatorDescriptor(
+ jobSpec, queryField, appContext.getStorageManagerInterface(), fileSplitProviders.first,
+ appContext.getIndexLifecycleManagerProvider(), tokenTypeTraits, tokenComparatorFactories,
+ invListsTypeTraits, invListsComparatorFactories, new LSMInvertedIndexDataflowHelperFactory(
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE),
+ 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/optimizer/base/RuleCollections.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
index 0f28239..09ef774 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
@@ -31,6 +31,7 @@
import edu.uci.ics.asterix.optimizer.rules.IntroduceDynamicTypeCastRule;
import edu.uci.ics.asterix.optimizer.rules.IntroduceSecondaryIndexInsertDeleteRule;
import edu.uci.ics.asterix.optimizer.rules.IntroduceStaticTypeCastRule;
+import edu.uci.ics.asterix.optimizer.rules.IntroduceTransactionCommitByAssignOpRule;
import edu.uci.ics.asterix.optimizer.rules.LoadRecordFieldsRule;
import edu.uci.ics.asterix.optimizer.rules.NestGroupByRule;
import edu.uci.ics.asterix.optimizer.rules.PullPositionalVariableFromUnnestRule;
@@ -195,6 +196,8 @@
public final static List<IAlgebraicRewriteRule> buildPhysicalRewritesAllLevelsRuleCollection() {
List<IAlgebraicRewriteRule> physicalRewritesAllLevels = new LinkedList<IAlgebraicRewriteRule>();
physicalRewritesAllLevels.add(new PullSelectOutOfEqJoin());
+ //Turned off the following rule for now not to change OptimizerTest results.
+ //physicalRewritesAllLevels.add(new IntroduceTransactionCommitByAssignOpRule());
physicalRewritesAllLevels.add(new SetAlgebricksPhysicalOperatorsRule());
physicalRewritesAllLevels.add(new SetAsterixPhysicalOperatorsRule());
physicalRewritesAllLevels.add(new EnforceStructuralPropertiesRule());
@@ -203,6 +206,7 @@
physicalRewritesAllLevels.add(new PullPositionalVariableFromUnnestRule());
physicalRewritesAllLevels.add(new PushProjectDownRule());
physicalRewritesAllLevels.add(new InsertProjectBeforeUnionRule());
+
return physicalRewritesAllLevels;
}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/FuzzyJoinRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/FuzzyJoinRule.java
index e700971..4b2a2ab 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/FuzzyJoinRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/FuzzyJoinRule.java
@@ -264,7 +264,7 @@
// The translator will compile metadata internally. Run this compilation
// under the same transaction id as the "outer" compilation.
AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
- AqlPlusExpressionToPlanTranslator translator = new AqlPlusExpressionToPlanTranslator(mp.getTxnId(),
+ AqlPlusExpressionToPlanTranslator translator = new AqlPlusExpressionToPlanTranslator(mp.getJobId(),
metadata.getMetadataTransactionContext(), counter, null);
LogicalOperatorDeepCopyVisitor deepCopyVisitor = new LogicalOperatorDeepCopyVisitor(counter);
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceTransactionCommitByAssignOpRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceTransactionCommitByAssignOpRule.java
new file mode 100644
index 0000000..d635942
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceTransactionCommitByAssignOpRule.java
@@ -0,0 +1,76 @@
+/*
+ * Copyright 2009-2012 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.optimizer.rules;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class IntroduceTransactionCommitByAssignOpRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (op.getOperatorTag() != LogicalOperatorTag.SELECT) {
+ return false;
+ }
+ SelectOperator selectOperator = (SelectOperator) op;
+
+ Mutable<ILogicalOperator> childOfSelect = selectOperator.getInputs().get(0);
+
+ //[Direction] SelectOp(cond1)<--ChildOps... ==> SelectOp(booleanValue of cond1)<--NewAssignOp(cond1)<--ChildOps...
+ //#. Create an assign-operator with a new local variable and the condition of the select-operator.
+ //#. Set the input(child operator) of the new assign-operator to input(child operator) of the select-operator.
+ // (Later, the newly created assign-operator will apply the condition on behalf of the select-operator,
+ // and set the variable of the assign-operator to a boolean value according to the condition evaluation.)
+ //#. Give the select-operator the result boolean value created by the newly created child assign-operator.
+
+ //create an assignOp with a variable and the condition of the select-operator.
+ LogicalVariable v = context.newVar();
+ AssignOperator assignOperator = new AssignOperator(v, new MutableObject<ILogicalExpression>(selectOperator
+ .getCondition().getValue()));
+
+ //set the input of the new assign-operator to the input of the select-operator.
+ assignOperator.getInputs().add(childOfSelect);
+
+ //set the result value of the assign-operator to the condition of the select-operator
+ selectOperator.getCondition().setValue(new VariableReferenceExpression(v));//scalarFunctionCallExpression);
+ selectOperator.getInputs().set(0, new MutableObject<ILogicalOperator>(assignOperator));
+
+ context.computeAndSetTypeEnvironmentForOperator(assignOperator);
+
+ //Once this rule is fired, don't apply again.
+ context.addToDontApplySet(this, selectOperator);
+ return true;
+ }
+}
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 dbd92c0..a0f4759 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
@@ -58,11 +58,11 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifierFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.searchmodifiers.ConjunctiveSearchModifierFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.searchmodifiers.EditDistanceSearchModifierFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.searchmodifiers.JaccardSearchModifierFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifierFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.ConjunctiveSearchModifierFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.EditDistanceSearchModifierFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.JaccardSearchModifierFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
/**
* Class for helping rewrite rules to choose and apply inverted indexes.
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 e5712df..2e0a09b 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
@@ -97,6 +97,7 @@
import edu.uci.ics.asterix.om.functions.AsterixFunctionInfo;
import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
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;
@@ -155,1521 +156,1268 @@
* source for the current subtree.
*/
-public class AqlExpressionToPlanTranslator extends AbstractAqlTranslator
- implements
- IAqlExpressionVisitor<Pair<ILogicalOperator, LogicalVariable>, Mutable<ILogicalOperator>> {
+public class AqlExpressionToPlanTranslator extends AbstractAqlTranslator implements
+ IAqlExpressionVisitor<Pair<ILogicalOperator, LogicalVariable>, Mutable<ILogicalOperator>> {
- private final MetadataTransactionContext mdTxnCtx;
- private final long txnId;
- private TranslationContext context;
- private String outputDatasetName;
- private Statement.Kind dmlKind;
- private static AtomicLong outputFileID = new AtomicLong(0);
- private static final String OUTPUT_FILE_PREFIX = "OUTPUT_";
+ private final MetadataTransactionContext mdTxnCtx;
+ private final JobId jobId;
+ private TranslationContext context;
+ private String outputDatasetName;
+ private Statement.Kind dmlKind;
+ private static AtomicLong outputFileID = new AtomicLong(0);
+ private static final String OUTPUT_FILE_PREFIX = "OUTPUT_";
- private static LogicalVariable METADATA_DUMMY_VAR = new LogicalVariable(-1);
+ private static LogicalVariable METADATA_DUMMY_VAR = new LogicalVariable(-1);
- public AqlExpressionToPlanTranslator(long txnId,
- MetadataTransactionContext mdTxnCtx, int currentVarCounter,
- String outputDatasetName, Statement.Kind dmlKind) {
- this.mdTxnCtx = mdTxnCtx;
- this.txnId = txnId;
- this.context = new TranslationContext(new Counter(currentVarCounter));
- this.outputDatasetName = outputDatasetName;
- this.dmlKind = dmlKind;
- }
+ public AqlExpressionToPlanTranslator(JobId jobId, MetadataTransactionContext mdTxnCtx, int currentVarCounter,
+ String outputDatasetName, Statement.Kind dmlKind) {
+ this.mdTxnCtx = mdTxnCtx;
+ this.jobId = jobId;
+ this.context = new TranslationContext(new Counter(currentVarCounter));
+ this.outputDatasetName = outputDatasetName;
+ this.dmlKind = dmlKind;
+ }
- public int getVarCounter() {
- return context.getVarCounter();
- }
+ public int getVarCounter() {
+ return context.getVarCounter();
+ }
- public ILogicalPlanAndMetadata translate(Query expr,
- AqlCompiledMetadataDeclarations compiledDeclarations)
- throws AlgebricksException, AsterixException {
- if (expr == null) {
- return null;
- }
- if (compiledDeclarations == null) {
- compiledDeclarations = compileMetadata(mdTxnCtx,
- expr.getPrologDeclList(), true);
- }
- if (!compiledDeclarations.isConnectedToDataverse())
- compiledDeclarations.connectToDataverse(compiledDeclarations
- .getDataverseName());
- IDataFormat format = compiledDeclarations.getFormat();
- if (format == null) {
- throw new AlgebricksException("Data format has not been set.");
- }
- format.registerRuntimeFunctions();
- Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this,
- new MutableObject<ILogicalOperator>(
- new EmptyTupleSourceOperator()));
+ public ILogicalPlanAndMetadata translate(Query expr, AqlCompiledMetadataDeclarations compiledDeclarations)
+ throws AlgebricksException, AsterixException {
+ if (expr == null) {
+ return null;
+ }
+ if (compiledDeclarations == null) {
+ compiledDeclarations = compileMetadata(mdTxnCtx, expr.getPrologDeclList(), true);
+ }
+ if (!compiledDeclarations.isConnectedToDataverse())
+ compiledDeclarations.connectToDataverse(compiledDeclarations.getDataverseName());
+ IDataFormat format = compiledDeclarations.getFormat();
+ if (format == null) {
+ throw new AlgebricksException("Data format has not been set.");
+ }
+ format.registerRuntimeFunctions();
+ Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, new MutableObject<ILogicalOperator>(
+ new EmptyTupleSourceOperator()));
- ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<Mutable<ILogicalOperator>>();
+ ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<Mutable<ILogicalOperator>>();
- boolean isTransactionalWrite = false;
- ILogicalOperator topOp = p.first;
- ProjectOperator project = (ProjectOperator) topOp;
- LogicalVariable resVar = project.getVariables().get(0);
- if (outputDatasetName == null) {
- FileSplit outputFileSplit = compiledDeclarations.getOutputFile();
- if (outputFileSplit == null) {
- outputFileSplit = getDefaultOutputFileLocation();
- }
- compiledDeclarations.setOutputFile(outputFileSplit);
- List<Mutable<ILogicalExpression>> writeExprList = new ArrayList<Mutable<ILogicalExpression>>(
- 1);
- writeExprList.add(new MutableObject<ILogicalExpression>(
- new VariableReferenceExpression(resVar)));
- FileSplitSinkId fssi = new FileSplitSinkId(outputFileSplit);
- FileSplitDataSink sink = new FileSplitDataSink(fssi, null);
- topOp = new WriteOperator(writeExprList, sink);
- topOp.getInputs().add(new MutableObject<ILogicalOperator>(project));
- } else {
- String dataVerseName = compiledDeclarations.getDataverseName();
- Dataset dataset = compiledDeclarations
- .findDataset(outputDatasetName);
- if (dataset == null) {
- throw new AlgebricksException("Cannot find dataset "
- + outputDatasetName);
- }
+ boolean isTransactionalWrite = false;
+ ILogicalOperator topOp = p.first;
+ ProjectOperator project = (ProjectOperator) topOp;
+ LogicalVariable resVar = project.getVariables().get(0);
+ if (outputDatasetName == null) {
+ FileSplit outputFileSplit = compiledDeclarations.getOutputFile();
+ if (outputFileSplit == null) {
+ outputFileSplit = getDefaultOutputFileLocation();
+ }
+ compiledDeclarations.setOutputFile(outputFileSplit);
+ List<Mutable<ILogicalExpression>> writeExprList = new ArrayList<Mutable<ILogicalExpression>>(1);
+ writeExprList.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(resVar)));
+ FileSplitSinkId fssi = new FileSplitSinkId(outputFileSplit);
+ FileSplitDataSink sink = new FileSplitDataSink(fssi, null);
+ topOp = new WriteOperator(writeExprList, sink);
+ topOp.getInputs().add(new MutableObject<ILogicalOperator>(project));
+ } else {
+ String dataVerseName = compiledDeclarations.getDataverseName();
+ Dataset dataset = compiledDeclarations.findDataset(outputDatasetName);
+ if (dataset == null) {
+ throw new AlgebricksException("Cannot find dataset " + outputDatasetName);
+ }
- AqlSourceId sourceId = new AqlSourceId(dataVerseName,
- outputDatasetName);
- String itemTypeName = dataset.getItemTypeName();
- IAType itemType = compiledDeclarations.findType(itemTypeName);
- 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>>();
+ AqlSourceId sourceId = new AqlSourceId(dataVerseName, outputDatasetName);
+ String itemTypeName = dataset.getItemTypeName();
+ IAType itemType = compiledDeclarations.findType(itemTypeName);
+ 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>>();
- 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)))));
- f.substituteVar(METADATA_DUMMY_VAR, resVar);
- exprs.add(new MutableObject<ILogicalExpression>(f));
- LogicalVariable v = context.newVar();
- vars.add(v);
- varRefsForLoading.add(new MutableObject<ILogicalExpression>(
- new VariableReferenceExpression(v)));
- }
- AssignOperator assign = new AssignOperator(vars, exprs);
- assign.getInputs()
- .add(new MutableObject<ILogicalOperator>(project));
+ 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)))));
+ f.substituteVar(METADATA_DUMMY_VAR, resVar);
+ exprs.add(new MutableObject<ILogicalExpression>(f));
+ LogicalVariable v = context.newVar();
+ vars.add(v);
+ varRefsForLoading.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(v)));
+ }
+ AssignOperator assign = new AssignOperator(vars, exprs);
+ assign.getInputs().add(new MutableObject<ILogicalOperator>(project));
- Mutable<ILogicalExpression> varRef = new MutableObject<ILogicalExpression>(
- new VariableReferenceExpression(resVar));
- ILogicalOperator load = null;
+ Mutable<ILogicalExpression> varRef = new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
+ resVar));
+ ILogicalOperator load = null;
- switch (dmlKind) {
- case WRITE_FROM_QUERY_RESULT: {
- load = new WriteResultOperator(dataSource, varRef,
- varRefsForLoading);
- load.getInputs().add(
- new MutableObject<ILogicalOperator>(assign));
- break;
- }
- case INSERT: {
- ILogicalOperator insertOp = new InsertDeleteOperator(
- dataSource, varRef, varRefsForLoading,
- InsertDeleteOperator.Kind.INSERT);
- insertOp.getInputs().add(
- new MutableObject<ILogicalOperator>(assign));
- load = new SinkOperator();
- load.getInputs().add(
- new MutableObject<ILogicalOperator>(insertOp));
- isTransactionalWrite = true;
- break;
- }
- case DELETE: {
- ILogicalOperator deleteOp = new InsertDeleteOperator(
- dataSource, varRef, varRefsForLoading,
- InsertDeleteOperator.Kind.DELETE);
- deleteOp.getInputs().add(
- new MutableObject<ILogicalOperator>(assign));
- load = new SinkOperator();
- load.getInputs().add(
- new MutableObject<ILogicalOperator>(deleteOp));
- isTransactionalWrite = true;
- break;
- }
- case BEGIN_FEED: {
- ILogicalOperator insertOp = new InsertDeleteOperator(
- dataSource, varRef, varRefsForLoading,
- InsertDeleteOperator.Kind.INSERT);
- insertOp.getInputs().add(
- new MutableObject<ILogicalOperator>(assign));
- load = new SinkOperator();
- load.getInputs().add(
- new MutableObject<ILogicalOperator>(insertOp));
- isTransactionalWrite = false;
- break;
- }
- }
- topOp = load;
- }
+ switch (dmlKind) {
+ case WRITE_FROM_QUERY_RESULT: {
+ load = new WriteResultOperator(dataSource, varRef, varRefsForLoading);
+ load.getInputs().add(new MutableObject<ILogicalOperator>(assign));
+ break;
+ }
+ case INSERT: {
+ ILogicalOperator insertOp = new InsertDeleteOperator(dataSource, varRef, varRefsForLoading,
+ InsertDeleteOperator.Kind.INSERT);
+ insertOp.getInputs().add(new MutableObject<ILogicalOperator>(assign));
+ load = new SinkOperator();
+ load.getInputs().add(new MutableObject<ILogicalOperator>(insertOp));
+ isTransactionalWrite = true;
+ break;
+ }
+ case DELETE: {
+ ILogicalOperator deleteOp = new InsertDeleteOperator(dataSource, varRef, varRefsForLoading,
+ InsertDeleteOperator.Kind.DELETE);
+ deleteOp.getInputs().add(new MutableObject<ILogicalOperator>(assign));
+ load = new SinkOperator();
+ load.getInputs().add(new MutableObject<ILogicalOperator>(deleteOp));
+ isTransactionalWrite = true;
+ break;
+ }
+ case BEGIN_FEED: {
+ ILogicalOperator insertOp = new InsertDeleteOperator(dataSource, varRef, varRefsForLoading,
+ InsertDeleteOperator.Kind.INSERT);
+ insertOp.getInputs().add(new MutableObject<ILogicalOperator>(assign));
+ load = new SinkOperator();
+ load.getInputs().add(new MutableObject<ILogicalOperator>(insertOp));
+ isTransactionalWrite = false;
+ break;
+ }
+ }
+ topOp = load;
+ }
- globalPlanRoots.add(new MutableObject<ILogicalOperator>(topOp));
- ILogicalPlan plan = new ALogicalPlanImpl(globalPlanRoots);
- AqlMetadataProvider metadataProvider = new AqlMetadataProvider(txnId,
- isTransactionalWrite, compiledDeclarations);
- ILogicalPlanAndMetadata planAndMetadata = new AqlLogicalPlanAndMetadataImpl(
- plan, metadataProvider);
- return planAndMetadata;
- }
+ globalPlanRoots.add(new MutableObject<ILogicalOperator>(topOp));
+ ILogicalPlan plan = new ALogicalPlanImpl(globalPlanRoots);
+ AqlMetadataProvider metadataProvider = new AqlMetadataProvider(jobId, isTransactionalWrite,
+ compiledDeclarations);
+ ILogicalPlanAndMetadata planAndMetadata = new AqlLogicalPlanAndMetadataImpl(plan, metadataProvider);
+ return planAndMetadata;
+ }
- private FileSplit getDefaultOutputFileLocation() throws MetadataException {
- if (AsterixProperties.INSTANCE.getOutputDir() == null) {
- throw new MetadataException(
- "Output location for query result not specified at the time of deployment, must specify explicitly using 'write output to ..' statement");
- }
- String filePath = AsterixProperties.INSTANCE.getOutputDir()
- + System.getProperty("file.separator") + OUTPUT_FILE_PREFIX
- + outputFileID.incrementAndGet();
- return new FileSplit(AsterixProperties.INSTANCE.getMetadataNodeName(),
- new FileReference(new File(filePath)));
- }
+ private FileSplit getDefaultOutputFileLocation() throws MetadataException {
+ if (AsterixProperties.INSTANCE.getOutputDir() == null) {
+ throw new MetadataException(
+ "Output location for query result not specified at the time of deployment, must specify explicitly using 'write output to ..' statement");
+ }
+ String filePath = AsterixProperties.INSTANCE.getOutputDir() + System.getProperty("file.separator")
+ + OUTPUT_FILE_PREFIX + outputFileID.incrementAndGet();
+ return new FileSplit(AsterixProperties.INSTANCE.getMetadataNodeName(), new FileReference(new File(filePath)));
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitForClause(ForClause fc,
- Mutable<ILogicalOperator> tupSource) throws AsterixException {
- LogicalVariable v = context.newVar(fc.getVarExpr());
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitForClause(ForClause fc, Mutable<ILogicalOperator> tupSource)
+ throws AsterixException {
+ LogicalVariable v = context.newVar(fc.getVarExpr());
- Expression inExpr = fc.getInExpr();
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
- inExpr, tupSource);
- ILogicalOperator returnedOp;
+ Expression inExpr = fc.getInExpr();
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(inExpr, tupSource);
+ ILogicalOperator returnedOp;
- if (fc.getPosVarExpr() == null) {
- 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.getInputs().add(eo.second);
+ if (fc.getPosVarExpr() == null) {
+ 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.getInputs().add(eo.second);
- return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
- }
+ return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitLetClause(LetClause lc,
- Mutable<ILogicalOperator> tupSource) throws AsterixException {
- LogicalVariable v;
- ILogicalOperator returnedOp;
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitLetClause(LetClause lc, Mutable<ILogicalOperator> tupSource)
+ throws AsterixException {
+ LogicalVariable v;
+ ILogicalOperator returnedOp;
- switch (lc.getBindingExpr().getKind()) {
- 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.getInputs().add(tupSource);
- break;
- }
- default: {
- v = context.newVar(lc.getVarExpr());
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
- lc.getBindingExpr(), tupSource);
- returnedOp = new AssignOperator(v,
- new MutableObject<ILogicalExpression>(eo.first));
- returnedOp.getInputs().add(eo.second);
- break;
- }
- }
- return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
- }
+ switch (lc.getBindingExpr().getKind()) {
+ 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.getInputs().add(tupSource);
+ break;
+ }
+ default: {
+ v = context.newVar(lc.getVarExpr());
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(lc.getBindingExpr(),
+ tupSource);
+ returnedOp = new AssignOperator(v, new MutableObject<ILogicalExpression>(eo.first));
+ returnedOp.getInputs().add(eo.second);
+ break;
+ }
+ }
+ return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitFlworExpression(
- FLWOGRExpression flwor, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
- Mutable<ILogicalOperator> flworPlan = tupSource;
- boolean isTop = context.isTopFlwor();
- if (isTop) {
- context.setTopFlwor(false);
- }
- for (Clause c : flwor.getClauseList()) {
- Pair<ILogicalOperator, LogicalVariable> pC = c.accept(this,
- flworPlan);
- flworPlan = new MutableObject<ILogicalOperator>(pC.first);
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitFlworExpression(FLWOGRExpression flwor,
+ Mutable<ILogicalOperator> tupSource) throws AsterixException {
+ Mutable<ILogicalOperator> flworPlan = tupSource;
+ boolean isTop = context.isTopFlwor();
+ if (isTop) {
+ context.setTopFlwor(false);
+ }
+ for (Clause c : flwor.getClauseList()) {
+ Pair<ILogicalOperator, LogicalVariable> pC = c.accept(this, flworPlan);
+ flworPlan = new MutableObject<ILogicalOperator>(pC.first);
+ }
- Expression r = flwor.getReturnExpr();
- boolean noFlworClause = flwor.noForClause();
+ Expression r = flwor.getReturnExpr();
+ boolean noFlworClause = flwor.noForClause();
- if (r.getKind() == Kind.VARIABLE_EXPRESSION) {
- VariableExpr v = (VariableExpr) r;
- LogicalVariable var = context.getVar(v.getVar().getId());
+ if (r.getKind() == Kind.VARIABLE_EXPRESSION) {
+ VariableExpr v = (VariableExpr) r;
+ LogicalVariable var = context.getVar(v.getVar().getId());
- return produceFlwrResult(noFlworClause, isTop, flworPlan, var);
+ return produceFlwrResult(noFlworClause, isTop, flworPlan, var);
- } else {
- Mutable<ILogicalOperator> baseOp = new MutableObject<ILogicalOperator>(
- flworPlan.getValue());
- Pair<ILogicalOperator, LogicalVariable> rRes = r.accept(this,
- baseOp);
- ILogicalOperator rOp = rRes.first;
- ILogicalOperator resOp;
- if (expressionNeedsNoNesting(r)) {
- baseOp.setValue(flworPlan.getValue());
- resOp = rOp;
- } else {
- SubplanOperator s = new SubplanOperator(rOp);
- s.getInputs().add(flworPlan);
- resOp = s;
- baseOp.setValue(new NestedTupleSourceOperator(
- new MutableObject<ILogicalOperator>(s)));
- }
- Mutable<ILogicalOperator> resOpRef = new MutableObject<ILogicalOperator>(
- resOp);
- return produceFlwrResult(noFlworClause, isTop, resOpRef,
- rRes.second);
- }
- }
+ } else {
+ Mutable<ILogicalOperator> baseOp = new MutableObject<ILogicalOperator>(flworPlan.getValue());
+ Pair<ILogicalOperator, LogicalVariable> rRes = r.accept(this, baseOp);
+ ILogicalOperator rOp = rRes.first;
+ ILogicalOperator resOp;
+ if (expressionNeedsNoNesting(r)) {
+ baseOp.setValue(flworPlan.getValue());
+ resOp = rOp;
+ } else {
+ SubplanOperator s = new SubplanOperator(rOp);
+ s.getInputs().add(flworPlan);
+ resOp = s;
+ baseOp.setValue(new NestedTupleSourceOperator(new MutableObject<ILogicalOperator>(s)));
+ }
+ Mutable<ILogicalOperator> resOpRef = new MutableObject<ILogicalOperator>(resOp);
+ return produceFlwrResult(noFlworClause, isTop, resOpRef, rRes.second);
+ }
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitFieldAccessor(
- FieldAccessor fa, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
- fa.getExpr(), tupSource);
- LogicalVariable v = context.newVar();
- AbstractFunctionCallExpression fldAccess = new ScalarFunctionCallExpression(
- FunctionUtils
- .getFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME));
- fldAccess.getArguments().add(
- new MutableObject<ILogicalExpression>(p.first));
- ILogicalExpression faExpr = new ConstantExpression(
- new AsterixConstantValue(new AString(fa.getIdent().getValue())));
- fldAccess.getArguments().add(
- new MutableObject<ILogicalExpression>(faExpr));
- AssignOperator a = new AssignOperator(v,
- new MutableObject<ILogicalExpression>(fldAccess));
- a.getInputs().add(p.second);
- return new Pair<ILogicalOperator, LogicalVariable>(a, v);
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitFieldAccessor(FieldAccessor fa,
+ Mutable<ILogicalOperator> tupSource) throws AsterixException {
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(fa.getExpr(), tupSource);
+ LogicalVariable v = context.newVar();
+ AbstractFunctionCallExpression fldAccess = new ScalarFunctionCallExpression(
+ FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME));
+ fldAccess.getArguments().add(new MutableObject<ILogicalExpression>(p.first));
+ ILogicalExpression faExpr = new ConstantExpression(new AsterixConstantValue(new AString(fa.getIdent()
+ .getValue())));
+ fldAccess.getArguments().add(new MutableObject<ILogicalExpression>(faExpr));
+ AssignOperator a = new AssignOperator(v, new MutableObject<ILogicalExpression>(fldAccess));
+ a.getInputs().add(p.second);
+ return new Pair<ILogicalOperator, LogicalVariable>(a, v);
- }
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitIndexAccessor(
- IndexAccessor ia, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
- ia.getExpr(), tupSource);
- LogicalVariable v = context.newVar();
- AbstractFunctionCallExpression f;
- int i = ia.getIndex();
- if (i == IndexAccessor.ANY) {
- f = new ScalarFunctionCallExpression(
- FunctionUtils
- .getFunctionInfo(AsterixBuiltinFunctions.ANY_COLLECTION_MEMBER));
- f.getArguments()
- .add(new MutableObject<ILogicalExpression>(p.first));
- } else {
- 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)))));
- }
- 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> visitIndexAccessor(IndexAccessor ia,
+ Mutable<ILogicalOperator> tupSource) throws AsterixException {
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(ia.getExpr(), tupSource);
+ LogicalVariable v = context.newVar();
+ AbstractFunctionCallExpression f;
+ int i = ia.getIndex();
+ if (i == IndexAccessor.ANY) {
+ f = new ScalarFunctionCallExpression(
+ FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.ANY_COLLECTION_MEMBER));
+ f.getArguments().add(new MutableObject<ILogicalExpression>(p.first));
+ } else {
+ 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)))));
+ }
+ 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)
- throws AsterixException {
- LogicalVariable v = context.newVar();
- AsterixFunction fid = fcall.getIdent();
- List<Mutable<ILogicalExpression>> args = new ArrayList<Mutable<ILogicalExpression>>();
- Mutable<ILogicalOperator> topOp = tupSource;
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitCallExpr(CallExpr fcall, Mutable<ILogicalOperator> tupSource)
+ throws AsterixException {
+ LogicalVariable v = context.newVar();
+ AsterixFunction fid = fcall.getIdent();
+ List<Mutable<ILogicalExpression>> args = new ArrayList<Mutable<ILogicalExpression>>();
+ Mutable<ILogicalOperator> topOp = tupSource;
- for (Expression expr : fcall.getExprList()) {
- switch (expr.getKind()) {
- case VARIABLE_EXPRESSION: {
- LogicalVariable var = context.getVar(((VariableExpr) expr)
- .getVar().getId());
- args.add(new MutableObject<ILogicalExpression>(
- new VariableReferenceExpression(var)));
- break;
- }
- case LITERAL_EXPRESSION: {
- LiteralExpr val = (LiteralExpr) expr;
- args.add(new MutableObject<ILogicalExpression>(
- new ConstantExpression(
- new AsterixConstantValue(ConstantHelper
- .objectFromLiteral(val.getValue())))));
- break;
- }
- default: {
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
- expr, topOp);
- AbstractLogicalOperator o1 = (AbstractLogicalOperator) eo.second
- .getValue();
- args.add(new MutableObject<ILogicalExpression>(eo.first));
- if (o1 != null
- && !(o1.getOperatorTag() == LogicalOperatorTag.ASSIGN && hasOnlyChild(
- o1, topOp))) {
- topOp = eo.second;
- }
- break;
- }
- }
- }
+ for (Expression expr : fcall.getExprList()) {
+ switch (expr.getKind()) {
+ case VARIABLE_EXPRESSION: {
+ LogicalVariable var = context.getVar(((VariableExpr) expr).getVar().getId());
+ args.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(var)));
+ break;
+ }
+ case LITERAL_EXPRESSION: {
+ LiteralExpr val = (LiteralExpr) expr;
+ args.add(new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(
+ ConstantHelper.objectFromLiteral(val.getValue())))));
+ break;
+ }
+ default: {
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(expr, topOp);
+ AbstractLogicalOperator o1 = (AbstractLogicalOperator) eo.second.getValue();
+ args.add(new MutableObject<ILogicalExpression>(eo.first));
+ if (o1 != null && !(o1.getOperatorTag() == LogicalOperatorTag.ASSIGN && hasOnlyChild(o1, topOp))) {
+ topOp = eo.second;
+ }
+ break;
+ }
+ }
+ }
- FunctionIdentifier fi = new FunctionIdentifier(
- AlgebricksBuiltinFunctions.ALGEBRICKS_NS, fid.getFunctionName());
- AsterixFunctionInfo afi = AsterixBuiltinFunctions.lookupFunction(fi);
- FunctionIdentifier builtinAquafi = afi == null ? null : afi
- .getFunctionIdentifier();
+ FunctionIdentifier fi = new FunctionIdentifier(AlgebricksBuiltinFunctions.ALGEBRICKS_NS, fid.getFunctionName());
+ AsterixFunctionInfo afi = AsterixBuiltinFunctions.lookupFunction(fi);
+ FunctionIdentifier builtinAquafi = afi == null ? null : afi.getFunctionIdentifier();
- if (builtinAquafi != null) {
- fi = builtinAquafi;
- } else {
- fi = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
- fid.getFunctionName());
- FunctionIdentifier builtinAsterixFi = AsterixBuiltinFunctions
- .getBuiltinFunctionIdentifier(fi);
- if (builtinAsterixFi != null) {
- fi = builtinAsterixFi;
- }
- }
- AbstractFunctionCallExpression f;
- if (AsterixBuiltinFunctions.isBuiltinAggregateFunction(fi)) {
- f = AsterixBuiltinFunctions.makeAggregateFunctionExpression(fi,
- args);
- } else if (AsterixBuiltinFunctions.isBuiltinUnnestingFunction(fi)) {
- UnnestingFunctionCallExpression ufce = new UnnestingFunctionCallExpression(
- FunctionUtils.getFunctionInfo(fi), args);
- ufce.setReturnsUniqueValues(AsterixBuiltinFunctions
- .returnsUniqueValues(fi));
- f = ufce;
- } else {
- f = new ScalarFunctionCallExpression(
- FunctionUtils.getFunctionInfo(fi), args);
- }
- // Put hints into function call expr.
- if (fcall.hasHints()) {
- for (IExpressionAnnotation hint : fcall.getHints()) {
- f.getAnnotations().put(hint, hint);
- }
- }
- AssignOperator op = new AssignOperator(v,
- new MutableObject<ILogicalExpression>(f));
- if (topOp != null) {
- op.getInputs().add(topOp);
- }
+ if (builtinAquafi != null) {
+ fi = builtinAquafi;
+ } else {
+ fi = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, fid.getFunctionName());
+ FunctionIdentifier builtinAsterixFi = AsterixBuiltinFunctions.getBuiltinFunctionIdentifier(fi);
+ if (builtinAsterixFi != null) {
+ fi = builtinAsterixFi;
+ }
+ }
+ AbstractFunctionCallExpression f;
+ if (AsterixBuiltinFunctions.isBuiltinAggregateFunction(fi)) {
+ f = AsterixBuiltinFunctions.makeAggregateFunctionExpression(fi, args);
+ } else if (AsterixBuiltinFunctions.isBuiltinUnnestingFunction(fi)) {
+ UnnestingFunctionCallExpression ufce = new UnnestingFunctionCallExpression(
+ FunctionUtils.getFunctionInfo(fi), args);
+ ufce.setReturnsUniqueValues(AsterixBuiltinFunctions.returnsUniqueValues(fi));
+ f = ufce;
+ } else {
+ f = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(fi), args);
+ }
+ // Put hints into function call expr.
+ if (fcall.hasHints()) {
+ for (IExpressionAnnotation hint : fcall.getHints()) {
+ f.getAnnotations().put(hint, hint);
+ }
+ }
+ AssignOperator op = new AssignOperator(v, new MutableObject<ILogicalExpression>(f));
+ if (topOp != null) {
+ op.getInputs().add(topOp);
+ }
- return new Pair<ILogicalOperator, LogicalVariable>(op, v);
- }
+ return new Pair<ILogicalOperator, LogicalVariable>(op, v);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitFunctionDecl(
- FunctionDecl fd, Mutable<ILogicalOperator> tupSource) {
- // TODO Auto-generated method stub
- throw new NotImplementedException();
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitFunctionDecl(FunctionDecl fd,
+ Mutable<ILogicalOperator> tupSource) {
+ // TODO Auto-generated method stub
+ throw new NotImplementedException();
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitGroupbyClause(
- GroupbyClause gc, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
- GroupByOperator gOp = new GroupByOperator();
- Mutable<ILogicalOperator> topOp = tupSource;
- for (GbyVariableExpressionPair ve : gc.getGbyPairList()) {
- LogicalVariable v;
- VariableExpr vexpr = ve.getVar();
- if (vexpr != null) {
- v = context.newVar(vexpr);
- } else {
- v = context.newVar();
- }
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
- ve.getExpr(), topOp);
- gOp.addGbyExpression(v, eo.first);
- topOp = eo.second;
- }
- for (GbyVariableExpressionPair ve : gc.getDecorPairList()) {
- LogicalVariable v;
- VariableExpr vexpr = ve.getVar();
- if (vexpr != null) {
- v = context.newVar(vexpr);
- } else {
- v = context.newVar();
- }
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
- ve.getExpr(), topOp);
- gOp.addDecorExpression(v, eo.first);
- topOp = eo.second;
- }
- gOp.getInputs().add(topOp);
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitGroupbyClause(GroupbyClause gc,
+ Mutable<ILogicalOperator> tupSource) throws AsterixException {
+ GroupByOperator gOp = new GroupByOperator();
+ Mutable<ILogicalOperator> topOp = tupSource;
+ for (GbyVariableExpressionPair ve : gc.getGbyPairList()) {
+ LogicalVariable v;
+ VariableExpr vexpr = ve.getVar();
+ if (vexpr != null) {
+ v = context.newVar(vexpr);
+ } else {
+ v = context.newVar();
+ }
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(ve.getExpr(), topOp);
+ gOp.addGbyExpression(v, eo.first);
+ topOp = eo.second;
+ }
+ for (GbyVariableExpressionPair ve : gc.getDecorPairList()) {
+ LogicalVariable v;
+ VariableExpr vexpr = ve.getVar();
+ if (vexpr != null) {
+ v = context.newVar(vexpr);
+ } else {
+ v = context.newVar();
+ }
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(ve.getExpr(), topOp);
+ gOp.addDecorExpression(v, eo.first);
+ topOp = eo.second;
+ }
+ gOp.getInputs().add(topOp);
- for (VariableExpr var : gc.getWithVarList()) {
- LogicalVariable aggVar = context.newVar();
- LogicalVariable oldVar = context.getVar(var);
- List<Mutable<ILogicalExpression>> flArgs = new ArrayList<Mutable<ILogicalExpression>>(
- 1);
- flArgs.add(new MutableObject<ILogicalExpression>(
- new VariableReferenceExpression(oldVar)));
- AggregateFunctionCallExpression fListify = AsterixBuiltinFunctions
- .makeAggregateFunctionExpression(
- AsterixBuiltinFunctions.LISTIFY, flArgs);
- AggregateOperator agg = new AggregateOperator(
- mkSingletonArrayList(aggVar),
- (List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(
- fListify)));
+ for (VariableExpr var : gc.getWithVarList()) {
+ LogicalVariable aggVar = context.newVar();
+ LogicalVariable oldVar = context.getVar(var);
+ List<Mutable<ILogicalExpression>> flArgs = new ArrayList<Mutable<ILogicalExpression>>(1);
+ flArgs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(oldVar)));
+ AggregateFunctionCallExpression fListify = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
+ AsterixBuiltinFunctions.LISTIFY, flArgs);
+ AggregateOperator agg = new AggregateOperator(mkSingletonArrayList(aggVar),
+ (List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(fListify)));
- agg.getInputs().add(
- 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
- // the one bound by the aggregation op.
- context.setVar(var, aggVar);
- }
+ agg.getInputs().add(
+ 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
+ // the one bound by the aggregation op.
+ context.setVar(var, aggVar);
+ }
- gOp.getAnnotations().put(OperatorAnnotations.USE_HASH_GROUP_BY,
- gc.hasHashGroupByHint());
- return new Pair<ILogicalOperator, LogicalVariable>(gOp, null);
- }
+ gOp.getAnnotations().put(OperatorAnnotations.USE_HASH_GROUP_BY, gc.hasHashGroupByHint());
+ return new Pair<ILogicalOperator, LogicalVariable>(gOp, null);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitIfExpr(IfExpr ifexpr,
- Mutable<ILogicalOperator> tupSource) throws AsterixException {
- // In the most general case, IfThenElse is translated in the following
- // way.
- //
- // We assign the result of the condition to one variable varCond.
- // We create one subplan which contains the plan for the "then" branch,
- // on top of which there is a selection whose condition is varCond.
- // Similarly, we create one subplan for the "else" branch, in which the
- // selection is not(varCond).
- // Finally, we concatenate the results. (??)
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitIfExpr(IfExpr ifexpr, Mutable<ILogicalOperator> tupSource)
+ throws AsterixException {
+ // In the most general case, IfThenElse is translated in the following
+ // way.
+ //
+ // We assign the result of the condition to one variable varCond.
+ // We create one subplan which contains the plan for the "then" branch,
+ // on top of which there is a selection whose condition is varCond.
+ // Similarly, we create one subplan for the "else" branch, in which the
+ // selection is not(varCond).
+ // Finally, we concatenate the results. (??)
- Pair<ILogicalOperator, LogicalVariable> pCond = ifexpr.getCondExpr()
- .accept(this, tupSource);
- ILogicalOperator opCond = pCond.first;
- LogicalVariable varCond = pCond.second;
+ Pair<ILogicalOperator, LogicalVariable> pCond = ifexpr.getCondExpr().accept(this, tupSource);
+ ILogicalOperator opCond = pCond.first;
+ LogicalVariable varCond = pCond.second;
- SubplanOperator sp = new SubplanOperator();
- Mutable<ILogicalOperator> nestedSource = new MutableObject<ILogicalOperator>(
- new NestedTupleSourceOperator(
- new MutableObject<ILogicalOperator>(sp)));
+ SubplanOperator sp = new SubplanOperator();
+ Mutable<ILogicalOperator> nestedSource = new MutableObject<ILogicalOperator>(new NestedTupleSourceOperator(
+ new MutableObject<ILogicalOperator>(sp)));
- Pair<ILogicalOperator, LogicalVariable> pThen = ifexpr.getThenExpr()
- .accept(this, nestedSource);
- SelectOperator sel1 = new SelectOperator(
- new MutableObject<ILogicalExpression>(
- new VariableReferenceExpression(varCond)));
- sel1.getInputs().add(new MutableObject<ILogicalOperator>(pThen.first));
+ Pair<ILogicalOperator, LogicalVariable> pThen = ifexpr.getThenExpr().accept(this, nestedSource);
+ 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);
- AbstractFunctionCallExpression notVarCond = new ScalarFunctionCallExpression(
- FunctionUtils.getFunctionInfo(AlgebricksBuiltinFunctions.NOT),
- new MutableObject<ILogicalExpression>(
- new VariableReferenceExpression(varCond)));
- SelectOperator sel2 = new SelectOperator(
- new MutableObject<ILogicalExpression>(notVarCond));
- sel2.getInputs().add(new MutableObject<ILogicalOperator>(pElse.first));
+ Pair<ILogicalOperator, LogicalVariable> pElse = ifexpr.getElseExpr().accept(this, nestedSource);
+ AbstractFunctionCallExpression notVarCond = new ScalarFunctionCallExpression(
+ FunctionUtils.getFunctionInfo(AlgebricksBuiltinFunctions.NOT), new MutableObject<ILogicalExpression>(
+ new VariableReferenceExpression(varCond)));
+ SelectOperator sel2 = new SelectOperator(new MutableObject<ILogicalExpression>(notVarCond));
+ sel2.getInputs().add(new MutableObject<ILogicalOperator>(pElse.first));
- ILogicalPlan p1 = new ALogicalPlanImpl(
- new MutableObject<ILogicalOperator>(sel1));
- sp.getNestedPlans().add(p1);
- ILogicalPlan p2 = new ALogicalPlanImpl(
- new MutableObject<ILogicalOperator>(sel2));
- sp.getNestedPlans().add(p2);
+ ILogicalPlan p1 = new ALogicalPlanImpl(new MutableObject<ILogicalOperator>(sel1));
+ sp.getNestedPlans().add(p1);
+ ILogicalPlan p2 = new ALogicalPlanImpl(new MutableObject<ILogicalOperator>(sel2));
+ sp.getNestedPlans().add(p2);
- Mutable<ILogicalOperator> opCondRef = new MutableObject<ILogicalOperator>(
- opCond);
- sp.getInputs().add(opCondRef);
+ Mutable<ILogicalOperator> opCondRef = new MutableObject<ILogicalOperator>(opCond);
+ sp.getInputs().add(opCondRef);
- 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)));
- AssignOperator a = new AssignOperator(resV,
- new MutableObject<ILogicalExpression>(concatNonNull));
- a.getInputs().add(new MutableObject<ILogicalOperator>(sp));
+ 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)));
+ AssignOperator a = new AssignOperator(resV, new MutableObject<ILogicalExpression>(concatNonNull));
+ a.getInputs().add(new MutableObject<ILogicalOperator>(sp));
- return new Pair<ILogicalOperator, LogicalVariable>(a, resV);
- }
+ return new Pair<ILogicalOperator, LogicalVariable>(a, resV);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitLiteralExpr(
- LiteralExpr l, Mutable<ILogicalOperator> tupSource) {
- LogicalVariable var = context.newVar();
- AssignOperator a = new AssignOperator(var,
- new MutableObject<ILogicalExpression>(new ConstantExpression(
- new AsterixConstantValue(ConstantHelper
- .objectFromLiteral(l.getValue())))));
- if (tupSource != null) {
- a.getInputs().add(tupSource);
- }
- return new Pair<ILogicalOperator, LogicalVariable>(a, var);
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitLiteralExpr(LiteralExpr l, Mutable<ILogicalOperator> tupSource) {
+ LogicalVariable var = context.newVar();
+ AssignOperator a = new AssignOperator(var, new MutableObject<ILogicalExpression>(new ConstantExpression(
+ new AsterixConstantValue(ConstantHelper.objectFromLiteral(l.getValue())))));
+ if (tupSource != null) {
+ a.getInputs().add(tupSource);
+ }
+ return new Pair<ILogicalOperator, LogicalVariable>(a, var);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitOperatorExpr(
- OperatorExpr op, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
- ArrayList<OperatorType> ops = op.getOpList();
- int nOps = ops.size();
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitOperatorExpr(OperatorExpr op,
+ Mutable<ILogicalOperator> tupSource) throws AsterixException {
+ ArrayList<OperatorType> ops = op.getOpList();
+ int nOps = ops.size();
- if (nOps > 0
- && (ops.get(0) == OperatorType.AND || ops.get(0) == OperatorType.OR)) {
- return visitAndOrOperator(op, tupSource);
- }
+ if (nOps > 0 && (ops.get(0) == OperatorType.AND || ops.get(0) == OperatorType.OR)) {
+ return visitAndOrOperator(op, tupSource);
+ }
- ArrayList<Expression> exprs = op.getExprList();
+ ArrayList<Expression> exprs = op.getExprList();
- Mutable<ILogicalOperator> topOp = tupSource;
+ Mutable<ILogicalOperator> topOp = tupSource;
- ILogicalExpression currExpr = null;
- for (int i = 0; i <= nOps; i++) {
+ ILogicalExpression currExpr = null;
+ for (int i = 0; i <= nOps; i++) {
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
- exprs.get(i), topOp);
- topOp = p.second;
- ILogicalExpression e = p.first;
- // now look at the operator
- if (i < nOps) {
- if (OperatorExpr.opIsComparison(ops.get(i))) {
- AbstractFunctionCallExpression c = createComparisonExpression(ops
- .get(i));
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(exprs.get(i), topOp);
+ topOp = p.second;
+ ILogicalExpression e = p.first;
+ // now look at the operator
+ if (i < nOps) {
+ if (OperatorExpr.opIsComparison(ops.get(i))) {
+ AbstractFunctionCallExpression c = createComparisonExpression(ops.get(i));
- // chain the operators
- if (i == 0) {
- c.getArguments().add(
- new MutableObject<ILogicalExpression>(e));
- currExpr = c;
- if (op.isBroadcastOperand(i)) {
- BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
- bcast.setObject(BroadcastSide.LEFT);
- c.getAnnotations()
- .put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY,
- bcast);
- }
- } else {
- ((AbstractFunctionCallExpression) currExpr)
- .getArguments()
- .add(new MutableObject<ILogicalExpression>(e));
- c.getArguments()
- .add(new MutableObject<ILogicalExpression>(
- currExpr));
- currExpr = c;
- if (i == 1 && op.isBroadcastOperand(i)) {
- BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
- bcast.setObject(BroadcastSide.RIGHT);
- c.getAnnotations()
- .put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY,
- bcast);
- }
- }
- } else {
- AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(ops
- .get(i));
+ // chain the operators
+ if (i == 0) {
+ c.getArguments().add(new MutableObject<ILogicalExpression>(e));
+ currExpr = c;
+ if (op.isBroadcastOperand(i)) {
+ BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
+ bcast.setObject(BroadcastSide.LEFT);
+ c.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
+ }
+ } else {
+ ((AbstractFunctionCallExpression) currExpr).getArguments().add(
+ new MutableObject<ILogicalExpression>(e));
+ c.getArguments().add(new MutableObject<ILogicalExpression>(currExpr));
+ currExpr = c;
+ if (i == 1 && op.isBroadcastOperand(i)) {
+ BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
+ bcast.setObject(BroadcastSide.RIGHT);
+ c.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
+ }
+ }
+ } else {
+ AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(ops.get(i));
- if (i == 0) {
- f.getArguments().add(
- new MutableObject<ILogicalExpression>(e));
- currExpr = f;
- } else {
- ((AbstractFunctionCallExpression) currExpr)
- .getArguments()
- .add(new MutableObject<ILogicalExpression>(e));
- f.getArguments()
- .add(new MutableObject<ILogicalExpression>(
- currExpr));
- currExpr = f;
- }
- }
- } else { // don't forget the last expression...
- ((AbstractFunctionCallExpression) currExpr).getArguments().add(
- new MutableObject<ILogicalExpression>(e));
- if (i == 1 && op.isBroadcastOperand(i)) {
- BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
- bcast.setObject(BroadcastSide.RIGHT);
- ((AbstractFunctionCallExpression) currExpr)
- .getAnnotations()
- .put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY,
- bcast);
- }
- }
- }
+ if (i == 0) {
+ f.getArguments().add(new MutableObject<ILogicalExpression>(e));
+ currExpr = f;
+ } else {
+ ((AbstractFunctionCallExpression) currExpr).getArguments().add(
+ new MutableObject<ILogicalExpression>(e));
+ f.getArguments().add(new MutableObject<ILogicalExpression>(currExpr));
+ currExpr = f;
+ }
+ }
+ } else { // don't forget the last expression...
+ ((AbstractFunctionCallExpression) currExpr).getArguments()
+ .add(new MutableObject<ILogicalExpression>(e));
+ if (i == 1 && op.isBroadcastOperand(i)) {
+ BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
+ bcast.setObject(BroadcastSide.RIGHT);
+ ((AbstractFunctionCallExpression) currExpr).getAnnotations().put(
+ BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
+ }
+ }
+ }
- // Add hints as annotations.
- if (op.hasHints() && currExpr instanceof AbstractFunctionCallExpression) {
- AbstractFunctionCallExpression currFuncExpr = (AbstractFunctionCallExpression) currExpr;
- for (IExpressionAnnotation hint : op.getHints()) {
- currFuncExpr.getAnnotations().put(hint, hint);
- }
- }
-
- LogicalVariable assignedVar = context.newVar();
- AssignOperator a = new AssignOperator(assignedVar,
- new MutableObject<ILogicalExpression>(currExpr));
+ // Add hints as annotations.
+ if (op.hasHints() && currExpr instanceof AbstractFunctionCallExpression) {
+ AbstractFunctionCallExpression currFuncExpr = (AbstractFunctionCallExpression) currExpr;
+ for (IExpressionAnnotation hint : op.getHints()) {
+ currFuncExpr.getAnnotations().put(hint, hint);
+ }
+ }
- a.getInputs().add(topOp);
+ LogicalVariable assignedVar = context.newVar();
+ AssignOperator a = new AssignOperator(assignedVar, new MutableObject<ILogicalExpression>(currExpr));
- return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
- }
+ a.getInputs().add(topOp);
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitOrderbyClause(
- OrderbyClause oc, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
+ return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
+ }
- OrderOperator ord = new OrderOperator();
- Iterator<OrderModifier> modifIter = oc.getModifierList().iterator();
- Mutable<ILogicalOperator> topOp = tupSource;
- for (Expression e : oc.getOrderbyList()) {
- 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)));
- topOp = p.second;
- }
- ord.getInputs().add(topOp);
- if (oc.getNumTuples() > 0) {
- ord.getAnnotations().put(OperatorAnnotations.CARDINALITY,
- oc.getNumTuples());
- }
- if (oc.getNumFrames() > 0) {
- ord.getAnnotations().put(OperatorAnnotations.MAX_NUMBER_FRAMES,
- oc.getNumFrames());
- }
- return new Pair<ILogicalOperator, LogicalVariable>(ord, null);
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitOrderbyClause(OrderbyClause oc,
+ Mutable<ILogicalOperator> tupSource) throws AsterixException {
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitQuantifiedExpression(
- QuantifiedExpression qe, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
- Mutable<ILogicalOperator> topOp = tupSource;
+ OrderOperator ord = new OrderOperator();
+ Iterator<OrderModifier> modifIter = oc.getModifierList().iterator();
+ Mutable<ILogicalOperator> topOp = tupSource;
+ for (Expression e : oc.getOrderbyList()) {
+ 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)));
+ topOp = p.second;
+ }
+ ord.getInputs().add(topOp);
+ if (oc.getNumTuples() > 0) {
+ ord.getAnnotations().put(OperatorAnnotations.CARDINALITY, oc.getNumTuples());
+ }
+ if (oc.getNumFrames() > 0) {
+ ord.getAnnotations().put(OperatorAnnotations.MAX_NUMBER_FRAMES, oc.getNumFrames());
+ }
+ return new Pair<ILogicalOperator, LogicalVariable>(ord, null);
+ }
- ILogicalOperator firstOp = null;
- Mutable<ILogicalOperator> lastOp = null;
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitQuantifiedExpression(QuantifiedExpression qe,
+ Mutable<ILogicalOperator> tupSource) throws AsterixException {
+ Mutable<ILogicalOperator> topOp = tupSource;
- for (QuantifiedPair qt : qe.getQuantifiedList()) {
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(
- qt.getExpr(), topOp);
- topOp = eo1.second;
- LogicalVariable uVar = context.newVar(qt.getVarExpr());
- ILogicalOperator u = new UnnestOperator(uVar,
- new MutableObject<ILogicalExpression>(
- makeUnnestExpression(eo1.first)));
+ ILogicalOperator firstOp = null;
+ Mutable<ILogicalOperator> lastOp = null;
- if (firstOp == null) {
- firstOp = u;
- }
- if (lastOp != null) {
- u.getInputs().add(lastOp);
- }
- lastOp = new MutableObject<ILogicalOperator>(u);
- }
+ for (QuantifiedPair qt : qe.getQuantifiedList()) {
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(qt.getExpr(), topOp);
+ topOp = eo1.second;
+ LogicalVariable uVar = context.newVar(qt.getVarExpr());
+ ILogicalOperator u = new UnnestOperator(uVar, new MutableObject<ILogicalExpression>(
+ makeUnnestExpression(eo1.first)));
- // We make all the unnest correspond. to quantif. vars. sit on top
- // in the hope of enabling joins & other optimiz.
- firstOp.getInputs().add(topOp);
- topOp = lastOp;
+ if (firstOp == null) {
+ firstOp = u;
+ }
+ if (lastOp != null) {
+ u.getInputs().add(lastOp);
+ }
+ lastOp = new MutableObject<ILogicalOperator>(u);
+ }
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(
- qe.getSatisfiesExpr(), topOp);
+ // We make all the unnest correspond. to quantif. vars. sit on top
+ // in the hope of enabling joins & other optimiz.
+ firstOp.getInputs().add(topOp);
+ topOp = lastOp;
- AggregateFunctionCallExpression fAgg;
- SelectOperator s;
- if (qe.getQuantifier() == Quantifier.SOME) {
- s = new SelectOperator(new MutableObject<ILogicalExpression>(
- eo2.first));
- s.getInputs().add(eo2.second);
- fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
- AsterixBuiltinFunctions.NON_EMPTY_STREAM,
- new ArrayList<Mutable<ILogicalExpression>>());
- } else { // EVERY
- List<Mutable<ILogicalExpression>> satExprList = new ArrayList<Mutable<ILogicalExpression>>(
- 1);
- satExprList.add(new MutableObject<ILogicalExpression>(eo2.first));
- s = new SelectOperator(new MutableObject<ILogicalExpression>(
- new ScalarFunctionCallExpression(FunctionUtils
- .getFunctionInfo(AlgebricksBuiltinFunctions.NOT),
- satExprList)));
- s.getInputs().add(eo2.second);
- fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
- AsterixBuiltinFunctions.EMPTY_STREAM,
- new ArrayList<Mutable<ILogicalExpression>>());
- }
- LogicalVariable qeVar = context.newVar();
- AggregateOperator a = new AggregateOperator(
- mkSingletonArrayList(qeVar),
- (List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(
- fAgg)));
- a.getInputs().add(new MutableObject<ILogicalOperator>(s));
- return new Pair<ILogicalOperator, LogicalVariable>(a, qeVar);
- }
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(qe.getSatisfiesExpr(), topOp);
+
+ AggregateFunctionCallExpression fAgg;
+ SelectOperator s;
+ if (qe.getQuantifier() == Quantifier.SOME) {
+ s = new SelectOperator(new MutableObject<ILogicalExpression>(eo2.first));
+ s.getInputs().add(eo2.second);
+ fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(AsterixBuiltinFunctions.NON_EMPTY_STREAM,
+ new ArrayList<Mutable<ILogicalExpression>>());
+ } else { // EVERY
+ List<Mutable<ILogicalExpression>> satExprList = new ArrayList<Mutable<ILogicalExpression>>(1);
+ satExprList.add(new MutableObject<ILogicalExpression>(eo2.first));
+ s = new SelectOperator(new MutableObject<ILogicalExpression>(new ScalarFunctionCallExpression(
+ FunctionUtils.getFunctionInfo(AlgebricksBuiltinFunctions.NOT), satExprList)));
+ s.getInputs().add(eo2.second);
+ fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(AsterixBuiltinFunctions.EMPTY_STREAM,
+ new ArrayList<Mutable<ILogicalExpression>>());
+ }
+ LogicalVariable qeVar = context.newVar();
+ AggregateOperator a = new AggregateOperator(mkSingletonArrayList(qeVar),
+ (List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(fAgg)));
+ a.getInputs().add(new MutableObject<ILogicalOperator>(s));
+ return new Pair<ILogicalOperator, LogicalVariable>(a, qeVar);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitQuery(Query q,
- Mutable<ILogicalOperator> tupSource) throws AsterixException {
- return q.getBody().accept(this, tupSource);
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitQuery(Query q, Mutable<ILogicalOperator> tupSource)
+ throws AsterixException {
+ return q.getBody().accept(this, tupSource);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitRecordConstructor(
- RecordConstructor rc, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
- AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(
- FunctionUtils
- .getFunctionInfo(AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR));
- LogicalVariable v1 = context.newVar();
- AssignOperator a = new AssignOperator(v1,
- new MutableObject<ILogicalExpression>(f));
- Mutable<ILogicalOperator> topOp = tupSource;
- for (FieldBinding fb : rc.getFbList()) {
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(
- fb.getLeftExpr(), topOp);
- f.getArguments().add(
- new MutableObject<ILogicalExpression>(eo1.first));
- topOp = eo1.second;
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(
- fb.getRightExpr(), topOp);
- f.getArguments().add(
- new MutableObject<ILogicalExpression>(eo2.first));
- topOp = eo2.second;
- }
- a.getInputs().add(topOp);
- return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitRecordConstructor(RecordConstructor rc,
+ Mutable<ILogicalOperator> tupSource) throws AsterixException {
+ AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(
+ FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR));
+ LogicalVariable v1 = context.newVar();
+ AssignOperator a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(f));
+ Mutable<ILogicalOperator> topOp = tupSource;
+ for (FieldBinding fb : rc.getFbList()) {
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(fb.getLeftExpr(), topOp);
+ f.getArguments().add(new MutableObject<ILogicalExpression>(eo1.first));
+ topOp = eo1.second;
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(fb.getRightExpr(), topOp);
+ f.getArguments().add(new MutableObject<ILogicalExpression>(eo2.first));
+ topOp = eo2.second;
+ }
+ a.getInputs().add(topOp);
+ return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitListConstructor(
- ListConstructor lc, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
- FunctionIdentifier fid = (lc.getType() == Type.ORDERED_LIST_CONSTRUCTOR) ? AsterixBuiltinFunctions.ORDERED_LIST_CONSTRUCTOR
- : AsterixBuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR;
- AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(
- FunctionUtils.getFunctionInfo(fid));
- LogicalVariable v1 = context.newVar();
- AssignOperator a = new AssignOperator(v1,
- new MutableObject<ILogicalExpression>(f));
- Mutable<ILogicalOperator> topOp = tupSource;
- for (Expression expr : lc.getExprList()) {
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
- expr, topOp);
- f.getArguments().add(
- new MutableObject<ILogicalExpression>(eo.first));
- topOp = eo.second;
- }
- a.getInputs().add(topOp);
- return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitListConstructor(ListConstructor lc,
+ Mutable<ILogicalOperator> tupSource) throws AsterixException {
+ FunctionIdentifier fid = (lc.getType() == Type.ORDERED_LIST_CONSTRUCTOR) ? AsterixBuiltinFunctions.ORDERED_LIST_CONSTRUCTOR
+ : AsterixBuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR;
+ AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(fid));
+ LogicalVariable v1 = context.newVar();
+ AssignOperator a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(f));
+ Mutable<ILogicalOperator> topOp = tupSource;
+ for (Expression expr : lc.getExprList()) {
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(expr, topOp);
+ f.getArguments().add(new MutableObject<ILogicalExpression>(eo.first));
+ topOp = eo.second;
+ }
+ a.getInputs().add(topOp);
+ return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitUnaryExpr(UnaryExpr u,
- Mutable<ILogicalOperator> tupSource) throws AsterixException {
- Expression expr = u.getExpr();
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
- expr, tupSource);
- LogicalVariable v1 = context.newVar();
- AssignOperator a;
- if (u.getSign() == Sign.POSITIVE) {
- a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(
- eo.first));
- } else {
- AbstractFunctionCallExpression m = new ScalarFunctionCallExpression(
- FunctionUtils
- .getFunctionInfo(AsterixBuiltinFunctions.NUMERIC_UNARY_MINUS));
- m.getArguments().add(
- new MutableObject<ILogicalExpression>(eo.first));
- a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(m));
- }
- a.getInputs().add(eo.second);
- return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitUnaryExpr(UnaryExpr u, Mutable<ILogicalOperator> tupSource)
+ throws AsterixException {
+ Expression expr = u.getExpr();
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(expr, tupSource);
+ LogicalVariable v1 = context.newVar();
+ AssignOperator a;
+ if (u.getSign() == Sign.POSITIVE) {
+ a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(eo.first));
+ } else {
+ AbstractFunctionCallExpression m = new ScalarFunctionCallExpression(
+ FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.NUMERIC_UNARY_MINUS));
+ m.getArguments().add(new MutableObject<ILogicalExpression>(eo.first));
+ a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(m));
+ }
+ a.getInputs().add(eo.second);
+ return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitVariableExpr(
- VariableExpr v, Mutable<ILogicalOperator> tupSource) {
- // 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)));
- a.getInputs().add(tupSource);
- return new Pair<ILogicalOperator, LogicalVariable>(a, var);
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitVariableExpr(VariableExpr v, Mutable<ILogicalOperator> tupSource) {
+ // 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)));
+ a.getInputs().add(tupSource);
+ return new Pair<ILogicalOperator, LogicalVariable>(a, var);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitWhereClause(
- WhereClause w, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
- w.getWhereExpr(), tupSource);
- SelectOperator s = new SelectOperator(
- new MutableObject<ILogicalExpression>(p.first));
- s.getInputs().add(p.second);
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitWhereClause(WhereClause w, Mutable<ILogicalOperator> tupSource)
+ throws AsterixException {
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(w.getWhereExpr(), tupSource);
+ SelectOperator s = new SelectOperator(new MutableObject<ILogicalExpression>(p.first));
+ s.getInputs().add(p.second);
- return new Pair<ILogicalOperator, LogicalVariable>(s, null);
- }
+ return new Pair<ILogicalOperator, LogicalVariable>(s, null);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitLimitClause(
- LimitClause lc, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(
- lc.getLimitExpr(), tupSource);
- LimitOperator opLim;
- Expression offset = lc.getOffset();
- if (offset != null) {
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> p2 = aqlExprToAlgExpression(
- offset, p1.second);
- opLim = new LimitOperator(p1.first, p2.first);
- opLim.getInputs().add(p2.second);
- } else {
- opLim = new LimitOperator(p1.first);
- opLim.getInputs().add(p1.second);
- }
- return new Pair<ILogicalOperator, LogicalVariable>(opLim, null);
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitLimitClause(LimitClause lc, Mutable<ILogicalOperator> tupSource)
+ throws AsterixException {
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(lc.getLimitExpr(), tupSource);
+ LimitOperator opLim;
+ Expression offset = lc.getOffset();
+ if (offset != null) {
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> p2 = aqlExprToAlgExpression(offset, p1.second);
+ opLim = new LimitOperator(p1.first, p2.first);
+ opLim.getInputs().add(p2.second);
+ } else {
+ opLim = new LimitOperator(p1.first);
+ opLim.getInputs().add(p1.second);
+ }
+ return new Pair<ILogicalOperator, LogicalVariable>(opLim, null);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitDieClause(DieClause lc,
- Mutable<ILogicalOperator> tupSource) throws AsterixException {
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(
- lc.getDieExpr(), tupSource);
- DieOperator opDie = new DieOperator(p1.first);
- opDie.getInputs().add(p1.second);
- return new Pair<ILogicalOperator, LogicalVariable>(opDie, null);
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitDieClause(DieClause lc, Mutable<ILogicalOperator> tupSource)
+ throws AsterixException {
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(lc.getDieExpr(), tupSource);
+ DieOperator opDie = new DieOperator(p1.first);
+ opDie.getInputs().add(p1.second);
+ return new Pair<ILogicalOperator, LogicalVariable>(opDie, null);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitDistinctClause(
- DistinctClause dc, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
- List<Mutable<ILogicalExpression>> exprList = new ArrayList<Mutable<ILogicalExpression>>();
- Mutable<ILogicalOperator> input = null;
- for (Expression expr : dc.getDistinctByExpr()) {
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
- expr, tupSource);
- exprList.add(new MutableObject<ILogicalExpression>(p.first));
- input = p.second;
- }
- DistinctOperator opDistinct = new DistinctOperator(exprList);
- opDistinct.getInputs().add(input);
- return new Pair<ILogicalOperator, LogicalVariable>(opDistinct, null);
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitDistinctClause(DistinctClause dc,
+ Mutable<ILogicalOperator> tupSource) throws AsterixException {
+ List<Mutable<ILogicalExpression>> exprList = new ArrayList<Mutable<ILogicalExpression>>();
+ Mutable<ILogicalOperator> input = null;
+ for (Expression expr : dc.getDistinctByExpr()) {
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(expr, tupSource);
+ exprList.add(new MutableObject<ILogicalExpression>(p.first));
+ input = p.second;
+ }
+ DistinctOperator opDistinct = new DistinctOperator(exprList);
+ opDistinct.getInputs().add(input);
+ return new Pair<ILogicalOperator, LogicalVariable>(opDistinct, null);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitUnionExpr(
- UnionExpr unionExpr, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
- Mutable<ILogicalOperator> ts = tupSource;
- ILogicalOperator lastOp = null;
- LogicalVariable lastVar = null;
- boolean first = true;
- for (Expression e : unionExpr.getExprs()) {
- if (first) {
- first = false;
- } else {
- ts = new MutableObject<ILogicalOperator>(
- new EmptyTupleSourceOperator());
- }
- Pair<ILogicalOperator, LogicalVariable> p1 = e.accept(this, ts);
- if (lastOp == null) {
- lastOp = p1.first;
- lastVar = p1.second;
- } else {
- LogicalVariable unnestVar1 = context.newVar();
- UnnestOperator unnest1 = new UnnestOperator(
- unnestVar1,
- new MutableObject<ILogicalExpression>(
- makeUnnestExpression(new VariableReferenceExpression(
- lastVar))));
- unnest1.getInputs().add(
- new MutableObject<ILogicalOperator>(lastOp));
- LogicalVariable unnestVar2 = context.newVar();
- UnnestOperator unnest2 = new UnnestOperator(
- unnestVar2,
- new MutableObject<ILogicalExpression>(
- makeUnnestExpression(new VariableReferenceExpression(
- p1.second))));
- unnest2.getInputs().add(
- new MutableObject<ILogicalOperator>(p1.first));
- List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap = new ArrayList<Triple<LogicalVariable, LogicalVariable, LogicalVariable>>(
- 1);
- LogicalVariable resultVar = context.newVar();
- Triple<LogicalVariable, LogicalVariable, LogicalVariable> triple = new Triple<LogicalVariable, LogicalVariable, LogicalVariable>(
- unnestVar1, unnestVar2, resultVar);
- varMap.add(triple);
- UnionAllOperator unionOp = new UnionAllOperator(varMap);
- unionOp.getInputs().add(
- new MutableObject<ILogicalOperator>(unnest1));
- unionOp.getInputs().add(
- new MutableObject<ILogicalOperator>(unnest2));
- lastVar = resultVar;
- lastOp = unionOp;
- }
- }
- LogicalVariable aggVar = context.newVar();
- ArrayList<LogicalVariable> aggregVars = new ArrayList<LogicalVariable>(
- 1);
- aggregVars.add(aggVar);
- List<Mutable<ILogicalExpression>> afcExprs = new ArrayList<Mutable<ILogicalExpression>>(
- 1);
- afcExprs.add(new MutableObject<ILogicalExpression>(
- new VariableReferenceExpression(lastVar)));
- AggregateFunctionCallExpression afc = AsterixBuiltinFunctions
- .makeAggregateFunctionExpression(
- AsterixBuiltinFunctions.LISTIFY, afcExprs);
- ArrayList<Mutable<ILogicalExpression>> aggregExprs = new ArrayList<Mutable<ILogicalExpression>>(
- 1);
- aggregExprs.add(new MutableObject<ILogicalExpression>(afc));
- AggregateOperator agg = new AggregateOperator(aggregVars, aggregExprs);
- agg.getInputs().add(new MutableObject<ILogicalOperator>(lastOp));
- return new Pair<ILogicalOperator, LogicalVariable>(agg, aggVar);
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitUnionExpr(UnionExpr unionExpr,
+ Mutable<ILogicalOperator> tupSource) throws AsterixException {
+ Mutable<ILogicalOperator> ts = tupSource;
+ ILogicalOperator lastOp = null;
+ LogicalVariable lastVar = null;
+ boolean first = true;
+ for (Expression e : unionExpr.getExprs()) {
+ if (first) {
+ first = false;
+ } else {
+ ts = new MutableObject<ILogicalOperator>(new EmptyTupleSourceOperator());
+ }
+ Pair<ILogicalOperator, LogicalVariable> p1 = e.accept(this, ts);
+ if (lastOp == null) {
+ lastOp = p1.first;
+ lastVar = p1.second;
+ } else {
+ LogicalVariable unnestVar1 = context.newVar();
+ UnnestOperator unnest1 = new UnnestOperator(unnestVar1, new MutableObject<ILogicalExpression>(
+ makeUnnestExpression(new VariableReferenceExpression(lastVar))));
+ unnest1.getInputs().add(new MutableObject<ILogicalOperator>(lastOp));
+ LogicalVariable unnestVar2 = context.newVar();
+ UnnestOperator unnest2 = new UnnestOperator(unnestVar2, new MutableObject<ILogicalExpression>(
+ makeUnnestExpression(new VariableReferenceExpression(p1.second))));
+ unnest2.getInputs().add(new MutableObject<ILogicalOperator>(p1.first));
+ List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap = new ArrayList<Triple<LogicalVariable, LogicalVariable, LogicalVariable>>(
+ 1);
+ LogicalVariable resultVar = context.newVar();
+ Triple<LogicalVariable, LogicalVariable, LogicalVariable> triple = new Triple<LogicalVariable, LogicalVariable, LogicalVariable>(
+ unnestVar1, unnestVar2, resultVar);
+ varMap.add(triple);
+ UnionAllOperator unionOp = new UnionAllOperator(varMap);
+ unionOp.getInputs().add(new MutableObject<ILogicalOperator>(unnest1));
+ unionOp.getInputs().add(new MutableObject<ILogicalOperator>(unnest2));
+ lastVar = resultVar;
+ lastOp = unionOp;
+ }
+ }
+ LogicalVariable aggVar = context.newVar();
+ ArrayList<LogicalVariable> aggregVars = new ArrayList<LogicalVariable>(1);
+ aggregVars.add(aggVar);
+ List<Mutable<ILogicalExpression>> afcExprs = new ArrayList<Mutable<ILogicalExpression>>(1);
+ afcExprs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(lastVar)));
+ AggregateFunctionCallExpression afc = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
+ AsterixBuiltinFunctions.LISTIFY, afcExprs);
+ ArrayList<Mutable<ILogicalExpression>> aggregExprs = new ArrayList<Mutable<ILogicalExpression>>(1);
+ aggregExprs.add(new MutableObject<ILogicalExpression>(afc));
+ AggregateOperator agg = new AggregateOperator(aggregVars, aggregExprs);
+ agg.getInputs().add(new MutableObject<ILogicalOperator>(lastOp));
+ return new Pair<ILogicalOperator, LogicalVariable>(agg, aggVar);
+ }
- private AbstractFunctionCallExpression createComparisonExpression(
- OperatorType t) {
- FunctionIdentifier fi = operatorTypeToFunctionIdentifier(t);
- IFunctionInfo finfo = FunctionUtils.getFunctionInfo(fi);
- return new ScalarFunctionCallExpression(finfo);
- }
+ private AbstractFunctionCallExpression createComparisonExpression(OperatorType t) {
+ FunctionIdentifier fi = operatorTypeToFunctionIdentifier(t);
+ IFunctionInfo finfo = FunctionUtils.getFunctionInfo(fi);
+ return new ScalarFunctionCallExpression(finfo);
+ }
- private FunctionIdentifier operatorTypeToFunctionIdentifier(OperatorType t) {
- switch (t) {
- case EQ: {
- return AlgebricksBuiltinFunctions.EQ;
- }
- case NEQ: {
- return AlgebricksBuiltinFunctions.NEQ;
- }
- case GT: {
- return AlgebricksBuiltinFunctions.GT;
- }
- case GE: {
- return AlgebricksBuiltinFunctions.GE;
- }
- case LT: {
- return AlgebricksBuiltinFunctions.LT;
- }
- case LE: {
- return AlgebricksBuiltinFunctions.LE;
- }
- default: {
- throw new IllegalStateException();
- }
- }
- }
+ private FunctionIdentifier operatorTypeToFunctionIdentifier(OperatorType t) {
+ switch (t) {
+ case EQ: {
+ return AlgebricksBuiltinFunctions.EQ;
+ }
+ case NEQ: {
+ return AlgebricksBuiltinFunctions.NEQ;
+ }
+ case GT: {
+ return AlgebricksBuiltinFunctions.GT;
+ }
+ case GE: {
+ return AlgebricksBuiltinFunctions.GE;
+ }
+ case LT: {
+ return AlgebricksBuiltinFunctions.LT;
+ }
+ case LE: {
+ return AlgebricksBuiltinFunctions.LE;
+ }
+ default: {
+ throw new IllegalStateException();
+ }
+ }
+ }
- private AbstractFunctionCallExpression createFunctionCallExpressionForBuiltinOperator(
- OperatorType t) throws AsterixException {
+ private AbstractFunctionCallExpression createFunctionCallExpressionForBuiltinOperator(OperatorType t)
+ throws AsterixException {
- FunctionIdentifier fid = null;
- switch (t) {
- case PLUS: {
- fid = AlgebricksBuiltinFunctions.NUMERIC_ADD;
- break;
- }
- case MINUS: {
- fid = AsterixBuiltinFunctions.NUMERIC_SUBTRACT;
- break;
- }
- case MUL: {
- fid = AsterixBuiltinFunctions.NUMERIC_MULTIPLY;
- break;
- }
- case DIV: {
- fid = AsterixBuiltinFunctions.NUMERIC_DIVIDE;
- break;
- }
- case MOD: {
- fid = AsterixBuiltinFunctions.NUMERIC_MOD;
- break;
- }
- case IDIV: {
- fid = AsterixBuiltinFunctions.NUMERIC_IDIV;
- break;
- }
- case CARET: {
- fid = AsterixBuiltinFunctions.CARET;
- break;
- }
- case AND: {
- fid = AlgebricksBuiltinFunctions.AND;
- break;
- }
- case OR: {
- fid = AlgebricksBuiltinFunctions.OR;
- break;
- }
- case FUZZY_EQ: {
- fid = AsterixBuiltinFunctions.FUZZY_EQ;
- break;
- }
+ FunctionIdentifier fid = null;
+ switch (t) {
+ case PLUS: {
+ fid = AlgebricksBuiltinFunctions.NUMERIC_ADD;
+ break;
+ }
+ case MINUS: {
+ fid = AsterixBuiltinFunctions.NUMERIC_SUBTRACT;
+ break;
+ }
+ case MUL: {
+ fid = AsterixBuiltinFunctions.NUMERIC_MULTIPLY;
+ break;
+ }
+ case DIV: {
+ fid = AsterixBuiltinFunctions.NUMERIC_DIVIDE;
+ break;
+ }
+ case MOD: {
+ fid = AsterixBuiltinFunctions.NUMERIC_MOD;
+ break;
+ }
+ case IDIV: {
+ fid = AsterixBuiltinFunctions.NUMERIC_IDIV;
+ break;
+ }
+ case CARET: {
+ fid = AsterixBuiltinFunctions.CARET;
+ break;
+ }
+ case AND: {
+ fid = AlgebricksBuiltinFunctions.AND;
+ break;
+ }
+ case OR: {
+ fid = AlgebricksBuiltinFunctions.OR;
+ break;
+ }
+ case FUZZY_EQ: {
+ fid = AsterixBuiltinFunctions.FUZZY_EQ;
+ break;
+ }
- default: {
- throw new NotImplementedException("Operator " + t
- + " is not yet implemented");
- }
- }
- return new ScalarFunctionCallExpression(
- FunctionUtils.getFunctionInfo(fid));
- }
+ default: {
+ throw new NotImplementedException("Operator " + t + " is not yet implemented");
+ }
+ }
+ return new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(fid));
+ }
- private static boolean hasOnlyChild(ILogicalOperator parent,
- Mutable<ILogicalOperator> childCandidate) {
- List<Mutable<ILogicalOperator>> inp = parent.getInputs();
- if (inp == null || inp.size() != 1) {
- return false;
- }
- return inp.get(0) == childCandidate;
- }
+ private static boolean hasOnlyChild(ILogicalOperator parent, Mutable<ILogicalOperator> childCandidate) {
+ List<Mutable<ILogicalOperator>> inp = parent.getInputs();
+ if (inp == null || inp.size() != 1) {
+ return false;
+ }
+ return inp.get(0) == childCandidate;
+ }
- private Pair<ILogicalExpression, Mutable<ILogicalOperator>> aqlExprToAlgExpression(
- Expression expr, Mutable<ILogicalOperator> topOp)
- throws AsterixException {
- switch (expr.getKind()) {
- case VARIABLE_EXPRESSION: {
- VariableReferenceExpression ve = new VariableReferenceExpression(
- context.getVar(((VariableExpr) expr).getVar().getId()));
- return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(ve,
- topOp);
- }
- case LITERAL_EXPRESSION: {
- LiteralExpr val = (LiteralExpr) expr;
- return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
- new ConstantExpression(new AsterixConstantValue(
- ConstantHelper.objectFromLiteral(val.getValue()))),
- topOp);
- }
- default: {
- // Mutable<ILogicalOperator> src = new
- // Mutable<ILogicalOperator>();
- // Mutable<ILogicalOperator> src = topOp;
- if (expressionNeedsNoNesting(expr)) {
- Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this,
- topOp);
- ILogicalExpression exp = ((AssignOperator) p.first)
- .getExpressions().get(0).getValue();
- return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
- exp, p.first.getInputs().get(0));
- } else {
- Mutable<ILogicalOperator> src = new MutableObject<ILogicalOperator>();
+ private Pair<ILogicalExpression, Mutable<ILogicalOperator>> aqlExprToAlgExpression(Expression expr,
+ Mutable<ILogicalOperator> topOp) throws AsterixException {
+ switch (expr.getKind()) {
+ case VARIABLE_EXPRESSION: {
+ VariableReferenceExpression ve = new VariableReferenceExpression(context.getVar(((VariableExpr) expr)
+ .getVar().getId()));
+ return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(ve, topOp);
+ }
+ case LITERAL_EXPRESSION: {
+ LiteralExpr val = (LiteralExpr) expr;
+ return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(new ConstantExpression(
+ new AsterixConstantValue(ConstantHelper.objectFromLiteral(val.getValue()))), topOp);
+ }
+ default: {
+ // Mutable<ILogicalOperator> src = new
+ // Mutable<ILogicalOperator>();
+ // Mutable<ILogicalOperator> src = topOp;
+ if (expressionNeedsNoNesting(expr)) {
+ Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, topOp);
+ ILogicalExpression exp = ((AssignOperator) p.first).getExpressions().get(0).getValue();
+ return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(exp, p.first.getInputs().get(0));
+ } else {
+ Mutable<ILogicalOperator> src = new MutableObject<ILogicalOperator>();
- Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this,
- src);
+ Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, src);
- if (((AbstractLogicalOperator) p.first).getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
- // src.setOperator(topOp.getOperator());
- Mutable<ILogicalOperator> top2 = new MutableObject<ILogicalOperator>(
- p.first);
- return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
- new VariableReferenceExpression(p.second), top2);
- } else {
- SubplanOperator s = new SubplanOperator();
- s.getInputs().add(topOp);
- src.setValue(new NestedTupleSourceOperator(
- new MutableObject<ILogicalOperator>(s)));
- Mutable<ILogicalOperator> planRoot = new MutableObject<ILogicalOperator>(
- p.first);
- s.setRootOp(planRoot);
- return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
- new VariableReferenceExpression(p.second),
- new MutableObject<ILogicalOperator>(s));
- }
- }
- }
- }
+ if (((AbstractLogicalOperator) p.first).getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
+ // src.setOperator(topOp.getOperator());
+ Mutable<ILogicalOperator> top2 = new MutableObject<ILogicalOperator>(p.first);
+ return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(new VariableReferenceExpression(
+ p.second), top2);
+ } else {
+ SubplanOperator s = new SubplanOperator();
+ s.getInputs().add(topOp);
+ src.setValue(new NestedTupleSourceOperator(new MutableObject<ILogicalOperator>(s)));
+ Mutable<ILogicalOperator> planRoot = new MutableObject<ILogicalOperator>(p.first);
+ s.setRootOp(planRoot);
+ return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(new VariableReferenceExpression(
+ p.second), new MutableObject<ILogicalOperator>(s));
+ }
+ }
+ }
+ }
- }
+ }
- private Pair<ILogicalOperator, LogicalVariable> produceFlwrResult(
- boolean noForClause, boolean isTop,
- Mutable<ILogicalOperator> resOpRef, LogicalVariable resVar) {
- if (isTop) {
- ProjectOperator pr = new ProjectOperator(resVar);
- pr.getInputs().add(resOpRef);
- return new Pair<ILogicalOperator, LogicalVariable>(pr, resVar);
+ private Pair<ILogicalOperator, LogicalVariable> produceFlwrResult(boolean noForClause, boolean isTop,
+ Mutable<ILogicalOperator> resOpRef, LogicalVariable resVar) {
+ if (isTop) {
+ ProjectOperator pr = new ProjectOperator(resVar);
+ pr.getInputs().add(resOpRef);
+ return new Pair<ILogicalOperator, LogicalVariable>(pr, resVar);
- } else if (noForClause) {
- return new Pair<ILogicalOperator, LogicalVariable>(
- resOpRef.getValue(), resVar);
- } else {
- return aggListify(resVar, resOpRef, false);
- }
- }
+ } else if (noForClause) {
+ return new Pair<ILogicalOperator, LogicalVariable>(resOpRef.getValue(), resVar);
+ } else {
+ return aggListify(resVar, resOpRef, false);
+ }
+ }
- private Pair<ILogicalOperator, LogicalVariable> aggListify(
- LogicalVariable var, Mutable<ILogicalOperator> opRef,
- boolean bProject) {
- AggregateFunctionCallExpression funAgg = AsterixBuiltinFunctions
- .makeAggregateFunctionExpression(
- AsterixBuiltinFunctions.LISTIFY,
- new ArrayList<Mutable<ILogicalExpression>>());
- 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)));
- agg.getInputs().add(opRef);
- ILogicalOperator res;
- if (bProject) {
- ProjectOperator pr = new ProjectOperator(varListified);
- pr.getInputs().add(new MutableObject<ILogicalOperator>(agg));
- res = pr;
- } else {
- res = agg;
- }
- return new Pair<ILogicalOperator, LogicalVariable>(res, varListified);
- }
+ private Pair<ILogicalOperator, LogicalVariable> aggListify(LogicalVariable var, Mutable<ILogicalOperator> opRef,
+ boolean bProject) {
+ AggregateFunctionCallExpression funAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
+ AsterixBuiltinFunctions.LISTIFY, new ArrayList<Mutable<ILogicalExpression>>());
+ 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)));
+ agg.getInputs().add(opRef);
+ ILogicalOperator res;
+ if (bProject) {
+ ProjectOperator pr = new ProjectOperator(varListified);
+ pr.getInputs().add(new MutableObject<ILogicalOperator>(agg));
+ res = pr;
+ } else {
+ res = agg;
+ }
+ return new Pair<ILogicalOperator, LogicalVariable>(res, varListified);
+ }
- private Pair<ILogicalOperator, LogicalVariable> visitAndOrOperator(
- OperatorExpr op, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
- ArrayList<OperatorType> ops = op.getOpList();
- int nOps = ops.size();
+ private Pair<ILogicalOperator, LogicalVariable> visitAndOrOperator(OperatorExpr op,
+ Mutable<ILogicalOperator> tupSource) throws AsterixException {
+ ArrayList<OperatorType> ops = op.getOpList();
+ int nOps = ops.size();
- ArrayList<Expression> exprs = op.getExprList();
+ ArrayList<Expression> exprs = op.getExprList();
- Mutable<ILogicalOperator> topOp = tupSource;
+ Mutable<ILogicalOperator> topOp = tupSource;
- OperatorType opLogical = ops.get(0);
- AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(opLogical);
+ OperatorType opLogical = ops.get(0);
+ AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(opLogical);
- for (int i = 0; i <= nOps; i++) {
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
- exprs.get(i), topOp);
- topOp = p.second;
- // now look at the operator
- if (i < nOps) {
- if (ops.get(i) != opLogical) {
- throw new TranslationException("Unexpected operator "
- + ops.get(i) + " in an OperatorExpr starting with "
- + opLogical);
- }
- }
- f.getArguments()
- .add(new MutableObject<ILogicalExpression>(p.first));
- }
+ for (int i = 0; i <= nOps; i++) {
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(exprs.get(i), topOp);
+ topOp = p.second;
+ // now look at the operator
+ if (i < nOps) {
+ if (ops.get(i) != opLogical) {
+ throw new TranslationException("Unexpected operator " + ops.get(i)
+ + " in an OperatorExpr starting with " + opLogical);
+ }
+ }
+ f.getArguments().add(new MutableObject<ILogicalExpression>(p.first));
+ }
- LogicalVariable assignedVar = context.newVar();
- AssignOperator a = new AssignOperator(assignedVar,
- new MutableObject<ILogicalExpression>(f));
- a.getInputs().add(topOp);
+ LogicalVariable assignedVar = context.newVar();
+ AssignOperator a = new AssignOperator(assignedVar, new MutableObject<ILogicalExpression>(f));
+ a.getInputs().add(topOp);
- return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
+ return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
- }
+ }
- private static boolean expressionNeedsNoNesting(Expression expr) {
- Kind k = expr.getKind();
- return k == Kind.LITERAL_EXPRESSION
- || k == Kind.LIST_CONSTRUCTOR_EXPRESSION
- || k == Kind.RECORD_CONSTRUCTOR_EXPRESSION
- || k == Kind.VARIABLE_EXPRESSION || k == Kind.CALL_EXPRESSION
- || k == Kind.OP_EXPRESSION
- || k == Kind.FIELD_ACCESSOR_EXPRESSION
- || k == Kind.INDEX_ACCESSOR_EXPRESSION
- || k == Kind.UNARY_EXPRESSION;
- }
+ private static boolean expressionNeedsNoNesting(Expression expr) {
+ Kind k = expr.getKind();
+ return k == Kind.LITERAL_EXPRESSION || k == Kind.LIST_CONSTRUCTOR_EXPRESSION
+ || k == Kind.RECORD_CONSTRUCTOR_EXPRESSION || k == Kind.VARIABLE_EXPRESSION
+ || k == Kind.CALL_EXPRESSION || k == Kind.OP_EXPRESSION || k == Kind.FIELD_ACCESSOR_EXPRESSION
+ || k == Kind.INDEX_ACCESSOR_EXPRESSION || k == Kind.UNARY_EXPRESSION;
+ }
- private <T> ArrayList<T> mkSingletonArrayList(T item) {
- ArrayList<T> array = new ArrayList<T>(1);
- array.add(item);
- return array;
- }
+ private <T> ArrayList<T> mkSingletonArrayList(T item) {
+ ArrayList<T> array = new ArrayList<T>(1);
+ array.add(item);
+ return array;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitTypeDecl(TypeDecl td,
- Mutable<ILogicalOperator> arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitTypeDecl(TypeDecl td, Mutable<ILogicalOperator> arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitRecordTypeDefiniton(
- RecordTypeDefinition tre, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitRecordTypeDefiniton(RecordTypeDefinition tre,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitTypeReferenceExpression(
- TypeReferenceExpression tre, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitTypeReferenceExpression(TypeReferenceExpression tre,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitNodegroupDecl(
- NodegroupDecl ngd, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitNodegroupDecl(NodegroupDecl ngd, Mutable<ILogicalOperator> arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitLoadFromFileStatement(
- LoadFromFileStatement stmtLoad, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitLoadFromFileStatement(LoadFromFileStatement stmtLoad,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitWriteFromQueryResultStatement(
- WriteFromQueryResultStatement stmtLoad,
- Mutable<ILogicalOperator> arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitWriteFromQueryResultStatement(
+ WriteFromQueryResultStatement stmtLoad, Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitDropStatement(
- DropStatement del, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitDropStatement(DropStatement del, Mutable<ILogicalOperator> arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitControlFeedStatement(
- ControlFeedStatement del, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitControlFeedStatement(ControlFeedStatement del,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitCreateIndexStatement(
- CreateIndexStatement cis, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitCreateIndexStatement(CreateIndexStatement cis,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitOrderedListTypeDefiniton(
- OrderedListTypeDefinition olte, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitOrderedListTypeDefiniton(OrderedListTypeDefinition olte,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitUnorderedListTypeDefiniton(
- UnorderedListTypeDefinition ulte, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitUnorderedListTypeDefiniton(UnorderedListTypeDefinition ulte,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- private ILogicalExpression makeUnnestExpression(ILogicalExpression expr) {
- switch (expr.getExpressionTag()) {
- case VARIABLE: {
- return new UnnestingFunctionCallExpression(
- FunctionUtils
- .getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
- new MutableObject<ILogicalExpression>(expr));
- }
- case FUNCTION_CALL: {
- AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
- if (fce.getKind() == FunctionKind.UNNEST) {
- return expr;
- } else {
- return new UnnestingFunctionCallExpression(
- FunctionUtils
- .getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
- new MutableObject<ILogicalExpression>(expr));
- }
- }
- default: {
- return expr;
- }
- }
- }
+ private ILogicalExpression makeUnnestExpression(ILogicalExpression expr) {
+ switch (expr.getExpressionTag()) {
+ case VARIABLE: {
+ return new UnnestingFunctionCallExpression(
+ FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
+ new MutableObject<ILogicalExpression>(expr));
+ }
+ case FUNCTION_CALL: {
+ AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
+ if (fce.getKind() == FunctionKind.UNNEST) {
+ return expr;
+ } else {
+ return new UnnestingFunctionCallExpression(
+ FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
+ new MutableObject<ILogicalExpression>(expr));
+ }
+ }
+ default: {
+ return expr;
+ }
+ }
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitInsertStatement(
- InsertStatement insert, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitInsertStatement(InsertStatement insert,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitDeleteStatement(
- DeleteStatement del, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitDeleteStatement(DeleteStatement del,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitUpdateStatement(
- UpdateStatement update, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitUpdateStatement(UpdateStatement update,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitUpdateClause(
- UpdateClause del, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitUpdateClause(UpdateClause del, Mutable<ILogicalOperator> arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitDataverseDecl(
- DataverseDecl dv, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitDataverseDecl(DataverseDecl dv, Mutable<ILogicalOperator> arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitDatasetDecl(
- DatasetDecl dd, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitDatasetDecl(DatasetDecl dd, Mutable<ILogicalOperator> arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitSetStatement(
- SetStatement ss, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitSetStatement(SetStatement ss, Mutable<ILogicalOperator> arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitWriteStatement(
- WriteStatement ws, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitWriteStatement(WriteStatement ws, Mutable<ILogicalOperator> arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitLoadFromQueryResultStatement(
- WriteFromQueryResultStatement stmtLoad,
- Mutable<ILogicalOperator> arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitLoadFromQueryResultStatement(
+ WriteFromQueryResultStatement stmtLoad, Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitCreateDataverseStatement(
- CreateDataverseStatement del, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitCreateDataverseStatement(CreateDataverseStatement del,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitIndexDropStatement(
- IndexDropStatement del, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitIndexDropStatement(IndexDropStatement del,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitNodeGroupDropStatement(
- NodeGroupDropStatement del, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitNodeGroupDropStatement(NodeGroupDropStatement del,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitDataverseDropStatement(
- DataverseDropStatement del, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitDataverseDropStatement(DataverseDropStatement del,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitTypeDropStatement(
- TypeDropStatement del, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitTypeDropStatement(TypeDropStatement del,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visit(
- CreateFunctionStatement cfs, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visit(CreateFunctionStatement cfs, Mutable<ILogicalOperator> arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitFunctionDropStatement(
- FunctionDropStatement del, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitFunctionDropStatement(FunctionDropStatement del,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitBeginFeedStatement(
- BeginFeedStatement bf, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitBeginFeedStatement(BeginFeedStatement bf,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
}
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 4fc1fc8..aa26d8a 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
@@ -97,6 +97,7 @@
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
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;
@@ -153,1576 +154,1320 @@
* source for the current subtree.
*/
-public class AqlPlusExpressionToPlanTranslator extends AbstractAqlTranslator
- implements
- IAqlPlusExpressionVisitor<Pair<ILogicalOperator, LogicalVariable>, Mutable<ILogicalOperator>> {
+public class AqlPlusExpressionToPlanTranslator extends AbstractAqlTranslator implements
+ IAqlPlusExpressionVisitor<Pair<ILogicalOperator, LogicalVariable>, Mutable<ILogicalOperator>> {
- private static final Logger LOGGER = Logger
- .getLogger(AqlPlusExpressionToPlanTranslator.class.getName());
+ private static final Logger LOGGER = Logger.getLogger(AqlPlusExpressionToPlanTranslator.class.getName());
- private class MetaScopeLogicalVariable {
- private HashMap<Identifier, LogicalVariable> map = new HashMap<Identifier, LogicalVariable>();
+ private class MetaScopeLogicalVariable {
+ private HashMap<Identifier, LogicalVariable> map = new HashMap<Identifier, LogicalVariable>();
- public VariableReferenceExpression getVariableReferenceExpression(
- Identifier id) throws AsterixException {
- LogicalVariable var = map.get(id);
- LOGGER.fine("get:" + id + ":" + var);
- if (var == null) {
- throw new AsterixException("Identifier " + id
- + " not found in AQL+ meta-scope.");
- }
- return new VariableReferenceExpression(var);
- }
+ public VariableReferenceExpression getVariableReferenceExpression(Identifier id) throws AsterixException {
+ LogicalVariable var = map.get(id);
+ LOGGER.fine("get:" + id + ":" + var);
+ if (var == null) {
+ throw new AsterixException("Identifier " + id + " not found in AQL+ meta-scope.");
+ }
+ return new VariableReferenceExpression(var);
+ }
- public void put(Identifier id, LogicalVariable var) {
- LOGGER.fine("put:" + id + ":" + var);
- map.put(id, var);
- }
- }
+ public void put(Identifier id, LogicalVariable var) {
+ LOGGER.fine("put:" + id + ":" + var);
+ map.put(id, var);
+ }
+ }
- private class MetaScopeILogicalOperator {
- private HashMap<Identifier, ILogicalOperator> map = new HashMap<Identifier, ILogicalOperator>();
+ private class MetaScopeILogicalOperator {
+ private HashMap<Identifier, ILogicalOperator> map = new HashMap<Identifier, ILogicalOperator>();
- public ILogicalOperator get(Identifier id) throws AsterixException {
- ILogicalOperator op = map.get(id);
- if (op == null) {
- throw new AsterixException("Identifier " + id
- + " not found in AQL+ meta-scope.");
- }
- return op;
- }
+ public ILogicalOperator get(Identifier id) throws AsterixException {
+ ILogicalOperator op = map.get(id);
+ if (op == null) {
+ throw new AsterixException("Identifier " + id + " not found in AQL+ meta-scope.");
+ }
+ return op;
+ }
- public void put(Identifier id, ILogicalOperator op) {
- LOGGER.fine("put:" + id + ":" + op);
- map.put(id, op);
- }
- }
+ public void put(Identifier id, ILogicalOperator op) {
+ LOGGER.fine("put:" + id + ":" + op);
+ map.put(id, op);
+ }
+ }
- private final long txnId;
- private final MetadataTransactionContext mdTxnCtx;
- private TranslationContext context;
- private String outputDatasetName;
- private MetaScopeLogicalVariable metaScopeExp = new MetaScopeLogicalVariable();
- private MetaScopeILogicalOperator metaScopeOp = new MetaScopeILogicalOperator();
- private static LogicalVariable METADATA_DUMMY_VAR = new LogicalVariable(-1);
+ private final JobId jobId;
+ private final MetadataTransactionContext mdTxnCtx;
+ private TranslationContext context;
+ private String outputDatasetName;
+ private MetaScopeLogicalVariable metaScopeExp = new MetaScopeLogicalVariable();
+ private MetaScopeILogicalOperator metaScopeOp = new MetaScopeILogicalOperator();
+ private static LogicalVariable METADATA_DUMMY_VAR = new LogicalVariable(-1);
- public AqlPlusExpressionToPlanTranslator(long txnId,
- MetadataTransactionContext mdTxnCtx, Counter currentVarCounter,
- String outputDatasetName) {
- this.txnId = txnId;
- this.mdTxnCtx = mdTxnCtx;
- this.context = new TranslationContext(currentVarCounter);
- this.outputDatasetName = outputDatasetName;
- this.context.setTopFlwor(false);
- }
+ public AqlPlusExpressionToPlanTranslator(JobId jobId, MetadataTransactionContext mdTxnCtx,
+ Counter currentVarCounter, String outputDatasetName) {
+ this.jobId = jobId;
+ this.mdTxnCtx = mdTxnCtx;
+ this.context = new TranslationContext(currentVarCounter);
+ this.outputDatasetName = outputDatasetName;
+ this.context.setTopFlwor(false);
+ }
- public int getVarCounter() {
- return context.getVarCounter();
- }
+ public int getVarCounter() {
+ return context.getVarCounter();
+ }
- public ILogicalPlanAndMetadata translate(Query expr)
- throws AlgebricksException, AsterixException {
- return translate(expr, null);
- }
+ public ILogicalPlanAndMetadata translate(Query expr) throws AlgebricksException, AsterixException {
+ return translate(expr, null);
+ }
- public ILogicalPlanAndMetadata translate(Query expr,
- AqlCompiledMetadataDeclarations compiledDeclarations)
- throws AlgebricksException, AsterixException {
- if (expr == null) {
- return null;
- }
- if (compiledDeclarations == null) {
- compiledDeclarations = compileMetadata(mdTxnCtx,
- expr.getPrologDeclList(), true);
- }
- if (!compiledDeclarations.isConnectedToDataverse())
- compiledDeclarations.connectToDataverse(compiledDeclarations
- .getDataverseName());
- IDataFormat format = compiledDeclarations.getFormat();
- if (format == null) {
- throw new AlgebricksException("Data format has not been set.");
- }
- format.registerRuntimeFunctions();
- Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this,
- new MutableObject<ILogicalOperator>(
- new EmptyTupleSourceOperator()));
+ public ILogicalPlanAndMetadata translate(Query expr, AqlCompiledMetadataDeclarations compiledDeclarations)
+ throws AlgebricksException, AsterixException {
+ if (expr == null) {
+ return null;
+ }
+ if (compiledDeclarations == null) {
+ compiledDeclarations = compileMetadata(mdTxnCtx, expr.getPrologDeclList(), true);
+ }
+ if (!compiledDeclarations.isConnectedToDataverse())
+ compiledDeclarations.connectToDataverse(compiledDeclarations.getDataverseName());
+ IDataFormat format = compiledDeclarations.getFormat();
+ if (format == null) {
+ throw new AlgebricksException("Data format has not been set.");
+ }
+ format.registerRuntimeFunctions();
+ Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, new MutableObject<ILogicalOperator>(
+ new EmptyTupleSourceOperator()));
- ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<Mutable<ILogicalOperator>>();
+ ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<Mutable<ILogicalOperator>>();
- boolean isTransactionalWrite = false;
- ILogicalOperator topOp = p.first;
- ProjectOperator project = (ProjectOperator) topOp;
- LogicalVariable resVar = project.getVariables().get(0);
- if (outputDatasetName == null) {
- List<Mutable<ILogicalExpression>> writeExprList = new ArrayList<Mutable<ILogicalExpression>>(
- 1);
- writeExprList.add(new MutableObject<ILogicalExpression>(
- new VariableReferenceExpression(resVar)));
- FileSplitSinkId fssi = new FileSplitSinkId(
- compiledDeclarations.getOutputFile());
- FileSplitDataSink sink = new FileSplitDataSink(fssi, null);
- topOp = new WriteOperator(writeExprList, sink);
- topOp.getInputs().add(new MutableObject<ILogicalOperator>(project));
- } else {
- Dataset dataset = compiledDeclarations
- .findDataset(outputDatasetName);
- if (dataset == null) {
- throw new AlgebricksException("Cannot find dataset "
- + outputDatasetName);
- }
- 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 (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));
- LogicalVariable v = context.newVar();
- vars.add(v);
- varRefsForLoading.add(new MutableObject<ILogicalExpression>(
- new VariableReferenceExpression(v)));
- }
- AssignOperator assign = new AssignOperator(vars, exprs);
- assign.getInputs()
- .add(new MutableObject<ILogicalOperator>(project));
- }
+ boolean isTransactionalWrite = false;
+ ILogicalOperator topOp = p.first;
+ ProjectOperator project = (ProjectOperator) topOp;
+ LogicalVariable resVar = project.getVariables().get(0);
+ if (outputDatasetName == null) {
+ List<Mutable<ILogicalExpression>> writeExprList = new ArrayList<Mutable<ILogicalExpression>>(1);
+ writeExprList.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(resVar)));
+ FileSplitSinkId fssi = new FileSplitSinkId(compiledDeclarations.getOutputFile());
+ FileSplitDataSink sink = new FileSplitDataSink(fssi, null);
+ topOp = new WriteOperator(writeExprList, sink);
+ topOp.getInputs().add(new MutableObject<ILogicalOperator>(project));
+ } else {
+ Dataset dataset = compiledDeclarations.findDataset(outputDatasetName);
+ if (dataset == null) {
+ throw new AlgebricksException("Cannot find dataset " + outputDatasetName);
+ }
+ 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 (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));
+ LogicalVariable v = context.newVar();
+ vars.add(v);
+ varRefsForLoading.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(v)));
+ }
+ AssignOperator assign = new AssignOperator(vars, exprs);
+ assign.getInputs().add(new MutableObject<ILogicalOperator>(project));
+ }
- globalPlanRoots.add(new MutableObject<ILogicalOperator>(topOp));
- ILogicalPlan plan = new ALogicalPlanImpl(globalPlanRoots);
- AqlMetadataProvider metadataProvider = new AqlMetadataProvider(txnId,
- isTransactionalWrite, compiledDeclarations);
- ILogicalPlanAndMetadata planAndMetadata = new AqlLogicalPlanAndMetadataImpl(
- plan, metadataProvider);
- return planAndMetadata;
- }
+ globalPlanRoots.add(new MutableObject<ILogicalOperator>(topOp));
+ ILogicalPlan plan = new ALogicalPlanImpl(globalPlanRoots);
+ AqlMetadataProvider metadataProvider = new AqlMetadataProvider(jobId, isTransactionalWrite,
+ compiledDeclarations);
+ ILogicalPlanAndMetadata planAndMetadata = new AqlLogicalPlanAndMetadataImpl(plan, metadataProvider);
+ return planAndMetadata;
+ }
- public ILogicalPlan translate(List<Clause> clauses)
- throws AlgebricksException, AsterixException {
+ public ILogicalPlan translate(List<Clause> clauses) throws AlgebricksException, AsterixException {
- if (clauses == null) {
- return null;
- }
+ if (clauses == null) {
+ return null;
+ }
- Mutable<ILogicalOperator> opRef = new MutableObject<ILogicalOperator>(
- new EmptyTupleSourceOperator());
- Pair<ILogicalOperator, LogicalVariable> p = null;
- for (Clause c : clauses) {
- p = c.accept(this, opRef);
- opRef = new MutableObject<ILogicalOperator>(p.first);
- }
+ Mutable<ILogicalOperator> opRef = new MutableObject<ILogicalOperator>(new EmptyTupleSourceOperator());
+ Pair<ILogicalOperator, LogicalVariable> p = null;
+ for (Clause c : clauses) {
+ p = c.accept(this, opRef);
+ opRef = new MutableObject<ILogicalOperator>(p.first);
+ }
- ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<Mutable<ILogicalOperator>>();
+ ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<Mutable<ILogicalOperator>>();
- ILogicalOperator topOp = p.first;
+ ILogicalOperator topOp = p.first;
- globalPlanRoots.add(new MutableObject<ILogicalOperator>(topOp));
- ILogicalPlan plan = new ALogicalPlanImpl(globalPlanRoots);
- return plan;
- }
+ globalPlanRoots.add(new MutableObject<ILogicalOperator>(topOp));
+ ILogicalPlan plan = new ALogicalPlanImpl(globalPlanRoots);
+ return plan;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitForClause(ForClause fc,
- Mutable<ILogicalOperator> tupSource) throws AsterixException {
- LogicalVariable v = context.newVar(fc.getVarExpr());
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitForClause(ForClause fc, Mutable<ILogicalOperator> tupSource)
+ throws AsterixException {
+ LogicalVariable v = context.newVar(fc.getVarExpr());
- Expression inExpr = fc.getInExpr();
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
- inExpr, tupSource);
- ILogicalOperator returnedOp;
+ Expression inExpr = fc.getInExpr();
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(inExpr, tupSource);
+ ILogicalOperator returnedOp;
- if (fc.getPosVarExpr() == null) {
- 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.getInputs().add(eo.second);
+ if (fc.getPosVarExpr() == null) {
+ 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.getInputs().add(eo.second);
- return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
- }
+ return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitLetClause(LetClause lc,
- Mutable<ILogicalOperator> tupSource) throws AsterixException {
- LogicalVariable v;
- ILogicalOperator returnedOp;
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitLetClause(LetClause lc, Mutable<ILogicalOperator> tupSource)
+ throws AsterixException {
+ LogicalVariable v;
+ ILogicalOperator returnedOp;
- switch (lc.getBindingExpr().getKind()) {
- 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.getInputs().add(tupSource);
- break;
- }
- default: {
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
- lc.getBindingExpr(), tupSource);
- v = context.newVar(lc.getVarExpr());
- returnedOp = new AssignOperator(v,
- new MutableObject<ILogicalExpression>(eo.first));
- returnedOp.getInputs().add(eo.second);
- break;
- }
- }
- return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
- }
+ switch (lc.getBindingExpr().getKind()) {
+ 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.getInputs().add(tupSource);
+ break;
+ }
+ default: {
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(lc.getBindingExpr(),
+ tupSource);
+ v = context.newVar(lc.getVarExpr());
+ returnedOp = new AssignOperator(v, new MutableObject<ILogicalExpression>(eo.first));
+ returnedOp.getInputs().add(eo.second);
+ break;
+ }
+ }
+ return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitFlworExpression(
- FLWOGRExpression flwor, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
- Mutable<ILogicalOperator> flworPlan = tupSource;
- boolean isTop = context.isTopFlwor();
- if (isTop) {
- context.setTopFlwor(false);
- }
- for (Clause c : flwor.getClauseList()) {
- Pair<ILogicalOperator, LogicalVariable> pC = c.accept(this,
- flworPlan);
- flworPlan = new MutableObject<ILogicalOperator>(pC.first);
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitFlworExpression(FLWOGRExpression flwor,
+ Mutable<ILogicalOperator> tupSource) throws AsterixException {
+ Mutable<ILogicalOperator> flworPlan = tupSource;
+ boolean isTop = context.isTopFlwor();
+ if (isTop) {
+ context.setTopFlwor(false);
+ }
+ for (Clause c : flwor.getClauseList()) {
+ Pair<ILogicalOperator, LogicalVariable> pC = c.accept(this, flworPlan);
+ flworPlan = new MutableObject<ILogicalOperator>(pC.first);
+ }
- Expression r = flwor.getReturnExpr();
- boolean noFlworClause = flwor.noForClause();
+ Expression r = flwor.getReturnExpr();
+ boolean noFlworClause = flwor.noForClause();
- if (r.getKind() == Kind.VARIABLE_EXPRESSION) {
- VariableExpr v = (VariableExpr) r;
- LogicalVariable var = context.getVar(v.getVar().getId());
+ if (r.getKind() == Kind.VARIABLE_EXPRESSION) {
+ VariableExpr v = (VariableExpr) r;
+ LogicalVariable var = context.getVar(v.getVar().getId());
- return produceFlwrResult(noFlworClause, isTop, flworPlan, var);
+ return produceFlwrResult(noFlworClause, isTop, flworPlan, var);
- } else {
- Mutable<ILogicalOperator> baseOp = new MutableObject<ILogicalOperator>(
- flworPlan.getValue());
- Pair<ILogicalOperator, LogicalVariable> rRes = r.accept(this,
- baseOp);
- ILogicalOperator rOp = rRes.first;
- ILogicalOperator resOp;
- if (expressionNeedsNoNesting(r)) {
- baseOp.setValue(flworPlan.getValue());
- resOp = rOp;
- } else {
- SubplanOperator s = new SubplanOperator(rOp);
- s.getInputs().add(flworPlan);
- resOp = s;
- baseOp.setValue(new NestedTupleSourceOperator(
- new MutableObject<ILogicalOperator>(s)));
- }
- Mutable<ILogicalOperator> resOpRef = new MutableObject<ILogicalOperator>(
- resOp);
- return produceFlwrResult(noFlworClause, isTop, resOpRef,
- rRes.second);
- }
- }
+ } else {
+ Mutable<ILogicalOperator> baseOp = new MutableObject<ILogicalOperator>(flworPlan.getValue());
+ Pair<ILogicalOperator, LogicalVariable> rRes = r.accept(this, baseOp);
+ ILogicalOperator rOp = rRes.first;
+ ILogicalOperator resOp;
+ if (expressionNeedsNoNesting(r)) {
+ baseOp.setValue(flworPlan.getValue());
+ resOp = rOp;
+ } else {
+ SubplanOperator s = new SubplanOperator(rOp);
+ s.getInputs().add(flworPlan);
+ resOp = s;
+ baseOp.setValue(new NestedTupleSourceOperator(new MutableObject<ILogicalOperator>(s)));
+ }
+ Mutable<ILogicalOperator> resOpRef = new MutableObject<ILogicalOperator>(resOp);
+ return produceFlwrResult(noFlworClause, isTop, resOpRef, rRes.second);
+ }
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitFieldAccessor(
- FieldAccessor fa, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
- fa.getExpr(), tupSource);
- LogicalVariable v = context.newVar();
- AbstractFunctionCallExpression fldAccess = new ScalarFunctionCallExpression(
- FunctionUtils
- .getFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME));
- fldAccess.getArguments().add(
- new MutableObject<ILogicalExpression>(p.first));
- ILogicalExpression faExpr = new ConstantExpression(
- new AsterixConstantValue(new AString(fa.getIdent().getValue())));
- fldAccess.getArguments().add(
- new MutableObject<ILogicalExpression>(faExpr));
- AssignOperator a = new AssignOperator(v,
- new MutableObject<ILogicalExpression>(fldAccess));
- a.getInputs().add(p.second);
- return new Pair<ILogicalOperator, LogicalVariable>(a, v);
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitFieldAccessor(FieldAccessor fa,
+ Mutable<ILogicalOperator> tupSource) throws AsterixException {
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(fa.getExpr(), tupSource);
+ LogicalVariable v = context.newVar();
+ AbstractFunctionCallExpression fldAccess = new ScalarFunctionCallExpression(
+ FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME));
+ fldAccess.getArguments().add(new MutableObject<ILogicalExpression>(p.first));
+ ILogicalExpression faExpr = new ConstantExpression(new AsterixConstantValue(new AString(fa.getIdent()
+ .getValue())));
+ fldAccess.getArguments().add(new MutableObject<ILogicalExpression>(faExpr));
+ AssignOperator a = new AssignOperator(v, new MutableObject<ILogicalExpression>(fldAccess));
+ a.getInputs().add(p.second);
+ return new Pair<ILogicalOperator, LogicalVariable>(a, v);
- }
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitIndexAccessor(
- IndexAccessor ia, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
- ia.getExpr(), tupSource);
- LogicalVariable v = context.newVar();
- AbstractFunctionCallExpression f;
- int i = ia.getIndex();
- if (i == IndexAccessor.ANY) {
- f = new ScalarFunctionCallExpression(
- FunctionUtils
- .getFunctionInfo(AsterixBuiltinFunctions.ANY_COLLECTION_MEMBER));
- f.getArguments()
- .add(new MutableObject<ILogicalExpression>(p.first));
- } else {
- 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)))));
- }
- 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> visitIndexAccessor(IndexAccessor ia,
+ Mutable<ILogicalOperator> tupSource) throws AsterixException {
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(ia.getExpr(), tupSource);
+ LogicalVariable v = context.newVar();
+ AbstractFunctionCallExpression f;
+ int i = ia.getIndex();
+ if (i == IndexAccessor.ANY) {
+ f = new ScalarFunctionCallExpression(
+ FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.ANY_COLLECTION_MEMBER));
+ f.getArguments().add(new MutableObject<ILogicalExpression>(p.first));
+ } else {
+ 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)))));
+ }
+ 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)
- throws AsterixException {
- LogicalVariable v = context.newVar();
- AsterixFunction fid = fcall.getIdent();
- List<Mutable<ILogicalExpression>> args = new ArrayList<Mutable<ILogicalExpression>>();
- Mutable<ILogicalOperator> topOp = tupSource;
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitCallExpr(CallExpr fcall, Mutable<ILogicalOperator> tupSource)
+ throws AsterixException {
+ LogicalVariable v = context.newVar();
+ AsterixFunction fid = fcall.getIdent();
+ List<Mutable<ILogicalExpression>> args = new ArrayList<Mutable<ILogicalExpression>>();
+ Mutable<ILogicalOperator> topOp = tupSource;
- for (Expression expr : fcall.getExprList()) {
- switch (expr.getKind()) {
- case VARIABLE_EXPRESSION: {
- LogicalVariable var = context.getVar(((VariableExpr) expr)
- .getVar().getId());
- args.add(new MutableObject<ILogicalExpression>(
- new VariableReferenceExpression(var)));
- break;
- }
- case LITERAL_EXPRESSION: {
- LiteralExpr val = (LiteralExpr) expr;
- args.add(new MutableObject<ILogicalExpression>(
- new ConstantExpression(
- new AsterixConstantValue(ConstantHelper
- .objectFromLiteral(val.getValue())))));
- break;
- }
- default: {
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
- expr, topOp);
- AbstractLogicalOperator o1 = (AbstractLogicalOperator) eo.second
- .getValue();
- args.add(new MutableObject<ILogicalExpression>(eo.first));
- if (o1 != null
- && !(o1.getOperatorTag() == LogicalOperatorTag.ASSIGN && hasOnlyChild(
- o1, topOp))) {
- topOp = eo.second;
- }
- break;
- }
- }
- }
+ for (Expression expr : fcall.getExprList()) {
+ switch (expr.getKind()) {
+ case VARIABLE_EXPRESSION: {
+ LogicalVariable var = context.getVar(((VariableExpr) expr).getVar().getId());
+ args.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(var)));
+ break;
+ }
+ case LITERAL_EXPRESSION: {
+ LiteralExpr val = (LiteralExpr) expr;
+ args.add(new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(
+ ConstantHelper.objectFromLiteral(val.getValue())))));
+ break;
+ }
+ default: {
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(expr, topOp);
+ AbstractLogicalOperator o1 = (AbstractLogicalOperator) eo.second.getValue();
+ args.add(new MutableObject<ILogicalExpression>(eo.first));
+ if (o1 != null && !(o1.getOperatorTag() == LogicalOperatorTag.ASSIGN && hasOnlyChild(o1, topOp))) {
+ topOp = eo.second;
+ }
+ break;
+ }
+ }
+ }
- FunctionIdentifier fi = new FunctionIdentifier(
- AlgebricksBuiltinFunctions.ALGEBRICKS_NS, fid.getFunctionName());
- AsterixFunctionInfo afi = AsterixBuiltinFunctions.lookupFunction(fi);
- FunctionIdentifier builtinAquafi = afi == null ? null : afi
- .getFunctionIdentifier();
+ FunctionIdentifier fi = new FunctionIdentifier(AlgebricksBuiltinFunctions.ALGEBRICKS_NS, fid.getFunctionName());
+ AsterixFunctionInfo afi = AsterixBuiltinFunctions.lookupFunction(fi);
+ FunctionIdentifier builtinAquafi = afi == null ? null : afi.getFunctionIdentifier();
- if (builtinAquafi != null) {
- fi = builtinAquafi;
- } else {
- fi = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
- fid.getFunctionName());
- FunctionIdentifier builtinAsterixFi = AsterixBuiltinFunctions
- .getBuiltinFunctionIdentifier(fi);
- if (builtinAsterixFi != null) {
- fi = builtinAsterixFi;
- }
- }
- AbstractFunctionCallExpression f;
- if (AsterixBuiltinFunctions.isBuiltinAggregateFunction(fi)) {
- f = AsterixBuiltinFunctions.makeAggregateFunctionExpression(fi,
- args);
- } else if (AsterixBuiltinFunctions.isBuiltinUnnestingFunction(fi)) {
- UnnestingFunctionCallExpression ufce = new UnnestingFunctionCallExpression(
- FunctionUtils.getFunctionInfo(fi), args);
- ufce.setReturnsUniqueValues(AsterixBuiltinFunctions
- .returnsUniqueValues(fi));
- f = ufce;
- } else {
- f = new ScalarFunctionCallExpression(
- FunctionUtils.getFunctionInfo(fi), args);
- }
- AssignOperator op = new AssignOperator(v,
- new MutableObject<ILogicalExpression>(f));
- if (topOp != null) {
- op.getInputs().add(topOp);
- }
+ if (builtinAquafi != null) {
+ fi = builtinAquafi;
+ } else {
+ fi = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, fid.getFunctionName());
+ FunctionIdentifier builtinAsterixFi = AsterixBuiltinFunctions.getBuiltinFunctionIdentifier(fi);
+ if (builtinAsterixFi != null) {
+ fi = builtinAsterixFi;
+ }
+ }
+ AbstractFunctionCallExpression f;
+ if (AsterixBuiltinFunctions.isBuiltinAggregateFunction(fi)) {
+ f = AsterixBuiltinFunctions.makeAggregateFunctionExpression(fi, args);
+ } else if (AsterixBuiltinFunctions.isBuiltinUnnestingFunction(fi)) {
+ UnnestingFunctionCallExpression ufce = new UnnestingFunctionCallExpression(
+ FunctionUtils.getFunctionInfo(fi), args);
+ ufce.setReturnsUniqueValues(AsterixBuiltinFunctions.returnsUniqueValues(fi));
+ f = ufce;
+ } else {
+ f = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(fi), args);
+ }
+ AssignOperator op = new AssignOperator(v, new MutableObject<ILogicalExpression>(f));
+ if (topOp != null) {
+ op.getInputs().add(topOp);
+ }
- return new Pair<ILogicalOperator, LogicalVariable>(op, v);
- }
+ return new Pair<ILogicalOperator, LogicalVariable>(op, v);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitFunctionDecl(
- FunctionDecl fd, Mutable<ILogicalOperator> tupSource) {
- // TODO Auto-generated method stub
- throw new NotImplementedException();
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitFunctionDecl(FunctionDecl fd,
+ Mutable<ILogicalOperator> tupSource) {
+ // TODO Auto-generated method stub
+ throw new NotImplementedException();
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitGroupbyClause(
- GroupbyClause gc, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
- GroupByOperator gOp = new GroupByOperator();
- Mutable<ILogicalOperator> topOp = tupSource;
- for (GbyVariableExpressionPair ve : gc.getGbyPairList()) {
- LogicalVariable v;
- VariableExpr vexpr = ve.getVar();
- if (vexpr != null) {
- v = context.newVar(vexpr);
- } else {
- v = context.newVar();
- }
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
- ve.getExpr(), topOp);
- gOp.addGbyExpression(v, eo.first);
- topOp = eo.second;
- }
- for (GbyVariableExpressionPair ve : gc.getDecorPairList()) {
- LogicalVariable v;
- VariableExpr vexpr = ve.getVar();
- if (vexpr != null) {
- v = context.newVar(vexpr);
- } else {
- v = context.newVar();
- }
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
- ve.getExpr(), topOp);
- gOp.addDecorExpression(v, eo.first);
- topOp = eo.second;
- }
- gOp.getInputs().add(topOp);
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitGroupbyClause(GroupbyClause gc,
+ Mutable<ILogicalOperator> tupSource) throws AsterixException {
+ GroupByOperator gOp = new GroupByOperator();
+ Mutable<ILogicalOperator> topOp = tupSource;
+ for (GbyVariableExpressionPair ve : gc.getGbyPairList()) {
+ LogicalVariable v;
+ VariableExpr vexpr = ve.getVar();
+ if (vexpr != null) {
+ v = context.newVar(vexpr);
+ } else {
+ v = context.newVar();
+ }
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(ve.getExpr(), topOp);
+ gOp.addGbyExpression(v, eo.first);
+ topOp = eo.second;
+ }
+ for (GbyVariableExpressionPair ve : gc.getDecorPairList()) {
+ LogicalVariable v;
+ VariableExpr vexpr = ve.getVar();
+ if (vexpr != null) {
+ v = context.newVar(vexpr);
+ } else {
+ v = context.newVar();
+ }
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(ve.getExpr(), topOp);
+ gOp.addDecorExpression(v, eo.first);
+ topOp = eo.second;
+ }
+ gOp.getInputs().add(topOp);
- for (VariableExpr var : gc.getWithVarList()) {
- LogicalVariable aggVar = context.newVar();
- LogicalVariable oldVar = context.getVar(var);
- List<Mutable<ILogicalExpression>> flArgs = new ArrayList<Mutable<ILogicalExpression>>(
- 1);
- flArgs.add(new MutableObject<ILogicalExpression>(
- new VariableReferenceExpression(oldVar)));
- AggregateFunctionCallExpression fListify = AsterixBuiltinFunctions
- .makeAggregateFunctionExpression(
- AsterixBuiltinFunctions.LISTIFY, flArgs);
- AggregateOperator agg = new AggregateOperator(
- mkSingletonArrayList(aggVar),
- (List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(
- fListify)));
- agg.getInputs().add(
- 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
- // the one bound by the aggregation op.
- context.setVar(var, aggVar);
- }
+ for (VariableExpr var : gc.getWithVarList()) {
+ LogicalVariable aggVar = context.newVar();
+ LogicalVariable oldVar = context.getVar(var);
+ List<Mutable<ILogicalExpression>> flArgs = new ArrayList<Mutable<ILogicalExpression>>(1);
+ flArgs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(oldVar)));
+ AggregateFunctionCallExpression fListify = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
+ AsterixBuiltinFunctions.LISTIFY, flArgs);
+ AggregateOperator agg = new AggregateOperator(mkSingletonArrayList(aggVar),
+ (List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(fListify)));
+ agg.getInputs().add(
+ 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
+ // the one bound by the aggregation op.
+ context.setVar(var, aggVar);
+ }
- gOp.getAnnotations().put(OperatorAnnotations.USE_HASH_GROUP_BY,
- gc.hasHashGroupByHint());
- return new Pair<ILogicalOperator, LogicalVariable>(gOp, null);
- }
+ gOp.getAnnotations().put(OperatorAnnotations.USE_HASH_GROUP_BY, gc.hasHashGroupByHint());
+ return new Pair<ILogicalOperator, LogicalVariable>(gOp, null);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitIfExpr(IfExpr ifexpr,
- Mutable<ILogicalOperator> tupSource) throws AsterixException {
- // In the most general case, IfThenElse is translated in the following
- // way.
- //
- // We assign the result of the condition to one variable varCond.
- // We create one subplan which contains the plan for the "then" branch,
- // on top of which there is a selection whose condition is varCond.
- // Similarly, we create one subplan for the "else" branch, in which the
- // selection is not(varCond).
- // Finally, we concatenate the results. (??)
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitIfExpr(IfExpr ifexpr, Mutable<ILogicalOperator> tupSource)
+ throws AsterixException {
+ // In the most general case, IfThenElse is translated in the following
+ // way.
+ //
+ // We assign the result of the condition to one variable varCond.
+ // We create one subplan which contains the plan for the "then" branch,
+ // on top of which there is a selection whose condition is varCond.
+ // Similarly, we create one subplan for the "else" branch, in which the
+ // selection is not(varCond).
+ // Finally, we concatenate the results. (??)
- Pair<ILogicalOperator, LogicalVariable> pCond = ifexpr.getCondExpr()
- .accept(this, tupSource);
- ILogicalOperator opCond = pCond.first;
- LogicalVariable varCond = pCond.second;
+ Pair<ILogicalOperator, LogicalVariable> pCond = ifexpr.getCondExpr().accept(this, tupSource);
+ ILogicalOperator opCond = pCond.first;
+ LogicalVariable varCond = pCond.second;
- SubplanOperator sp = new SubplanOperator();
- Mutable<ILogicalOperator> nestedSource = new MutableObject<ILogicalOperator>(
- new NestedTupleSourceOperator(
- new MutableObject<ILogicalOperator>(sp)));
+ SubplanOperator sp = new SubplanOperator();
+ Mutable<ILogicalOperator> nestedSource = new MutableObject<ILogicalOperator>(new NestedTupleSourceOperator(
+ new MutableObject<ILogicalOperator>(sp)));
- Pair<ILogicalOperator, LogicalVariable> pThen = ifexpr.getThenExpr()
- .accept(this, nestedSource);
- SelectOperator sel1 = new SelectOperator(
- new MutableObject<ILogicalExpression>(
- new VariableReferenceExpression(varCond)));
- sel1.getInputs().add(new MutableObject<ILogicalOperator>(pThen.first));
+ Pair<ILogicalOperator, LogicalVariable> pThen = ifexpr.getThenExpr().accept(this, nestedSource);
+ 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);
- AbstractFunctionCallExpression notVarCond = new ScalarFunctionCallExpression(
- FunctionUtils.getFunctionInfo(AlgebricksBuiltinFunctions.NOT),
- new MutableObject<ILogicalExpression>(
- new VariableReferenceExpression(varCond)));
- SelectOperator sel2 = new SelectOperator(
- new MutableObject<ILogicalExpression>(notVarCond));
- sel2.getInputs().add(new MutableObject<ILogicalOperator>(pElse.first));
+ Pair<ILogicalOperator, LogicalVariable> pElse = ifexpr.getElseExpr().accept(this, nestedSource);
+ AbstractFunctionCallExpression notVarCond = new ScalarFunctionCallExpression(
+ FunctionUtils.getFunctionInfo(AlgebricksBuiltinFunctions.NOT), new MutableObject<ILogicalExpression>(
+ new VariableReferenceExpression(varCond)));
+ SelectOperator sel2 = new SelectOperator(new MutableObject<ILogicalExpression>(notVarCond));
+ sel2.getInputs().add(new MutableObject<ILogicalOperator>(pElse.first));
- ILogicalPlan p1 = new ALogicalPlanImpl(
- new MutableObject<ILogicalOperator>(sel1));
- sp.getNestedPlans().add(p1);
- ILogicalPlan p2 = new ALogicalPlanImpl(
- new MutableObject<ILogicalOperator>(sel2));
- sp.getNestedPlans().add(p2);
+ ILogicalPlan p1 = new ALogicalPlanImpl(new MutableObject<ILogicalOperator>(sel1));
+ sp.getNestedPlans().add(p1);
+ ILogicalPlan p2 = new ALogicalPlanImpl(new MutableObject<ILogicalOperator>(sel2));
+ sp.getNestedPlans().add(p2);
- Mutable<ILogicalOperator> opCondRef = new MutableObject<ILogicalOperator>(
- opCond);
- sp.getInputs().add(opCondRef);
+ Mutable<ILogicalOperator> opCondRef = new MutableObject<ILogicalOperator>(opCond);
+ sp.getInputs().add(opCondRef);
- 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)));
- AssignOperator a = new AssignOperator(resV,
- new MutableObject<ILogicalExpression>(concatNonNull));
- a.getInputs().add(new MutableObject<ILogicalOperator>(sp));
+ 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)));
+ AssignOperator a = new AssignOperator(resV, new MutableObject<ILogicalExpression>(concatNonNull));
+ a.getInputs().add(new MutableObject<ILogicalOperator>(sp));
- return new Pair<ILogicalOperator, LogicalVariable>(a, resV);
- }
+ return new Pair<ILogicalOperator, LogicalVariable>(a, resV);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitLiteralExpr(
- LiteralExpr l, Mutable<ILogicalOperator> tupSource) {
- LogicalVariable var = context.newVar();
- AssignOperator a = new AssignOperator(var,
- new MutableObject<ILogicalExpression>(new ConstantExpression(
- new AsterixConstantValue(ConstantHelper
- .objectFromLiteral(l.getValue())))));
- if (tupSource != null) {
- a.getInputs().add(tupSource);
- }
- return new Pair<ILogicalOperator, LogicalVariable>(a, var);
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitLiteralExpr(LiteralExpr l, Mutable<ILogicalOperator> tupSource) {
+ LogicalVariable var = context.newVar();
+ AssignOperator a = new AssignOperator(var, new MutableObject<ILogicalExpression>(new ConstantExpression(
+ new AsterixConstantValue(ConstantHelper.objectFromLiteral(l.getValue())))));
+ if (tupSource != null) {
+ a.getInputs().add(tupSource);
+ }
+ return new Pair<ILogicalOperator, LogicalVariable>(a, var);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitOperatorExpr(
- OperatorExpr op, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
- ArrayList<OperatorType> ops = op.getOpList();
- int nOps = ops.size();
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitOperatorExpr(OperatorExpr op,
+ Mutable<ILogicalOperator> tupSource) throws AsterixException {
+ ArrayList<OperatorType> ops = op.getOpList();
+ int nOps = ops.size();
- if (nOps > 0
- && (ops.get(0) == OperatorType.AND || ops.get(0) == OperatorType.OR)) {
- return visitAndOrOperator(op, tupSource);
- }
+ if (nOps > 0 && (ops.get(0) == OperatorType.AND || ops.get(0) == OperatorType.OR)) {
+ return visitAndOrOperator(op, tupSource);
+ }
- ArrayList<Expression> exprs = op.getExprList();
+ ArrayList<Expression> exprs = op.getExprList();
- Mutable<ILogicalOperator> topOp = tupSource;
+ Mutable<ILogicalOperator> topOp = tupSource;
- ILogicalExpression currExpr = null;
- for (int i = 0; i <= nOps; i++) {
+ ILogicalExpression currExpr = null;
+ for (int i = 0; i <= nOps; i++) {
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
- exprs.get(i), topOp);
- topOp = p.second;
- ILogicalExpression e = p.first;
- // now look at the operator
- if (i < nOps) {
- if (OperatorExpr.opIsComparison(ops.get(i))) {
- AbstractFunctionCallExpression c = createComparisonExpression(ops
- .get(i));
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(exprs.get(i), topOp);
+ topOp = p.second;
+ ILogicalExpression e = p.first;
+ // now look at the operator
+ if (i < nOps) {
+ if (OperatorExpr.opIsComparison(ops.get(i))) {
+ AbstractFunctionCallExpression c = createComparisonExpression(ops.get(i));
- // chain the operators
- if (i == 0) {
- c.getArguments().add(
- new MutableObject<ILogicalExpression>(e));
- currExpr = c;
- if (op.isBroadcastOperand(i)) {
- BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
- bcast.setObject(BroadcastSide.LEFT);
- c.getAnnotations()
- .put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY,
- bcast);
- }
- } else {
- ((AbstractFunctionCallExpression) currExpr)
- .getArguments()
- .add(new MutableObject<ILogicalExpression>(e));
- c.getArguments()
- .add(new MutableObject<ILogicalExpression>(
- currExpr));
- currExpr = c;
- if (i == 1 && op.isBroadcastOperand(i)) {
- BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
- bcast.setObject(BroadcastSide.RIGHT);
- c.getAnnotations()
- .put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY,
- bcast);
- }
- }
- } else {
- AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(ops
- .get(i));
+ // chain the operators
+ if (i == 0) {
+ c.getArguments().add(new MutableObject<ILogicalExpression>(e));
+ currExpr = c;
+ if (op.isBroadcastOperand(i)) {
+ BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
+ bcast.setObject(BroadcastSide.LEFT);
+ c.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
+ }
+ } else {
+ ((AbstractFunctionCallExpression) currExpr).getArguments().add(
+ new MutableObject<ILogicalExpression>(e));
+ c.getArguments().add(new MutableObject<ILogicalExpression>(currExpr));
+ currExpr = c;
+ if (i == 1 && op.isBroadcastOperand(i)) {
+ BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
+ bcast.setObject(BroadcastSide.RIGHT);
+ c.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
+ }
+ }
+ } else {
+ AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(ops.get(i));
- if (i == 0) {
- f.getArguments().add(
- new MutableObject<ILogicalExpression>(e));
- currExpr = f;
- } else {
- ((AbstractFunctionCallExpression) currExpr)
- .getArguments()
- .add(new MutableObject<ILogicalExpression>(e));
- f.getArguments()
- .add(new MutableObject<ILogicalExpression>(
- currExpr));
- currExpr = f;
- }
- }
- } else { // don't forget the last expression...
- ((AbstractFunctionCallExpression) currExpr).getArguments().add(
- new MutableObject<ILogicalExpression>(e));
- if (i == 1 && op.isBroadcastOperand(i)) {
- BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
- bcast.setObject(BroadcastSide.RIGHT);
- ((AbstractFunctionCallExpression) currExpr)
- .getAnnotations()
- .put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY,
- bcast);
- }
- }
- }
+ if (i == 0) {
+ f.getArguments().add(new MutableObject<ILogicalExpression>(e));
+ currExpr = f;
+ } else {
+ ((AbstractFunctionCallExpression) currExpr).getArguments().add(
+ new MutableObject<ILogicalExpression>(e));
+ f.getArguments().add(new MutableObject<ILogicalExpression>(currExpr));
+ currExpr = f;
+ }
+ }
+ } else { // don't forget the last expression...
+ ((AbstractFunctionCallExpression) currExpr).getArguments()
+ .add(new MutableObject<ILogicalExpression>(e));
+ if (i == 1 && op.isBroadcastOperand(i)) {
+ BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
+ bcast.setObject(BroadcastSide.RIGHT);
+ ((AbstractFunctionCallExpression) currExpr).getAnnotations().put(
+ BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
+ }
+ }
+ }
- LogicalVariable assignedVar = context.newVar();
- AssignOperator a = new AssignOperator(assignedVar,
- new MutableObject<ILogicalExpression>(currExpr));
+ LogicalVariable assignedVar = context.newVar();
+ AssignOperator a = new AssignOperator(assignedVar, new MutableObject<ILogicalExpression>(currExpr));
- a.getInputs().add(topOp);
+ a.getInputs().add(topOp);
- return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
- }
+ return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitOrderbyClause(
- OrderbyClause oc, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitOrderbyClause(OrderbyClause oc,
+ Mutable<ILogicalOperator> tupSource) throws AsterixException {
- OrderOperator ord = new OrderOperator();
- Iterator<OrderModifier> modifIter = oc.getModifierList().iterator();
- Mutable<ILogicalOperator> topOp = tupSource;
- for (Expression e : oc.getOrderbyList()) {
- 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)));
- topOp = p.second;
- }
- ord.getInputs().add(topOp);
- if (oc.getNumTuples() > 0) {
- ord.getAnnotations().put(OperatorAnnotations.CARDINALITY,
- oc.getNumTuples());
- }
- if (oc.getNumFrames() > 0) {
- ord.getAnnotations().put(OperatorAnnotations.MAX_NUMBER_FRAMES,
- oc.getNumFrames());
- }
- return new Pair<ILogicalOperator, LogicalVariable>(ord, null);
- }
+ OrderOperator ord = new OrderOperator();
+ Iterator<OrderModifier> modifIter = oc.getModifierList().iterator();
+ Mutable<ILogicalOperator> topOp = tupSource;
+ for (Expression e : oc.getOrderbyList()) {
+ 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)));
+ topOp = p.second;
+ }
+ ord.getInputs().add(topOp);
+ if (oc.getNumTuples() > 0) {
+ ord.getAnnotations().put(OperatorAnnotations.CARDINALITY, oc.getNumTuples());
+ }
+ if (oc.getNumFrames() > 0) {
+ ord.getAnnotations().put(OperatorAnnotations.MAX_NUMBER_FRAMES, oc.getNumFrames());
+ }
+ return new Pair<ILogicalOperator, LogicalVariable>(ord, null);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitQuantifiedExpression(
- QuantifiedExpression qe, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
- Mutable<ILogicalOperator> topOp = tupSource;
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitQuantifiedExpression(QuantifiedExpression qe,
+ Mutable<ILogicalOperator> tupSource) throws AsterixException {
+ Mutable<ILogicalOperator> topOp = tupSource;
- ILogicalOperator firstOp = null;
- Mutable<ILogicalOperator> lastOp = null;
+ ILogicalOperator firstOp = null;
+ Mutable<ILogicalOperator> lastOp = null;
- for (QuantifiedPair qt : qe.getQuantifiedList()) {
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(
- qt.getExpr(), topOp);
- topOp = eo1.second;
- LogicalVariable uVar = context.newVar(qt.getVarExpr());
- ILogicalOperator u = new UnnestOperator(uVar,
- new MutableObject<ILogicalExpression>(
- makeUnnestExpression(eo1.first)));
+ for (QuantifiedPair qt : qe.getQuantifiedList()) {
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(qt.getExpr(), topOp);
+ topOp = eo1.second;
+ LogicalVariable uVar = context.newVar(qt.getVarExpr());
+ ILogicalOperator u = new UnnestOperator(uVar, new MutableObject<ILogicalExpression>(
+ makeUnnestExpression(eo1.first)));
- if (firstOp == null) {
- firstOp = u;
- }
- if (lastOp != null) {
- u.getInputs().add(lastOp);
- }
- lastOp = new MutableObject<ILogicalOperator>(u);
- }
+ if (firstOp == null) {
+ firstOp = u;
+ }
+ if (lastOp != null) {
+ u.getInputs().add(lastOp);
+ }
+ lastOp = new MutableObject<ILogicalOperator>(u);
+ }
- // We make all the unnest correspond. to quantif. vars. sit on top
- // in the hope of enabling joins & other optimiz.
- firstOp.getInputs().add(topOp);
- topOp = lastOp;
+ // We make all the unnest correspond. to quantif. vars. sit on top
+ // in the hope of enabling joins & other optimiz.
+ firstOp.getInputs().add(topOp);
+ topOp = lastOp;
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(
- qe.getSatisfiesExpr(), topOp);
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(qe.getSatisfiesExpr(), topOp);
- AggregateFunctionCallExpression fAgg;
- SelectOperator s;
- if (qe.getQuantifier() == Quantifier.SOME) {
- s = new SelectOperator(new MutableObject<ILogicalExpression>(
- eo2.first));
- s.getInputs().add(eo2.second);
- fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
- AsterixBuiltinFunctions.NON_EMPTY_STREAM,
- new ArrayList<Mutable<ILogicalExpression>>());
- } else { // EVERY
- List<Mutable<ILogicalExpression>> satExprList = new ArrayList<Mutable<ILogicalExpression>>(
- 1);
- satExprList.add(new MutableObject<ILogicalExpression>(eo2.first));
- s = new SelectOperator(new MutableObject<ILogicalExpression>(
- new ScalarFunctionCallExpression(FunctionUtils
- .getFunctionInfo(AlgebricksBuiltinFunctions.NOT),
- satExprList)));
- s.getInputs().add(eo2.second);
- fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
- AsterixBuiltinFunctions.EMPTY_STREAM,
- new ArrayList<Mutable<ILogicalExpression>>());
- }
- LogicalVariable qeVar = context.newVar();
- AggregateOperator a = new AggregateOperator(
- mkSingletonArrayList(qeVar),
- (List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(
- fAgg)));
- a.getInputs().add(new MutableObject<ILogicalOperator>(s));
- return new Pair<ILogicalOperator, LogicalVariable>(a, qeVar);
- }
+ AggregateFunctionCallExpression fAgg;
+ SelectOperator s;
+ if (qe.getQuantifier() == Quantifier.SOME) {
+ s = new SelectOperator(new MutableObject<ILogicalExpression>(eo2.first));
+ s.getInputs().add(eo2.second);
+ fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(AsterixBuiltinFunctions.NON_EMPTY_STREAM,
+ new ArrayList<Mutable<ILogicalExpression>>());
+ } else { // EVERY
+ List<Mutable<ILogicalExpression>> satExprList = new ArrayList<Mutable<ILogicalExpression>>(1);
+ satExprList.add(new MutableObject<ILogicalExpression>(eo2.first));
+ s = new SelectOperator(new MutableObject<ILogicalExpression>(new ScalarFunctionCallExpression(
+ FunctionUtils.getFunctionInfo(AlgebricksBuiltinFunctions.NOT), satExprList)));
+ s.getInputs().add(eo2.second);
+ fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(AsterixBuiltinFunctions.EMPTY_STREAM,
+ new ArrayList<Mutable<ILogicalExpression>>());
+ }
+ LogicalVariable qeVar = context.newVar();
+ AggregateOperator a = new AggregateOperator(mkSingletonArrayList(qeVar),
+ (List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(fAgg)));
+ a.getInputs().add(new MutableObject<ILogicalOperator>(s));
+ return new Pair<ILogicalOperator, LogicalVariable>(a, qeVar);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitQuery(Query q,
- Mutable<ILogicalOperator> tupSource) throws AsterixException {
- return q.getBody().accept(this, tupSource);
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitQuery(Query q, Mutable<ILogicalOperator> tupSource)
+ throws AsterixException {
+ return q.getBody().accept(this, tupSource);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitRecordConstructor(
- RecordConstructor rc, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
- AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(
- FunctionUtils
- .getFunctionInfo(AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR));
- LogicalVariable v1 = context.newVar();
- AssignOperator a = new AssignOperator(v1,
- new MutableObject<ILogicalExpression>(f));
- Mutable<ILogicalOperator> topOp = tupSource;
- for (FieldBinding fb : rc.getFbList()) {
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(
- fb.getLeftExpr(), topOp);
- f.getArguments().add(
- new MutableObject<ILogicalExpression>(eo1.first));
- topOp = eo1.second;
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(
- fb.getRightExpr(), topOp);
- f.getArguments().add(
- new MutableObject<ILogicalExpression>(eo2.first));
- topOp = eo2.second;
- }
- a.getInputs().add(topOp);
- return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitRecordConstructor(RecordConstructor rc,
+ Mutable<ILogicalOperator> tupSource) throws AsterixException {
+ AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(
+ FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR));
+ LogicalVariable v1 = context.newVar();
+ AssignOperator a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(f));
+ Mutable<ILogicalOperator> topOp = tupSource;
+ for (FieldBinding fb : rc.getFbList()) {
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(fb.getLeftExpr(), topOp);
+ f.getArguments().add(new MutableObject<ILogicalExpression>(eo1.first));
+ topOp = eo1.second;
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(fb.getRightExpr(), topOp);
+ f.getArguments().add(new MutableObject<ILogicalExpression>(eo2.first));
+ topOp = eo2.second;
+ }
+ a.getInputs().add(topOp);
+ return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitListConstructor(
- ListConstructor lc, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
- FunctionIdentifier fid = (lc.getType() == Type.ORDERED_LIST_CONSTRUCTOR) ? AsterixBuiltinFunctions.ORDERED_LIST_CONSTRUCTOR
- : AsterixBuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR;
- AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(
- FunctionUtils.getFunctionInfo(fid));
- LogicalVariable v1 = context.newVar();
- AssignOperator a = new AssignOperator(v1,
- new MutableObject<ILogicalExpression>(f));
- Mutable<ILogicalOperator> topOp = tupSource;
- for (Expression expr : lc.getExprList()) {
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
- expr, topOp);
- f.getArguments().add(
- new MutableObject<ILogicalExpression>(eo.first));
- topOp = eo.second;
- }
- a.getInputs().add(topOp);
- return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitListConstructor(ListConstructor lc,
+ Mutable<ILogicalOperator> tupSource) throws AsterixException {
+ FunctionIdentifier fid = (lc.getType() == Type.ORDERED_LIST_CONSTRUCTOR) ? AsterixBuiltinFunctions.ORDERED_LIST_CONSTRUCTOR
+ : AsterixBuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR;
+ AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(fid));
+ LogicalVariable v1 = context.newVar();
+ AssignOperator a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(f));
+ Mutable<ILogicalOperator> topOp = tupSource;
+ for (Expression expr : lc.getExprList()) {
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(expr, topOp);
+ f.getArguments().add(new MutableObject<ILogicalExpression>(eo.first));
+ topOp = eo.second;
+ }
+ a.getInputs().add(topOp);
+ return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitUnaryExpr(UnaryExpr u,
- Mutable<ILogicalOperator> tupSource) throws AsterixException {
- Expression expr = u.getExpr();
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
- expr, tupSource);
- LogicalVariable v1 = context.newVar();
- AssignOperator a;
- if (u.getSign() == Sign.POSITIVE) {
- a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(
- eo.first));
- } else {
- AbstractFunctionCallExpression m = new ScalarFunctionCallExpression(
- FunctionUtils
- .getFunctionInfo(AsterixBuiltinFunctions.NUMERIC_UNARY_MINUS));
- m.getArguments().add(
- new MutableObject<ILogicalExpression>(eo.first));
- a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(m));
- }
- a.getInputs().add(eo.second);
- return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitUnaryExpr(UnaryExpr u, Mutable<ILogicalOperator> tupSource)
+ throws AsterixException {
+ Expression expr = u.getExpr();
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(expr, tupSource);
+ LogicalVariable v1 = context.newVar();
+ AssignOperator a;
+ if (u.getSign() == Sign.POSITIVE) {
+ a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(eo.first));
+ } else {
+ AbstractFunctionCallExpression m = new ScalarFunctionCallExpression(
+ FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.NUMERIC_UNARY_MINUS));
+ m.getArguments().add(new MutableObject<ILogicalExpression>(eo.first));
+ a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(m));
+ }
+ a.getInputs().add(eo.second);
+ return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitVariableExpr(
- VariableExpr v, Mutable<ILogicalOperator> tupSource) {
- // 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)));
- a.getInputs().add(tupSource);
- return new Pair<ILogicalOperator, LogicalVariable>(a, var);
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitVariableExpr(VariableExpr v, Mutable<ILogicalOperator> tupSource) {
+ // 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)));
+ a.getInputs().add(tupSource);
+ return new Pair<ILogicalOperator, LogicalVariable>(a, var);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitWhereClause(
- WhereClause w, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
- w.getWhereExpr(), tupSource);
- SelectOperator s = new SelectOperator(
- new MutableObject<ILogicalExpression>(p.first));
- s.getInputs().add(p.second);
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitWhereClause(WhereClause w, Mutable<ILogicalOperator> tupSource)
+ throws AsterixException {
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(w.getWhereExpr(), tupSource);
+ SelectOperator s = new SelectOperator(new MutableObject<ILogicalExpression>(p.first));
+ s.getInputs().add(p.second);
- return new Pair<ILogicalOperator, LogicalVariable>(s, null);
- }
+ return new Pair<ILogicalOperator, LogicalVariable>(s, null);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitLimitClause(
- LimitClause lc, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(
- lc.getLimitExpr(), tupSource);
- LimitOperator opLim;
- Expression offset = lc.getOffset();
- if (offset != null) {
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> p2 = aqlExprToAlgExpression(
- offset, p1.second);
- opLim = new LimitOperator(p1.first, p2.first);
- opLim.getInputs().add(p2.second);
- } else {
- opLim = new LimitOperator(p1.first);
- opLim.getInputs().add(p1.second);
- }
- return new Pair<ILogicalOperator, LogicalVariable>(opLim, null);
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitLimitClause(LimitClause lc, Mutable<ILogicalOperator> tupSource)
+ throws AsterixException {
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(lc.getLimitExpr(), tupSource);
+ LimitOperator opLim;
+ Expression offset = lc.getOffset();
+ if (offset != null) {
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> p2 = aqlExprToAlgExpression(offset, p1.second);
+ opLim = new LimitOperator(p1.first, p2.first);
+ opLim.getInputs().add(p2.second);
+ } else {
+ opLim = new LimitOperator(p1.first);
+ opLim.getInputs().add(p1.second);
+ }
+ return new Pair<ILogicalOperator, LogicalVariable>(opLim, null);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitDieClause(DieClause lc,
- Mutable<ILogicalOperator> tupSource) throws AsterixException {
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(
- lc.getDieExpr(), tupSource);
- DieOperator opDie = new DieOperator(p1.first);
- opDie.getInputs().add(p1.second);
- return new Pair<ILogicalOperator, LogicalVariable>(opDie, null);
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitDieClause(DieClause lc, Mutable<ILogicalOperator> tupSource)
+ throws AsterixException {
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(lc.getDieExpr(), tupSource);
+ DieOperator opDie = new DieOperator(p1.first);
+ opDie.getInputs().add(p1.second);
+ return new Pair<ILogicalOperator, LogicalVariable>(opDie, null);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitDistinctClause(
- DistinctClause dc, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
- List<Mutable<ILogicalExpression>> exprList = new ArrayList<Mutable<ILogicalExpression>>();
- Mutable<ILogicalOperator> input = null;
- for (Expression expr : dc.getDistinctByExpr()) {
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
- expr, tupSource);
- exprList.add(new MutableObject<ILogicalExpression>(p.first));
- input = p.second;
- }
- DistinctOperator opDistinct = new DistinctOperator(exprList);
- opDistinct.getInputs().add(input);
- return new Pair<ILogicalOperator, LogicalVariable>(opDistinct, null);
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitDistinctClause(DistinctClause dc,
+ Mutable<ILogicalOperator> tupSource) throws AsterixException {
+ List<Mutable<ILogicalExpression>> exprList = new ArrayList<Mutable<ILogicalExpression>>();
+ Mutable<ILogicalOperator> input = null;
+ for (Expression expr : dc.getDistinctByExpr()) {
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(expr, tupSource);
+ exprList.add(new MutableObject<ILogicalExpression>(p.first));
+ input = p.second;
+ }
+ DistinctOperator opDistinct = new DistinctOperator(exprList);
+ opDistinct.getInputs().add(input);
+ return new Pair<ILogicalOperator, LogicalVariable>(opDistinct, null);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitUnionExpr(
- UnionExpr unionExpr, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
- Mutable<ILogicalOperator> ts = tupSource;
- ILogicalOperator lastOp = null;
- LogicalVariable lastVar = null;
- boolean first = true;
- for (Expression e : unionExpr.getExprs()) {
- if (first) {
- first = false;
- } else {
- ts = new MutableObject<ILogicalOperator>(
- new EmptyTupleSourceOperator());
- }
- Pair<ILogicalOperator, LogicalVariable> p1 = e.accept(this, ts);
- if (lastOp == null) {
- lastOp = p1.first;
- lastVar = p1.second;
- } else {
- LogicalVariable unnestVar1 = context.newVar();
- UnnestOperator unnest1 = new UnnestOperator(
- unnestVar1,
- new MutableObject<ILogicalExpression>(
- makeUnnestExpression(new VariableReferenceExpression(
- lastVar))));
- unnest1.getInputs().add(
- new MutableObject<ILogicalOperator>(lastOp));
- LogicalVariable unnestVar2 = context.newVar();
- UnnestOperator unnest2 = new UnnestOperator(
- unnestVar2,
- new MutableObject<ILogicalExpression>(
- makeUnnestExpression(new VariableReferenceExpression(
- p1.second))));
- unnest2.getInputs().add(
- new MutableObject<ILogicalOperator>(p1.first));
- List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap = new ArrayList<Triple<LogicalVariable, LogicalVariable, LogicalVariable>>(
- 1);
- LogicalVariable resultVar = context.newVar();
- Triple<LogicalVariable, LogicalVariable, LogicalVariable> triple = new Triple<LogicalVariable, LogicalVariable, LogicalVariable>(
- unnestVar1, unnestVar2, resultVar);
- varMap.add(triple);
- UnionAllOperator unionOp = new UnionAllOperator(varMap);
- unionOp.getInputs().add(
- new MutableObject<ILogicalOperator>(unnest1));
- unionOp.getInputs().add(
- new MutableObject<ILogicalOperator>(unnest2));
- lastVar = resultVar;
- lastOp = unionOp;
- }
- }
- LogicalVariable aggVar = context.newVar();
- ArrayList<LogicalVariable> aggregVars = new ArrayList<LogicalVariable>(
- 1);
- aggregVars.add(aggVar);
- List<Mutable<ILogicalExpression>> afcExprs = new ArrayList<Mutable<ILogicalExpression>>(
- 1);
- afcExprs.add(new MutableObject<ILogicalExpression>(
- new VariableReferenceExpression(lastVar)));
- AggregateFunctionCallExpression afc = AsterixBuiltinFunctions
- .makeAggregateFunctionExpression(
- AsterixBuiltinFunctions.LISTIFY, afcExprs);
- ArrayList<Mutable<ILogicalExpression>> aggregExprs = new ArrayList<Mutable<ILogicalExpression>>(
- 1);
- aggregExprs.add(new MutableObject<ILogicalExpression>(afc));
- AggregateOperator agg = new AggregateOperator(aggregVars, aggregExprs);
- agg.getInputs().add(new MutableObject<ILogicalOperator>(lastOp));
- return new Pair<ILogicalOperator, LogicalVariable>(agg, aggVar);
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitUnionExpr(UnionExpr unionExpr,
+ Mutable<ILogicalOperator> tupSource) throws AsterixException {
+ Mutable<ILogicalOperator> ts = tupSource;
+ ILogicalOperator lastOp = null;
+ LogicalVariable lastVar = null;
+ boolean first = true;
+ for (Expression e : unionExpr.getExprs()) {
+ if (first) {
+ first = false;
+ } else {
+ ts = new MutableObject<ILogicalOperator>(new EmptyTupleSourceOperator());
+ }
+ Pair<ILogicalOperator, LogicalVariable> p1 = e.accept(this, ts);
+ if (lastOp == null) {
+ lastOp = p1.first;
+ lastVar = p1.second;
+ } else {
+ LogicalVariable unnestVar1 = context.newVar();
+ UnnestOperator unnest1 = new UnnestOperator(unnestVar1, new MutableObject<ILogicalExpression>(
+ makeUnnestExpression(new VariableReferenceExpression(lastVar))));
+ unnest1.getInputs().add(new MutableObject<ILogicalOperator>(lastOp));
+ LogicalVariable unnestVar2 = context.newVar();
+ UnnestOperator unnest2 = new UnnestOperator(unnestVar2, new MutableObject<ILogicalExpression>(
+ makeUnnestExpression(new VariableReferenceExpression(p1.second))));
+ unnest2.getInputs().add(new MutableObject<ILogicalOperator>(p1.first));
+ List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap = new ArrayList<Triple<LogicalVariable, LogicalVariable, LogicalVariable>>(
+ 1);
+ LogicalVariable resultVar = context.newVar();
+ Triple<LogicalVariable, LogicalVariable, LogicalVariable> triple = new Triple<LogicalVariable, LogicalVariable, LogicalVariable>(
+ unnestVar1, unnestVar2, resultVar);
+ varMap.add(triple);
+ UnionAllOperator unionOp = new UnionAllOperator(varMap);
+ unionOp.getInputs().add(new MutableObject<ILogicalOperator>(unnest1));
+ unionOp.getInputs().add(new MutableObject<ILogicalOperator>(unnest2));
+ lastVar = resultVar;
+ lastOp = unionOp;
+ }
+ }
+ LogicalVariable aggVar = context.newVar();
+ ArrayList<LogicalVariable> aggregVars = new ArrayList<LogicalVariable>(1);
+ aggregVars.add(aggVar);
+ List<Mutable<ILogicalExpression>> afcExprs = new ArrayList<Mutable<ILogicalExpression>>(1);
+ afcExprs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(lastVar)));
+ AggregateFunctionCallExpression afc = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
+ AsterixBuiltinFunctions.LISTIFY, afcExprs);
+ ArrayList<Mutable<ILogicalExpression>> aggregExprs = new ArrayList<Mutable<ILogicalExpression>>(1);
+ aggregExprs.add(new MutableObject<ILogicalExpression>(afc));
+ AggregateOperator agg = new AggregateOperator(aggregVars, aggregExprs);
+ agg.getInputs().add(new MutableObject<ILogicalOperator>(lastOp));
+ return new Pair<ILogicalOperator, LogicalVariable>(agg, aggVar);
+ }
- private AbstractFunctionCallExpression createComparisonExpression(
- OperatorType t) {
- FunctionIdentifier fi = operatorTypeToFunctionIdentifier(t);
- IFunctionInfo finfo = FunctionUtils.getFunctionInfo(fi);
- return new ScalarFunctionCallExpression(finfo);
- }
+ private AbstractFunctionCallExpression createComparisonExpression(OperatorType t) {
+ FunctionIdentifier fi = operatorTypeToFunctionIdentifier(t);
+ IFunctionInfo finfo = FunctionUtils.getFunctionInfo(fi);
+ return new ScalarFunctionCallExpression(finfo);
+ }
- private FunctionIdentifier operatorTypeToFunctionIdentifier(OperatorType t) {
- switch (t) {
- case EQ: {
- return AlgebricksBuiltinFunctions.EQ;
- }
- case NEQ: {
- return AlgebricksBuiltinFunctions.NEQ;
- }
- case GT: {
- return AlgebricksBuiltinFunctions.GT;
- }
- case GE: {
- return AlgebricksBuiltinFunctions.GE;
- }
- case LT: {
- return AlgebricksBuiltinFunctions.LT;
- }
- case LE: {
- return AlgebricksBuiltinFunctions.LE;
- }
- default: {
- throw new IllegalStateException();
- }
- }
- }
+ private FunctionIdentifier operatorTypeToFunctionIdentifier(OperatorType t) {
+ switch (t) {
+ case EQ: {
+ return AlgebricksBuiltinFunctions.EQ;
+ }
+ case NEQ: {
+ return AlgebricksBuiltinFunctions.NEQ;
+ }
+ case GT: {
+ return AlgebricksBuiltinFunctions.GT;
+ }
+ case GE: {
+ return AlgebricksBuiltinFunctions.GE;
+ }
+ case LT: {
+ return AlgebricksBuiltinFunctions.LT;
+ }
+ case LE: {
+ return AlgebricksBuiltinFunctions.LE;
+ }
+ default: {
+ throw new IllegalStateException();
+ }
+ }
+ }
- private AbstractFunctionCallExpression createFunctionCallExpressionForBuiltinOperator(
- OperatorType t) throws AsterixException {
+ private AbstractFunctionCallExpression createFunctionCallExpressionForBuiltinOperator(OperatorType t)
+ throws AsterixException {
- FunctionIdentifier fid = null;
- switch (t) {
- case PLUS: {
- fid = AlgebricksBuiltinFunctions.NUMERIC_ADD;
- break;
- }
- case MINUS: {
- fid = AsterixBuiltinFunctions.NUMERIC_SUBTRACT;
- break;
- }
- case MUL: {
- fid = AsterixBuiltinFunctions.NUMERIC_MULTIPLY;
- break;
- }
- case DIV: {
- fid = AsterixBuiltinFunctions.NUMERIC_DIVIDE;
- break;
- }
- case MOD: {
- fid = AsterixBuiltinFunctions.NUMERIC_MOD;
- break;
- }
- case IDIV: {
- fid = AsterixBuiltinFunctions.NUMERIC_IDIV;
- break;
- }
- case CARET: {
- fid = AsterixBuiltinFunctions.CARET;
- break;
- }
- case AND: {
- fid = AlgebricksBuiltinFunctions.AND;
- break;
- }
- case OR: {
- fid = AlgebricksBuiltinFunctions.OR;
- break;
- }
- case FUZZY_EQ: {
- fid = AsterixBuiltinFunctions.FUZZY_EQ;
- break;
- }
+ FunctionIdentifier fid = null;
+ switch (t) {
+ case PLUS: {
+ fid = AlgebricksBuiltinFunctions.NUMERIC_ADD;
+ break;
+ }
+ case MINUS: {
+ fid = AsterixBuiltinFunctions.NUMERIC_SUBTRACT;
+ break;
+ }
+ case MUL: {
+ fid = AsterixBuiltinFunctions.NUMERIC_MULTIPLY;
+ break;
+ }
+ case DIV: {
+ fid = AsterixBuiltinFunctions.NUMERIC_DIVIDE;
+ break;
+ }
+ case MOD: {
+ fid = AsterixBuiltinFunctions.NUMERIC_MOD;
+ break;
+ }
+ case IDIV: {
+ fid = AsterixBuiltinFunctions.NUMERIC_IDIV;
+ break;
+ }
+ case CARET: {
+ fid = AsterixBuiltinFunctions.CARET;
+ break;
+ }
+ case AND: {
+ fid = AlgebricksBuiltinFunctions.AND;
+ break;
+ }
+ case OR: {
+ fid = AlgebricksBuiltinFunctions.OR;
+ break;
+ }
+ case FUZZY_EQ: {
+ fid = AsterixBuiltinFunctions.FUZZY_EQ;
+ break;
+ }
- default: {
- throw new NotImplementedException("Operator " + t
- + " is not yet implemented");
- }
- }
- return new ScalarFunctionCallExpression(
- FunctionUtils.getFunctionInfo(fid));
- }
+ default: {
+ throw new NotImplementedException("Operator " + t + " is not yet implemented");
+ }
+ }
+ return new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(fid));
+ }
- private static boolean hasOnlyChild(ILogicalOperator parent,
- Mutable<ILogicalOperator> childCandidate) {
- List<Mutable<ILogicalOperator>> inp = parent.getInputs();
- if (inp == null || inp.size() != 1) {
- return false;
- }
- return inp.get(0) == childCandidate;
- }
+ private static boolean hasOnlyChild(ILogicalOperator parent, Mutable<ILogicalOperator> childCandidate) {
+ List<Mutable<ILogicalOperator>> inp = parent.getInputs();
+ if (inp == null || inp.size() != 1) {
+ return false;
+ }
+ return inp.get(0) == childCandidate;
+ }
- private Pair<ILogicalExpression, Mutable<ILogicalOperator>> aqlExprToAlgExpression(
- Expression expr, Mutable<ILogicalOperator> topOp)
- throws AsterixException {
- switch (expr.getKind()) {
- case VARIABLE_EXPRESSION: {
- VariableReferenceExpression ve = new VariableReferenceExpression(
- context.getVar(((VariableExpr) expr).getVar().getId()));
- return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(ve,
- topOp);
- }
- case METAVARIABLE_EXPRESSION: {
- ILogicalExpression le = metaScopeExp
- .getVariableReferenceExpression(((VariableExpr) expr)
- .getVar());
- return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(le,
- topOp);
- }
- case LITERAL_EXPRESSION: {
- LiteralExpr val = (LiteralExpr) expr;
- return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
- new ConstantExpression(new AsterixConstantValue(
- ConstantHelper.objectFromLiteral(val.getValue()))),
- topOp);
- }
- default: {
- // Mutable<ILogicalExpression> src = new
- // Mutable<ILogicalExpression>();
- // Mutable<ILogicalExpression> src = topOp;
- if (expressionNeedsNoNesting(expr)) {
- Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this,
- topOp);
- ILogicalExpression exp = ((AssignOperator) p.first)
- .getExpressions().get(0).getValue();
- return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
- exp, p.first.getInputs().get(0));
- } else {
- Mutable<ILogicalOperator> src = new MutableObject<ILogicalOperator>();
+ private Pair<ILogicalExpression, Mutable<ILogicalOperator>> aqlExprToAlgExpression(Expression expr,
+ Mutable<ILogicalOperator> topOp) throws AsterixException {
+ switch (expr.getKind()) {
+ case VARIABLE_EXPRESSION: {
+ VariableReferenceExpression ve = new VariableReferenceExpression(context.getVar(((VariableExpr) expr)
+ .getVar().getId()));
+ return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(ve, topOp);
+ }
+ case METAVARIABLE_EXPRESSION: {
+ ILogicalExpression le = metaScopeExp.getVariableReferenceExpression(((VariableExpr) expr).getVar());
+ return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(le, topOp);
+ }
+ case LITERAL_EXPRESSION: {
+ LiteralExpr val = (LiteralExpr) expr;
+ return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(new ConstantExpression(
+ new AsterixConstantValue(ConstantHelper.objectFromLiteral(val.getValue()))), topOp);
+ }
+ default: {
+ // Mutable<ILogicalExpression> src = new
+ // Mutable<ILogicalExpression>();
+ // Mutable<ILogicalExpression> src = topOp;
+ if (expressionNeedsNoNesting(expr)) {
+ Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, topOp);
+ ILogicalExpression exp = ((AssignOperator) p.first).getExpressions().get(0).getValue();
+ return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(exp, p.first.getInputs().get(0));
+ } else {
+ Mutable<ILogicalOperator> src = new MutableObject<ILogicalOperator>();
- Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this,
- src);
+ Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, src);
- if (((AbstractLogicalOperator) p.first).getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
- // src.setOperator(topOp.getOperator());
- Mutable<ILogicalOperator> top2 = new MutableObject<ILogicalOperator>(
- p.first);
- return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
- new VariableReferenceExpression(p.second), top2);
- } else {
- SubplanOperator s = new SubplanOperator();
- s.getInputs().add(topOp);
- src.setValue(new NestedTupleSourceOperator(
- new MutableObject<ILogicalOperator>(s)));
- Mutable<ILogicalOperator> planRoot = new MutableObject<ILogicalOperator>(
- p.first);
- s.setRootOp(planRoot);
- return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
- new VariableReferenceExpression(p.second),
- new MutableObject<ILogicalOperator>(s));
- }
- }
- }
- }
+ if (((AbstractLogicalOperator) p.first).getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
+ // src.setOperator(topOp.getOperator());
+ Mutable<ILogicalOperator> top2 = new MutableObject<ILogicalOperator>(p.first);
+ return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(new VariableReferenceExpression(
+ p.second), top2);
+ } else {
+ SubplanOperator s = new SubplanOperator();
+ s.getInputs().add(topOp);
+ src.setValue(new NestedTupleSourceOperator(new MutableObject<ILogicalOperator>(s)));
+ Mutable<ILogicalOperator> planRoot = new MutableObject<ILogicalOperator>(p.first);
+ s.setRootOp(planRoot);
+ return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(new VariableReferenceExpression(
+ p.second), new MutableObject<ILogicalOperator>(s));
+ }
+ }
+ }
+ }
- }
+ }
- private Pair<ILogicalOperator, LogicalVariable> produceFlwrResult(
- boolean noForClause, boolean isTop,
- Mutable<ILogicalOperator> resOpRef, LogicalVariable resVar) {
- if (isTop) {
- ProjectOperator pr = new ProjectOperator(resVar);
- pr.getInputs().add(resOpRef);
- return new Pair<ILogicalOperator, LogicalVariable>(pr, resVar);
+ private Pair<ILogicalOperator, LogicalVariable> produceFlwrResult(boolean noForClause, boolean isTop,
+ Mutable<ILogicalOperator> resOpRef, LogicalVariable resVar) {
+ if (isTop) {
+ ProjectOperator pr = new ProjectOperator(resVar);
+ pr.getInputs().add(resOpRef);
+ return new Pair<ILogicalOperator, LogicalVariable>(pr, resVar);
- } else if (noForClause) {
- return new Pair<ILogicalOperator, LogicalVariable>(
- resOpRef.getValue(), resVar);
- } else {
- return aggListify(resVar, resOpRef, false);
- }
- }
+ } else if (noForClause) {
+ return new Pair<ILogicalOperator, LogicalVariable>(resOpRef.getValue(), resVar);
+ } else {
+ return aggListify(resVar, resOpRef, false);
+ }
+ }
- private Pair<ILogicalOperator, LogicalVariable> aggListify(
- LogicalVariable var, Mutable<ILogicalOperator> opRef,
- boolean bProject) {
- AggregateFunctionCallExpression funAgg = AsterixBuiltinFunctions
- .makeAggregateFunctionExpression(
- AsterixBuiltinFunctions.LISTIFY,
- new ArrayList<Mutable<ILogicalExpression>>());
- 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)));
- agg.getInputs().add(opRef);
- ILogicalOperator res;
- if (bProject) {
- ProjectOperator pr = new ProjectOperator(varListified);
- pr.getInputs().add(new MutableObject<ILogicalOperator>(agg));
- res = pr;
- } else {
- res = agg;
- }
- return new Pair<ILogicalOperator, LogicalVariable>(res, varListified);
- }
+ private Pair<ILogicalOperator, LogicalVariable> aggListify(LogicalVariable var, Mutable<ILogicalOperator> opRef,
+ boolean bProject) {
+ AggregateFunctionCallExpression funAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
+ AsterixBuiltinFunctions.LISTIFY, new ArrayList<Mutable<ILogicalExpression>>());
+ 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)));
+ agg.getInputs().add(opRef);
+ ILogicalOperator res;
+ if (bProject) {
+ ProjectOperator pr = new ProjectOperator(varListified);
+ pr.getInputs().add(new MutableObject<ILogicalOperator>(agg));
+ res = pr;
+ } else {
+ res = agg;
+ }
+ return new Pair<ILogicalOperator, LogicalVariable>(res, varListified);
+ }
- private Pair<ILogicalOperator, LogicalVariable> visitAndOrOperator(
- OperatorExpr op, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
- ArrayList<OperatorType> ops = op.getOpList();
- int nOps = ops.size();
+ private Pair<ILogicalOperator, LogicalVariable> visitAndOrOperator(OperatorExpr op,
+ Mutable<ILogicalOperator> tupSource) throws AsterixException {
+ ArrayList<OperatorType> ops = op.getOpList();
+ int nOps = ops.size();
- ArrayList<Expression> exprs = op.getExprList();
+ ArrayList<Expression> exprs = op.getExprList();
- Mutable<ILogicalOperator> topOp = tupSource;
+ Mutable<ILogicalOperator> topOp = tupSource;
- OperatorType opLogical = ops.get(0);
- AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(opLogical);
+ OperatorType opLogical = ops.get(0);
+ AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(opLogical);
- for (int i = 0; i <= nOps; i++) {
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
- exprs.get(i), topOp);
- topOp = p.second;
- // now look at the operator
- if (i < nOps) {
- if (ops.get(i) != opLogical) {
- throw new TranslationException("Unexpected operator "
- + ops.get(i) + " in an OperatorExpr starting with "
- + opLogical);
- }
- }
- f.getArguments()
- .add(new MutableObject<ILogicalExpression>(p.first));
- }
+ for (int i = 0; i <= nOps; i++) {
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(exprs.get(i), topOp);
+ topOp = p.second;
+ // now look at the operator
+ if (i < nOps) {
+ if (ops.get(i) != opLogical) {
+ throw new TranslationException("Unexpected operator " + ops.get(i)
+ + " in an OperatorExpr starting with " + opLogical);
+ }
+ }
+ f.getArguments().add(new MutableObject<ILogicalExpression>(p.first));
+ }
- LogicalVariable assignedVar = context.newVar();
- AssignOperator a = new AssignOperator(assignedVar,
- new MutableObject<ILogicalExpression>(f));
- a.getInputs().add(topOp);
+ LogicalVariable assignedVar = context.newVar();
+ AssignOperator a = new AssignOperator(assignedVar, new MutableObject<ILogicalExpression>(f));
+ a.getInputs().add(topOp);
- return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
+ return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
- }
+ }
- private static boolean expressionNeedsNoNesting(Expression expr) {
- Kind k = expr.getKind();
- return k == Kind.LITERAL_EXPRESSION
- || k == Kind.LIST_CONSTRUCTOR_EXPRESSION
- || k == Kind.RECORD_CONSTRUCTOR_EXPRESSION
- || k == Kind.VARIABLE_EXPRESSION || k == Kind.CALL_EXPRESSION
- || k == Kind.OP_EXPRESSION
- || k == Kind.FIELD_ACCESSOR_EXPRESSION
- || k == Kind.INDEX_ACCESSOR_EXPRESSION
- || k == Kind.UNARY_EXPRESSION;
- }
+ private static boolean expressionNeedsNoNesting(Expression expr) {
+ Kind k = expr.getKind();
+ return k == Kind.LITERAL_EXPRESSION || k == Kind.LIST_CONSTRUCTOR_EXPRESSION
+ || k == Kind.RECORD_CONSTRUCTOR_EXPRESSION || k == Kind.VARIABLE_EXPRESSION
+ || k == Kind.CALL_EXPRESSION || k == Kind.OP_EXPRESSION || k == Kind.FIELD_ACCESSOR_EXPRESSION
+ || k == Kind.INDEX_ACCESSOR_EXPRESSION || k == Kind.UNARY_EXPRESSION;
+ }
- private <T> ArrayList<T> mkSingletonArrayList(T item) {
- ArrayList<T> array = new ArrayList<T>(1);
- array.add(item);
- return array;
- }
+ private <T> ArrayList<T> mkSingletonArrayList(T item) {
+ ArrayList<T> array = new ArrayList<T>(1);
+ array.add(item);
+ return array;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitTypeDecl(TypeDecl td,
- Mutable<ILogicalOperator> arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitTypeDecl(TypeDecl td, Mutable<ILogicalOperator> arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitRecordTypeDefiniton(
- RecordTypeDefinition tre, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitRecordTypeDefiniton(RecordTypeDefinition tre,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitTypeReferenceExpression(
- TypeReferenceExpression tre, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitTypeReferenceExpression(TypeReferenceExpression tre,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitNodegroupDecl(
- NodegroupDecl ngd, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitNodegroupDecl(NodegroupDecl ngd, Mutable<ILogicalOperator> arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitLoadFromFileStatement(
- LoadFromFileStatement stmtLoad, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitLoadFromFileStatement(LoadFromFileStatement stmtLoad,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitWriteFromQueryResultStatement(
- WriteFromQueryResultStatement stmtLoad,
- Mutable<ILogicalOperator> arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitWriteFromQueryResultStatement(
+ WriteFromQueryResultStatement stmtLoad, Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitDropStatement(
- DropStatement del, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitDropStatement(DropStatement del, Mutable<ILogicalOperator> arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitCreateIndexStatement(
- CreateIndexStatement cis, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitCreateIndexStatement(CreateIndexStatement cis,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitOrderedListTypeDefiniton(
- OrderedListTypeDefinition olte, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitOrderedListTypeDefiniton(OrderedListTypeDefinition olte,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitUnorderedListTypeDefiniton(
- UnorderedListTypeDefinition ulte, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitUnorderedListTypeDefiniton(UnorderedListTypeDefinition ulte,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitMetaVariableClause(
- MetaVariableClause mc, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
- return new Pair<ILogicalOperator, LogicalVariable>(metaScopeOp.get(mc
- .getVar()), null);
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitMetaVariableClause(MetaVariableClause mc,
+ Mutable<ILogicalOperator> tupSource) throws AsterixException {
+ return new Pair<ILogicalOperator, LogicalVariable>(metaScopeOp.get(mc.getVar()), null);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitJoinClause(
- JoinClause jc, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
- // Pair<ILogicalOperator, LogicalVariable> leftSide =
- // jc.getLeftExpr().accept(this, tupSource);
- Mutable<ILogicalOperator> opRef = tupSource;
- Pair<ILogicalOperator, LogicalVariable> leftSide = null;
- for (Clause c : jc.getLeftClauses()) {
- leftSide = c.accept(this, opRef);
- opRef = new MutableObject<ILogicalOperator>(leftSide.first);
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitJoinClause(JoinClause jc, Mutable<ILogicalOperator> tupSource)
+ throws AsterixException {
+ // Pair<ILogicalOperator, LogicalVariable> leftSide =
+ // jc.getLeftExpr().accept(this, tupSource);
+ Mutable<ILogicalOperator> opRef = tupSource;
+ Pair<ILogicalOperator, LogicalVariable> leftSide = null;
+ for (Clause c : jc.getLeftClauses()) {
+ leftSide = c.accept(this, opRef);
+ opRef = new MutableObject<ILogicalOperator>(leftSide.first);
+ }
- // Pair<ILogicalOperator, LogicalVariable> rightSide =
- // jc.getRightExpr().accept(this, tupSource);
- opRef = tupSource;
- Pair<ILogicalOperator, LogicalVariable> rightSide = null;
- for (Clause c : jc.getRightClauses()) {
- rightSide = c.accept(this, opRef);
- opRef = new MutableObject<ILogicalOperator>(rightSide.first);
- }
+ // Pair<ILogicalOperator, LogicalVariable> rightSide =
+ // jc.getRightExpr().accept(this, tupSource);
+ opRef = tupSource;
+ Pair<ILogicalOperator, LogicalVariable> rightSide = null;
+ for (Clause c : jc.getRightClauses()) {
+ rightSide = c.accept(this, opRef);
+ opRef = new MutableObject<ILogicalOperator>(rightSide.first);
+ }
- Pair<ILogicalExpression, Mutable<ILogicalOperator>> whereCond = aqlExprToAlgExpression(
- jc.getWhereExpr(), tupSource);
+ Pair<ILogicalExpression, Mutable<ILogicalOperator>> whereCond = aqlExprToAlgExpression(jc.getWhereExpr(),
+ tupSource);
- AbstractBinaryJoinOperator join;
- switch (jc.getKind()) {
- case INNER: {
- join = new InnerJoinOperator(new MutableObject<ILogicalExpression>(
- whereCond.first));
- break;
- }
- case LEFT_OUTER: {
- join = new LeftOuterJoinOperator(
- new MutableObject<ILogicalExpression>(whereCond.first));
- break;
- }
- default: {
- throw new IllegalStateException();
- }
- }
- join.getInputs().add(
- new MutableObject<ILogicalOperator>(leftSide.first));
- join.getInputs().add(
- new MutableObject<ILogicalOperator>(rightSide.first));
- return new Pair<ILogicalOperator, LogicalVariable>(join, null);
- }
+ AbstractBinaryJoinOperator join;
+ switch (jc.getKind()) {
+ case INNER: {
+ join = new InnerJoinOperator(new MutableObject<ILogicalExpression>(whereCond.first));
+ break;
+ }
+ case LEFT_OUTER: {
+ join = new LeftOuterJoinOperator(new MutableObject<ILogicalExpression>(whereCond.first));
+ break;
+ }
+ default: {
+ throw new IllegalStateException();
+ }
+ }
+ join.getInputs().add(new MutableObject<ILogicalOperator>(leftSide.first));
+ join.getInputs().add(new MutableObject<ILogicalOperator>(rightSide.first));
+ return new Pair<ILogicalOperator, LogicalVariable>(join, null);
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitMetaVariableExpr(
- MetaVariableExpr me, Mutable<ILogicalOperator> tupSource)
- throws AsterixException {
- LogicalVariable var = context.newVar();
- AssignOperator a = new AssignOperator(var,
- new MutableObject<ILogicalExpression>(metaScopeExp
- .getVariableReferenceExpression(me.getVar())));
- a.getInputs().add(tupSource);
- return new Pair<ILogicalOperator, LogicalVariable>(a, var);
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitMetaVariableExpr(MetaVariableExpr me,
+ Mutable<ILogicalOperator> tupSource) throws AsterixException {
+ LogicalVariable var = context.newVar();
+ AssignOperator a = new AssignOperator(var, new MutableObject<ILogicalExpression>(
+ metaScopeExp.getVariableReferenceExpression(me.getVar())));
+ a.getInputs().add(tupSource);
+ return new Pair<ILogicalOperator, LogicalVariable>(a, var);
+ }
- public void addOperatorToMetaScope(Identifier id, ILogicalOperator op) {
- metaScopeOp.put(id, op);
- }
+ public void addOperatorToMetaScope(Identifier id, ILogicalOperator op) {
+ metaScopeOp.put(id, op);
+ }
- public void addVariableToMetaScope(Identifier id, LogicalVariable var) {
- metaScopeExp.put(id, var);
- }
+ public void addVariableToMetaScope(Identifier id, LogicalVariable var) {
+ metaScopeExp.put(id, var);
+ }
- private ILogicalExpression makeUnnestExpression(ILogicalExpression expr) {
- switch (expr.getExpressionTag()) {
- case VARIABLE: {
- return new UnnestingFunctionCallExpression(
- FunctionUtils
- .getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
- new MutableObject<ILogicalExpression>(expr));
- }
- case FUNCTION_CALL: {
- AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
- if (fce.getKind() == FunctionKind.UNNEST) {
- return expr;
- } else {
- return new UnnestingFunctionCallExpression(
- FunctionUtils
- .getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
- new MutableObject<ILogicalExpression>(expr));
- }
- }
- default: {
- return expr;
- }
- }
- }
+ private ILogicalExpression makeUnnestExpression(ILogicalExpression expr) {
+ switch (expr.getExpressionTag()) {
+ case VARIABLE: {
+ return new UnnestingFunctionCallExpression(
+ FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
+ new MutableObject<ILogicalExpression>(expr));
+ }
+ case FUNCTION_CALL: {
+ AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
+ if (fce.getKind() == FunctionKind.UNNEST) {
+ return expr;
+ } else {
+ return new UnnestingFunctionCallExpression(
+ FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
+ new MutableObject<ILogicalExpression>(expr));
+ }
+ }
+ default: {
+ return expr;
+ }
+ }
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitInsertStatement(
- InsertStatement insert, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitInsertStatement(InsertStatement insert,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitDeleteStatement(
- DeleteStatement del, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitDeleteStatement(DeleteStatement del,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitUpdateStatement(
- UpdateStatement update, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitUpdateStatement(UpdateStatement update,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitUpdateClause(
- UpdateClause del, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitUpdateClause(UpdateClause del, Mutable<ILogicalOperator> arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitDataverseDecl(
- DataverseDecl dv, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitDataverseDecl(DataverseDecl dv, Mutable<ILogicalOperator> arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitDatasetDecl(
- DatasetDecl dd, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitDatasetDecl(DatasetDecl dd, Mutable<ILogicalOperator> arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitSetStatement(
- SetStatement ss, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitSetStatement(SetStatement ss, Mutable<ILogicalOperator> arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitWriteStatement(
- WriteStatement ws, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitWriteStatement(WriteStatement ws, Mutable<ILogicalOperator> arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitLoadFromQueryResultStatement(
- WriteFromQueryResultStatement stmtLoad,
- Mutable<ILogicalOperator> arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitLoadFromQueryResultStatement(
+ WriteFromQueryResultStatement stmtLoad, Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitCreateDataverseStatement(
- CreateDataverseStatement del, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitCreateDataverseStatement(CreateDataverseStatement del,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitIndexDropStatement(
- IndexDropStatement del, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitIndexDropStatement(IndexDropStatement del,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitNodeGroupDropStatement(
- NodeGroupDropStatement del, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitNodeGroupDropStatement(NodeGroupDropStatement del,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitDataverseDropStatement(
- DataverseDropStatement del, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitDataverseDropStatement(DataverseDropStatement del,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitTypeDropStatement(
- TypeDropStatement del, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitTypeDropStatement(TypeDropStatement del,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitControlFeedStatement(
- ControlFeedStatement del, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitControlFeedStatement(ControlFeedStatement del,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visit(
- CreateFunctionStatement cfs, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visit(CreateFunctionStatement cfs, Mutable<ILogicalOperator> arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitFunctionDropStatement(
- FunctionDropStatement del, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitFunctionDropStatement(FunctionDropStatement del,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Pair<ILogicalOperator, LogicalVariable> visitBeginFeedStatement(
- BeginFeedStatement bf, Mutable<ILogicalOperator> arg)
- throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitBeginFeedStatement(BeginFeedStatement bf,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/APIFramework.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/APIFramework.java
index 57b2ff0..60b7e6c 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/APIFramework.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/APIFramework.java
@@ -33,7 +33,7 @@
import edu.uci.ics.asterix.optimizer.base.RuleCollections;
import edu.uci.ics.asterix.runtime.job.listener.JobEventListenerFactory;
import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionIDFactory;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobIdFactory;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
import edu.uci.ics.asterix.translator.AqlExpressionToPlanTranslator;
import edu.uci.ics.asterix.translator.DmlTranslator;
@@ -102,6 +102,8 @@
RuleCollections.buildConsolidationRuleCollection()));
defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,
RuleCollections.buildOpPushDownRuleCollection()));
+
+ //put TXnRuleCollection!
return defaultLogicalRewrites;
}
@@ -149,7 +151,7 @@
// Begin a transaction against the metadata.
// Lock the metadata in X mode to protect against other DDL and DML.
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
- MetadataManager.INSTANCE.lock(mdTxnCtx, LockMode.EXCLUSIVE);
+ MetadataManager.INSTANCE.lock(mdTxnCtx, LockMode.X);
try {
DdlTranslator ddlt = new DdlTranslator(mdTxnCtx, query.getPrologDeclList(), out, pc, pdf);
ddlt.translate(hcc, false);
@@ -170,7 +172,7 @@
// Lock the metadata in S mode to protect against other DDL
// modifications.
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
- MetadataManager.INSTANCE.lock(mdTxnCtx, LockMode.SHARED);
+ MetadataManager.INSTANCE.lock(mdTxnCtx, LockMode.S);
try {
DmlTranslator dmlt = new DmlTranslator(mdTxnCtx, query.getPrologDeclList());
dmlt.translate();
@@ -305,7 +307,7 @@
JSONException, RemoteException, ACIDException {
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
try {
- MetadataManager.INSTANCE.lock(mdTxnCtx, LockMode.SHARED);
+ MetadataManager.INSTANCE.lock(mdTxnCtx, LockMode.S);
Pair<AqlCompiledMetadataDeclarations, JobSpecification> result = compileQueryInternal(mdTxnCtx,
dataverseName, q, varCounter, outputDatasetName, metadataDecls, pc, out, pdf, dmlKind);
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -390,8 +392,8 @@
}
}
- long txnId = TransactionIDFactory.generateTransactionId();
- AqlExpressionToPlanTranslator t = new AqlExpressionToPlanTranslator(txnId, mdTxnCtx, rw.getVarCounter(),
+ edu.uci.ics.asterix.transaction.management.service.transaction.JobId asterixJobId = JobIdFactory.generateJobId();
+ AqlExpressionToPlanTranslator t = new AqlExpressionToPlanTranslator(asterixJobId, mdTxnCtx, rw.getVarCounter(),
outputDatasetName, dmlKind);
ILogicalPlanAndMetadata planAndMetadata = t.translate(rwQ, metadataDecls);
@@ -526,7 +528,7 @@
JobSpecification spec = compiler.createJob(AsterixAppContextInfoImpl.INSTANCE);
// set the job event listener
- spec.setJobletEventListenerFactory(new JobEventListenerFactory(txnId, isWriteTransaction));
+ spec.setJobletEventListenerFactory(new JobEventListenerFactory(asterixJobId, isWriteTransaction));
if (pc.isPrintJob()) {
switch (pdf) {
case HTML: {
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 c68817c..6f02b90 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
@@ -82,6 +82,7 @@
import edu.uci.ics.asterix.om.types.AbstractCollectionType;
import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetIdFactory;
import edu.uci.ics.asterix.translator.AbstractAqlTranslator;
import edu.uci.ics.asterix.translator.DmlTranslator.CompiledCreateIndexStatement;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -93,1090 +94,924 @@
public class DdlTranslator extends AbstractAqlTranslator {
- private final MetadataTransactionContext mdTxnCtx;
- private final List<Statement> aqlStatements;
- private final PrintWriter out;
- private final SessionConfig pc;
- private final DisplayFormat pdf;
- private AqlCompiledMetadataDeclarations compiledDeclarations;
+ private final MetadataTransactionContext mdTxnCtx;
+ private final List<Statement> aqlStatements;
+ private final PrintWriter out;
+ private final SessionConfig pc;
+ private final DisplayFormat pdf;
+ private AqlCompiledMetadataDeclarations compiledDeclarations;
- private static Map<String, BuiltinType> builtinTypeMap;
+ private static Map<String, BuiltinType> builtinTypeMap;
- public DdlTranslator(MetadataTransactionContext mdTxnCtx,
- List<Statement> aqlStatements, PrintWriter out, SessionConfig pc,
- DisplayFormat pdf) {
- this.mdTxnCtx = mdTxnCtx;
- this.aqlStatements = aqlStatements;
- this.out = out;
- this.pc = pc;
- this.pdf = pdf;
- builtinTypeMap = AsterixBuiltinTypeMap.getBuiltinTypes();
- }
+ public DdlTranslator(MetadataTransactionContext mdTxnCtx, List<Statement> aqlStatements, PrintWriter out,
+ SessionConfig pc, DisplayFormat pdf) {
+ this.mdTxnCtx = mdTxnCtx;
+ this.aqlStatements = aqlStatements;
+ this.out = out;
+ this.pc = pc;
+ this.pdf = pdf;
+ builtinTypeMap = AsterixBuiltinTypeMap.getBuiltinTypes();
+ }
- public void translate(IHyracksClientConnection hcc,
- boolean disconnectFromDataverse) throws AlgebricksException {
- try {
- compiledDeclarations = compileMetadata(mdTxnCtx, aqlStatements,
- true);
- compileAndExecuteDDLstatements(hcc, mdTxnCtx,
- disconnectFromDataverse);
- } catch (Exception e) {
- throw new AlgebricksException(e);
- }
- }
+ public void translate(IHyracksClientConnection hcc, boolean disconnectFromDataverse) throws AlgebricksException {
+ try {
+ compiledDeclarations = compileMetadata(mdTxnCtx, aqlStatements, true);
+ compileAndExecuteDDLstatements(hcc, mdTxnCtx, disconnectFromDataverse);
+ } catch (Exception e) {
+ throw new AlgebricksException(e);
+ }
+ }
- private void compileAndExecuteDDLstatements(IHyracksClientConnection hcc,
- MetadataTransactionContext mdTxnCtx, boolean disconnectFromDataverse)
- throws Exception {
- for (Statement stmt : aqlStatements) {
- validateOperation(compiledDeclarations, stmt);
- switch (stmt.getKind()) {
- case DATAVERSE_DECL: {
- checkForDataverseConnection(false);
- DataverseDecl dvd = (DataverseDecl) stmt;
- String dataverseName = dvd.getDataverseName().getValue();
- compiledDeclarations.connectToDataverse(dataverseName);
- break;
- }
+ private void compileAndExecuteDDLstatements(IHyracksClientConnection hcc, MetadataTransactionContext mdTxnCtx,
+ boolean disconnectFromDataverse) throws Exception {
+ for (Statement stmt : aqlStatements) {
+ validateOperation(compiledDeclarations, stmt);
+ switch (stmt.getKind()) {
+ case DATAVERSE_DECL: {
+ checkForDataverseConnection(false);
+ DataverseDecl dvd = (DataverseDecl) stmt;
+ String dataverseName = dvd.getDataverseName().getValue();
+ compiledDeclarations.connectToDataverse(dataverseName);
+ break;
+ }
- case CREATE_DATAVERSE: {
- checkForDataverseConnection(false);
- CreateDataverseStatement stmtCreateDataverse = (CreateDataverseStatement) stmt;
- String dvName = stmtCreateDataverse.getDataverseName()
- .getValue();
- Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx,
- dvName);
- if (dv != null && !stmtCreateDataverse.getIfNotExists()) {
- throw new AlgebricksException("A dataverse with this name "
- + dvName + " already exists.");
- }
- MetadataManager.INSTANCE.addDataverse(mdTxnCtx, new Dataverse(
- dvName, stmtCreateDataverse.getFormat()));
- break;
- }
+ case CREATE_DATAVERSE: {
+ checkForDataverseConnection(false);
+ CreateDataverseStatement stmtCreateDataverse = (CreateDataverseStatement) stmt;
+ String dvName = stmtCreateDataverse.getDataverseName().getValue();
+ Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dvName);
+ if (dv != null && !stmtCreateDataverse.getIfNotExists()) {
+ throw new AlgebricksException("A dataverse with this name " + dvName + " already exists.");
+ }
+ MetadataManager.INSTANCE.addDataverse(mdTxnCtx,
+ new Dataverse(dvName, stmtCreateDataverse.getFormat()));
+ break;
+ }
- case DATASET_DECL: {
- checkForDataverseConnection(true);
- DatasetDecl dd = (DatasetDecl) stmt;
- String datasetName = dd.getName().getValue();
- DatasetType dsType = dd.getDatasetType();
- String itemTypeName = null;
- IDatasetDetails datasetDetails = null;
- Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx,
- compiledDeclarations.getDataverseName(), datasetName);
- if (ds != null) {
- if (dd.getIfNotExists()) {
- continue;
- } else {
- throw new AlgebricksException(
- "A dataset with this name " + datasetName
- + " already exists.");
- }
- }
- itemTypeName = dd.getItemTypeName().getValue();
- Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
- compiledDeclarations.getDataverseName(), itemTypeName);
- if (dt == null) {
- throw new AlgebricksException(": type " + itemTypeName
- + " could not be found.");
- }
- switch (dd.getDatasetType()) {
- case INTERNAL: {
- IAType itemType = dt.getDatatype();
- if (itemType.getTypeTag() != ATypeTag.RECORD) {
- throw new AlgebricksException(
- "Can only partition ARecord's.");
- }
- List<String> partitioningExprs = ((InternalDetailsDecl) dd
- .getDatasetDetailsDecl()).getPartitioningExprs();
- String ngName = ((InternalDetailsDecl) dd
- .getDatasetDetailsDecl()).getNodegroupName()
- .getValue();
- datasetDetails = new InternalDatasetDetails(
- InternalDatasetDetails.FileStructure.BTREE,
- InternalDatasetDetails.PartitioningStrategy.HASH,
- partitioningExprs, partitioningExprs, ngName);
- break;
- }
- case EXTERNAL: {
- String adapter = ((ExternalDetailsDecl) dd
- .getDatasetDetailsDecl()).getAdapter();
- Map<String, String> properties = ((ExternalDetailsDecl) dd
- .getDatasetDetailsDecl()).getProperties();
- datasetDetails = new ExternalDatasetDetails(adapter,
- properties);
- break;
- }
- case FEED: {
- IAType itemType = dt.getDatatype();
- if (itemType.getTypeTag() != ATypeTag.RECORD) {
- throw new AlgebricksException(
- "Can only partition ARecord's.");
- }
- List<String> partitioningExprs = ((FeedDetailsDecl) dd
- .getDatasetDetailsDecl()).getPartitioningExprs();
- String ngName = ((FeedDetailsDecl) dd
- .getDatasetDetailsDecl()).getNodegroupName()
- .getValue();
- String adapter = ((FeedDetailsDecl) dd
- .getDatasetDetailsDecl()).getAdapterClassname();
- Map<String, String> properties = ((FeedDetailsDecl) dd
- .getDatasetDetailsDecl()).getProperties();
- String functionIdentifier = ((FeedDetailsDecl) dd
- .getDatasetDetailsDecl()).getFunctionIdentifier();
- datasetDetails = new FeedDatasetDetails(
- InternalDatasetDetails.FileStructure.BTREE,
- InternalDatasetDetails.PartitioningStrategy.HASH,
- partitioningExprs, partitioningExprs, ngName,
- adapter, properties, functionIdentifier,
- FeedDatasetDetails.FeedState.INACTIVE.toString());
- break;
- }
- }
- MetadataManager.INSTANCE.addDataset(mdTxnCtx, new Dataset(
- compiledDeclarations.getDataverseName(), datasetName,
- itemTypeName, datasetDetails, dsType));
- if (dd.getDatasetType() == DatasetType.INTERNAL
- || dd.getDatasetType() == DatasetType.FEED) {
- runCreateDatasetJob(hcc, datasetName);
- }
- break;
- }
+ case DATASET_DECL: {
+ checkForDataverseConnection(true);
+ DatasetDecl dd = (DatasetDecl) stmt;
+ String datasetName = dd.getName().getValue();
+ DatasetType dsType = dd.getDatasetType();
+ String itemTypeName = null;
+ IDatasetDetails datasetDetails = null;
+ Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx, compiledDeclarations.getDataverseName(),
+ datasetName);
+ if (ds != null) {
+ if (dd.getIfNotExists()) {
+ continue;
+ } else {
+ throw new AlgebricksException("A dataset with this name " + datasetName
+ + " already exists.");
+ }
+ }
+ itemTypeName = dd.getItemTypeName().getValue();
+ Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
+ compiledDeclarations.getDataverseName(), itemTypeName);
+ if (dt == null) {
+ throw new AlgebricksException(": type " + itemTypeName + " could not be found.");
+ }
+ switch (dd.getDatasetType()) {
+ case INTERNAL: {
+ IAType itemType = dt.getDatatype();
+ if (itemType.getTypeTag() != ATypeTag.RECORD) {
+ throw new AlgebricksException("Can only partition ARecord's.");
+ }
+ List<String> partitioningExprs = ((InternalDetailsDecl) dd.getDatasetDetailsDecl())
+ .getPartitioningExprs();
+ String ngName = ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).getNodegroupName()
+ .getValue();
+ datasetDetails = new InternalDatasetDetails(InternalDatasetDetails.FileStructure.BTREE,
+ InternalDatasetDetails.PartitioningStrategy.HASH, partitioningExprs,
+ partitioningExprs, ngName);
+ break;
+ }
+ case EXTERNAL: {
+ String adapter = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl()).getAdapter();
+ Map<String, String> properties = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl())
+ .getProperties();
+ datasetDetails = new ExternalDatasetDetails(adapter, properties);
+ break;
+ }
+ case FEED: {
+ IAType itemType = dt.getDatatype();
+ if (itemType.getTypeTag() != ATypeTag.RECORD) {
+ throw new AlgebricksException("Can only partition ARecord's.");
+ }
+ List<String> partitioningExprs = ((FeedDetailsDecl) dd.getDatasetDetailsDecl())
+ .getPartitioningExprs();
+ String ngName = ((FeedDetailsDecl) dd.getDatasetDetailsDecl()).getNodegroupName()
+ .getValue();
+ String adapter = ((FeedDetailsDecl) dd.getDatasetDetailsDecl()).getAdapterClassname();
+ Map<String, String> properties = ((FeedDetailsDecl) dd.getDatasetDetailsDecl())
+ .getProperties();
+ String functionIdentifier = ((FeedDetailsDecl) dd.getDatasetDetailsDecl())
+ .getFunctionIdentifier();
+ datasetDetails = new FeedDatasetDetails(InternalDatasetDetails.FileStructure.BTREE,
+ InternalDatasetDetails.PartitioningStrategy.HASH, partitioningExprs,
+ partitioningExprs, ngName, adapter, properties, functionIdentifier,
+ FeedDatasetDetails.FeedState.INACTIVE.toString());
+ break;
+ }
+ }
+ MetadataManager.INSTANCE.addDataset(mdTxnCtx, new Dataset(compiledDeclarations.getDataverseName(),
+ datasetName, itemTypeName, datasetDetails, dsType, DatasetIdFactory.generateDatasetId()));
+ if (dd.getDatasetType() == DatasetType.INTERNAL || dd.getDatasetType() == DatasetType.FEED) {
+ runCreateDatasetJob(hcc, datasetName);
+ }
+ break;
+ }
- case CREATE_INDEX: {
- checkForDataverseConnection(true);
- CreateIndexStatement stmtCreateIndex = (CreateIndexStatement) stmt;
- String datasetName = stmtCreateIndex.getDatasetName()
- .getValue();
- Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx,
- compiledDeclarations.getDataverseName(), datasetName);
- if (ds == null) {
- throw new AlgebricksException(
- "There is no dataset with this name " + datasetName);
- }
- String indexName = stmtCreateIndex.getIndexName().getValue();
- Index idx = MetadataManager.INSTANCE.getIndex(mdTxnCtx,
- compiledDeclarations.getDataverseName(), datasetName,
- indexName);
- if (idx != null) {
- if (!stmtCreateIndex.getIfNotExists()) {
- throw new AlgebricksException(
- "An index with this name " + indexName
- + " already exists.");
- } else {
- stmtCreateIndex.setNeedToCreate(false);
- }
- } else {
- MetadataManager.INSTANCE.addIndex(
- mdTxnCtx,
- new Index(compiledDeclarations.getDataverseName(),
- datasetName, indexName, stmtCreateIndex
- .getIndexType(), stmtCreateIndex
- .getFieldExprs(), stmtCreateIndex
- .getGramLength(), false));
- runCreateIndexJob(hcc, stmtCreateIndex);
- }
- break;
- }
- case TYPE_DECL: {
- checkForDataverseConnection(true);
- TypeDecl stmtCreateType = (TypeDecl) stmt;
- String typeName = stmtCreateType.getIdent().getValue();
- Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
- compiledDeclarations.getDataverseName(), typeName);
- if (dt != null) {
- if (!stmtCreateType.getIfNotExists())
- throw new AlgebricksException(
- "A datatype with this name " + typeName
- + " already exists.");
- } else {
- if (builtinTypeMap.get(typeName) != null) {
- throw new AlgebricksException(
- "Cannot redefine builtin type " + typeName
- + ".");
- } else {
- Map<String, IAType> typeMap = computeTypes(mdTxnCtx,
- (TypeDecl) stmt);
- IAType type = typeMap.get(typeName);
- MetadataManager.INSTANCE.addDatatype(
- mdTxnCtx,
- new Datatype(compiledDeclarations
- .getDataverseName(), typeName, type,
- false));
- }
- }
- break;
- }
- case NODEGROUP_DECL: {
- NodegroupDecl stmtCreateNodegroup = (NodegroupDecl) stmt;
- String ngName = stmtCreateNodegroup.getNodegroupName()
- .getValue();
- NodeGroup ng = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx,
- ngName);
- if (ng != null) {
- if (!stmtCreateNodegroup.getIfNotExists())
- throw new AlgebricksException(
- "A nodegroup with this name " + ngName
- + " already exists.");
- } else {
- List<Identifier> ncIdentifiers = stmtCreateNodegroup
- .getNodeControllerNames();
- List<String> ncNames = new ArrayList<String>(
- ncIdentifiers.size());
- for (Identifier id : ncIdentifiers) {
- ncNames.add(id.getValue());
- }
- MetadataManager.INSTANCE.addNodegroup(mdTxnCtx,
- new NodeGroup(ngName, ncNames));
- }
- break;
- }
- // drop statements
- case DATAVERSE_DROP: {
- DataverseDropStatement stmtDelete = (DataverseDropStatement) stmt;
- String dvName = stmtDelete.getDataverseName().getValue();
- if (AsterixBuiltinArtifactMap.isSystemProtectedArtifact(
- ARTIFACT_KIND.DATAVERSE, dvName)) {
- throw new AsterixException(
- "Invalid Operation cannot drop dataverse " + dvName
- + " (protected by system)");
- }
+ case CREATE_INDEX: {
+ checkForDataverseConnection(true);
+ CreateIndexStatement stmtCreateIndex = (CreateIndexStatement) stmt;
+ String datasetName = stmtCreateIndex.getDatasetName().getValue();
+ Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx, compiledDeclarations.getDataverseName(),
+ datasetName);
+ if (ds == null) {
+ throw new AlgebricksException("There is no dataset with this name " + datasetName);
+ }
+ String indexName = stmtCreateIndex.getIndexName().getValue();
+ Index idx = MetadataManager.INSTANCE.getIndex(mdTxnCtx, compiledDeclarations.getDataverseName(),
+ datasetName, indexName);
+ if (idx != null) {
+ if (!stmtCreateIndex.getIfNotExists()) {
+ throw new AlgebricksException("An index with this name " + indexName + " already exists.");
+ } else {
+ stmtCreateIndex.setNeedToCreate(false);
+ }
+ } else {
+ MetadataManager.INSTANCE.addIndex(mdTxnCtx, new Index(compiledDeclarations.getDataverseName(),
+ datasetName, indexName, stmtCreateIndex.getIndexType(),
+ stmtCreateIndex.getFieldExprs(), stmtCreateIndex.getGramLength(), false));
+ runCreateIndexJob(hcc, stmtCreateIndex);
+ }
+ break;
+ }
+ case TYPE_DECL: {
+ checkForDataverseConnection(true);
+ TypeDecl stmtCreateType = (TypeDecl) stmt;
+ String typeName = stmtCreateType.getIdent().getValue();
+ Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
+ compiledDeclarations.getDataverseName(), typeName);
+ if (dt != null) {
+ if (!stmtCreateType.getIfNotExists())
+ throw new AlgebricksException("A datatype with this name " + typeName + " already exists.");
+ } else {
+ if (builtinTypeMap.get(typeName) != null) {
+ throw new AlgebricksException("Cannot redefine builtin type " + typeName + ".");
+ } else {
+ Map<String, IAType> typeMap = computeTypes(mdTxnCtx, (TypeDecl) stmt);
+ IAType type = typeMap.get(typeName);
+ MetadataManager.INSTANCE.addDatatype(mdTxnCtx,
+ new Datatype(compiledDeclarations.getDataverseName(), typeName, type, false));
+ }
+ }
+ break;
+ }
+ case NODEGROUP_DECL: {
+ NodegroupDecl stmtCreateNodegroup = (NodegroupDecl) stmt;
+ String ngName = stmtCreateNodegroup.getNodegroupName().getValue();
+ NodeGroup ng = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, ngName);
+ if (ng != null) {
+ if (!stmtCreateNodegroup.getIfNotExists())
+ throw new AlgebricksException("A nodegroup with this name " + ngName + " already exists.");
+ } else {
+ List<Identifier> ncIdentifiers = stmtCreateNodegroup.getNodeControllerNames();
+ List<String> ncNames = new ArrayList<String>(ncIdentifiers.size());
+ for (Identifier id : ncIdentifiers) {
+ ncNames.add(id.getValue());
+ }
+ MetadataManager.INSTANCE.addNodegroup(mdTxnCtx, new NodeGroup(ngName, ncNames));
+ }
+ break;
+ }
+ // drop statements
+ case DATAVERSE_DROP: {
+ DataverseDropStatement stmtDelete = (DataverseDropStatement) stmt;
+ String dvName = stmtDelete.getDataverseName().getValue();
+ if (AsterixBuiltinArtifactMap.isSystemProtectedArtifact(ARTIFACT_KIND.DATAVERSE, dvName)) {
+ throw new AsterixException("Invalid Operation cannot drop dataverse " + dvName
+ + " (protected by system)");
+ }
- if (compiledDeclarations.isConnectedToDataverse())
- compiledDeclarations.disconnectFromDataverse();
- checkForDataverseConnection(false);
+ if (compiledDeclarations.isConnectedToDataverse())
+ compiledDeclarations.disconnectFromDataverse();
+ checkForDataverseConnection(false);
- Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx,
- dvName);
- if (dv == null) {
- if (!stmtDelete.getIfExists()) {
- throw new AlgebricksException(
- "There is no dataverse with this name "
- + dvName + ".");
- }
- } else {
- compiledDeclarations.connectToDataverse(dvName);
- 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).getDatasetType();
- if (dsType == DatasetType.INTERNAL
- || dsType == DatasetType.FEED) {
- List<Index> indexes = MetadataManager.INSTANCE
- .getDatasetIndexes(mdTxnCtx, dvName,
- datasetName);
- for (int k = 0; k < indexes.size(); k++) {
- if (indexes.get(k).isSecondaryIndex()) {
- compileIndexDropStatement(hcc, mdTxnCtx,
- datasetName, indexes.get(k)
- .getIndexName());
- }
- }
- }
- compileDatasetDropStatement(hcc, mdTxnCtx, datasetName);
- }
- MetadataManager.INSTANCE.dropDataverse(mdTxnCtx, dvName);
- if (compiledDeclarations.isConnectedToDataverse())
- compiledDeclarations.disconnectFromDataverse();
- }
- break;
- }
- case DATASET_DROP: {
- checkForDataverseConnection(true);
- DropStatement stmtDelete = (DropStatement) stmt;
- String datasetName = stmtDelete.getDatasetName().getValue();
- if (AsterixBuiltinArtifactMap.isSystemProtectedArtifact(
- ARTIFACT_KIND.DATASET, datasetName)) {
- throw new AsterixException(
- "Invalid Operation cannot drop dataset "
- + datasetName + " (protected by system)");
- }
- Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx,
- compiledDeclarations.getDataverseName(), datasetName);
- if (ds == null) {
- if (!stmtDelete.getIfExists())
- throw new AlgebricksException(
- "There is no dataset with this name "
- + datasetName + ".");
- } else {
- 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++) {
- if (indexes.get(j).isPrimaryIndex()) {
- compileIndexDropStatement(hcc, mdTxnCtx,
- datasetName, indexes.get(j)
- .getIndexName());
- }
- }
- }
- compileDatasetDropStatement(hcc, mdTxnCtx, datasetName);
- }
- break;
- }
- case INDEX_DROP: {
- checkForDataverseConnection(true);
- IndexDropStatement stmtDelete = (IndexDropStatement) stmt;
- String datasetName = stmtDelete.getDatasetName().getValue();
- Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx,
- compiledDeclarations.getDataverseName(), datasetName);
- if (ds == null)
- throw new AlgebricksException(
- "There is no dataset with this name " + datasetName
- + ".");
- if (ds.getDatasetType() == DatasetType.INTERNAL
- || ds.getDatasetType() == DatasetType.FEED) {
- String indexName = stmtDelete.getIndexName().getValue();
- Index idx = MetadataManager.INSTANCE.getIndex(mdTxnCtx,
- compiledDeclarations.getDataverseName(),
- datasetName, indexName);
- if (idx == null) {
- if (!stmtDelete.getIfExists())
- throw new AlgebricksException(
- "There is no index with this name "
- + indexName + ".");
- } else
- compileIndexDropStatement(hcc, mdTxnCtx, datasetName,
- indexName);
- } else {
- throw new AlgebricksException(
- datasetName
- + " is an external dataset. Indexes are not maintained for external datasets.");
- }
- break;
- }
- case TYPE_DROP: {
- checkForDataverseConnection(true);
- TypeDropStatement stmtDelete = (TypeDropStatement) stmt;
- String typeName = stmtDelete.getTypeName().getValue();
- Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
- compiledDeclarations.getDataverseName(), typeName);
- if (dt == null) {
- if (!stmtDelete.getIfExists())
- throw new AlgebricksException(
- "There is no datatype with this name "
- + typeName + ".");
- } else
- MetadataManager.INSTANCE.dropDatatype(mdTxnCtx,
- compiledDeclarations.getDataverseName(), typeName);
- break;
- }
- case NODEGROUP_DROP: {
- NodeGroupDropStatement stmtDelete = (NodeGroupDropStatement) stmt;
- String nodegroupName = stmtDelete.getNodeGroupName().getValue();
- if (AsterixBuiltinArtifactMap.isSystemProtectedArtifact(
- ARTIFACT_KIND.NODEGROUP, nodegroupName)) {
- throw new AsterixException(
- "Invalid Operation cannot drop nodegroup "
- + nodegroupName + " (protected by system)");
- }
- NodeGroup ng = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx,
- nodegroupName);
- if (ng == null) {
- if (!stmtDelete.getIfExists())
- throw new AlgebricksException(
- "There is no nodegroup with this name "
- + nodegroupName + ".");
- } else
- MetadataManager.INSTANCE.dropNodegroup(mdTxnCtx,
- nodegroupName);
- break;
- }
+ Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dvName);
+ if (dv == null) {
+ if (!stmtDelete.getIfExists()) {
+ throw new AlgebricksException("There is no dataverse with this name " + dvName + ".");
+ }
+ } else {
+ compiledDeclarations.connectToDataverse(dvName);
+ 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).getDatasetType();
+ if (dsType == DatasetType.INTERNAL || dsType == DatasetType.FEED) {
+ List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dvName,
+ datasetName);
+ for (int k = 0; k < indexes.size(); k++) {
+ if (indexes.get(k).isSecondaryIndex()) {
+ compileIndexDropStatement(hcc, mdTxnCtx, datasetName, indexes.get(k)
+ .getIndexName());
+ }
+ }
+ }
+ compileDatasetDropStatement(hcc, mdTxnCtx, datasetName);
+ }
+ MetadataManager.INSTANCE.dropDataverse(mdTxnCtx, dvName);
+ if (compiledDeclarations.isConnectedToDataverse())
+ compiledDeclarations.disconnectFromDataverse();
+ }
+ break;
+ }
+ case DATASET_DROP: {
+ checkForDataverseConnection(true);
+ DropStatement stmtDelete = (DropStatement) stmt;
+ String datasetName = stmtDelete.getDatasetName().getValue();
+ if (AsterixBuiltinArtifactMap.isSystemProtectedArtifact(ARTIFACT_KIND.DATASET, datasetName)) {
+ throw new AsterixException("Invalid Operation cannot drop dataset " + datasetName
+ + " (protected by system)");
+ }
+ Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx, compiledDeclarations.getDataverseName(),
+ datasetName);
+ if (ds == null) {
+ if (!stmtDelete.getIfExists())
+ throw new AlgebricksException("There is no dataset with this name " + datasetName + ".");
+ } else {
+ 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++) {
+ if (indexes.get(j).isPrimaryIndex()) {
+ compileIndexDropStatement(hcc, mdTxnCtx, datasetName, indexes.get(j).getIndexName());
+ }
+ }
+ }
+ compileDatasetDropStatement(hcc, mdTxnCtx, datasetName);
+ }
+ break;
+ }
+ case INDEX_DROP: {
+ checkForDataverseConnection(true);
+ IndexDropStatement stmtDelete = (IndexDropStatement) stmt;
+ String datasetName = stmtDelete.getDatasetName().getValue();
+ Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx, compiledDeclarations.getDataverseName(),
+ datasetName);
+ if (ds == null)
+ throw new AlgebricksException("There is no dataset with this name " + datasetName + ".");
+ if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED) {
+ String indexName = stmtDelete.getIndexName().getValue();
+ Index idx = MetadataManager.INSTANCE.getIndex(mdTxnCtx,
+ compiledDeclarations.getDataverseName(), datasetName, indexName);
+ if (idx == null) {
+ if (!stmtDelete.getIfExists())
+ throw new AlgebricksException("There is no index with this name " + indexName + ".");
+ } else
+ compileIndexDropStatement(hcc, mdTxnCtx, datasetName, indexName);
+ } else {
+ throw new AlgebricksException(datasetName
+ + " is an external dataset. Indexes are not maintained for external datasets.");
+ }
+ break;
+ }
+ case TYPE_DROP: {
+ checkForDataverseConnection(true);
+ TypeDropStatement stmtDelete = (TypeDropStatement) stmt;
+ String typeName = stmtDelete.getTypeName().getValue();
+ Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
+ compiledDeclarations.getDataverseName(), typeName);
+ if (dt == null) {
+ if (!stmtDelete.getIfExists())
+ throw new AlgebricksException("There is no datatype with this name " + typeName + ".");
+ } else
+ MetadataManager.INSTANCE.dropDatatype(mdTxnCtx, compiledDeclarations.getDataverseName(),
+ typeName);
+ break;
+ }
+ case NODEGROUP_DROP: {
+ NodeGroupDropStatement stmtDelete = (NodeGroupDropStatement) stmt;
+ String nodegroupName = stmtDelete.getNodeGroupName().getValue();
+ if (AsterixBuiltinArtifactMap.isSystemProtectedArtifact(ARTIFACT_KIND.NODEGROUP, nodegroupName)) {
+ throw new AsterixException("Invalid Operation cannot drop nodegroup " + nodegroupName
+ + " (protected by system)");
+ }
+ NodeGroup ng = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, nodegroupName);
+ if (ng == null) {
+ if (!stmtDelete.getIfExists())
+ throw new AlgebricksException("There is no nodegroup with this name " + nodegroupName + ".");
+ } else
+ MetadataManager.INSTANCE.dropNodegroup(mdTxnCtx, nodegroupName);
+ break;
+ }
- case CREATE_FUNCTION: {
- CreateFunctionStatement cfs = (CreateFunctionStatement) stmt;
- Function function = new Function(
- compiledDeclarations.getDataverseName(), cfs
- .getFunctionIdentifier().getFunctionName(), cfs
- .getFunctionIdentifier().getArity(),
- cfs.getParamList(), cfs.getFunctionBody());
- try {
- FunctionUtils.getFunctionDecl(function);
- } catch (Exception e) {
- throw new AsterixException(
- "Unable to compile function definition", e);
- }
- MetadataManager.INSTANCE
- .addFunction(mdTxnCtx, new Function(
- compiledDeclarations.getDataverseName(), cfs
- .getFunctionIdentifier()
- .getFunctionName(), cfs
- .getFunctionIdentifier().getArity(),
- cfs.getParamList(), cfs.getFunctionBody()));
- break;
- }
+ case CREATE_FUNCTION: {
+ CreateFunctionStatement cfs = (CreateFunctionStatement) stmt;
+ Function function = new Function(compiledDeclarations.getDataverseName(), cfs
+ .getFunctionIdentifier().getFunctionName(), cfs.getFunctionIdentifier().getArity(),
+ cfs.getParamList(), cfs.getFunctionBody());
+ try {
+ FunctionUtils.getFunctionDecl(function);
+ } catch (Exception e) {
+ throw new AsterixException("Unable to compile function definition", e);
+ }
+ MetadataManager.INSTANCE.addFunction(mdTxnCtx, new Function(
+ compiledDeclarations.getDataverseName(), cfs.getFunctionIdentifier().getFunctionName(), cfs
+ .getFunctionIdentifier().getArity(), cfs.getParamList(), cfs.getFunctionBody()));
+ break;
+ }
- case FUNCTION_DROP: {
- checkForDataverseConnection(true);
- FunctionDropStatement stmtDropFunction = (FunctionDropStatement) stmt;
- String functionName = stmtDropFunction.getFunctionName()
- .getValue();
- FunctionIdentifier fId = new FunctionIdentifier(
- FunctionConstants.ASTERIX_NS, functionName,
- stmtDropFunction.getArity());
- if (AsterixBuiltinArtifactMap.isSystemProtectedArtifact(
- ARTIFACT_KIND.FUNCTION, fId)) {
- throw new AsterixException(
- "Invalid Operation cannot drop function "
- + functionName + " (protected by system)");
- }
- Function function = MetadataManager.INSTANCE.getFunction(
- mdTxnCtx, compiledDeclarations.getDataverseName(),
- functionName, stmtDropFunction.getArity());
- if (function == null) {
- if (!stmtDropFunction.getIfExists())
- throw new AlgebricksException(
- "There is no function with this name "
- + functionName + ".");
- } else {
- MetadataManager.INSTANCE.dropFunction(mdTxnCtx,
- compiledDeclarations.getDataverseName(),
- functionName, stmtDropFunction.getArity());
- }
- break;
- }
- }
- }
+ case FUNCTION_DROP: {
+ checkForDataverseConnection(true);
+ FunctionDropStatement stmtDropFunction = (FunctionDropStatement) stmt;
+ String functionName = stmtDropFunction.getFunctionName().getValue();
+ FunctionIdentifier fId = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, functionName,
+ stmtDropFunction.getArity());
+ if (AsterixBuiltinArtifactMap.isSystemProtectedArtifact(ARTIFACT_KIND.FUNCTION, fId)) {
+ throw new AsterixException("Invalid Operation cannot drop function " + functionName
+ + " (protected by system)");
+ }
+ Function function = MetadataManager.INSTANCE.getFunction(mdTxnCtx,
+ compiledDeclarations.getDataverseName(), functionName, stmtDropFunction.getArity());
+ if (function == null) {
+ if (!stmtDropFunction.getIfExists())
+ throw new AlgebricksException("There is no function with this name " + functionName + ".");
+ } else {
+ MetadataManager.INSTANCE.dropFunction(mdTxnCtx, compiledDeclarations.getDataverseName(),
+ functionName, stmtDropFunction.getArity());
+ }
+ break;
+ }
+ }
+ }
- if (disconnectFromDataverse) {
- if (compiledDeclarations.isConnectedToDataverse()) {
- compiledDeclarations.disconnectFromDataverse();
- }
- }
- }
+ if (disconnectFromDataverse) {
+ if (compiledDeclarations.isConnectedToDataverse()) {
+ compiledDeclarations.disconnectFromDataverse();
+ }
+ }
+ }
- private void checkForDataverseConnection(boolean needConnection)
- throws AlgebricksException {
- if (compiledDeclarations.isConnectedToDataverse() != needConnection) {
- if (needConnection)
- throw new AlgebricksException(
- "You need first to connect to a dataverse.");
- else
- throw new AlgebricksException(
- "You need first to disconnect from the dataverse.");
- }
- }
+ private void checkForDataverseConnection(boolean needConnection) throws AlgebricksException {
+ if (compiledDeclarations.isConnectedToDataverse() != needConnection) {
+ if (needConnection)
+ throw new AlgebricksException("You need first to connect to a dataverse.");
+ else
+ throw new AlgebricksException("You need first to disconnect from the dataverse.");
+ }
+ }
- private void runJob(IHyracksClientConnection hcc, JobSpecification jobSpec)
- throws Exception {
- System.out.println(jobSpec.toString());
- executeJobArray(hcc, new JobSpecification[] { jobSpec }, out, pdf);
- }
+ private void runJob(IHyracksClientConnection hcc, JobSpecification jobSpec) throws Exception {
+ System.out.println(jobSpec.toString());
+ executeJobArray(hcc, new JobSpecification[] { jobSpec }, out, pdf);
+ }
- public void executeJobArray(IHyracksClientConnection hcc,
- JobSpecification[] specs, PrintWriter out, DisplayFormat pdf)
- throws Exception {
- for (int i = 0; i < specs.length; i++) {
- specs[i].setMaxReattempts(0);
- JobId jobId = hcc.startJob(GlobalConfig.HYRACKS_APP_NAME, specs[i]);
- hcc.waitForCompletion(jobId);
- }
- }
+ public void executeJobArray(IHyracksClientConnection hcc, JobSpecification[] specs, PrintWriter out,
+ DisplayFormat pdf) throws Exception {
+ for (int i = 0; i < specs.length; i++) {
+ specs[i].setMaxReattempts(0);
+ JobId jobId = hcc.startJob(GlobalConfig.HYRACKS_APP_NAME, specs[i]);
+ hcc.waitForCompletion(jobId);
+ }
+ }
- private void runCreateDatasetJob(IHyracksClientConnection hcc,
- String datasetName) throws AsterixException, AlgebricksException,
- Exception {
- runJob(hcc, DatasetOperations.createDatasetJobSpec(datasetName,
- compiledDeclarations));
- }
+ private void runCreateDatasetJob(IHyracksClientConnection hcc, String datasetName) throws AsterixException,
+ AlgebricksException, Exception {
+ runJob(hcc, DatasetOperations.createDatasetJobSpec(datasetName, compiledDeclarations));
+ }
- private void runCreateIndexJob(IHyracksClientConnection hcc,
- CreateIndexStatement stmtCreateIndex) throws Exception {
- // TODO: Eventually CreateIndexStatement and
- // CompiledCreateIndexStatement should be replaced by the corresponding
- // metadata entity.
- // For now we must still convert to a CompiledCreateIndexStatement here.
- CompiledCreateIndexStatement createIndexStmt = new CompiledCreateIndexStatement(
- stmtCreateIndex.getIndexName().getValue(), stmtCreateIndex
- .getDatasetName().getValue(),
- stmtCreateIndex.getFieldExprs(),
- stmtCreateIndex.getGramLength(), stmtCreateIndex.getIndexType());
- JobSpecification spec = IndexOperations
- .buildSecondaryIndexCreationJobSpec(createIndexStmt,
- compiledDeclarations);
- if (spec == null) {
- throw new AsterixException(
- "Failed to create job spec for creating index '"
- + stmtCreateIndex.getDatasetName() + "."
- + stmtCreateIndex.getIndexName() + "'");
- }
- runJob(hcc, spec);
- }
+ private void runCreateIndexJob(IHyracksClientConnection hcc, CreateIndexStatement stmtCreateIndex) throws Exception {
+ // TODO: Eventually CreateIndexStatement and
+ // CompiledCreateIndexStatement should be replaced by the corresponding
+ // metadata entity.
+ // For now we must still convert to a CompiledCreateIndexStatement here.
+ CompiledCreateIndexStatement createIndexStmt = new CompiledCreateIndexStatement(stmtCreateIndex.getIndexName()
+ .getValue(), stmtCreateIndex.getDatasetName().getValue(), stmtCreateIndex.getFieldExprs(),
+ stmtCreateIndex.getGramLength(), stmtCreateIndex.getIndexType());
+ JobSpecification spec = IndexOperations.buildSecondaryIndexCreationJobSpec(createIndexStmt,
+ compiledDeclarations);
+ if (spec == null) {
+ throw new AsterixException("Failed to create job spec for creating index '"
+ + stmtCreateIndex.getDatasetName() + "." + stmtCreateIndex.getIndexName() + "'");
+ }
+ runJob(hcc, spec);
+ }
- private void compileDatasetDropStatement(IHyracksClientConnection hcc,
- MetadataTransactionContext mdTxnCtx, String datasetName)
- throws Exception {
- CompiledDatasetDropStatement cds = new CompiledDatasetDropStatement(
- datasetName);
- Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx,
- compiledDeclarations.getDataverseName(), datasetName);
- if (ds.getDatasetType() == DatasetType.INTERNAL
- || ds.getDatasetType() == DatasetType.FEED) {
- JobSpecification[] jobs = DatasetOperations
- .createDropDatasetJobSpec(cds, compiledDeclarations);
- for (JobSpecification job : jobs)
- runJob(hcc, job);
- }
- MetadataManager.INSTANCE.dropDataset(mdTxnCtx,
- compiledDeclarations.getDataverseName(), datasetName);
- }
+ private void compileDatasetDropStatement(IHyracksClientConnection hcc, MetadataTransactionContext mdTxnCtx,
+ String datasetName) throws Exception {
+ CompiledDatasetDropStatement cds = new CompiledDatasetDropStatement(datasetName);
+ Dataset ds = MetadataManager.INSTANCE
+ .getDataset(mdTxnCtx, compiledDeclarations.getDataverseName(), datasetName);
+ if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED) {
+ JobSpecification[] jobs = DatasetOperations.createDropDatasetJobSpec(cds, compiledDeclarations);
+ for (JobSpecification job : jobs)
+ runJob(hcc, job);
+ }
+ MetadataManager.INSTANCE.dropDataset(mdTxnCtx, compiledDeclarations.getDataverseName(), datasetName);
+ }
- public AqlCompiledMetadataDeclarations getCompiledDeclarations() {
- return compiledDeclarations;
- }
+ public AqlCompiledMetadataDeclarations getCompiledDeclarations() {
+ return compiledDeclarations;
+ }
- private void compileIndexDropStatement(IHyracksClientConnection hcc,
- MetadataTransactionContext mdTxnCtx, String datasetName,
- String indexName) throws Exception {
- CompiledIndexDropStatement cds = new CompiledIndexDropStatement(
- datasetName, indexName);
- runJob(hcc, IndexOperations.buildDropSecondaryIndexJobSpec(cds,
- compiledDeclarations));
- MetadataManager.INSTANCE
- .dropIndex(mdTxnCtx, compiledDeclarations.getDataverseName(),
- datasetName, indexName);
- }
+ private void compileIndexDropStatement(IHyracksClientConnection hcc, MetadataTransactionContext mdTxnCtx,
+ String datasetName, String indexName) throws Exception {
+ CompiledIndexDropStatement cds = new CompiledIndexDropStatement(datasetName, indexName);
+ runJob(hcc, IndexOperations.buildDropSecondaryIndexJobSpec(cds, compiledDeclarations));
+ MetadataManager.INSTANCE.dropIndex(mdTxnCtx, compiledDeclarations.getDataverseName(), datasetName, indexName);
+ }
- private Map<String, IAType> computeTypes(
- MetadataTransactionContext mdTxnCtx, TypeDecl tDec)
- throws AlgebricksException, MetadataException {
- Map<String, IAType> typeMap = new HashMap<String, IAType>();
- Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes = new HashMap<String, Map<ARecordType, List<Integer>>>();
- Map<String, List<AbstractCollectionType>> incompleteItemTypes = new HashMap<String, List<AbstractCollectionType>>();
- Map<String, List<String>> incompleteTopLevelTypeReferences = new HashMap<String, List<String>>();
+ private Map<String, IAType> computeTypes(MetadataTransactionContext mdTxnCtx, TypeDecl tDec)
+ throws AlgebricksException, MetadataException {
+ Map<String, IAType> typeMap = new HashMap<String, IAType>();
+ Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes = new HashMap<String, Map<ARecordType, List<Integer>>>();
+ Map<String, List<AbstractCollectionType>> incompleteItemTypes = new HashMap<String, List<AbstractCollectionType>>();
+ Map<String, List<String>> incompleteTopLevelTypeReferences = new HashMap<String, List<String>>();
- firstPass(tDec, typeMap, incompleteFieldTypes, incompleteItemTypes,
- incompleteTopLevelTypeReferences);
- secondPass(mdTxnCtx, typeMap, incompleteFieldTypes,
- incompleteItemTypes, incompleteTopLevelTypeReferences);
+ firstPass(tDec, typeMap, incompleteFieldTypes, incompleteItemTypes, incompleteTopLevelTypeReferences);
+ secondPass(mdTxnCtx, typeMap, incompleteFieldTypes, incompleteItemTypes, incompleteTopLevelTypeReferences);
- return typeMap;
- }
+ return typeMap;
+ }
- private void secondPass(MetadataTransactionContext mdTxnCtx,
- Map<String, IAType> typeMap,
- Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
- Map<String, List<AbstractCollectionType>> incompleteItemTypes,
- Map<String, List<String>> incompleteTopLevelTypeReferences)
- throws AlgebricksException, MetadataException {
- // solve remaining top level references
- for (String trefName : incompleteTopLevelTypeReferences.keySet()) {
- IAType t;// = typeMap.get(trefName);
- Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
- compiledDeclarations.getDataverseName(), trefName);
- if (dt == null) {
- throw new AlgebricksException("Could not resolve type "
- + trefName);
- } else
- t = dt.getDatatype();
- for (String tname : incompleteTopLevelTypeReferences.get(trefName)) {
- typeMap.put(tname, t);
- }
- }
- // solve remaining field type references
- for (String trefName : incompleteFieldTypes.keySet()) {
- IAType t;// = typeMap.get(trefName);
- Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
- compiledDeclarations.getDataverseName(), trefName);
- if (dt == null) {
- throw new AlgebricksException("Could not resolve type "
- + trefName);
- } else
- t = dt.getDatatype();
- Map<ARecordType, List<Integer>> fieldsToFix = incompleteFieldTypes
- .get(trefName);
- for (ARecordType recType : fieldsToFix.keySet()) {
- List<Integer> positions = fieldsToFix.get(recType);
- IAType[] fldTypes = recType.getFieldTypes();
- for (Integer pos : positions) {
- if (fldTypes[pos] == null) {
- fldTypes[pos] = t;
- } else { // nullable
- AUnionType nullableUnion = (AUnionType) fldTypes[pos];
- nullableUnion.setTypeAtIndex(t, 1);
- }
- }
- }
- }
- // solve remaining item type references
- for (String trefName : incompleteItemTypes.keySet()) {
- IAType t;// = typeMap.get(trefName);
- Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
- compiledDeclarations.getDataverseName(), trefName);
- if (dt == null) {
- throw new AlgebricksException("Could not resolve type "
- + trefName);
- } else
- t = dt.getDatatype();
- for (AbstractCollectionType act : incompleteItemTypes.get(trefName)) {
- act.setItemType(t);
- }
- }
- }
+ private void secondPass(MetadataTransactionContext mdTxnCtx, Map<String, IAType> typeMap,
+ Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
+ Map<String, List<AbstractCollectionType>> incompleteItemTypes,
+ Map<String, List<String>> incompleteTopLevelTypeReferences) throws AlgebricksException, MetadataException {
+ // solve remaining top level references
+ for (String trefName : incompleteTopLevelTypeReferences.keySet()) {
+ IAType t;// = typeMap.get(trefName);
+ Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, compiledDeclarations.getDataverseName(),
+ trefName);
+ if (dt == null) {
+ throw new AlgebricksException("Could not resolve type " + trefName);
+ } else
+ t = dt.getDatatype();
+ for (String tname : incompleteTopLevelTypeReferences.get(trefName)) {
+ typeMap.put(tname, t);
+ }
+ }
+ // solve remaining field type references
+ for (String trefName : incompleteFieldTypes.keySet()) {
+ IAType t;// = typeMap.get(trefName);
+ Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, compiledDeclarations.getDataverseName(),
+ trefName);
+ if (dt == null) {
+ throw new AlgebricksException("Could not resolve type " + trefName);
+ } else
+ t = dt.getDatatype();
+ Map<ARecordType, List<Integer>> fieldsToFix = incompleteFieldTypes.get(trefName);
+ for (ARecordType recType : fieldsToFix.keySet()) {
+ List<Integer> positions = fieldsToFix.get(recType);
+ IAType[] fldTypes = recType.getFieldTypes();
+ for (Integer pos : positions) {
+ if (fldTypes[pos] == null) {
+ fldTypes[pos] = t;
+ } else { // nullable
+ AUnionType nullableUnion = (AUnionType) fldTypes[pos];
+ nullableUnion.setTypeAtIndex(t, 1);
+ }
+ }
+ }
+ }
+ // solve remaining item type references
+ for (String trefName : incompleteItemTypes.keySet()) {
+ IAType t;// = typeMap.get(trefName);
+ Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, compiledDeclarations.getDataverseName(),
+ trefName);
+ if (dt == null) {
+ throw new AlgebricksException("Could not resolve type " + trefName);
+ } else
+ t = dt.getDatatype();
+ for (AbstractCollectionType act : incompleteItemTypes.get(trefName)) {
+ act.setItemType(t);
+ }
+ }
+ }
- private void firstPass(TypeDecl td, Map<String, IAType> typeMap,
- Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
- Map<String, List<AbstractCollectionType>> incompleteItemTypes,
- Map<String, List<String>> incompleteTopLevelTypeReferences)
- throws AlgebricksException {
+ private void firstPass(TypeDecl td, Map<String, IAType> typeMap,
+ Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
+ Map<String, List<AbstractCollectionType>> incompleteItemTypes,
+ Map<String, List<String>> incompleteTopLevelTypeReferences) throws AlgebricksException {
- TypeExpression texpr = td.getTypeDef();
- String tdname = td.getIdent().getValue();
- if (builtinTypeMap.get(tdname) != null) {
- throw new AlgebricksException("Cannot redefine builtin type "
- + tdname + " .");
- }
- switch (texpr.getTypeKind()) {
- case TYPEREFERENCE: {
- TypeReferenceExpression tre = (TypeReferenceExpression) texpr;
- IAType t = solveTypeReference(tre, typeMap);
- if (t != null) {
- typeMap.put(tdname, t);
- } else {
- addIncompleteTopLevelTypeReference(tdname, tre,
- incompleteTopLevelTypeReferences);
- }
- break;
- }
- case RECORD: {
- RecordTypeDefinition rtd = (RecordTypeDefinition) texpr;
- ARecordType recType = computeRecordType(tdname, rtd, typeMap,
- incompleteFieldTypes, incompleteItemTypes);
- typeMap.put(tdname, recType);
- break;
- }
- case ORDEREDLIST: {
- OrderedListTypeDefinition oltd = (OrderedListTypeDefinition) texpr;
- AOrderedListType olType = computeOrderedListType(tdname, oltd,
- typeMap, incompleteItemTypes, incompleteFieldTypes);
- typeMap.put(tdname, olType);
- break;
- }
- case UNORDEREDLIST: {
- UnorderedListTypeDefinition ultd = (UnorderedListTypeDefinition) texpr;
- AUnorderedListType ulType = computeUnorderedListType(tdname, ultd,
- typeMap, incompleteItemTypes, incompleteFieldTypes);
- typeMap.put(tdname, ulType);
- break;
- }
- default: {
- throw new IllegalStateException();
- }
- }
- }
+ TypeExpression texpr = td.getTypeDef();
+ String tdname = td.getIdent().getValue();
+ if (builtinTypeMap.get(tdname) != null) {
+ throw new AlgebricksException("Cannot redefine builtin type " + tdname + " .");
+ }
+ switch (texpr.getTypeKind()) {
+ case TYPEREFERENCE: {
+ TypeReferenceExpression tre = (TypeReferenceExpression) texpr;
+ IAType t = solveTypeReference(tre, typeMap);
+ if (t != null) {
+ typeMap.put(tdname, t);
+ } else {
+ addIncompleteTopLevelTypeReference(tdname, tre, incompleteTopLevelTypeReferences);
+ }
+ break;
+ }
+ case RECORD: {
+ RecordTypeDefinition rtd = (RecordTypeDefinition) texpr;
+ ARecordType recType = computeRecordType(tdname, rtd, typeMap, incompleteFieldTypes, incompleteItemTypes);
+ typeMap.put(tdname, recType);
+ break;
+ }
+ case ORDEREDLIST: {
+ OrderedListTypeDefinition oltd = (OrderedListTypeDefinition) texpr;
+ AOrderedListType olType = computeOrderedListType(tdname, oltd, typeMap, incompleteItemTypes,
+ incompleteFieldTypes);
+ typeMap.put(tdname, olType);
+ break;
+ }
+ case UNORDEREDLIST: {
+ UnorderedListTypeDefinition ultd = (UnorderedListTypeDefinition) texpr;
+ AUnorderedListType ulType = computeUnorderedListType(tdname, ultd, typeMap, incompleteItemTypes,
+ incompleteFieldTypes);
+ typeMap.put(tdname, ulType);
+ break;
+ }
+ default: {
+ throw new IllegalStateException();
+ }
+ }
+ }
- private AOrderedListType computeOrderedListType(String typeName,
- OrderedListTypeDefinition oltd, Map<String, IAType> typeMap,
- Map<String, List<AbstractCollectionType>> incompleteItemTypes,
- Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes) {
- TypeExpression tExpr = oltd.getItemTypeExpression();
- AOrderedListType aolt = new AOrderedListType(null, typeName);
- setCollectionItemType(tExpr, typeMap, incompleteItemTypes,
- incompleteFieldTypes, aolt);
- return aolt;
- }
+ private AOrderedListType computeOrderedListType(String typeName, OrderedListTypeDefinition oltd,
+ Map<String, IAType> typeMap, Map<String, List<AbstractCollectionType>> incompleteItemTypes,
+ Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes) {
+ TypeExpression tExpr = oltd.getItemTypeExpression();
+ AOrderedListType aolt = new AOrderedListType(null, typeName);
+ setCollectionItemType(tExpr, typeMap, incompleteItemTypes, incompleteFieldTypes, aolt);
+ return aolt;
+ }
- private AUnorderedListType computeUnorderedListType(String typeName,
- UnorderedListTypeDefinition ultd, Map<String, IAType> typeMap,
- Map<String, List<AbstractCollectionType>> incompleteItemTypes,
- Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes) {
- TypeExpression tExpr = ultd.getItemTypeExpression();
- AUnorderedListType ault = new AUnorderedListType(null, typeName);
- setCollectionItemType(tExpr, typeMap, incompleteItemTypes,
- incompleteFieldTypes, ault);
- return ault;
- }
+ private AUnorderedListType computeUnorderedListType(String typeName, UnorderedListTypeDefinition ultd,
+ Map<String, IAType> typeMap, Map<String, List<AbstractCollectionType>> incompleteItemTypes,
+ Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes) {
+ TypeExpression tExpr = ultd.getItemTypeExpression();
+ AUnorderedListType ault = new AUnorderedListType(null, typeName);
+ setCollectionItemType(tExpr, typeMap, incompleteItemTypes, incompleteFieldTypes, ault);
+ return ault;
+ }
- private void setCollectionItemType(TypeExpression tExpr,
- Map<String, IAType> typeMap,
- Map<String, List<AbstractCollectionType>> incompleteItemTypes,
- Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
- AbstractCollectionType act) {
- switch (tExpr.getTypeKind()) {
- case ORDEREDLIST: {
- OrderedListTypeDefinition oltd = (OrderedListTypeDefinition) tExpr;
- IAType t = computeOrderedListType(null, oltd, typeMap,
- incompleteItemTypes, incompleteFieldTypes);
- act.setItemType(t);
- break;
- }
- case UNORDEREDLIST: {
- UnorderedListTypeDefinition ultd = (UnorderedListTypeDefinition) tExpr;
- IAType t = computeUnorderedListType(null, ultd, typeMap,
- incompleteItemTypes, incompleteFieldTypes);
- act.setItemType(t);
- break;
- }
- case RECORD: {
- RecordTypeDefinition rtd = (RecordTypeDefinition) tExpr;
- IAType t = computeRecordType(null, rtd, typeMap,
- incompleteFieldTypes, incompleteItemTypes);
- act.setItemType(t);
- break;
- }
- case TYPEREFERENCE: {
- TypeReferenceExpression tre = (TypeReferenceExpression) tExpr;
- IAType tref = solveTypeReference(tre, typeMap);
- if (tref != null) {
- act.setItemType(tref);
- } else {
- addIncompleteCollectionTypeReference(act, tre,
- incompleteItemTypes);
- }
- break;
- }
- default: {
- throw new IllegalStateException();
- }
- }
- }
+ private void setCollectionItemType(TypeExpression tExpr, Map<String, IAType> typeMap,
+ Map<String, List<AbstractCollectionType>> incompleteItemTypes,
+ Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes, AbstractCollectionType act) {
+ switch (tExpr.getTypeKind()) {
+ case ORDEREDLIST: {
+ OrderedListTypeDefinition oltd = (OrderedListTypeDefinition) tExpr;
+ IAType t = computeOrderedListType(null, oltd, typeMap, incompleteItemTypes, incompleteFieldTypes);
+ act.setItemType(t);
+ break;
+ }
+ case UNORDEREDLIST: {
+ UnorderedListTypeDefinition ultd = (UnorderedListTypeDefinition) tExpr;
+ IAType t = computeUnorderedListType(null, ultd, typeMap, incompleteItemTypes, incompleteFieldTypes);
+ act.setItemType(t);
+ break;
+ }
+ case RECORD: {
+ RecordTypeDefinition rtd = (RecordTypeDefinition) tExpr;
+ IAType t = computeRecordType(null, rtd, typeMap, incompleteFieldTypes, incompleteItemTypes);
+ act.setItemType(t);
+ break;
+ }
+ case TYPEREFERENCE: {
+ TypeReferenceExpression tre = (TypeReferenceExpression) tExpr;
+ IAType tref = solveTypeReference(tre, typeMap);
+ if (tref != null) {
+ act.setItemType(tref);
+ } else {
+ addIncompleteCollectionTypeReference(act, tre, incompleteItemTypes);
+ }
+ break;
+ }
+ default: {
+ throw new IllegalStateException();
+ }
+ }
+ }
- private ARecordType computeRecordType(String typeName,
- RecordTypeDefinition rtd, Map<String, IAType> typeMap,
- Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
- Map<String, List<AbstractCollectionType>> incompleteItemTypes) {
- List<String> names = rtd.getFieldNames();
- int n = names.size();
- String[] fldNames = new String[n];
- IAType[] fldTypes = new IAType[n];
- int i = 0;
- for (String s : names) {
- fldNames[i++] = s;
- }
- boolean isOpen = rtd.getRecordKind() == RecordKind.OPEN;
- ARecordType recType = new ARecordType(typeName, fldNames, fldTypes,
- isOpen);
- for (int j = 0; j < n; j++) {
- TypeExpression texpr = rtd.getFieldTypes().get(j);
- switch (texpr.getTypeKind()) {
- case TYPEREFERENCE: {
- TypeReferenceExpression tre = (TypeReferenceExpression) texpr;
- IAType tref = solveTypeReference(tre, typeMap);
- if (tref != null) {
- if (!rtd.getNullableFields().get(j)) { // not nullable
- fldTypes[j] = tref;
- } else { // nullable
- fldTypes[j] = makeUnionWithNull(null, tref);
- }
- } else {
- addIncompleteFieldTypeReference(recType, j, tre,
- incompleteFieldTypes);
- if (rtd.getNullableFields().get(j)) {
- fldTypes[j] = makeUnionWithNull(null, null);
- }
- }
- break;
- }
- case RECORD: {
- RecordTypeDefinition recTypeDef2 = (RecordTypeDefinition) texpr;
- IAType t2 = computeRecordType(null, recTypeDef2, typeMap,
- incompleteFieldTypes, incompleteItemTypes);
- if (!rtd.getNullableFields().get(j)) { // not nullable
- fldTypes[j] = t2;
- } else { // nullable
- fldTypes[j] = makeUnionWithNull(null, t2);
- }
- break;
- }
- case ORDEREDLIST: {
- OrderedListTypeDefinition oltd = (OrderedListTypeDefinition) texpr;
- IAType t2 = computeOrderedListType(null, oltd, typeMap,
- incompleteItemTypes, incompleteFieldTypes);
- fldTypes[j] = (rtd.getNullableFields().get(j)) ? makeUnionWithNull(
- null, t2) : t2;
- break;
- }
- case UNORDEREDLIST: {
- UnorderedListTypeDefinition ultd = (UnorderedListTypeDefinition) texpr;
- IAType t2 = computeUnorderedListType(null, ultd, typeMap,
- incompleteItemTypes, incompleteFieldTypes);
- fldTypes[j] = (rtd.getNullableFields().get(j)) ? makeUnionWithNull(
- null, t2) : t2;
- break;
- }
- default: {
- throw new IllegalStateException();
- }
- }
+ private ARecordType computeRecordType(String typeName, RecordTypeDefinition rtd, Map<String, IAType> typeMap,
+ Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
+ Map<String, List<AbstractCollectionType>> incompleteItemTypes) {
+ List<String> names = rtd.getFieldNames();
+ int n = names.size();
+ String[] fldNames = new String[n];
+ IAType[] fldTypes = new IAType[n];
+ int i = 0;
+ for (String s : names) {
+ fldNames[i++] = s;
+ }
+ boolean isOpen = rtd.getRecordKind() == RecordKind.OPEN;
+ ARecordType recType = new ARecordType(typeName, fldNames, fldTypes, isOpen);
+ for (int j = 0; j < n; j++) {
+ TypeExpression texpr = rtd.getFieldTypes().get(j);
+ switch (texpr.getTypeKind()) {
+ case TYPEREFERENCE: {
+ TypeReferenceExpression tre = (TypeReferenceExpression) texpr;
+ IAType tref = solveTypeReference(tre, typeMap);
+ if (tref != null) {
+ if (!rtd.getNullableFields().get(j)) { // not nullable
+ fldTypes[j] = tref;
+ } else { // nullable
+ fldTypes[j] = makeUnionWithNull(null, tref);
+ }
+ } else {
+ addIncompleteFieldTypeReference(recType, j, tre, incompleteFieldTypes);
+ if (rtd.getNullableFields().get(j)) {
+ fldTypes[j] = makeUnionWithNull(null, null);
+ }
+ }
+ break;
+ }
+ case RECORD: {
+ RecordTypeDefinition recTypeDef2 = (RecordTypeDefinition) texpr;
+ IAType t2 = computeRecordType(null, recTypeDef2, typeMap, incompleteFieldTypes, incompleteItemTypes);
+ if (!rtd.getNullableFields().get(j)) { // not nullable
+ fldTypes[j] = t2;
+ } else { // nullable
+ fldTypes[j] = makeUnionWithNull(null, t2);
+ }
+ break;
+ }
+ case ORDEREDLIST: {
+ OrderedListTypeDefinition oltd = (OrderedListTypeDefinition) texpr;
+ IAType t2 = computeOrderedListType(null, oltd, typeMap, incompleteItemTypes, incompleteFieldTypes);
+ fldTypes[j] = (rtd.getNullableFields().get(j)) ? makeUnionWithNull(null, t2) : t2;
+ break;
+ }
+ case UNORDEREDLIST: {
+ UnorderedListTypeDefinition ultd = (UnorderedListTypeDefinition) texpr;
+ IAType t2 = computeUnorderedListType(null, ultd, typeMap, incompleteItemTypes, incompleteFieldTypes);
+ fldTypes[j] = (rtd.getNullableFields().get(j)) ? makeUnionWithNull(null, t2) : t2;
+ break;
+ }
+ default: {
+ throw new IllegalStateException();
+ }
+ }
- }
+ }
- return recType;
- }
+ return recType;
+ }
- private AUnionType makeUnionWithNull(String unionTypeName, IAType type) {
- ArrayList<IAType> unionList = new ArrayList<IAType>(2);
- unionList.add(BuiltinType.ANULL);
- unionList.add(type);
- return new AUnionType(unionList, unionTypeName);
- }
+ private AUnionType makeUnionWithNull(String unionTypeName, IAType type) {
+ ArrayList<IAType> unionList = new ArrayList<IAType>(2);
+ unionList.add(BuiltinType.ANULL);
+ unionList.add(type);
+ return new AUnionType(unionList, unionTypeName);
+ }
- private void addIncompleteCollectionTypeReference(
- AbstractCollectionType collType, TypeReferenceExpression tre,
- Map<String, List<AbstractCollectionType>> incompleteItemTypes) {
- String typeName = tre.getIdent().getValue();
- List<AbstractCollectionType> typeList = incompleteItemTypes
- .get(typeName);
- if (typeList == null) {
- typeList = new LinkedList<AbstractCollectionType>();
- incompleteItemTypes.put(typeName, typeList);
- }
- typeList.add(collType);
- }
+ private void addIncompleteCollectionTypeReference(AbstractCollectionType collType, TypeReferenceExpression tre,
+ Map<String, List<AbstractCollectionType>> incompleteItemTypes) {
+ String typeName = tre.getIdent().getValue();
+ List<AbstractCollectionType> typeList = incompleteItemTypes.get(typeName);
+ if (typeList == null) {
+ typeList = new LinkedList<AbstractCollectionType>();
+ incompleteItemTypes.put(typeName, typeList);
+ }
+ typeList.add(collType);
+ }
- private void addIncompleteFieldTypeReference(ARecordType recType,
- int fldPosition, TypeReferenceExpression tre,
- Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes) {
- String typeName = tre.getIdent().getValue();
- Map<ARecordType, List<Integer>> refMap = incompleteFieldTypes
- .get(typeName);
- if (refMap == null) {
- refMap = new HashMap<ARecordType, List<Integer>>();
- incompleteFieldTypes.put(typeName, refMap);
- }
- List<Integer> typeList = refMap.get(recType);
- if (typeList == null) {
- typeList = new ArrayList<Integer>();
- refMap.put(recType, typeList);
- }
- typeList.add(fldPosition);
- }
+ private void addIncompleteFieldTypeReference(ARecordType recType, int fldPosition, TypeReferenceExpression tre,
+ Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes) {
+ String typeName = tre.getIdent().getValue();
+ Map<ARecordType, List<Integer>> refMap = incompleteFieldTypes.get(typeName);
+ if (refMap == null) {
+ refMap = new HashMap<ARecordType, List<Integer>>();
+ incompleteFieldTypes.put(typeName, refMap);
+ }
+ List<Integer> typeList = refMap.get(recType);
+ if (typeList == null) {
+ typeList = new ArrayList<Integer>();
+ refMap.put(recType, typeList);
+ }
+ typeList.add(fldPosition);
+ }
- private void addIncompleteTopLevelTypeReference(String tdeclName,
- TypeReferenceExpression tre,
- Map<String, List<String>> incompleteTopLevelTypeReferences) {
- String name = tre.getIdent().getValue();
- List<String> refList = incompleteTopLevelTypeReferences.get(name);
- if (refList == null) {
- refList = new LinkedList<String>();
- incompleteTopLevelTypeReferences.put(name, refList);
- }
- refList.add(tdeclName);
- }
+ private void addIncompleteTopLevelTypeReference(String tdeclName, TypeReferenceExpression tre,
+ Map<String, List<String>> incompleteTopLevelTypeReferences) {
+ String name = tre.getIdent().getValue();
+ List<String> refList = incompleteTopLevelTypeReferences.get(name);
+ if (refList == null) {
+ refList = new LinkedList<String>();
+ incompleteTopLevelTypeReferences.put(name, refList);
+ }
+ refList.add(tdeclName);
+ }
- private IAType solveTypeReference(TypeReferenceExpression tre,
- Map<String, IAType> typeMap) {
- String name = tre.getIdent().getValue();
- IAType builtin = builtinTypeMap.get(name);
- if (builtin != null) {
- return builtin;
- } else {
- return typeMap.get(name);
- }
- }
+ private IAType solveTypeReference(TypeReferenceExpression tre, Map<String, IAType> typeMap) {
+ String name = tre.getIdent().getValue();
+ IAType builtin = builtinTypeMap.get(name);
+ if (builtin != null) {
+ return builtin;
+ } else {
+ return typeMap.get(name);
+ }
+ }
- public static interface ICompiledStatement {
+ public static interface ICompiledStatement {
- public abstract Kind getKind();
- }
+ public abstract Kind getKind();
+ }
- public static class CompiledLoadFromFileStatement implements
- ICompiledStatement, IParseFileSplitsDecl {
- private String datasetName;
- private FileSplit[] splits;
- private boolean alreadySorted;
- private Character delimChar;
+ public static class CompiledLoadFromFileStatement implements ICompiledStatement, IParseFileSplitsDecl {
+ private String datasetName;
+ private FileSplit[] splits;
+ private boolean alreadySorted;
+ private Character delimChar;
- public CompiledLoadFromFileStatement(String datasetName,
- FileSplit[] splits, Character delimChar, boolean alreadySorted) {
- this.datasetName = datasetName;
- this.splits = splits;
- this.delimChar = delimChar;
- this.alreadySorted = alreadySorted;
- }
+ public CompiledLoadFromFileStatement(String datasetName, FileSplit[] splits, Character delimChar,
+ boolean alreadySorted) {
+ this.datasetName = datasetName;
+ this.splits = splits;
+ this.delimChar = delimChar;
+ this.alreadySorted = alreadySorted;
+ }
- public String getDatasetName() {
- return datasetName;
- }
+ public String getDatasetName() {
+ return datasetName;
+ }
- @Override
- public FileSplit[] getSplits() {
- return splits;
- }
+ @Override
+ public FileSplit[] getSplits() {
+ return splits;
+ }
- @Override
- public Character getDelimChar() {
- return delimChar;
- }
+ @Override
+ public Character getDelimChar() {
+ return delimChar;
+ }
- public boolean alreadySorted() {
- return alreadySorted;
- }
+ public boolean alreadySorted() {
+ return alreadySorted;
+ }
- @Override
- public boolean isDelimitedFileFormat() {
- return delimChar != null;
- }
+ @Override
+ public boolean isDelimitedFileFormat() {
+ return delimChar != null;
+ }
- @Override
- public Kind getKind() {
- return Kind.LOAD_FROM_FILE;
- }
- }
+ @Override
+ public Kind getKind() {
+ return Kind.LOAD_FROM_FILE;
+ }
+ }
- public static class CompiledWriteFromQueryResultStatement implements
- ICompiledStatement {
+ public static class CompiledWriteFromQueryResultStatement implements ICompiledStatement {
- private String datasetName;
- private Query query;
- private int varCounter;
+ private String datasetName;
+ private Query query;
+ private int varCounter;
- public CompiledWriteFromQueryResultStatement(String datasetName,
- Query query, int varCounter) {
- this.datasetName = datasetName;
- this.query = query;
- this.varCounter = varCounter;
- }
+ public CompiledWriteFromQueryResultStatement(String datasetName, Query query, int varCounter) {
+ this.datasetName = datasetName;
+ this.query = query;
+ this.varCounter = varCounter;
+ }
- public String getDatasetName() {
- return datasetName;
- }
+ public String getDatasetName() {
+ return datasetName;
+ }
- public int getVarCounter() {
- return varCounter;
- }
+ public int getVarCounter() {
+ return varCounter;
+ }
- public Query getQuery() {
- return query;
- }
+ public Query getQuery() {
+ return query;
+ }
- @Override
- public Kind getKind() {
- return Kind.WRITE_FROM_QUERY_RESULT;
- }
+ @Override
+ public Kind getKind() {
+ return Kind.WRITE_FROM_QUERY_RESULT;
+ }
- }
+ }
- public static class CompiledDatasetDropStatement implements
- ICompiledStatement {
- private String datasetName;
+ public static class CompiledDatasetDropStatement implements ICompiledStatement {
+ private String datasetName;
- public CompiledDatasetDropStatement(String datasetName) {
- this.datasetName = datasetName;
- }
+ public CompiledDatasetDropStatement(String datasetName) {
+ this.datasetName = datasetName;
+ }
- public String getDatasetName() {
- return datasetName;
- }
+ public String getDatasetName() {
+ return datasetName;
+ }
- @Override
- public Kind getKind() {
- return Kind.DATASET_DROP;
- }
- }
+ @Override
+ public Kind getKind() {
+ return Kind.DATASET_DROP;
+ }
+ }
- // added by yasser
- public static class CompiledCreateDataverseStatement implements
- ICompiledStatement {
- private String dataverseName;
- private String format;
+ // added by yasser
+ public static class CompiledCreateDataverseStatement implements ICompiledStatement {
+ private String dataverseName;
+ private String format;
- public CompiledCreateDataverseStatement(String dataverseName,
- String format) {
- this.dataverseName = dataverseName;
- this.format = format;
- }
+ public CompiledCreateDataverseStatement(String dataverseName, String format) {
+ this.dataverseName = dataverseName;
+ this.format = format;
+ }
- public String getDataverseName() {
- return dataverseName;
- }
+ public String getDataverseName() {
+ return dataverseName;
+ }
- public String getFormat() {
- return format;
- }
+ public String getFormat() {
+ return format;
+ }
- @Override
- public Kind getKind() {
- return Kind.CREATE_DATAVERSE;
- }
- }
+ @Override
+ public Kind getKind() {
+ return Kind.CREATE_DATAVERSE;
+ }
+ }
- public static class CompiledNodeGroupDropStatement implements
- ICompiledStatement {
- private String nodeGroupName;
+ public static class CompiledNodeGroupDropStatement implements ICompiledStatement {
+ private String nodeGroupName;
- public CompiledNodeGroupDropStatement(String nodeGroupName) {
- this.nodeGroupName = nodeGroupName;
- }
+ public CompiledNodeGroupDropStatement(String nodeGroupName) {
+ this.nodeGroupName = nodeGroupName;
+ }
- public String getNodeGroupName() {
- return nodeGroupName;
- }
+ public String getNodeGroupName() {
+ return nodeGroupName;
+ }
- @Override
- public Kind getKind() {
- return Kind.NODEGROUP_DROP;
- }
- }
+ @Override
+ public Kind getKind() {
+ return Kind.NODEGROUP_DROP;
+ }
+ }
- public static class CompiledIndexDropStatement implements
- ICompiledStatement {
- private String datasetName;
- private String indexName;
+ public static class CompiledIndexDropStatement implements ICompiledStatement {
+ private String datasetName;
+ private String indexName;
- public CompiledIndexDropStatement(String datasetName, String indexName) {
- this.datasetName = datasetName;
- this.indexName = indexName;
- }
+ public CompiledIndexDropStatement(String datasetName, String indexName) {
+ this.datasetName = datasetName;
+ this.indexName = indexName;
+ }
- public String getDatasetName() {
- return datasetName;
- }
+ public String getDatasetName() {
+ return datasetName;
+ }
- public String getIndexName() {
- return indexName;
- }
+ public String getIndexName() {
+ return indexName;
+ }
- @Override
- public Kind getKind() {
- return Kind.INDEX_DROP;
- }
- }
+ @Override
+ public Kind getKind() {
+ return Kind.INDEX_DROP;
+ }
+ }
- public static class CompiledDataverseDropStatement implements
- ICompiledStatement {
- private String dataverseName;
- private boolean ifExists;
+ public static class CompiledDataverseDropStatement implements ICompiledStatement {
+ private String dataverseName;
+ private boolean ifExists;
- public CompiledDataverseDropStatement(String dataverseName,
- boolean ifExists) {
- this.dataverseName = dataverseName;
- this.ifExists = ifExists;
- }
+ public CompiledDataverseDropStatement(String dataverseName, boolean ifExists) {
+ this.dataverseName = dataverseName;
+ this.ifExists = ifExists;
+ }
- public String getDataverseName() {
- return dataverseName;
- }
+ public String getDataverseName() {
+ return dataverseName;
+ }
- public boolean getIfExists() {
- return ifExists;
- }
+ public boolean getIfExists() {
+ return ifExists;
+ }
- @Override
- public Kind getKind() {
- return Kind.DATAVERSE_DROP;
- }
- }
+ @Override
+ public Kind getKind() {
+ return Kind.DATAVERSE_DROP;
+ }
+ }
- public static class CompiledTypeDropStatement implements ICompiledStatement {
- private String typeName;
+ public static class CompiledTypeDropStatement implements ICompiledStatement {
+ private String typeName;
- public CompiledTypeDropStatement(String nodeGroupName) {
- this.typeName = nodeGroupName;
- }
+ public CompiledTypeDropStatement(String nodeGroupName) {
+ this.typeName = nodeGroupName;
+ }
- public String getTypeName() {
- return typeName;
- }
+ public String getTypeName() {
+ return typeName;
+ }
- @Override
- public Kind getKind() {
- return Kind.TYPE_DROP;
- }
- }
+ @Override
+ public Kind getKind() {
+ return Kind.TYPE_DROP;
+ }
+ }
}
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 b028e06..b12e6d8 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
@@ -23,8 +23,7 @@
import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
import edu.uci.ics.asterix.common.config.GlobalConfig;
import edu.uci.ics.asterix.common.config.OptimizationConfUtil;
-import edu.uci.ics.asterix.common.context.AsterixIndexRegistryProvider;
-import edu.uci.ics.asterix.common.context.AsterixStorageManagerInterface;
+import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.formats.base.IDataFormat;
import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
@@ -35,7 +34,9 @@
import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.runtime.job.listener.JobEventListenerFactory;
import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobIdFactory;
import edu.uci.ics.asterix.translator.DmlTranslator.CompiledLoadFromFileStatement;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
@@ -67,14 +68,17 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexCreateOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDropOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.dataflow.LSMRTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+import edu.uci.ics.hyracks.storage.common.file.TransientLocalResourceFactoryProvider;
public class DatasetOperations {
@@ -92,9 +96,6 @@
LOGGER.info("DROP DATASETPATH: " + datasetPath);
- IIndexRegistryProvider<IIndex> indexRegistryProvider = AsterixIndexRegistryProvider.INSTANCE;
- IStorageManagerInterface storageManager = AsterixStorageManagerInterface.INSTANCE;
-
Dataset dataset = metadata.findDataset(datasetName);
if (dataset == null) {
throw new AlgebricksException("DROP DATASET: No metadata for dataset " + datasetName);
@@ -121,8 +122,36 @@
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> idxSplitsAndConstraint = metadata
.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName,
index.getIndexName());
- TreeIndexDropOperatorDescriptor secondaryBtreeDrop = new TreeIndexDropOperatorDescriptor(specs[i],
- storageManager, indexRegistryProvider, idxSplitsAndConstraint.first);
+ IIndexDataflowHelperFactory dfhFactory;
+ switch (index.getIndexType()) {
+ case BTREE:
+ dfhFactory = new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE);
+ break;
+ case RTREE:
+ dfhFactory = new LSMRTreeDataflowHelperFactory(
+ new IPrimitiveValueProviderFactory[] { null }, RTreePolicyType.RTREE,
+ new IBinaryComparatorFactory[] { null }, AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE, null);
+ break;
+ case NGRAM_INVIX:
+ case WORD_INVIX:
+ dfhFactory = new LSMInvertedIndexDataflowHelperFactory(
+ AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE);
+ break;
+ default:
+ throw new AsterixException("Unknown index type provided.");
+ }
+ IndexDropOperatorDescriptor secondaryBtreeDrop = new IndexDropOperatorDescriptor(specs[i],
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+ idxSplitsAndConstraint.first, dfhFactory);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(specs[i], secondaryBtreeDrop,
idxSplitsAndConstraint.second);
i++;
@@ -136,8 +165,11 @@
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, datasetName);
- TreeIndexDropOperatorDescriptor primaryBtreeDrop = new TreeIndexDropOperatorDescriptor(specPrimary,
- storageManager, indexRegistryProvider, splitsAndConstraint.first);
+ IndexDropOperatorDescriptor primaryBtreeDrop = new IndexDropOperatorDescriptor(specPrimary,
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+ splitsAndConstraint.first, new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE));
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(specPrimary, primaryBtreeDrop,
splitsAndConstraint.second);
@@ -165,11 +197,14 @@
sb.append(stringOf(fs[i]) + " ");
}
LOGGER.info("CREATING File Splits: " + sb.toString());
- IIndexRegistryProvider<IIndex> indexRegistryProvider = AsterixIndexRegistryProvider.INSTANCE;
- IStorageManagerInterface storageManager = AsterixStorageManagerInterface.INSTANCE;
- TreeIndexCreateOperatorDescriptor indexCreateOp = new TreeIndexCreateOperatorDescriptor(spec, storageManager,
- indexRegistryProvider, splitsAndConstraint.first, typeTraits, comparatorFactories,
- new BTreeDataflowHelperFactory(), NoOpOperationCallbackProvider.INSTANCE);
+ //TODO replace this transient one to the persistent one
+ TransientLocalResourceFactoryProvider localResourceFactoryProvider = new TransientLocalResourceFactoryProvider();
+ TreeIndexCreateOperatorDescriptor indexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+ splitsAndConstraint.first, typeTraits, comparatorFactories, new LSMBTreeDataflowHelperFactory(
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE),
+ localResourceFactoryProvider);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, indexCreateOp,
splitsAndConstraint.second);
spec.addRoot(indexCreateOp);
@@ -190,6 +225,9 @@
}
JobSpecification spec = new JobSpecification();
+ edu.uci.ics.asterix.transaction.management.service.transaction.JobId asterixJobId = JobIdFactory
+ .generateJobId();
+
ARecordType itemType = (ARecordType) metadata.findType(dataset.getItemTypeName());
IDataFormat format = metadata.getFormat();
ISerializerDeserializer payloadSerde = format.getSerdeProvider().getSerializerDeserializer(itemType);
@@ -238,11 +276,12 @@
}
LOGGER.info("LOAD into File Splits: " + sb.toString());
- IIndexRegistryProvider<IIndex> indexRegistryProvider = AsterixIndexRegistryProvider.INSTANCE;
- IStorageManagerInterface storageManager = AsterixStorageManagerInterface.INSTANCE;
TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
- storageManager, indexRegistryProvider, splitsAndConstraint.first, typeTraits, comparatorFactories,
- fieldPermutation, GlobalConfig.DEFAULT_BTREE_FILL_FACTOR, new BTreeDataflowHelperFactory(),
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+ splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation,
+ GlobalConfig.DEFAULT_BTREE_FILL_FACTOR, false, new LSMBTreeDataflowHelperFactory(
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE),
NoOpOperationCallbackProvider.INSTANCE);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, btreeBulkLoad,
splitsAndConstraint.second);
@@ -266,6 +305,7 @@
}
spec.addRoot(btreeBulkLoad);
spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+ spec.setJobletEventListenerFactory(new JobEventListenerFactory(asterixJobId, false));
return new Job(spec);
}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java
index 7bd7ae6..8b2d583 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java
@@ -2,8 +2,7 @@
import edu.uci.ics.asterix.aql.translator.DdlTranslator.CompiledIndexDropStatement;
import edu.uci.ics.asterix.common.config.OptimizationConfUtil;
-import edu.uci.ics.asterix.common.context.AsterixIndexRegistryProvider;
-import edu.uci.ics.asterix.common.context.AsterixStorageManagerInterface;
+import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.metadata.MetadataException;
import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
@@ -15,10 +14,8 @@
import edu.uci.ics.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDropOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
public class IndexOperations {
@@ -45,13 +42,14 @@
String indexName = deleteStmt.getIndexName();
JobSpecification spec = new JobSpecification();
- IIndexRegistryProvider<IIndex> indexRegistryProvider = AsterixIndexRegistryProvider.INSTANCE;
- IStorageManagerInterface storageManager = AsterixStorageManagerInterface.INSTANCE;
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = datasetDecls
.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
- TreeIndexDropOperatorDescriptor btreeDrop = new TreeIndexDropOperatorDescriptor(spec, storageManager,
- indexRegistryProvider, splitsAndConstraint.first);
+ IndexDropOperatorDescriptor btreeDrop = new IndexDropOperatorDescriptor(spec,
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+ splitsAndConstraint.first, new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE));
AlgebricksPartitionConstraintHelper
.setPartitionConstraintInJobSpec(spec, btreeDrop, splitsAndConstraint.second);
spec.addRoot(btreeDrop);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
index b9ad908..ce26312 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
@@ -1,7 +1,6 @@
package edu.uci.ics.asterix.file;
-import edu.uci.ics.asterix.common.context.AsterixIndexRegistryProvider;
-import edu.uci.ics.asterix.common.context.AsterixStorageManagerInterface;
+import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -12,12 +11,12 @@
import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexCreateOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.common.file.TransientLocalResourceFactoryProvider;
public class SecondaryBTreeCreator extends SecondaryIndexCreator {
@@ -28,10 +27,14 @@
@Override
public JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException {
JobSpecification spec = new JobSpecification();
+ //TODO replace this transient one to the persistent one
+ TransientLocalResourceFactoryProvider localResourceFactoryProvider = new TransientLocalResourceFactoryProvider();
TreeIndexCreateOperatorDescriptor secondaryIndexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
- AsterixStorageManagerInterface.INSTANCE, AsterixIndexRegistryProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
- new BTreeDataflowHelperFactory(), NoOpOperationCallbackProvider.INSTANCE);
+ new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE), localResourceFactoryProvider);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,
secondaryPartitionConstraint);
spec.addRoot(secondaryIndexCreateOp);
@@ -63,7 +66,9 @@
// Create secondary BTree bulk load op.
TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(spec, numSecondaryKeys,
- new BTreeDataflowHelperFactory(), BTree.DEFAULT_FILL_FACTOR);
+ new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE), BTree.DEFAULT_FILL_FACTOR);
// Connect the operators.
spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
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 65f6f5e..6cda5ab 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
@@ -4,8 +4,7 @@
import java.util.List;
import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.common.context.AsterixIndexRegistryProvider;
-import edu.uci.ics.asterix.common.context.AsterixStorageManagerInterface;
+import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.formats.nontagged.AqlBinaryBooleanInspectorImpl;
import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
@@ -50,11 +49,11 @@
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
@SuppressWarnings("rawtypes")
// TODO: We should eventually have a hierarchy of classes that can create all possible index job specs,
@@ -225,10 +224,11 @@
// +Infinity
int[] highKeyFields = null;
BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
- AsterixStorageManagerInterface.INSTANCE, AsterixIndexRegistryProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
primaryFileSplitProvider, primaryRecDesc.getTypeTraits(), primaryComparatorFactories, lowKeyFields,
- highKeyFields, true, true, new BTreeDataflowHelperFactory(), false,
- NoOpOperationCallbackProvider.INSTANCE);
+ highKeyFields, true, true, new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE), false, NoOpOperationCallbackProvider.INSTANCE);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, primarySearchOp,
primaryPartitionConstraint);
return primarySearchOp;
@@ -283,9 +283,9 @@
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadata
.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, secondaryIndexName);
TreeIndexBulkLoadOperatorDescriptor treeIndexBulkLoadOp = new TreeIndexBulkLoadOperatorDescriptor(spec,
- AsterixStorageManagerInterface.INSTANCE, AsterixIndexRegistryProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
secondarySplitsAndConstraint.first, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
- fieldPermutation, fillFactor, dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+ fieldPermutation, fillFactor, false, dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, treeIndexBulkLoadOp,
secondarySplitsAndConstraint.second);
return treeIndexBulkLoadOp;
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 d5e8222..1deb278 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
@@ -2,8 +2,7 @@
import java.util.List;
-import edu.uci.ics.asterix.common.context.AsterixIndexRegistryProvider;
-import edu.uci.ics.asterix.common.context.AsterixStorageManagerInterface;
+import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.metadata.entities.Index;
import edu.uci.ics.asterix.om.types.IAType;
@@ -27,13 +26,15 @@
import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.BinaryTokenizerOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.InvertedIndexBulkLoadOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.InvertedIndexCreateOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.BinaryTokenizerOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexBulkLoadOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexCreateOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactoryProvider;
+import edu.uci.ics.hyracks.storage.common.file.TransientLocalResourceFactoryProvider;
public class SecondaryInvertedIndexCreator extends SecondaryIndexCreator {
@@ -63,7 +64,8 @@
if (numSecondaryKeys > 1) {
throw new AsterixException("Cannot create composite inverted index on multiple fields.");
}
- // Prepare record descriptor used in the assign op, and the optional select op.
+ // Prepare record descriptor used in the assign op, and the optional
+ // select op.
List<String> secondaryKeyFields = createIndexStmt.getKeyFields();
secondaryFieldAccessEvalFactories = new ICopyEvaluatorFactory[numSecondaryKeys];
ISerializerDeserializer[] secondaryRecFields = new ISerializerDeserializer[numPrimaryKeys + numSecondaryKeys];
@@ -87,11 +89,12 @@
tokenComparatorFactories[0] = InvertedIndexAccessMethod.getTokenBinaryComparatorFactory(secondaryKeyType);
tokenTypeTraits[0] = InvertedIndexAccessMethod.getTokenTypeTrait(secondaryKeyType);
// Set tokenizer factory.
- // TODO: We might want to expose the hashing option at the AQL level,
+ // TODO: We might want to expose the hashing option at the AQL level,
// and add the choice to the index metadata.
tokenizerFactory = InvertedIndexAccessMethod.getBinaryTokenizerFactory(secondaryKeyType.getTypeTag(),
createIndexStmt.getIndexType(), createIndexStmt.getGramLength());
- // Type traits for inverted-list elements. Inverted lists contain primary keys.
+ // Type traits for inverted-list elements. Inverted lists contain
+ // primary keys.
invListsTypeTraits = new ITypeTraits[numPrimaryKeys];
for (int i = 0; i < numPrimaryKeys; i++) {
invListsTypeTraits[i] = primaryRecDesc.getTypeTraits()[i];
@@ -119,11 +122,16 @@
@Override
public JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException {
JobSpecification spec = new JobSpecification();
- InvertedIndexCreateOperatorDescriptor invIndexCreateOp = new InvertedIndexCreateOperatorDescriptor(spec,
- AsterixStorageManagerInterface.INSTANCE, fileSplitProviders.first, fileSplitProviders.second,
- AsterixIndexRegistryProvider.INSTANCE, tokenTypeTraits, tokenComparatorFactories, invListsTypeTraits,
- primaryComparatorFactories, tokenizerFactory, new BTreeDataflowHelperFactory(),
- NoOpOperationCallbackProvider.INSTANCE);
+ //TODO replace the transient one to persistent one
+ ILocalResourceFactoryProvider localResourceFactoryProvider = new TransientLocalResourceFactoryProvider();
+ LSMInvertedIndexCreateOperatorDescriptor invIndexCreateOp = new LSMInvertedIndexCreateOperatorDescriptor(spec,
+ AsterixRuntimeComponentsProvider.INSTANCE, fileSplitProviders.first,
+ AsterixRuntimeComponentsProvider.INSTANCE, tokenTypeTraits, tokenComparatorFactories,
+ invListsTypeTraits, primaryComparatorFactories, tokenizerFactory,
+ new LSMInvertedIndexDataflowHelperFactory(AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE), NoOpOperationCallbackProvider.INSTANCE,
+ localResourceFactoryProvider);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, invIndexCreateOp,
secondaryPartitionConstraint);
spec.addRoot(invIndexCreateOp);
@@ -135,7 +143,7 @@
public JobSpecification buildLoadingJobSpec() throws AsterixException, AlgebricksException {
JobSpecification spec = new JobSpecification();
- // Create dummy key provider for feeding the primary index scan.
+ // Create dummy key provider for feeding the primary index scan.
AbstractOperatorDescriptor keyProviderOp = createDummyKeyProviderOp(spec);
// Create primary index scan op.
@@ -144,7 +152,8 @@
// Assign op.
AlgebricksMetaOperatorDescriptor asterixAssignOp = createAssignOp(spec, primaryScanOp, numSecondaryKeys);
- // If any of the secondary fields are nullable, then add a select op that filters nulls.
+ // If any of the secondary fields are nullable, then add a select op
+ // that filters nulls.
AlgebricksMetaOperatorDescriptor selectOp = null;
if (anySecondaryKeyIsNullable) {
selectOp = createFilterNullsSelectOp(spec, numSecondaryKeys);
@@ -157,7 +166,7 @@
ExternalSortOperatorDescriptor sortOp = createSortOp(spec, tokenKeyPairComparatorFactories, tokenKeyPairRecDesc);
// Create secondary inverted index bulk load op.
- InvertedIndexBulkLoadOperatorDescriptor invIndexBulkLoadOp = createInvertedIndexBulkLoadOp(spec);
+ LSMInvertedIndexBulkLoadOperatorDescriptor invIndexBulkLoadOp = createInvertedIndexBulkLoadOp(spec);
// Connect the operators.
spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
@@ -205,16 +214,18 @@
return sortOp;
}
- private InvertedIndexBulkLoadOperatorDescriptor createInvertedIndexBulkLoadOp(JobSpecification spec) {
+ private LSMInvertedIndexBulkLoadOperatorDescriptor createInvertedIndexBulkLoadOp(JobSpecification spec) {
int[] fieldPermutation = new int[numSecondaryKeys + numPrimaryKeys];
for (int i = 0; i < numTokenKeyPairFields; i++) {
fieldPermutation[i] = i;
}
- InvertedIndexBulkLoadOperatorDescriptor invIndexBulkLoadOp = new InvertedIndexBulkLoadOperatorDescriptor(spec,
- fieldPermutation, AsterixStorageManagerInterface.INSTANCE, fileSplitProviders.first,
- fileSplitProviders.second, AsterixIndexRegistryProvider.INSTANCE, tokenTypeTraits,
- tokenComparatorFactories, invListsTypeTraits, primaryComparatorFactories, tokenizerFactory,
- new BTreeDataflowHelperFactory(), NoOpOperationCallbackProvider.INSTANCE);
+ LSMInvertedIndexBulkLoadOperatorDescriptor invIndexBulkLoadOp = new LSMInvertedIndexBulkLoadOperatorDescriptor(
+ spec, fieldPermutation, false, AsterixRuntimeComponentsProvider.INSTANCE, fileSplitProviders.first,
+ AsterixRuntimeComponentsProvider.INSTANCE, tokenTypeTraits, tokenComparatorFactories,
+ invListsTypeTraits, primaryComparatorFactories, tokenizerFactory,
+ new LSMInvertedIndexDataflowHelperFactory(AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE), NoOpOperationCallbackProvider.INSTANCE);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, invIndexBulkLoadOp,
secondaryPartitionConstraint);
return invIndexBulkLoadOp;
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 3127573..a3069b2 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
@@ -2,14 +2,15 @@
import java.util.List;
-import edu.uci.ics.asterix.common.context.AsterixIndexRegistryProvider;
-import edu.uci.ics.asterix.common.context.AsterixStorageManagerInterface;
+import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.dataflow.data.nontagged.valueproviders.AqlPrimitiveValueProviderFactory;
import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
+import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
import edu.uci.ics.asterix.metadata.entities.Index;
+import edu.uci.ics.asterix.om.types.ATypeTag;
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
import edu.uci.ics.asterix.translator.DmlTranslator.CompiledCreateIndexStatement;
@@ -31,14 +32,16 @@
import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexCreateOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.am.rtree.dataflow.RTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.dataflow.LSMRTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+import edu.uci.ics.hyracks.storage.common.file.TransientLocalResourceFactoryProvider;
@SuppressWarnings("rawtypes")
public class SecondaryRTreeCreator extends SecondaryIndexCreator {
protected IPrimitiveValueProviderFactory[] valueProviderFactories;
protected int numNestedSecondaryKeyFields;
+ protected ATypeTag keyType;
protected SecondaryRTreeCreator(PhysicalOptimizationConfig physOptConf) {
super(physOptConf);
@@ -47,10 +50,16 @@
@Override
public JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException {
JobSpecification spec = new JobSpecification();
+ //TODO replace this transient one to the persistent one
+ TransientLocalResourceFactoryProvider localResourceFactoryProvider = new TransientLocalResourceFactoryProvider();
TreeIndexCreateOperatorDescriptor secondaryIndexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
- AsterixStorageManagerInterface.INSTANCE, AsterixIndexRegistryProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
- new RTreeDataflowHelperFactory(valueProviderFactories), NoOpOperationCallbackProvider.INSTANCE);
+ new LSMRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
+ primaryComparatorFactories, AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE, AqlMetadataProvider.proposeLinearizer(keyType,
+ secondaryComparatorFactories.length)), localResourceFactoryProvider);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,
secondaryPartitionConstraint);
spec.addRoot(secondaryIndexCreateOp);
@@ -85,6 +94,7 @@
+ numNestedSecondaryKeyFields];
ITypeTraits[] secondaryTypeTraits = new ITypeTraits[numNestedSecondaryKeyFields + numPrimaryKeys];
IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(spatialType.getTypeTag());
+ keyType = nestedKeyType.getTypeTag();
for (int i = 0; i < numNestedSecondaryKeyFields; i++) {
ISerializerDeserializer keySerde = AqlSerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(nestedKeyType);
@@ -123,9 +133,14 @@
}
// Create secondary RTree bulk load op.
- TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(spec,
- numNestedSecondaryKeyFields, new RTreeDataflowHelperFactory(valueProviderFactories),
- BTree.DEFAULT_FILL_FACTOR);
+ TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(
+ spec,
+ numNestedSecondaryKeyFields,
+ new LSMRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
+ primaryComparatorFactories, AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE, AqlMetadataProvider.proposeLinearizer(keyType,
+ secondaryComparatorFactories.length)), BTree.DEFAULT_FILL_FACTOR);
// Connect the operators.
spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/TestKeywordIndexJob.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/TestKeywordIndexJob.java
index c295f55..4811bb7 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/TestKeywordIndexJob.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/TestKeywordIndexJob.java
@@ -6,8 +6,7 @@
import java.util.HashMap;
import java.util.List;
-import edu.uci.ics.asterix.common.context.AsterixIndexRegistryProvider;
-import edu.uci.ics.asterix.common.context.AsterixStorageManagerInterface;
+import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AObjectSerializerDeserializer;
@@ -38,12 +37,9 @@
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.PrinterOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
public class TestKeywordIndexJob {
@@ -66,9 +62,6 @@
// ---------- START GENERAL BTREE STUFF
- IIndexRegistryProvider<IIndex> indexRegistryProvider = AsterixIndexRegistryProvider.INSTANCE;
- IStorageManagerInterface storageManager = AsterixStorageManagerInterface.INSTANCE;
-
// ---------- END GENERAL BTREE STUFF
List<String> nodeGroup = new ArrayList<String>();
@@ -144,9 +137,11 @@
new FileSplit("nc1", new FileReference(new File("/tmp/nc1/demo1112/Customers_idx_NameInvIndex"))),
new FileSplit("nc2", new FileReference(new File("/tmp/nc2/demo1112/Customers_idx_NameInvIndex"))) });
BTreeSearchOperatorDescriptor secondarySearchOp = new BTreeSearchOperatorDescriptor(spec, secondaryRecDesc,
- storageManager, indexRegistryProvider, secondarySplitProvider, secondaryTypeTraits,
- secondaryComparatorFactories, lowKeyFields, highKeyFields, true, true,
- new BTreeDataflowHelperFactory(), false, NoOpOperationCallbackProvider.INSTANCE);
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+ secondarySplitProvider, secondaryTypeTraits, secondaryComparatorFactories, lowKeyFields, highKeyFields,
+ true, true, new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE), false, NoOpOperationCallbackProvider.INSTANCE);
String[] secondarySearchOpLocationConstraint = new String[nodeGroup.size()];
for (int p = 0; p < nodeGroup.size(); p++) {
secondarySearchOpLocationConstraint[p] = nodeGroup.get(p);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/TestSecondaryIndexJob.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/TestSecondaryIndexJob.java
index 5d2cdc5..c04e8dd 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/TestSecondaryIndexJob.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/TestSecondaryIndexJob.java
@@ -6,8 +6,7 @@
import java.util.HashMap;
import java.util.List;
-import edu.uci.ics.asterix.common.context.AsterixIndexRegistryProvider;
-import edu.uci.ics.asterix.common.context.AsterixStorageManagerInterface;
+import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AObjectSerializerDeserializer;
@@ -38,12 +37,9 @@
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.PrinterOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
public class TestSecondaryIndexJob {
@@ -66,9 +62,6 @@
// ---------- START GENERAL BTREE STUFF
- IIndexRegistryProvider<IIndex> indexRegistryProvider = AsterixIndexRegistryProvider.INSTANCE;
- IStorageManagerInterface storageManager = AsterixStorageManagerInterface.INSTANCE;
-
// ---------- END GENERAL BTREE STUFF
List<String> nodeGroup = new ArrayList<String>();
@@ -144,9 +137,11 @@
new FileSplit("nc1", new FileReference(new File("/tmp/nc1/demo1112/Customers_idx_NameBtreeIndex"))),
new FileSplit("nc2", new FileReference(new File("/tmp/nc2/demo1112/Customers_idx_NameBtreeIndex"))) });
BTreeSearchOperatorDescriptor secondarySearchOp = new BTreeSearchOperatorDescriptor(spec, secondaryRecDesc,
- storageManager, indexRegistryProvider, secondarySplitProvider, secondaryTypeTraits,
- secondaryComparatorFactories, lowKeyFields, highKeyFields, true, true,
- new BTreeDataflowHelperFactory(), false, NoOpOperationCallbackProvider.INSTANCE);
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+ secondarySplitProvider, secondaryTypeTraits, secondaryComparatorFactories, lowKeyFields, highKeyFields,
+ true, true, new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE), false, NoOpOperationCallbackProvider.INSTANCE);
String[] secondarySearchOpLocationConstraint = new String[nodeGroup.size()];
for (int p = 0; p < nodeGroup.size(); p++) {
secondarySearchOpLocationConstraint[p] = nodeGroup.get(p);
diff --git a/asterix-app/src/test/resources/runtimets/ignore.txt b/asterix-app/src/test/resources/runtimets/ignore.txt
index 2b011ec..ec49149 100644
--- a/asterix-app/src/test/resources/runtimets/ignore.txt
+++ b/asterix-app/src/test/resources/runtimets/ignore.txt
@@ -33,3 +33,7 @@
flwor
string/startwith03.aql
aggregate/droptype.aql
+failure/insert-rtree.aql
+failure/insert.aql
+failure/delete-rtree.aql
+failure/delete.aql
\ No newline at end of file
diff --git a/asterix-common/pom.xml b/asterix-common/pom.xml
index 048c037..67377d5 100644
--- a/asterix-common/pom.xml
+++ b/asterix-common/pom.xml
@@ -40,6 +40,11 @@
<type>jar</type>
<scope>compile</scope>
</dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-storage-am-lsm-common</artifactId>
+ <version>0.2.1-SNAPSHOT</version>
+ </dependency>
</dependencies>
</project>
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixAppContextInfoImpl.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixAppContextInfoImpl.java
index 144a8824..dca790d 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixAppContextInfoImpl.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixAppContextInfoImpl.java
@@ -3,11 +3,9 @@
import java.util.Map;
import java.util.Set;
-import edu.uci.ics.asterix.common.context.AsterixIndexRegistryProvider;
-import edu.uci.ics.asterix.common.context.AsterixStorageManagerInterface;
+import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
import edu.uci.ics.asterix.common.dataflow.IAsterixApplicationContextInfo;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
public class AsterixAppContextInfoImpl implements IAsterixApplicationContextInfo {
@@ -20,13 +18,8 @@
}
@Override
- public IIndexRegistryProvider<IIndex> getIndexRegistryProvider() {
- return AsterixIndexRegistryProvider.INSTANCE;
- }
-
- @Override
public IStorageManagerInterface getStorageManagerInterface() {
- return AsterixStorageManagerInterface.INSTANCE;
+ return AsterixRuntimeComponentsProvider.INSTANCE;
}
public static void setNodeControllerInfo(Map<String, Set<String>> nodeControllerInfo) {
@@ -37,4 +30,9 @@
return nodeControllerMap;
}
+ @Override
+ public IIndexLifecycleManagerProvider getIndexLifecycleManagerProvider() {
+ return AsterixRuntimeComponentsProvider.INSTANCE;
+ }
+
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixAppRuntimeContext.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixAppRuntimeContext.java
index d676cb5..52a67d3 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixAppRuntimeContext.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixAppRuntimeContext.java
@@ -8,8 +8,16 @@
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
import edu.uci.ics.hyracks.api.application.INCApplicationContext;
import edu.uci.ics.hyracks.api.io.IIOManager;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexLifecycleManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMFlushController;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ConstantMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.FlushController;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ImmediateScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ReferenceCountingOperationTracker;
import edu.uci.ics.hyracks.storage.common.buffercache.BufferCache;
import edu.uci.ics.hyracks.storage.common.buffercache.ClockPageReplacementStrategy;
import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
@@ -18,16 +26,29 @@
import edu.uci.ics.hyracks.storage.common.buffercache.IPageReplacementStrategy;
import edu.uci.ics.hyracks.storage.common.file.IFileMapManager;
import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepositoryFactory;
+import edu.uci.ics.hyracks.storage.common.file.PersistentLocalResourceRepositoryFactory;
+import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactory;
+import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactoryProvider;
public class AsterixAppRuntimeContext {
private static final int DEFAULT_BUFFER_CACHE_PAGE_SIZE = 32768;
+ private static final int DEFAULT_LIFECYCLEMANAGER_MEMORY_BUDGET = 1024 * 1024 * 1024; // 1GB
private final INCApplicationContext ncApplicationContext;
-
- private IndexRegistry<IIndex> indexRegistry;
+
+ private IIndexLifecycleManager indexLifecycleManager;
private IFileMapManager fileMapManager;
private IBufferCache bufferCache;
private TransactionProvider provider;
+ private ILSMFlushController flushController;
+ private ILSMMergePolicy mergePolicy;
+ private ILSMOperationTracker opTracker;
+ private ILSMIOOperationScheduler lsmIOScheduler;
+ private ILocalResourceRepository localResourceRepository;
+ private ResourceIdFactory resourceIdFactory;
+
public AsterixAppRuntimeContext(INCApplicationContext ncApplicationContext) {
this.ncApplicationContext = ncApplicationContext;
}
@@ -36,20 +57,23 @@
int pageSize = getBufferCachePageSize();
int numPages = getBufferCacheNumPages();
- // Initialize file map manager
fileMapManager = new AsterixFileMapManager();
-
- // Initialize the buffer cache
ICacheMemoryAllocator allocator = new HeapBufferAllocator();
IPageReplacementStrategy prs = new ClockPageReplacementStrategy();
IIOManager ioMgr = ncApplicationContext.getRootContext().getIOManager();
bufferCache = new BufferCache(ioMgr, allocator, prs, fileMapManager, pageSize, numPages, Integer.MAX_VALUE);
-
- // Initialize the index registry
- indexRegistry = new IndexRegistry<IIndex>();
-
- // Initialize the transaction sub-system
+ indexLifecycleManager = new IndexLifecycleManager(DEFAULT_LIFECYCLEMANAGER_MEMORY_BUDGET);
provider = new TransactionProvider(ncApplicationContext.getNodeId());
+
+ flushController = new FlushController();
+ lsmIOScheduler = ImmediateScheduler.INSTANCE;
+ mergePolicy = new ConstantMergePolicy(lsmIOScheduler, 3);
+ opTracker = new ReferenceCountingOperationTracker();
+
+ ILocalResourceRepositoryFactory persistentLocalResourceRepositoryFactory = new PersistentLocalResourceRepositoryFactory(
+ ioMgr);
+ localResourceRepository = persistentLocalResourceRepositoryFactory.createRepository();
+ resourceIdFactory = (new ResourceIdFactoryProvider(localResourceRepository)).createResourceIdFactory();
}
private int getBufferCachePageSize() {
@@ -107,12 +131,35 @@
return fileMapManager;
}
- public IndexRegistry<IIndex> getIndexRegistry() {
- return indexRegistry;
- }
-
public TransactionProvider getTransactionProvider() {
return provider;
}
+ public IIndexLifecycleManager getIndexLifecycleManager() {
+ return indexLifecycleManager;
+ }
+
+ public ILSMFlushController getFlushController() {
+ return flushController;
+ }
+
+ public ILSMMergePolicy getLSMMergePolicy() {
+ return mergePolicy;
+ }
+
+ public ILSMOperationTracker getLSMOperationTracker() {
+ return opTracker;
+ }
+
+ public ILSMIOOperationScheduler getLSMIOScheduler() {
+ return lsmIOScheduler;
+ }
+
+ public ILocalResourceRepository getLocalResourceRepository() {
+ return localResourceRepository;
+ }
+
+ public ResourceIdFactory getResourceIdFactory() {
+ return resourceIdFactory;
+ }
}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixIndexRegistryProvider.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixIndexRegistryProvider.java
deleted file mode 100644
index bc59291..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixIndexRegistryProvider.java
+++ /dev/null
@@ -1,23 +0,0 @@
-package edu.uci.ics.asterix.common.context;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
-
-public class AsterixIndexRegistryProvider implements IIndexRegistryProvider<IIndex> {
-
- private static final long serialVersionUID = 1L;
-
- public static final AsterixIndexRegistryProvider INSTANCE = new AsterixIndexRegistryProvider();
-
- private AsterixIndexRegistryProvider() {
- }
-
- @Override
- public IndexRegistry<IIndex> getRegistry(IHyracksTaskContext ctx) {
- return ((AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
- .getIndexRegistry();
- }
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixRuntimeComponentsProvider.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixRuntimeComponentsProvider.java
new file mode 100644
index 0000000..49f59a0
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixRuntimeComponentsProvider.java
@@ -0,0 +1,78 @@
+package edu.uci.ics.asterix.common.context;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMFlushController;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMFlushControllerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
+import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactory;
+
+public enum AsterixRuntimeComponentsProvider implements IIndexLifecycleManagerProvider, IStorageManagerInterface,
+ ILSMIOOperationSchedulerProvider, ILSMFlushControllerProvider, ILSMOperationTrackerProvider,
+ ILSMMergePolicyProvider {
+ INSTANCE;
+
+ @Override
+ public ILSMOperationTracker getOperationTracker(IHyracksTaskContext ctx) {
+ return ((AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
+ .getLSMOperationTracker();
+ }
+
+ @Override
+ public ILSMFlushController getFlushController(IHyracksTaskContext ctx) {
+ return ((AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
+ .getFlushController();
+ }
+
+ @Override
+ public ILSMIOOperationScheduler getIOScheduler(IHyracksTaskContext ctx) {
+ return ((AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
+ .getLSMIOScheduler();
+ }
+
+ @Override
+ public ILSMMergePolicy getMergePolicy(IHyracksTaskContext ctx) {
+ return ((AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
+ .getLSMMergePolicy();
+ }
+
+ @Override
+ public IBufferCache getBufferCache(IHyracksTaskContext ctx) {
+ return ((AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
+ .getBufferCache();
+ }
+
+ @Override
+ public IFileMapProvider getFileMapProvider(IHyracksTaskContext ctx) {
+ return ((AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
+ .getFileMapManager();
+ }
+
+ @Override
+ public ILocalResourceRepository getLocalResourceRepository(IHyracksTaskContext ctx) {
+ return ((AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
+ .getLocalResourceRepository();
+ }
+
+ @Override
+ public IIndexLifecycleManager getLifecycleManager(IHyracksTaskContext ctx) {
+ return ((AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
+ .getIndexLifecycleManager();
+ }
+
+ @Override
+ public ResourceIdFactory getResourceIdFactory(IHyracksTaskContext ctx) {
+ return ((AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
+ .getResourceIdFactory();
+ }
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixStorageManagerInterface.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixStorageManagerInterface.java
deleted file mode 100644
index a1b5a94..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixStorageManagerInterface.java
+++ /dev/null
@@ -1,24 +0,0 @@
-package edu.uci.ics.asterix.common.context;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
-
-public class AsterixStorageManagerInterface implements IStorageManagerInterface {
- private static final long serialVersionUID = 1L;
-
- public static AsterixStorageManagerInterface INSTANCE = new AsterixStorageManagerInterface();
-
- @Override
- public IBufferCache getBufferCache(IHyracksTaskContext ctx) {
- return ((AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
- .getBufferCache();
- }
-
- @Override
- public IFileMapProvider getFileMapProvider(IHyracksTaskContext ctx) {
- return ((AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
- .getFileMapManager();
- }
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/IAsterixApplicationContextInfo.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/IAsterixApplicationContextInfo.java
index 7bb0fd6..200c2e19 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/IAsterixApplicationContextInfo.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/IAsterixApplicationContextInfo.java
@@ -1,11 +1,10 @@
package edu.uci.ics.asterix.common.dataflow;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
public interface IAsterixApplicationContextInfo {
- public IIndexRegistryProvider<IIndex> getIndexRegistryProvider();
+ public IIndexLifecycleManagerProvider getIndexLifecycleManagerProvider();
public IStorageManagerInterface getStorageManagerInterface();
}
diff --git a/asterix-hyracks-glue/pom.xml b/asterix-hyracks-glue/pom.xml
deleted file mode 100644
index 5f991a4..0000000
--- a/asterix-hyracks-glue/pom.xml
+++ /dev/null
@@ -1,51 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
- <modelVersion>4.0.0</modelVersion>
- <parent>
- <artifactId>asterix</artifactId>
- <groupId>edu.uci.ics.asterix</groupId>
- <version>0.0.4-SNAPSHOT</version>
- </parent>
- <groupId>edu.uci.ics.asterix</groupId>
- <artifactId>asterix-hyracks-glue</artifactId>
- <version>0.0.4-SNAPSHOT</version>
- <build>
- <plugins>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-compiler-plugin</artifactId>
- <version>2.0.2</version>
- <configuration>
- <source>1.6</source>
- <target>1.6</target>
- </configuration>
- </plugin>
- </plugins>
- </build>
- <name>asterix-hyracks-glue</name>
- <url>http://maven.apache.org</url>
- <dependencies>
- <dependency>
- <groupId>junit</groupId>
- <artifactId>junit</artifactId>
- <version>3.8.1</version>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>edu.uci.ics.asterix</groupId>
- <artifactId>asterix-transactions</artifactId>
- <version>0.0.4-SNAPSHOT</version>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>edu.uci.ics.asterix</groupId>
- <artifactId>asterix-common</artifactId>
- <version>0.0.4-SNAPSHOT</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- </dependencies>
- <properties>
- <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
- </properties>
-</project>
diff --git a/asterix-hyracks-glue/src/main/java/edu/uci/ics/asterix/runtime/transaction/TreeIndexInsertUpdateDeleteOperatorDescriptor.java b/asterix-hyracks-glue/src/main/java/edu/uci/ics/asterix/runtime/transaction/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
deleted file mode 100644
index f96d3da..0000000
--- a/asterix-hyracks-glue/src/main/java/edu/uci/ics/asterix/runtime/transaction/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
+++ /dev/null
@@ -1,82 +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.runtime.transaction;
-
-import edu.uci.ics.asterix.common.context.AsterixAppRuntimeContext;
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionManager;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.IOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.ITupleFilterFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-
-public class TreeIndexInsertUpdateDeleteOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
-
- private static final long serialVersionUID = 1L;
-
- private final int[] fieldPermutation;
-
- private final IndexOp op;
-
- private final long transactionId;
-
- /**
- * TODO: Index operators should live in Hyracks. Right now, they are needed
- * here in Asterix as a hack to provide transactionIDs. The Asterix verions
- * of this operator will disappear and the operator will come from Hyracks
- * once the LSM/Recovery/Transactions world has been introduced.
- */
- public TreeIndexInsertUpdateDeleteOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
- IStorageManagerInterface storageManager, IIndexRegistryProvider<IIndex> indexRegistryProvider,
- IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
- IBinaryComparatorFactory[] comparatorFactories, int[] fieldPermutation, IndexOp op,
- IIndexDataflowHelperFactory dataflowHelperFactory, ITupleFilterFactory tupleFilterFactory,
- IOperationCallbackProvider opCallbackProvider, long transactionId) {
- super(spec, 1, 1, recDesc, storageManager, indexRegistryProvider, fileSplitProvider, typeTraits,
- comparatorFactories, dataflowHelperFactory, tupleFilterFactory, false, opCallbackProvider);
- this.fieldPermutation = fieldPermutation;
- this.op = op;
- this.transactionId = transactionId;
- }
-
- @Override
- public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
- IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
- TransactionContext txnContext;
- try {
- ITransactionManager transactionManager = ((AsterixAppRuntimeContext) ctx.getJobletContext()
- .getApplicationContext().getApplicationObject()).getTransactionProvider().getTransactionManager();
- txnContext = transactionManager.getTransactionContext(transactionId);
- } catch (ACIDException ae) {
- throw new RuntimeException(" could not obtain context for invalid transaction id " + transactionId);
- }
- return new TreeIndexInsertUpdateDeleteOperatorNodePushable(txnContext, this, ctx, partition, fieldPermutation,
- recordDescProvider, op);
- }
-}
diff --git a/asterix-hyracks-glue/src/main/java/edu/uci/ics/asterix/runtime/transaction/TreeIndexInsertUpdateDeleteOperatorNodePushable.java b/asterix-hyracks-glue/src/main/java/edu/uci/ics/asterix/runtime/transaction/TreeIndexInsertUpdateDeleteOperatorNodePushable.java
deleted file mode 100644
index 1fd8fee..0000000
--- a/asterix-hyracks-glue/src/main/java/edu/uci/ics/asterix/runtime/transaction/TreeIndexInsertUpdateDeleteOperatorNodePushable.java
+++ /dev/null
@@ -1,214 +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.runtime.transaction;
-
-import java.nio.ByteBuffer;
-
-import edu.uci.ics.asterix.common.context.AsterixAppRuntimeContext;
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.resource.ICloseable;
-import edu.uci.ics.asterix.transaction.management.resource.TransactionalResourceRepository;
-import edu.uci.ics.asterix.transaction.management.service.locking.ILockManager;
-import edu.uci.ics.asterix.transaction.management.service.logging.DataUtil;
-import edu.uci.ics.asterix.transaction.management.service.logging.TreeLogger;
-import edu.uci.ics.asterix.transaction.management.service.logging.TreeResourceManager;
-import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
-import edu.uci.ics.hyracks.storage.am.common.api.ITupleFilter;
-import edu.uci.ics.hyracks.storage.am.common.api.ITupleFilterFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.PermutingFrameTupleReference;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
-
-public class TreeIndexInsertUpdateDeleteOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
- private final IHyracksTaskContext ctx;
- private FrameTupleAccessor accessor;
- private TreeIndexDataflowHelper treeIndexHelper;
- private final IRecordDescriptorProvider recordDescProvider;
- private final IndexOp op;
- private final PermutingFrameTupleReference tuple = new PermutingFrameTupleReference();
- private FrameTupleReference frameTuple;
- private ByteBuffer writeBuffer;
- private IIndexAccessor indexAccessor;
- private ILockManager lockManager;
- private final TransactionContext txnContext;
- private TreeLogger treeLogger;
- private ITupleFilter tupleFilter;
- private final TransactionProvider transactionProvider;
-
- /* TODO: Index operators should live in Hyracks. Right now, they are needed here in Asterix
- * as a hack to provide transactionIDs. The Asterix verions of this operator will disappear
- * and the operator will come from Hyracks once the LSM/Recovery/Transactions world has
- * been introduced.
- */
- public TreeIndexInsertUpdateDeleteOperatorNodePushable(TransactionContext txnContext,
- AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition, int[] fieldPermutation,
- IRecordDescriptorProvider recordDescProvider, IndexOp op) {
- this.ctx = ctx;
- treeIndexHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
- opDesc, ctx, partition);
- this.recordDescProvider = recordDescProvider;
- this.op = op;
- tuple.setFieldPermutation(fieldPermutation);
- this.txnContext = txnContext;
- AsterixAppRuntimeContext runtimeContext = (AsterixAppRuntimeContext) ctx.getJobletContext()
- .getApplicationContext().getApplicationObject();
- transactionProvider = runtimeContext.getTransactionProvider();
- }
-
- public void initializeTransactionSupport() {
- TransactionalResourceRepository resourceRepository = transactionProvider.getTransactionalResourceRepository();
- IResourceManager resourceMgr = resourceRepository.getTransactionalResourceMgr(TreeResourceManager.ID);
- if (resourceMgr == null) {
- resourceRepository.registerTransactionalResourceManager(TreeResourceManager.ID, new TreeResourceManager(
- transactionProvider));
- }
- int fileId = treeIndexHelper.getIndexFileId();
- byte[] resourceId = DataUtil.intToByteArray(fileId);
- transactionProvider.getTransactionalResourceRepository().registerTransactionalResource(resourceId,
- treeIndexHelper.getIndex());
- lockManager = transactionProvider.getLockManager();
- treeLogger = transactionProvider.getTreeLoggerRepository().getTreeLogger(resourceId);
- }
-
- @Override
- public void open() throws HyracksDataException {
- AbstractTreeIndexOperatorDescriptor opDesc = (AbstractTreeIndexOperatorDescriptor) treeIndexHelper
- .getOperatorDescriptor();
- RecordDescriptor inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
- accessor = new FrameTupleAccessor(treeIndexHelper.getHyracksTaskContext().getFrameSize(), inputRecDesc);
- writeBuffer = treeIndexHelper.getHyracksTaskContext().allocateFrame();
- writer.open();
- try {
- treeIndexHelper.init(false);
- ITreeIndex treeIndex = (ITreeIndex) treeIndexHelper.getIndex();
- indexAccessor = treeIndex.createAccessor();
- ITupleFilterFactory tupleFilterFactory = opDesc.getTupleFilterFactory();
- if (tupleFilterFactory != null) {
- tupleFilter = tupleFilterFactory.createTupleFilter(ctx);
- frameTuple = new FrameTupleReference();
- }
- initializeTransactionSupport();
- } catch (Exception e) {
- // cleanup in case of failure
- treeIndexHelper.deinit();
- throw new HyracksDataException(e);
- }
- }
-
- @Override
- public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
- final IIndex treeIndex = treeIndexHelper.getIndex();
- accessor.reset(buffer);
- int fileId = treeIndexHelper.getIndexFileId();
- byte[] resourceId = DataUtil.intToByteArray(fileId);
- int tupleCount = accessor.getTupleCount();
- try {
- for (int i = 0; i < tupleCount; i++) {
- if (tupleFilter != null) {
- frameTuple.reset(accessor, i);
- if (!tupleFilter.accept(frameTuple)) {
- continue;
- }
- }
- tuple.reset(accessor, i);
- switch (op) {
- case INSERT: {
- lockManager.lock(txnContext, resourceId,
- TransactionManagementConstants.LockManagerConstants.LockMode.EXCLUSIVE);
- indexAccessor.insert(tuple);
- treeLogger.generateLogRecord(transactionProvider, txnContext, op, tuple);
- break;
- }
-
- case DELETE: {
- lockManager.lock(txnContext, resourceId,
- TransactionManagementConstants.LockManagerConstants.LockMode.EXCLUSIVE);
- indexAccessor.delete(tuple);
- treeLogger.generateLogRecord(transactionProvider, txnContext, op, tuple);
- break;
- }
-
- default: {
- throw new HyracksDataException("Unsupported operation " + op
- + " in tree index InsertUpdateDelete operator");
- }
- }
- }
- } catch (ACIDException ae) {
- throw new HyracksDataException("exception in locking/logging during operation " + op + " on tree "
- + treeIndex, ae);
- } catch (Exception e) {
- e.printStackTrace();
- throw new HyracksDataException(e);
- }
-
- // pass a copy of the frame to next op
- System.arraycopy(buffer.array(), 0, writeBuffer.array(), 0, buffer.capacity());
- FrameUtils.flushFrame(writeBuffer, writer);
-
- }
-
- @Override
- public void close() throws HyracksDataException {
- try {
- writer.close();
- } finally {
- txnContext.addCloseableResource(new ICloseable() {
- @Override
- public void close(TransactionContext txnContext) throws ACIDException {
- try {
- treeIndexHelper.deinit();
- } catch (Exception e) {
- throw new ACIDException(txnContext, "could not de-initialize " + treeIndexHelper, e);
- }
- }
- });
- }
- }
-
- @Override
- public void fail() throws HyracksDataException {
- try {
- writer.fail();
- } finally {
- txnContext.addCloseableResource(new ICloseable() {
- @Override
- public void close(TransactionContext txnContext) throws ACIDException {
- try {
- treeIndexHelper.deinit();
- } catch (Exception e) {
- throw new ACIDException(txnContext, "could not de-initialize " + treeIndexHelper, e);
- }
- }
- });
- }
- }
-
-}
diff --git a/asterix-metadata/pom.xml b/asterix-metadata/pom.xml
index 67f16ff..d95e29a 100644
--- a/asterix-metadata/pom.xml
+++ b/asterix-metadata/pom.xml
@@ -1,4 +1,5 @@
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<artifactId>asterix</artifactId>
@@ -43,27 +44,32 @@
<scope>compile</scope>
</dependency>
<dependency>
- <groupId>edu.uci.ics.hyracks</groupId>
- <artifactId>hyracks-storage-am-invertedindex</artifactId>
- <version>0.2.2-SNAPSHOT</version>
- </dependency>
- <dependency>
- <groupId>edu.uci.ics.hyracks</groupId>
- <artifactId>hyracks-storage-am-rtree</artifactId>
- <version>0.2.2-SNAPSHOT</version>
- </dependency>
- <dependency>
- <groupId>edu.uci.ics.asterix</groupId>
- <artifactId>asterix-hyracks-glue</artifactId>
- <version>0.0.4-SNAPSHOT</version>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-core</artifactId>
+ <version>0.20.2</version>
+ <type>jar</type>
<scope>compile</scope>
</dependency>
- <dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-core</artifactId>
- <version>0.20.2</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>
+ hyracks-storage-am-lsm-invertedindex
+ </artifactId>
+ <version>0.2.2-SNAPSHOT</version>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>
+ hyracks-storage-am-lsm-btree
+ </artifactId>
+ <version>0.2.2-SNAPSHOT</version>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>
+ hyracks-storage-am-lsm-rtree
+ </artifactId>
+ <version>0.2.2-SNAPSHOT</version>
+ </dependency>
</dependencies>
</project>
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java
index bd4f9bb..dfb272c 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java
@@ -29,7 +29,8 @@
import edu.uci.ics.asterix.metadata.entities.Node;
import edu.uci.ics.asterix.metadata.entities.NodeGroup;
import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionIDFactory;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobIdFactory;
/**
* Provides access to Asterix metadata via remote methods to the metadata node.
@@ -105,36 +106,36 @@
@Override
public MetadataTransactionContext beginTransaction() throws RemoteException, ACIDException {
- long txnId = TransactionIDFactory.generateTransactionId();
- metadataNode.beginTransaction(txnId);
- return new MetadataTransactionContext(txnId);
+ JobId jobId = JobIdFactory.generateJobId();
+ metadataNode.beginTransaction(jobId);
+ return new MetadataTransactionContext(jobId);
}
@Override
public void commitTransaction(MetadataTransactionContext ctx) throws RemoteException, ACIDException {
- metadataNode.commitTransaction(ctx.getTxnId());
+ metadataNode.commitTransaction(ctx.getJobId());
cache.commit(ctx);
}
@Override
public void abortTransaction(MetadataTransactionContext ctx) throws RemoteException, ACIDException {
- metadataNode.abortTransaction(ctx.getTxnId());
+ metadataNode.abortTransaction(ctx.getJobId());
}
@Override
- public void lock(MetadataTransactionContext ctx, int lockMode) throws RemoteException, ACIDException {
- metadataNode.lock(ctx.getTxnId(), lockMode);
+ public void lock(MetadataTransactionContext ctx, byte lockMode) throws RemoteException, ACIDException {
+ metadataNode.lock(ctx.getJobId(), lockMode);
}
@Override
public void unlock(MetadataTransactionContext ctx) throws RemoteException, ACIDException {
- metadataNode.unlock(ctx.getTxnId());
+ metadataNode.unlock(ctx.getJobId());
}
@Override
public void addDataverse(MetadataTransactionContext ctx, Dataverse dataverse) throws MetadataException {
try {
- metadataNode.addDataverse(ctx.getTxnId(), dataverse);
+ metadataNode.addDataverse(ctx.getJobId(), dataverse);
} catch (RemoteException e) {
throw new MetadataException(e);
}
@@ -144,7 +145,7 @@
@Override
public void dropDataverse(MetadataTransactionContext ctx, String dataverseName) throws MetadataException {
try {
- metadataNode.dropDataverse(ctx.getTxnId(), dataverseName);
+ metadataNode.dropDataverse(ctx.getJobId(), dataverseName);
} catch (RemoteException e) {
throw new MetadataException(e);
}
@@ -172,7 +173,7 @@
return dataverse;
}
try {
- dataverse = metadataNode.getDataverse(ctx.getTxnId(), dataverseName);
+ dataverse = metadataNode.getDataverse(ctx.getJobId(), dataverseName);
} catch (RemoteException e) {
throw new MetadataException(e);
}
@@ -191,7 +192,7 @@
try {
// Assuming that the transaction can read its own writes on the
// metadata node.
- dataverseDatasets = metadataNode.getDataverseDatasets(ctx.getTxnId(), dataverseName);
+ dataverseDatasets = metadataNode.getDataverseDatasets(ctx.getJobId(), dataverseName);
} catch (RemoteException e) {
throw new MetadataException(e);
}
@@ -203,7 +204,7 @@
@Override
public void addDataset(MetadataTransactionContext ctx, Dataset dataset) throws MetadataException {
try {
- metadataNode.addDataset(ctx.getTxnId(), dataset);
+ metadataNode.addDataset(ctx.getJobId(), dataset);
} catch (RemoteException e) {
throw new MetadataException(e);
}
@@ -214,7 +215,7 @@
public void dropDataset(MetadataTransactionContext ctx, String dataverseName, String datasetName)
throws MetadataException {
try {
- metadataNode.dropDataset(ctx.getTxnId(), dataverseName, datasetName);
+ metadataNode.dropDataset(ctx.getJobId(), dataverseName, datasetName);
} catch (RemoteException e) {
throw new MetadataException(e);
}
@@ -248,7 +249,7 @@
return dataset;
}
try {
- dataset = metadataNode.getDataset(ctx.getTxnId(), dataverseName, datasetName);
+ dataset = metadataNode.getDataset(ctx.getJobId(), dataverseName, datasetName);
} catch (RemoteException e) {
throw new MetadataException(e);
}
@@ -265,7 +266,7 @@
throws MetadataException {
List<Index> datsetIndexes;
try {
- datsetIndexes = metadataNode.getDatasetIndexes(ctx.getTxnId(), dataverseName, datasetName);
+ datsetIndexes = metadataNode.getDatasetIndexes(ctx.getJobId(), dataverseName, datasetName);
} catch (RemoteException e) {
throw new MetadataException(e);
}
@@ -275,7 +276,7 @@
@Override
public void addDatatype(MetadataTransactionContext ctx, Datatype datatype) throws MetadataException {
try {
- metadataNode.addDatatype(ctx.getTxnId(), datatype);
+ metadataNode.addDatatype(ctx.getJobId(), datatype);
} catch (RemoteException e) {
throw new MetadataException(e);
}
@@ -286,7 +287,7 @@
public void dropDatatype(MetadataTransactionContext ctx, String dataverseName, String datatypeName)
throws MetadataException {
try {
- metadataNode.dropDatatype(ctx.getTxnId(), dataverseName, datatypeName);
+ metadataNode.dropDatatype(ctx.getJobId(), dataverseName, datatypeName);
} catch (RemoteException e) {
throw new MetadataException(e);
}
@@ -320,7 +321,7 @@
return datatype;
}
try {
- datatype = metadataNode.getDatatype(ctx.getTxnId(), dataverseName, datatypeName);
+ datatype = metadataNode.getDatatype(ctx.getJobId(), dataverseName, datatypeName);
} catch (RemoteException e) {
throw new MetadataException(e);
}
@@ -335,7 +336,7 @@
@Override
public void addIndex(MetadataTransactionContext ctx, Index index) throws MetadataException {
try {
- metadataNode.addIndex(ctx.getTxnId(), index);
+ metadataNode.addIndex(ctx.getJobId(), index);
} catch (RemoteException e) {
throw new MetadataException(e);
}
@@ -345,7 +346,7 @@
public void dropIndex(MetadataTransactionContext ctx, String dataverseName, String datasetName, String indexName)
throws MetadataException {
try {
- metadataNode.dropIndex(ctx.getTxnId(), dataverseName, datasetName, indexName);
+ metadataNode.dropIndex(ctx.getJobId(), dataverseName, datasetName, indexName);
} catch (RemoteException e) {
throw new MetadataException(e);
}
@@ -355,7 +356,7 @@
public Index getIndex(MetadataTransactionContext ctx, String dataverseName, String datasetName, String indexName)
throws MetadataException {
try {
- return metadataNode.getIndex(ctx.getTxnId(), dataverseName, datasetName, indexName);
+ return metadataNode.getIndex(ctx.getJobId(), dataverseName, datasetName, indexName);
} catch (RemoteException e) {
throw new MetadataException(e);
}
@@ -364,7 +365,7 @@
@Override
public void addNode(MetadataTransactionContext ctx, Node node) throws MetadataException {
try {
- metadataNode.addNode(ctx.getTxnId(), node);
+ metadataNode.addNode(ctx.getJobId(), node);
} catch (RemoteException e) {
throw new MetadataException(e);
}
@@ -373,7 +374,7 @@
@Override
public void addNodegroup(MetadataTransactionContext ctx, NodeGroup nodeGroup) throws MetadataException {
try {
- metadataNode.addNodeGroup(ctx.getTxnId(), nodeGroup);
+ metadataNode.addNodeGroup(ctx.getJobId(), nodeGroup);
} catch (RemoteException e) {
throw new MetadataException(e);
}
@@ -383,7 +384,7 @@
@Override
public void dropNodegroup(MetadataTransactionContext ctx, String nodeGroupName) throws MetadataException {
try {
- metadataNode.dropNodegroup(ctx.getTxnId(), nodeGroupName);
+ metadataNode.dropNodegroup(ctx.getJobId(), nodeGroupName);
} catch (RemoteException e) {
throw new MetadataException(e);
}
@@ -411,7 +412,7 @@
return nodeGroup;
}
try {
- nodeGroup = metadataNode.getNodeGroup(ctx.getTxnId(), nodeGroupName);
+ nodeGroup = metadataNode.getNodeGroup(ctx.getJobId(), nodeGroupName);
} catch (RemoteException e) {
throw new MetadataException(e);
}
@@ -426,7 +427,7 @@
@Override
public void addFunction(MetadataTransactionContext mdTxnCtx, Function function) throws MetadataException {
try {
- metadataNode.addFunction(mdTxnCtx.getTxnId(), function);
+ metadataNode.addFunction(mdTxnCtx.getJobId(), function);
} catch (RemoteException e) {
throw new MetadataException(e);
}
@@ -437,7 +438,7 @@
public void dropFunction(MetadataTransactionContext ctx, String dataverseName, String functionName, int arity)
throws MetadataException {
try {
- metadataNode.dropFunction(ctx.getTxnId(), dataverseName, functionName, arity);
+ metadataNode.dropFunction(ctx.getJobId(), dataverseName, functionName, arity);
} catch (RemoteException e) {
throw new MetadataException(e);
}
@@ -471,7 +472,7 @@
return function;
}
try {
- function = metadataNode.getFunction(ctx.getTxnId(), dataverseName, functionName, arity);
+ function = metadataNode.getFunction(ctx.getJobId(), dataverseName, functionName, arity);
} catch (RemoteException e) {
throw new MetadataException(e);
}
@@ -483,4 +484,13 @@
return function;
}
+
+ @Override
+ public void initializeDatasetIdFactory(MetadataTransactionContext ctx) throws MetadataException {
+ try {
+ metadataNode.initializeDatasetIdFactory(ctx.getJobId());
+ } catch (RemoteException e) {
+ throw new MetadataException(e);
+ }
+ }
}
\ No newline at end of file
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 591154a..c634d95 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
@@ -53,6 +53,9 @@
import edu.uci.ics.asterix.om.base.AString;
import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
+import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetIdFactory;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
@@ -63,27 +66,27 @@
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeDuplicateKeyException;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTreeRangeSearchCursor;
public class MetadataNode implements IMetadataNode {
private static final long serialVersionUID = 1L;
// TODO: Temporary transactional resource id for metadata.
private static final byte[] metadataResourceId = MetadataNode.class.toString().getBytes();
+ private static final DatasetId METADATA_DATASET_ID = new DatasetId(MetadataPrimaryIndexes.METADATA_DATASET_ID);
- private IndexRegistry<IIndex> indexRegistry;
+ private IIndexLifecycleManager indexLifecycleManager;
private TransactionProvider transactionProvider;
public static final MetadataNode INSTANCE = new MetadataNode();
@@ -94,24 +97,24 @@
public void initialize(AsterixAppRuntimeContext runtimeContext) {
this.transactionProvider = runtimeContext.getTransactionProvider();
- this.indexRegistry = runtimeContext.getIndexRegistry();
+ this.indexLifecycleManager = runtimeContext.getIndexLifecycleManager();
}
@Override
- public void beginTransaction(long transactionId) throws ACIDException, RemoteException {
+ public void beginTransaction(JobId transactionId) throws ACIDException, RemoteException {
transactionProvider.getTransactionManager().beginTransaction(transactionId);
}
@Override
- public void commitTransaction(long txnId) throws RemoteException, ACIDException {
- TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(txnId);
+ public void commitTransaction(JobId jobId) throws RemoteException, ACIDException {
+ TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(jobId);
transactionProvider.getTransactionManager().commitTransaction(txnCtx);
}
@Override
- public void abortTransaction(long txnId) throws RemoteException, ACIDException {
+ public void abortTransaction(JobId jobId) throws RemoteException, ACIDException {
try {
- TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(txnId);
+ TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(jobId);
transactionProvider.getTransactionManager().abortTransaction(txnCtx);
} catch (ACIDException e) {
e.printStackTrace();
@@ -120,23 +123,23 @@
}
@Override
- public boolean lock(long txnId, int lockMode) throws ACIDException, RemoteException {
- TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(txnId);
- return transactionProvider.getLockManager().lock(txnCtx, metadataResourceId, lockMode);
+ public void lock(JobId jobId, byte lockMode) throws ACIDException, RemoteException {
+ TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(jobId);
+ transactionProvider.getLockManager().lock(METADATA_DATASET_ID, -1, lockMode, txnCtx);
}
@Override
- public boolean unlock(long txnId) throws ACIDException, RemoteException {
- TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(txnId);
- return transactionProvider.getLockManager().unlock(txnCtx, metadataResourceId);
+ public void unlock(JobId jobId) throws ACIDException, RemoteException {
+ TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(jobId);
+ transactionProvider.getLockManager().unlock(METADATA_DATASET_ID, -1, txnCtx);
}
@Override
- public void addDataverse(long txnId, Dataverse dataverse) throws MetadataException, RemoteException {
+ public void addDataverse(JobId jobId, Dataverse dataverse) throws MetadataException, RemoteException {
try {
DataverseTupleTranslator tupleReaderWriter = new DataverseTupleTranslator(true);
ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(dataverse);
- insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.DATAVERSE_DATASET, tuple);
+ insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, tuple);
} catch (BTreeDuplicateKeyException e) {
throw new MetadataException("A dataverse with this name " + dataverse.getDataverseName()
+ " already exists.", e);
@@ -146,26 +149,26 @@
}
@Override
- public void addDataset(long txnId, Dataset dataset) throws MetadataException, RemoteException {
+ public void addDataset(JobId jobId, Dataset dataset) throws MetadataException, RemoteException {
try {
// Insert into the 'dataset' dataset.
DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(true);
ITupleReference datasetTuple = tupleReaderWriter.getTupleFromMetadataEntity(dataset);
- insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
+ insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
// Add the primary index for the dataset.
InternalDatasetDetails id = (InternalDatasetDetails) dataset.getDatasetDetails();
Index primaryIndex = new Index(dataset.getDataverseName(), dataset.getDatasetName(),
dataset.getDatasetName(), IndexType.BTREE, id.getPrimaryKey(), true);
- addIndex(txnId, primaryIndex);
+ addIndex(jobId, primaryIndex);
ITupleReference nodeGroupTuple = createTuple(id.getNodeGroupName(), dataset.getDataverseName(),
dataset.getDatasetName());
- insertTupleIntoIndex(txnId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, nodeGroupTuple);
+ insertTupleIntoIndex(jobId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, nodeGroupTuple);
}
// Add entry in datatype secondary index.
ITupleReference dataTypeTuple = createTuple(dataset.getDataverseName(), dataset.getItemTypeName(),
dataset.getDatasetName());
- insertTupleIntoIndex(txnId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeTuple);
+ insertTupleIntoIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeTuple);
} catch (BTreeDuplicateKeyException e) {
throw new MetadataException("A dataset with this name " + dataset.getDatasetName()
+ " already exists in dataverse '" + dataset.getDataverseName() + "'.", e);
@@ -175,11 +178,11 @@
}
@Override
- public void addIndex(long txnId, Index index) throws MetadataException, RemoteException {
+ public void addIndex(JobId jobId, Index index) throws MetadataException, RemoteException {
try {
IndexTupleTranslator tupleWriter = new IndexTupleTranslator(true);
ITupleReference tuple = tupleWriter.getTupleFromMetadataEntity(index);
- insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.INDEX_DATASET, tuple);
+ insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, tuple);
} catch (BTreeDuplicateKeyException e) {
throw new MetadataException("An index with name '" + index.getIndexName() + "' already exists.", e);
} catch (Exception e) {
@@ -188,11 +191,11 @@
}
@Override
- public void addNode(long txnId, Node node) throws MetadataException, RemoteException {
+ public void addNode(JobId jobId, Node node) throws MetadataException, RemoteException {
try {
NodeTupleTranslator tupleReaderWriter = new NodeTupleTranslator(true);
ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(node);
- insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.NODE_DATASET, tuple);
+ insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.NODE_DATASET, tuple);
} catch (BTreeDuplicateKeyException e) {
throw new MetadataException("A node with name '" + node.getNodeName() + "' already exists.", e);
} catch (Exception e) {
@@ -201,11 +204,11 @@
}
@Override
- public void addNodeGroup(long txnId, NodeGroup nodeGroup) throws MetadataException, RemoteException {
+ public void addNodeGroup(JobId jobId, NodeGroup nodeGroup) throws MetadataException, RemoteException {
try {
NodeGroupTupleTranslator tupleReaderWriter = new NodeGroupTupleTranslator(true);
ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(nodeGroup);
- insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.NODEGROUP_DATASET, tuple);
+ insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, tuple);
} catch (BTreeDuplicateKeyException e) {
throw new MetadataException("A nodegroup with name '" + nodeGroup.getNodeGroupName() + "' already exists.",
e);
@@ -215,11 +218,11 @@
}
@Override
- public void addDatatype(long txnId, Datatype datatype) throws MetadataException, RemoteException {
+ public void addDatatype(JobId jobId, Datatype datatype) throws MetadataException, RemoteException {
try {
- DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(txnId, this, true);
+ DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(jobId, this, true);
ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(datatype);
- insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
+ insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
} catch (BTreeDuplicateKeyException e) {
throw new MetadataException("A datatype with name '" + datatype.getDatatypeName() + "' already exists.", e);
} catch (Exception e) {
@@ -228,12 +231,12 @@
}
@Override
- public void addFunction(long txnId, Function function) throws MetadataException, RemoteException {
+ public void addFunction(JobId jobId, Function function) throws MetadataException, RemoteException {
try {
// Insert into the 'function' dataset.
FunctionTupleTranslator tupleReaderWriter = new FunctionTupleTranslator(true);
ITupleReference functionTuple = tupleReaderWriter.getTupleFromMetadataEntity(function);
- insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.FUNCTION_DATASET, functionTuple);
+ insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, functionTuple);
} catch (BTreeDuplicateKeyException e) {
throw new MetadataException("A dataset with this name " + function.getFunctionName() + " and arity "
@@ -244,53 +247,55 @@
}
}
- public void insertIntoDatatypeSecondaryIndex(long txnId, String dataverseName, String nestedTypeName,
+ public void insertIntoDatatypeSecondaryIndex(JobId jobId, String dataverseName, String nestedTypeName,
String topTypeName) throws Exception {
ITupleReference tuple = createTuple(dataverseName, nestedTypeName, topTypeName);
- insertTupleIntoIndex(txnId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, tuple);
+ insertTupleIntoIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, tuple);
}
- private void insertTupleIntoIndex(long txnId, IMetadataIndex index, ITupleReference tuple) throws Exception {
- int fileId = index.getFileId();
- BTree btree = (BTree) indexRegistry.get(fileId);
- btree.open(fileId);
- ITreeIndexAccessor indexAccessor = btree.createAccessor();
- TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(txnId);
- transactionProvider.getLockManager().lock(txnCtx, index.getResourceId(), LockMode.EXCLUSIVE);
+ private void insertTupleIntoIndex(JobId jobId, IMetadataIndex index, ITupleReference tuple) throws Exception {
+ long resourceID = index.getResourceID();
+ IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
+ indexLifecycleManager.open(resourceID);
+ IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
+ NoOpOperationCallback.INSTANCE);
+ TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(jobId);
+ transactionProvider.getLockManager().lock(index.getDatasetId(), -1, LockMode.X, txnCtx);
// TODO: fix exceptions once new BTree exception model is in hyracks.
indexAccessor.insert(tuple);
- index.getTreeLogger().generateLogRecord(transactionProvider, txnCtx, IndexOp.INSERT, tuple);
+ index.getTreeLogger().generateLogRecord(transactionProvider, txnCtx, IndexOperation.INSERT, tuple);
+ indexLifecycleManager.close(resourceID);
}
@Override
- public void dropDataverse(long txnId, String dataverseName) throws MetadataException, RemoteException {
+ public void dropDataverse(JobId jobId, String dataverseName) throws MetadataException, RemoteException {
try {
List<Dataset> dataverseDatasets;
// As a side effect, acquires an S lock on the 'dataset' dataset
// on behalf of txnId.
- dataverseDatasets = getDataverseDatasets(txnId, dataverseName);
+ dataverseDatasets = getDataverseDatasets(jobId, dataverseName);
if (dataverseDatasets != null && dataverseDatasets.size() > 0) {
// Drop all datasets in this dataverse.
for (int i = 0; i < dataverseDatasets.size(); i++) {
- dropDataset(txnId, dataverseName, dataverseDatasets.get(i).getDatasetName());
+ dropDataset(jobId, dataverseName, dataverseDatasets.get(i).getDatasetName());
}
}
List<Datatype> dataverseDatatypes;
// As a side effect, acquires an S lock on the 'datatype' dataset
// on behalf of txnId.
- dataverseDatatypes = getDataverseDatatypes(txnId, dataverseName);
+ dataverseDatatypes = getDataverseDatatypes(jobId, dataverseName);
if (dataverseDatatypes != null && dataverseDatatypes.size() > 0) {
// Drop all types in this dataverse.
for (int i = 0; i < dataverseDatatypes.size(); i++) {
- forceDropDatatype(txnId, dataverseName, dataverseDatatypes.get(i).getDatatypeName());
+ forceDropDatatype(jobId, dataverseName, dataverseDatatypes.get(i).getDatatypeName());
}
}
// Delete the dataverse entry from the 'dataverse' dataset.
ITupleReference searchKey = createTuple(dataverseName);
// As a side effect, acquires an S lock on the 'dataverse' dataset
// on behalf of txnId.
- ITupleReference tuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.DATAVERSE_DATASET, searchKey);
- deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.DATAVERSE_DATASET, tuple);
+ ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, searchKey);
+ deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, tuple);
// TODO: Change this to be a BTree specific exception, e.g.,
// BTreeKeyDoesNotExistException.
} catch (TreeIndexException e) {
@@ -301,11 +306,11 @@
}
@Override
- public void dropDataset(long txnId, String dataverseName, String datasetName) throws MetadataException,
+ public void dropDataset(JobId jobId, String dataverseName, String datasetName) throws MetadataException,
RemoteException {
Dataset dataset;
try {
- dataset = getDataset(txnId, dataverseName, datasetName);
+ dataset = getDataset(jobId, dataverseName, datasetName);
} catch (Exception e) {
throw new MetadataException(e);
}
@@ -317,30 +322,30 @@
ITupleReference searchKey = createTuple(dataverseName, datasetName);
// Searches the index for the tuple to be deleted. Acquires an S
// lock on the 'dataset' dataset.
- ITupleReference datasetTuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey);
- deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
+ ITupleReference datasetTuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey);
+ deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
// Delete entry from secondary index 'group'.
if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
InternalDatasetDetails id = (InternalDatasetDetails) dataset.getDatasetDetails();
ITupleReference groupNameSearchKey = createTuple(id.getNodeGroupName(), dataverseName, datasetName);
// Searches the index for the tuple to be deleted. Acquires an S
// lock on the GROUPNAME_ON_DATASET_INDEX index.
- ITupleReference groupNameTuple = getTupleToBeDeleted(txnId,
+ ITupleReference groupNameTuple = getTupleToBeDeleted(jobId,
MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, groupNameSearchKey);
- deleteTupleFromIndex(txnId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, groupNameTuple);
+ deleteTupleFromIndex(jobId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, groupNameTuple);
}
// Delete entry from secondary index 'type'.
ITupleReference dataTypeSearchKey = createTuple(dataverseName, dataset.getItemTypeName(), datasetName);
// Searches the index for the tuple to be deleted. Acquires an S
// lock on the DATATYPENAME_ON_DATASET_INDEX index.
- ITupleReference dataTypeTuple = getTupleToBeDeleted(txnId,
+ ITupleReference dataTypeTuple = getTupleToBeDeleted(jobId,
MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeSearchKey);
- deleteTupleFromIndex(txnId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeTuple);
+ deleteTupleFromIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeTuple);
// Delete entry(s) from the 'indexes' dataset.
if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
- List<Index> datasetIndexes = getDatasetIndexes(txnId, dataverseName, datasetName);
+ List<Index> datasetIndexes = getDatasetIndexes(jobId, dataverseName, datasetName);
for (Index index : datasetIndexes) {
- dropIndex(txnId, dataverseName, datasetName, index.getIndexName());
+ dropIndex(jobId, dataverseName, datasetName, index.getIndexName());
}
}
// TODO: Change this to be a BTree specific exception, e.g.,
@@ -353,14 +358,14 @@
}
@Override
- public void dropIndex(long txnId, String dataverseName, String datasetName, String indexName)
+ public void dropIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
throws MetadataException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName, datasetName, indexName);
// Searches the index for the tuple to be deleted. Acquires an S
// lock on the 'index' dataset.
- ITupleReference tuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey);
- deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.INDEX_DATASET, tuple);
+ ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey);
+ deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, tuple);
// TODO: Change this to be a BTree specific exception, e.g.,
// BTreeKeyDoesNotExistException.
} catch (TreeIndexException e) {
@@ -372,10 +377,10 @@
}
@Override
- public void dropNodegroup(long txnId, String nodeGroupName) throws MetadataException, RemoteException {
+ public void dropNodegroup(JobId jobId, String nodeGroupName) throws MetadataException, RemoteException {
List<String> datasetNames;
try {
- datasetNames = getDatasetNamesPartitionedOnThisNodeGroup(txnId, nodeGroupName);
+ datasetNames = getDatasetNamesPartitionedOnThisNodeGroup(jobId, nodeGroupName);
} catch (Exception e) {
throw new MetadataException(e);
}
@@ -391,8 +396,8 @@
ITupleReference searchKey = createTuple(nodeGroupName);
// Searches the index for the tuple to be deleted. Acquires an S
// lock on the 'nodegroup' dataset.
- ITupleReference tuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.NODEGROUP_DATASET, searchKey);
- deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.NODEGROUP_DATASET, tuple);
+ ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, searchKey);
+ deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, tuple);
// TODO: Change this to be a BTree specific exception, e.g.,
// BTreeKeyDoesNotExistException.
} catch (TreeIndexException e) {
@@ -403,13 +408,13 @@
}
@Override
- public void dropDatatype(long txnId, String dataverseName, String datatypeName) throws MetadataException,
+ public void dropDatatype(JobId jobId, String dataverseName, String datatypeName) throws MetadataException,
RemoteException {
List<String> datasetNames;
List<String> usedDatatypes;
try {
- datasetNames = getDatasetNamesDeclaredByThisDatatype(txnId, dataverseName, datatypeName);
- usedDatatypes = getDatatypeNamesUsingThisDatatype(txnId, dataverseName, datatypeName);
+ datasetNames = getDatasetNamesDeclaredByThisDatatype(jobId, dataverseName, datatypeName);
+ usedDatatypes = getDatatypeNamesUsingThisDatatype(jobId, dataverseName, datatypeName);
} catch (Exception e) {
throw new MetadataException(e);
}
@@ -434,14 +439,14 @@
ITupleReference searchKey = createTuple(dataverseName, datatypeName);
// Searches the index for the tuple to be deleted. Acquires an S
// lock on the 'datatype' dataset.
- ITupleReference tuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey);
- deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
- deleteFromDatatypeSecondaryIndex(txnId, dataverseName, datatypeName);
- List<String> nestedTypes = getNestedDatatypeNames(txnId, dataverseName, datatypeName);
+ ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey);
+ deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
+ deleteFromDatatypeSecondaryIndex(jobId, dataverseName, datatypeName);
+ List<String> nestedTypes = getNestedDatatypeNames(jobId, dataverseName, datatypeName);
for (String nestedType : nestedTypes) {
- Datatype dt = getDatatype(txnId, dataverseName, nestedType);
+ Datatype dt = getDatatype(jobId, dataverseName, nestedType);
if (dt != null && dt.getIsAnonymous()) {
- dropDatatype(txnId, dataverseName, dt.getDatatypeName());
+ dropDatatype(jobId, dataverseName, dt.getDatatypeName());
}
}
// TODO: Change this to be a BTree specific exception, e.g.,
@@ -453,14 +458,14 @@
}
}
- private void forceDropDatatype(long txnId, String dataverseName, String datatypeName) throws AsterixException {
+ private void forceDropDatatype(JobId jobId, String dataverseName, String datatypeName) throws AsterixException {
try {
ITupleReference searchKey = createTuple(dataverseName, datatypeName);
// Searches the index for the tuple to be deleted. Acquires an S
// lock on the 'datatype' dataset.
- ITupleReference tuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey);
- deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
- deleteFromDatatypeSecondaryIndex(txnId, dataverseName, datatypeName);
+ ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey);
+ deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
+ deleteFromDatatypeSecondaryIndex(jobId, dataverseName, datatypeName);
// TODO: Change this to be a BTree specific exception, e.g.,
// BTreeKeyDoesNotExistException.
} catch (TreeIndexException e) {
@@ -472,17 +477,17 @@
}
}
- private void deleteFromDatatypeSecondaryIndex(long txnId, String dataverseName, String datatypeName)
+ private void deleteFromDatatypeSecondaryIndex(JobId jobId, String dataverseName, String datatypeName)
throws AsterixException {
try {
- List<String> nestedTypes = getNestedDatatypeNames(txnId, dataverseName, datatypeName);
+ List<String> nestedTypes = getNestedDatatypeNames(jobId, dataverseName, datatypeName);
for (String nestedType : nestedTypes) {
ITupleReference searchKey = createTuple(dataverseName, nestedType, datatypeName);
// Searches the index for the tuple to be deleted. Acquires an S
// lock on the DATATYPENAME_ON_DATATYPE_INDEX index.
- ITupleReference tuple = getTupleToBeDeleted(txnId,
+ ITupleReference tuple = getTupleToBeDeleted(jobId,
MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, searchKey);
- deleteTupleFromIndex(txnId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, tuple);
+ deleteTupleFromIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, tuple);
}
// TODO: Change this to be a BTree specific exception, e.g.,
// BTreeKeyDoesNotExistException.
@@ -495,31 +500,32 @@
}
}
- private void deleteTupleFromIndex(long txnId, IMetadataIndex index, ITupleReference tuple) throws Exception {
- int fileId = index.getFileId();
- BTree btree = (BTree) indexRegistry.get(fileId);
- btree.open(fileId);
-
- ITreeIndexAccessor indexAccessor = btree.createAccessor();
- TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(txnId);
+ private void deleteTupleFromIndex(JobId jobId, IMetadataIndex index, ITupleReference tuple) throws Exception {
+ long resourceID = index.getResourceID();
+ IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
+ indexLifecycleManager.open(resourceID);
+ IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
+ NoOpOperationCallback.INSTANCE);
+ TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(jobId);
// This lock is actually an upgrade, because a deletion must be preceded
// by a search, in order to be able to undo an aborted deletion.
// The transaction with txnId will have an S lock on the
// resource. Note that lock converters have a higher priority than
// regular waiters in the LockManager.
- transactionProvider.getLockManager().lock(txnCtx, index.getResourceId(), LockMode.EXCLUSIVE);
+ transactionProvider.getLockManager().lock(index.getDatasetId(), -1, LockMode.X, txnCtx);
indexAccessor.delete(tuple);
- index.getTreeLogger().generateLogRecord(transactionProvider, txnCtx, IndexOp.DELETE, tuple);
+ index.getTreeLogger().generateLogRecord(transactionProvider, txnCtx, IndexOperation.DELETE, tuple);
+ indexLifecycleManager.close(resourceID);
}
@Override
- public Dataverse getDataverse(long txnId, String dataverseName) throws MetadataException, RemoteException {
+ public Dataverse getDataverse(JobId jobId, String dataverseName) throws MetadataException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName);
DataverseTupleTranslator tupleReaderWriter = new DataverseTupleTranslator(false);
IValueExtractor<Dataverse> valueExtractor = new MetadataEntityValueExtractor<Dataverse>(tupleReaderWriter);
List<Dataverse> results = new ArrayList<Dataverse>();
- searchIndex(txnId, MetadataPrimaryIndexes.DATAVERSE_DATASET, searchKey, valueExtractor, results);
+ searchIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, searchKey, valueExtractor, results);
if (results.isEmpty()) {
return null;
}
@@ -531,28 +537,28 @@
}
@Override
- public List<Dataset> getDataverseDatasets(long txnId, String dataverseName) throws MetadataException,
+ public List<Dataset> getDataverseDatasets(JobId jobId, String dataverseName) throws MetadataException,
RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName);
DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(false);
IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<Dataset>(tupleReaderWriter);
List<Dataset> results = new ArrayList<Dataset>();
- searchIndex(txnId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey, valueExtractor, results);
+ searchIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey, valueExtractor, results);
return results;
} catch (Exception e) {
throw new MetadataException(e);
}
}
- private List<Datatype> getDataverseDatatypes(long txnId, String dataverseName) throws MetadataException,
+ private List<Datatype> getDataverseDatatypes(JobId jobId, String dataverseName) throws MetadataException,
RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName);
- DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(txnId, this, false);
+ DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(jobId, this, false);
IValueExtractor<Datatype> valueExtractor = new MetadataEntityValueExtractor<Datatype>(tupleReaderWriter);
List<Datatype> results = new ArrayList<Datatype>();
- searchIndex(txnId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey, valueExtractor, results);
+ searchIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey, valueExtractor, results);
return results;
} catch (Exception e) {
throw new MetadataException(e);
@@ -560,14 +566,14 @@
}
@Override
- public Dataset getDataset(long txnId, String dataverseName, String datasetName) throws MetadataException,
+ public Dataset getDataset(JobId jobId, String dataverseName, String datasetName) throws MetadataException,
RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName, datasetName);
DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(false);
List<Dataset> results = new ArrayList<Dataset>();
IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<Dataset>(tupleReaderWriter);
- searchIndex(txnId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey, valueExtractor, results);
+ searchIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey, valueExtractor, results);
if (results.isEmpty()) {
return null;
}
@@ -577,13 +583,13 @@
}
}
- private List<String> getDatasetNamesDeclaredByThisDatatype(long txnId, String dataverseName, String datatypeName)
+ private List<String> getDatasetNamesDeclaredByThisDatatype(JobId jobId, String dataverseName, String datatypeName)
throws MetadataException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName, dataverseName);
List<String> results = new ArrayList<String>();
IValueExtractor<String> valueExtractor = new DatasetNameValueExtractor();
- searchIndex(txnId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, searchKey, valueExtractor,
+ searchIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, searchKey, valueExtractor,
results);
return results;
} catch (Exception e) {
@@ -591,13 +597,13 @@
}
}
- public List<String> getDatatypeNamesUsingThisDatatype(long txnId, String dataverseName, String datatypeName)
+ public List<String> getDatatypeNamesUsingThisDatatype(JobId jobId, String dataverseName, String datatypeName)
throws MetadataException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName, datatypeName);
List<String> results = new ArrayList<String>();
IValueExtractor<String> valueExtractor = new DatatypeNameValueExtractor(dataverseName, this);
- searchIndex(txnId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, searchKey, valueExtractor,
+ searchIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, searchKey, valueExtractor,
results);
return results;
} catch (Exception e) {
@@ -605,13 +611,13 @@
}
}
- private List<String> getNestedDatatypeNames(long txnId, String dataverseName, String datatypeName)
+ private List<String> getNestedDatatypeNames(JobId jobId, String dataverseName, String datatypeName)
throws MetadataException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName);
List<String> results = new ArrayList<String>();
IValueExtractor<String> valueExtractor = new NestedDatatypeNameValueExtractor(datatypeName);
- searchIndex(txnId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, searchKey, valueExtractor,
+ searchIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, searchKey, valueExtractor,
results);
return results;
} catch (Exception e) {
@@ -619,13 +625,13 @@
}
}
- public List<String> getDatasetNamesPartitionedOnThisNodeGroup(long txnId, String nodegroup)
+ public List<String> getDatasetNamesPartitionedOnThisNodeGroup(JobId jobId, String nodegroup)
throws MetadataException, RemoteException {
try {
ITupleReference searchKey = createTuple(nodegroup);
List<String> results = new ArrayList<String>();
IValueExtractor<String> valueExtractor = new DatasetNameValueExtractor();
- searchIndex(txnId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, searchKey, valueExtractor, results);
+ searchIndex(jobId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, searchKey, valueExtractor, results);
return results;
} catch (Exception e) {
throw new MetadataException(e);
@@ -633,14 +639,14 @@
}
@Override
- public Index getIndex(long txnId, String dataverseName, String datasetName, String indexName)
+ public Index getIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
throws MetadataException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName, datasetName, indexName);
IndexTupleTranslator tupleReaderWriter = new IndexTupleTranslator(false);
IValueExtractor<Index> valueExtractor = new MetadataEntityValueExtractor<Index>(tupleReaderWriter);
List<Index> results = new ArrayList<Index>();
- searchIndex(txnId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey, valueExtractor, results);
+ searchIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey, valueExtractor, results);
if (results.isEmpty()) {
return null;
}
@@ -651,14 +657,14 @@
}
@Override
- public List<Index> getDatasetIndexes(long txnId, String dataverseName, String datasetName)
+ public List<Index> getDatasetIndexes(JobId jobId, String dataverseName, String datasetName)
throws MetadataException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName, datasetName);
IndexTupleTranslator tupleReaderWriter = new IndexTupleTranslator(false);
IValueExtractor<Index> valueExtractor = new MetadataEntityValueExtractor<Index>(tupleReaderWriter);
List<Index> results = new ArrayList<Index>();
- searchIndex(txnId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey, valueExtractor, results);
+ searchIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey, valueExtractor, results);
return results;
} catch (Exception e) {
throw new MetadataException(e);
@@ -666,14 +672,14 @@
}
@Override
- public Datatype getDatatype(long txnId, String dataverseName, String datatypeName) throws MetadataException,
+ public Datatype getDatatype(JobId jobId, String dataverseName, String datatypeName) throws MetadataException,
RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName, datatypeName);
- DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(txnId, this, false);
+ DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(jobId, this, false);
IValueExtractor<Datatype> valueExtractor = new MetadataEntityValueExtractor<Datatype>(tupleReaderWriter);
List<Datatype> results = new ArrayList<Datatype>();
- searchIndex(txnId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey, valueExtractor, results);
+ searchIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey, valueExtractor, results);
if (results.isEmpty()) {
return null;
}
@@ -684,13 +690,13 @@
}
@Override
- public NodeGroup getNodeGroup(long txnId, String nodeGroupName) throws MetadataException, RemoteException {
+ public NodeGroup getNodeGroup(JobId jobId, String nodeGroupName) throws MetadataException, RemoteException {
try {
ITupleReference searchKey = createTuple(nodeGroupName);
NodeGroupTupleTranslator tupleReaderWriter = new NodeGroupTupleTranslator(false);
IValueExtractor<NodeGroup> valueExtractor = new MetadataEntityValueExtractor<NodeGroup>(tupleReaderWriter);
List<NodeGroup> results = new ArrayList<NodeGroup>();
- searchIndex(txnId, MetadataPrimaryIndexes.NODEGROUP_DATASET, searchKey, valueExtractor, results);
+ searchIndex(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, searchKey, valueExtractor, results);
if (results.isEmpty()) {
return null;
}
@@ -701,14 +707,14 @@
}
@Override
- public Function getFunction(long txnId, String dataverseName, String functionName, int arity)
+ public Function getFunction(JobId jobId, String dataverseName, String functionName, int arity)
throws MetadataException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName, functionName, "" + arity);
FunctionTupleTranslator tupleReaderWriter = new FunctionTupleTranslator(false);
List<Function> results = new ArrayList<Function>();
IValueExtractor<Function> valueExtractor = new MetadataEntityValueExtractor<Function>(tupleReaderWriter);
- searchIndex(txnId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey, valueExtractor, results);
+ searchIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey, valueExtractor, results);
if (results.isEmpty()) {
return null;
}
@@ -719,11 +725,11 @@
}
@Override
- public void dropFunction(long txnId, String dataverseName, String functionName, int arity)
+ public void dropFunction(JobId jobId, String dataverseName, String functionName, int arity)
throws MetadataException, RemoteException {
Function function;
try {
- function = getFunction(txnId, dataverseName, functionName, arity);
+ function = getFunction(jobId, dataverseName, functionName, arity);
} catch (Exception e) {
throw new MetadataException(e);
}
@@ -736,9 +742,9 @@
ITupleReference searchKey = createTuple(dataverseName, functionName, "" + arity);
// Searches the index for the tuple to be deleted. Acquires an S
// lock on the 'function' dataset.
- ITupleReference datasetTuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.FUNCTION_DATASET,
+ ITupleReference datasetTuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET,
searchKey);
- deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.FUNCTION_DATASET, datasetTuple);
+ deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, datasetTuple);
// TODO: Change this to be a BTree specific exception, e.g.,
// BTreeKeyDoesNotExistException.
@@ -750,11 +756,11 @@
}
}
- private ITupleReference getTupleToBeDeleted(long txnId, IMetadataIndex metadataIndex, ITupleReference searchKey)
+ private ITupleReference getTupleToBeDeleted(JobId jobId, IMetadataIndex metadataIndex, ITupleReference searchKey)
throws Exception {
IValueExtractor<ITupleReference> valueExtractor = new TupleCopyValueExtractor(metadataIndex.getTypeTraits());
List<ITupleReference> results = new ArrayList<ITupleReference>();
- searchIndex(txnId, metadataIndex, searchKey, valueExtractor, results);
+ searchIndex(jobId, metadataIndex, searchKey, valueExtractor, results);
if (results.isEmpty()) {
// TODO: Temporarily a TreeIndexException to make it get caught by
// caller in the appropriate catch block.
@@ -764,17 +770,17 @@
return results.get(0);
}
- private <ResultType> void searchIndex(long txnId, IMetadataIndex index, ITupleReference searchKey,
+ private <ResultType> void searchIndex(JobId jobId, IMetadataIndex index, ITupleReference searchKey,
IValueExtractor<ResultType> valueExtractor, List<ResultType> results) throws Exception {
- TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(txnId);
- transactionProvider.getLockManager().lock(txnCtx, index.getResourceId(), LockMode.SHARED);
+ TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(jobId);
+ transactionProvider.getLockManager().lock(index.getDatasetId(), -1, LockMode.S, txnCtx);
IBinaryComparatorFactory[] comparatorFactories = index.getKeyBinaryComparatorFactory();
- int fileId = index.getFileId();
- BTree btree = (BTree) indexRegistry.get(fileId);
- btree.open(fileId);
- ITreeIndexFrame leafFrame = btree.getLeafFrameFactory().createFrame();
- ITreeIndexAccessor indexAccessor = btree.createAccessor();
- ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) leafFrame, false);
+ long resourceID = index.getResourceID();
+ IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
+ indexLifecycleManager.open(resourceID);
+ IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
+ NoOpOperationCallback.INSTANCE);
+ ITreeIndexCursor rangeCursor = new LSMBTreeRangeSearchCursor();
IBinaryComparator[] searchCmps = new IBinaryComparator[searchKey.getFieldCount()];
for (int i = 0; i < searchKey.getFieldCount(); i++) {
searchCmps[i] = comparatorFactories[i].createBinaryComparator();
@@ -786,7 +792,7 @@
try {
while (rangeCursor.hasNext()) {
rangeCursor.next();
- ResultType result = valueExtractor.getValue(txnId, rangeCursor.getTuple());
+ ResultType result = valueExtractor.getValue(jobId, rangeCursor.getTuple());
if (result != null) {
results.add(result);
}
@@ -794,6 +800,44 @@
} finally {
rangeCursor.close();
}
+ indexLifecycleManager.close(resourceID);
+ }
+
+ @Override
+ public void initializeDatasetIdFactory(JobId jobId) throws MetadataException, RemoteException {
+ int mostRecentDatasetId = MetadataPrimaryIndexes.FIRST_AVAILABLE_USER_DATASET_ID;
+ long resourceID = MetadataPrimaryIndexes.DATASET_DATASET.getResourceID();
+ IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
+ try {
+ indexLifecycleManager.open(resourceID);
+ IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
+ NoOpOperationCallback.INSTANCE);
+ IIndexCursor rangeCursor = indexAccessor.createSearchCursor();
+
+ DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(false);
+ IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<Dataset>(tupleReaderWriter);
+ RangePredicate rangePred = new RangePredicate(null, null, true, true, null, null);
+
+ indexAccessor.search(rangeCursor, rangePred);
+ int datasetId;
+
+ try {
+ while (rangeCursor.hasNext()) {
+ rangeCursor.next();
+ datasetId = ((Dataset) valueExtractor.getValue(jobId, rangeCursor.getTuple())).getDatasetId();
+ if (mostRecentDatasetId < datasetId) {
+ mostRecentDatasetId = datasetId;
+ }
+ }
+ } finally {
+ rangeCursor.close();
+ }
+
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+
+ DatasetIdFactory.initialize(mostRecentDatasetId);
}
// TODO: Can use Hyrack's TupleUtils for this, once we switch to a newer
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataTransactionContext.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataTransactionContext.java
index e0aa69b..329b770 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataTransactionContext.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataTransactionContext.java
@@ -22,6 +22,7 @@
import edu.uci.ics.asterix.metadata.entities.Dataverse;
import edu.uci.ics.asterix.metadata.entities.Function;
import edu.uci.ics.asterix.metadata.entities.NodeGroup;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
/**
* Used to implement serializable transactions against the MetadataCache.
@@ -53,14 +54,14 @@
protected MetadataCache droppedCache = new MetadataCache();
protected ArrayList<MetadataLogicalOperation> opLog = new ArrayList<MetadataLogicalOperation>();
- private final long txnId;
+ private final JobId jobId;
- public MetadataTransactionContext(long txnId) {
- this.txnId = txnId;
+ public MetadataTransactionContext(JobId jobId) {
+ this.jobId = jobId;
}
- public long getTxnId() {
- return txnId;
+ public JobId getJobId() {
+ return jobId;
}
public void addDataverse(Dataverse dataverse) {
@@ -96,7 +97,7 @@
public void dropDataset(String dataverseName, String datasetName) {
Dataset dataset = new Dataset(dataverseName, datasetName, null, null,
- null);
+ null, -1);
droppedCache.addDatasetIfNotExists(dataset);
logAndApply(new MetadataLogicalOperation(dataset, false));
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataIndex.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataIndex.java
index f9e5540..88dac70 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataIndex.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataIndex.java
@@ -20,10 +20,12 @@
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
import edu.uci.ics.asterix.transaction.management.service.logging.TreeLogger;
+import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
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;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
/**
@@ -58,13 +60,23 @@
public ARecordType getPayloadRecordType();
+ public void setFile(FileReference file);
+
+ public FileReference getFile();
+
public void setFileId(int fileId);
public void initTreeLogger(ITreeIndex treeIndex) throws ACIDException;
public int getFileId();
+ public void setResourceID(long resourceID);
+
+ public long getResourceID();
+
public byte[] getResourceId();
public TreeLogger getTreeLogger();
+
+ public DatasetId getDatasetId();
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataManager.java
index adba1c2..779cbe4 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataManager.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataManager.java
@@ -28,6 +28,7 @@
import edu.uci.ics.asterix.metadata.entities.Node;
import edu.uci.ics.asterix.metadata.entities.NodeGroup;
import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
/**
* A metadata manager provides user access to Asterix metadata (e.g., types,
@@ -92,7 +93,7 @@
* @throws ACIDException
* @throws RemoteException
*/
- public void lock(MetadataTransactionContext ctx, int lockMode) throws ACIDException, RemoteException;
+ public void lock(MetadataTransactionContext ctx, byte lockMode) throws ACIDException, RemoteException;
/**
* Releases all locks on the metadata held by the given transaction id.
@@ -397,4 +398,6 @@
public void dropFunction(MetadataTransactionContext ctx,
String dataverseName, String functionName, int arity)
throws MetadataException;
+
+ public void initializeDatasetIdFactory(MetadataTransactionContext ctx) throws MetadataException;
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataNode.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataNode.java
index 10555be..4bc513a 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataNode.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataNode.java
@@ -29,6 +29,7 @@
import edu.uci.ics.asterix.metadata.entities.Node;
import edu.uci.ics.asterix.metadata.entities.NodeGroup;
import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
/**
* A metadata node stores metadata in its local storage structures (currently
@@ -47,7 +48,7 @@
* @throws ACIDException
* @throws RemoteException
*/
- public void beginTransaction(long txnId) throws ACIDException, RemoteException;
+ public void beginTransaction(JobId jobId) throws ACIDException, RemoteException;
/**
* Commits a local transaction against the metadata.
@@ -55,7 +56,7 @@
* @throws ACIDException
* @throws RemoteException
*/
- public void commitTransaction(long txnId) throws ACIDException, RemoteException;
+ public void commitTransaction(JobId jobId) throws ACIDException, RemoteException;
/**
* Aborts a local transaction against the metadata.
@@ -63,7 +64,7 @@
* @throws ACIDException
* @throws RemoteException
*/
- public void abortTransaction(long txnId) throws ACIDException, RemoteException;
+ public void abortTransaction(JobId jobId) throws ACIDException, RemoteException;
/**
* Locally locks the entire metadata in given mode on behalf of given
@@ -72,7 +73,7 @@
* @throws ACIDException
* @throws RemoteException
*/
- public boolean lock(long txnId, int lockMode) throws ACIDException, RemoteException;
+ public void lock(JobId jobId, byte lockMode) throws ACIDException, RemoteException;
/**
* Releases all local locks of given transaction id.
@@ -80,13 +81,13 @@
* @throws ACIDException
* @throws RemoteException
*/
- public boolean unlock(long txnId) throws ACIDException, RemoteException;
+ public void unlock(JobId jobId) throws ACIDException, RemoteException;
/**
* Inserts a new dataverse into the metadata, acquiring local locks on
* behalf of the given transaction id.
*
- * @param txnId
+ * @param jobId
* A globally unique id for an active metadata transaction.
* @param dataverse
* Dataverse instance to be inserted.
@@ -94,13 +95,13 @@
* For example, if the dataverse already exists.
* @throws RemoteException
*/
- public void addDataverse(long txnId, Dataverse dataverse) throws MetadataException, RemoteException;
+ public void addDataverse(JobId jobId, Dataverse dataverse) throws MetadataException, RemoteException;
/**
* Retrieves a dataverse with given name, acquiring local locks on behalf of
* the given transaction id.
*
- * @param txnId
+ * @param jobId
* A globally unique id for an active metadata transaction.
* @param dataverseName
* Name of the dataverse to retrieve.
@@ -109,13 +110,13 @@
* For example, if the dataverse does not exist.
* @throws RemoteException
*/
- public Dataverse getDataverse(long txnId, String dataverseName) throws MetadataException, RemoteException;
+ public Dataverse getDataverse(JobId jobId, String dataverseName) throws MetadataException, RemoteException;
/**
* Retrieves all datasets belonging to the given dataverse, acquiring local
* locks on behalf of the given transaction id.
*
- * @param txnId
+ * @param jobId
* A globally unique id for an active metadata transaction.
* @param dataverseName
* Name of the dataverse of which to find all datasets.
@@ -123,7 +124,7 @@
* @throws MetadataException
* For example, if the dataverse does not exist. RemoteException
*/
- public List<Dataset> getDataverseDatasets(long txnId, String dataverseName) throws MetadataException,
+ public List<Dataset> getDataverseDatasets(JobId jobId, String dataverseName) throws MetadataException,
RemoteException;
/**
@@ -131,20 +132,20 @@
* indexes, and types, acquiring local locks on behalf of the given
* transaction id.
*
- * @param txnId
+ * @param jobId
* A globally unique id for an active metadata transaction.
* @return A list of dataset instances.
* @throws MetadataException
* For example, if the dataverse does not exist.
* @throws RemoteException
*/
- public void dropDataverse(long txnId, String dataverseName) throws MetadataException, RemoteException;
+ public void dropDataverse(JobId jobId, String dataverseName) throws MetadataException, RemoteException;
/**
* Inserts a new dataset into the metadata, acquiring local locks on behalf
* of the given transaction id.
*
- * @param txnId
+ * @param jobId
* A globally unique id for an active metadata transaction.
* @param dataset
* Dataset instance to be inserted.
@@ -152,13 +153,13 @@
* For example, if the dataset already exists.
* @throws RemoteException
*/
- public void addDataset(long txnId, Dataset dataset) throws MetadataException, RemoteException;
+ public void addDataset(JobId jobId, Dataset dataset) throws MetadataException, RemoteException;
/**
* Retrieves a dataset within a given dataverse, acquiring local locks on
* behalf of the given transaction id.
*
- * @param txnId
+ * @param jobId
* A globally unique id for an active metadata transaction.
* @param dataverseName
* Dataverse name to look for the dataset.
@@ -169,14 +170,14 @@
* For example, if the dataset does not exist.
* @throws RemoteException
*/
- public Dataset getDataset(long txnId, String dataverseName, String datasetName) throws MetadataException,
+ public Dataset getDataset(JobId jobId, String dataverseName, String datasetName) throws MetadataException,
RemoteException;
/**
* Retrieves all indexes of a dataset, acquiring local locks on behalf of
* the given transaction id.
*
- * @param txnId
+ * @param jobId
* A globally unique id for an active metadata transaction.
* @param dataverseName
* Name of dataverse which holds the given dataset.
@@ -187,14 +188,14 @@
* For example, if the dataset and/or dataverse does not exist.
* @throws RemoteException
*/
- public List<Index> getDatasetIndexes(long txnId, String dataverseName, String datasetName)
+ public List<Index> getDatasetIndexes(JobId jobId, String dataverseName, String datasetName)
throws MetadataException, RemoteException;
/**
* Deletes the dataset with given name, and all it's associated indexes,
* acquiring local locks on behalf of the given transaction id.
*
- * @param txnId
+ * @param jobId
* A globally unique id for an active metadata transaction.
* @param dataverseName
* Name of dataverse which holds the given dataset.
@@ -204,7 +205,7 @@
* For example, if the dataset and/or dataverse does not exist.
* @throws RemoteException
*/
- public void dropDataset(long txnId, String dataverseName, String datasetName) throws MetadataException,
+ public void dropDataset(JobId jobId, String dataverseName, String datasetName) throws MetadataException,
RemoteException;
/**
@@ -212,7 +213,7 @@
* the given transaction id. The index itself knows its name, and which
* dataset it belongs to.
*
- * @param txnId
+ * @param jobId
* A globally unique id for an active metadata transaction.
* @param index
* Index instance to be inserted.
@@ -220,13 +221,13 @@
* For example, if the index already exists.
* @throws RemoteException
*/
- public void addIndex(long txnId, Index index) throws MetadataException, RemoteException;
+ public void addIndex(JobId jobId, Index index) throws MetadataException, RemoteException;
/**
* Retrieves the index with given name, in given dataverse and dataset,
* acquiring local locks on behalf of the given transaction id.
*
- * @param txnId
+ * @param jobId
* A globally unique id for an active metadata transaction.
* @param dataverseName
* Name of the datavers holding the given dataset.
@@ -238,14 +239,14 @@
* For example, if the index does not exist.
* @throws RemoteException
*/
- public Index getIndex(long txnId, String dataverseName, String datasetName, String indexName)
+ public Index getIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
throws MetadataException, RemoteException;
/**
* Deletes the index with given name, in given dataverse and dataset,
* acquiring local locks on behalf of the given transaction id.
*
- * @param txnId
+ * @param jobId
* A globally unique id for an active metadata transaction.
* @param dataverseName
* Name of the datavers holding the given dataset.
@@ -256,14 +257,14 @@
* For example, if the index does not exist.
* @throws RemoteException
*/
- public void dropIndex(long txnId, String dataverseName, String datasetName, String indexName)
+ public void dropIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
throws MetadataException, RemoteException;
/**
* Inserts a datatype, acquiring local locks on behalf of the given
* transaction id.
*
- * @param txnId
+ * @param jobId
* A globally unique id for an active metadata transaction.
* @param datatype
* Datatype instance to be inserted.
@@ -271,13 +272,13 @@
* For example, if the datatype already exists.
* @throws RemoteException
*/
- public void addDatatype(long txnId, Datatype datatype) throws MetadataException, RemoteException;
+ public void addDatatype(JobId jobId, Datatype datatype) throws MetadataException, RemoteException;
/**
* Retrieves the datatype with given name in given dataverse, acquiring
* local locks on behalf of the given transaction id.
*
- * @param txnId
+ * @param jobId
* A globally unique id for an active metadata transaction.
* @param dataverseName
* Name of dataverse holding the datatype.
@@ -288,14 +289,14 @@
* For example, if the datatype does not exist.
* @throws RemoteException
*/
- public Datatype getDatatype(long txnId, String dataverseName, String datatypeName) throws MetadataException,
+ public Datatype getDatatype(JobId jobId, String dataverseName, String datatypeName) throws MetadataException,
RemoteException;
/**
* Deletes the given datatype in given dataverse, acquiring local locks on
* behalf of the given transaction id.
*
- * @param txnId
+ * @param jobId
* A globally unique id for an active metadata transaction.
* @param dataverseName
* Name of dataverse holding the datatype.
@@ -306,14 +307,14 @@
* deleted.
* @throws RemoteException
*/
- public void dropDatatype(long txnId, String dataverseName, String datatypeName) throws MetadataException,
+ public void dropDatatype(JobId jobId, String dataverseName, String datatypeName) throws MetadataException,
RemoteException;
/**
* Inserts a node group, acquiring local locks on behalf of the given
* transaction id.
*
- * @param txnId
+ * @param jobId
* A globally unique id for an active metadata transaction.
* @param nodeGroup
* Node group instance to insert.
@@ -321,13 +322,13 @@
* For example, if the node group already exists.
* @throws RemoteException
*/
- public void addNodeGroup(long txnId, NodeGroup nodeGroup) throws MetadataException, RemoteException;
+ public void addNodeGroup(JobId jobId, NodeGroup nodeGroup) throws MetadataException, RemoteException;
/**
* Retrieves a node group, acquiring local locks on behalf of the given
* transaction id.
*
- * @param txnId
+ * @param jobId
* A globally unique id for an active metadata transaction.
* @param nodeGroupName
* Name of node group to be retrieved.
@@ -335,13 +336,13 @@
* For example, if the node group does not exist.
* @throws RemoteException
*/
- public NodeGroup getNodeGroup(long txnId, String nodeGroupName) throws MetadataException, RemoteException;
+ public NodeGroup getNodeGroup(JobId jobId, String nodeGroupName) throws MetadataException, RemoteException;
/**
* Deletes a node group, acquiring local locks on behalf of the given
* transaction id.
*
- * @param txnId
+ * @param jobId
* A globally unique id for an active metadata transaction.
* @param nodeGroupName
* Name of node group to be deleted.
@@ -350,13 +351,13 @@
* group to be deleted.
* @throws RemoteException
*/
- public void dropNodegroup(long txnId, String nodeGroupName) throws MetadataException, RemoteException;
+ public void dropNodegroup(JobId jobId, String nodeGroupName) throws MetadataException, RemoteException;
/**
* Inserts a node (compute node), acquiring local locks on behalf of the
* given transaction id.
*
- * @param txnId
+ * @param jobId
* A globally unique id for an active metadata transaction.
* @param node
* Node instance to be inserted.
@@ -364,11 +365,11 @@
* For example, if the node already exists.
* @throws RemoteException
*/
- public void addNode(long txnId, Node node) throws MetadataException, RemoteException;
+ public void addNode(JobId jobId, Node node) throws MetadataException, RemoteException;
/**
*
- * @param txnId
+ * @param jobId
* A globally unique id for an active metadata transaction.
* @param dataverseName
* dataverse asociated with the function that is to be deleted.
@@ -380,7 +381,7 @@
* @throws MetadataException
* @throws RemoteException
*/
- public Function getFunction(long txnId, String dataverseName,
+ public Function getFunction(JobId jobId, String dataverseName,
String functionName, int arity) throws MetadataException,
RemoteException;
@@ -388,7 +389,7 @@
* Deletes a function , acquiring local locks on behalf of the given
* transaction id.
*
- * @param txnId
+ * @param jobId
* A globally unique id for an active metadata transaction.
* @param dataverseName
* dataverse asociated with the function that is to be deleted.
@@ -401,13 +402,13 @@
* group to be deleted.
* @throws RemoteException
*/
- public void dropFunction(long txnId, String dataverseName,
+ public void dropFunction(JobId jobId, String dataverseName,
String functionName, int arity) throws MetadataException,
RemoteException;
/**
*
- * @param txnId
+ * @param jobId
* A globally unique id for an active metadata transaction.
* @param function
* Function to be inserted
@@ -416,6 +417,8 @@
* unknown function
* @throws RemoteException
*/
- public void addFunction(long txnId, Function function)
+ public void addFunction(JobId jobId, Function function)
throws MetadataException, RemoteException;
+
+ public void initializeDatasetIdFactory(JobId jobId) throws MetadataException, RemoteException;
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IValueExtractor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IValueExtractor.java
index aec55b5..d8958dd 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IValueExtractor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IValueExtractor.java
@@ -18,6 +18,7 @@
import java.io.IOException;
import edu.uci.ics.asterix.metadata.MetadataException;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -32,7 +33,7 @@
/**
* Extracts an object of type T from a given tuple.
*
- * @param txnId
+ * @param jobId
* A globally unique transaction id.
* @param tuple
* Tuple from which an object shall be extracted.
@@ -41,5 +42,5 @@
* @throws HyracksDataException
* @throws IOException
*/
- public T getValue(long txnId, ITupleReference tuple) throws MetadataException, HyracksDataException, IOException;
+ public T getValue(JobId jobId, ITupleReference tuple) throws MetadataException, HyracksDataException, IOException;
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
index 6c388c2..3b5af14 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -52,21 +52,25 @@
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
-import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTree;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactory;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactoryProvider;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
+import edu.uci.ics.hyracks.storage.common.file.TransientFileMapManager;
+import edu.uci.ics.hyracks.storage.common.file.TransientLocalResourceFactoryProvider;
/**
* Initializes the remote metadata storage facilities ("universe") using a
@@ -78,17 +82,23 @@
* stopUniverse() should be called upon application undeployment.
*/
public class MetadataBootstrap {
+ private static final Logger LOGGER = Logger.getLogger(MetadataBootstrap.class.getName());
+ private static final int DEFAULT_MEM_PAGE_SIZE = 32768;
+ private static final int DEFAULT_MEM_NUM_PAGES = 100;
+
+ private static AsterixAppRuntimeContext runtimeContext;
+
private static IBufferCache bufferCache;
private static IFileMapProvider fileMapProvider;
- private static IndexRegistry<IIndex> btreeRegistry;
+ private static IIndexLifecycleManager indexLifecycleManager;
+ private static ILocalResourceRepository localResourceRepository;
+ private static IIOManager ioManager;
private static String metadataNodeName;
private static String metadataStore;
private static HashSet<String> nodeNames;
private static String outputDir;
- private static final Logger LOGGER = Logger.getLogger(MetadataBootstrap.class.getName());
-
private static IMetadataIndex[] primaryIndexes;
private static IMetadataIndex[] secondaryIndexes;
@@ -104,8 +114,7 @@
public static void startUniverse(AsterixProperties asterixProperties, INCApplicationContext ncApplicationContext)
throws Exception {
- AsterixAppRuntimeContext runtimeContext = (AsterixAppRuntimeContext) ncApplicationContext
- .getApplicationObject();
+ runtimeContext = (AsterixAppRuntimeContext) ncApplicationContext.getApplicationObject();
// Initialize static metadata objects, such as record types and metadata
// index descriptors.
@@ -133,31 +142,25 @@
(new File(outputDir)).mkdirs();
}
- btreeRegistry = runtimeContext.getIndexRegistry();
+ indexLifecycleManager = runtimeContext.getIndexLifecycleManager();
+ localResourceRepository = runtimeContext.getLocalResourceRepository();
bufferCache = runtimeContext.getBufferCache();
fileMapProvider = runtimeContext.getFileMapManager();
-
- // Create fileRefs to all BTree files and open them in BufferCache.
- for (int i = 0; i < primaryIndexes.length; i++) {
- openIndexFile(primaryIndexes[i]);
- }
- for (int i = 0; i < secondaryIndexes.length; i++) {
- openIndexFile(secondaryIndexes[i]);
- }
+ ioManager = ncApplicationContext.getRootContext().getIOManager();
// Begin a transaction against the metadata.
// Lock the metadata in X mode.
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
- MetadataManager.INSTANCE.lock(mdTxnCtx, LockMode.EXCLUSIVE);
+ MetadataManager.INSTANCE.lock(mdTxnCtx, LockMode.X);
try {
if (isNewUniverse) {
for (int i = 0; i < primaryIndexes.length; i++) {
- createIndex(primaryIndexes[i]);
+ enlistMetadataDataset(primaryIndexes[i], true);
registerTransactionalResource(primaryIndexes[i], resourceRepository);
}
for (int i = 0; i < secondaryIndexes.length; i++) {
- createIndex(secondaryIndexes[i]);
+ enlistMetadataDataset(secondaryIndexes[i], true);
registerTransactionalResource(secondaryIndexes[i], resourceRepository);
}
insertInitialDataverses(mdTxnCtx);
@@ -169,15 +172,16 @@
LOGGER.info("FINISHED CREATING METADATA B-TREES.");
} else {
for (int i = 0; i < primaryIndexes.length; i++) {
- enlistMetadataDataset(primaryIndexes[i]);
+ enlistMetadataDataset(primaryIndexes[i], false);
registerTransactionalResource(primaryIndexes[i], resourceRepository);
}
for (int i = 0; i < secondaryIndexes.length; i++) {
- enlistMetadataDataset(secondaryIndexes[i]);
+ enlistMetadataDataset(secondaryIndexes[i], false);
registerTransactionalResource(secondaryIndexes[i], resourceRepository);
}
LOGGER.info("FINISHED ENLISTMENT OF METADATA B-TREES.");
}
+ MetadataManager.INSTANCE.initializeDatasetIdFactory(mdTxnCtx);
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
} catch (Exception e) {
MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
@@ -186,44 +190,26 @@
}
public static void stopUniverse() throws HyracksDataException {
- try {
- // Close all BTree files in BufferCache.
- for (int i = 0; i < primaryIndexes.length; i++) {
- bufferCache.closeFile(primaryIndexes[i].getFileId());
- }
- for (int i = 0; i < secondaryIndexes.length; i++) {
- bufferCache.closeFile(secondaryIndexes[i].getFileId());
- }
- } catch (HyracksDataException e) {
- // Ignore for now.
- // TODO: If multiple NCs are running in the same VM, then we could
- // have multiple NCs undeploying asterix concurrently.
- // It would also mean that there is only one BufferCache. A
- // pathological sequence of events would be that NC2
- // closes the BufferCache and then NC1 enters this portion of the
- // code and tries to close unopened files.
- // What we really want is to check whether the BufferCache is open
- // in a synchronized block.
- // The BufferCache api currently does not allow us to check for
- // openness.
- // Swallowing the exceptions is a simple fix for now.
+ // Close all BTree files in BufferCache.
+ for (int i = 0; i < primaryIndexes.length; i++) {
+ long resourceID = localResourceRepository
+ .getResourceByName(primaryIndexes[i].getFile().getFile().getPath()).getResourceId();
+ indexLifecycleManager.close(resourceID);
+ indexLifecycleManager.unregister(resourceID);
}
- }
-
- private static void openIndexFile(IMetadataIndex index) throws HyracksDataException, ACIDException {
- String filePath = metadataStore + index.getFileNameRelativePath();
- FileReference file = new FileReference(new File(filePath));
- bufferCache.createFile(file);
- int fileId = fileMapProvider.lookupFileId(file);
- bufferCache.openFile(fileId);
- index.setFileId(fileId);
+ for (int i = 0; i < secondaryIndexes.length; i++) {
+ long resourceID = localResourceRepository.getResourceByName(
+ secondaryIndexes[i].getFile().getFile().getPath()).getResourceId();
+ indexLifecycleManager.close(resourceID);
+ indexLifecycleManager.unregister(resourceID);
+ }
}
private static void registerTransactionalResource(IMetadataIndex index,
TransactionalResourceRepository resourceRepository) throws ACIDException {
- int fileId = index.getFileId();
- ITreeIndex treeIndex = (ITreeIndex) btreeRegistry.get(fileId);
- byte[] resourceId = DataUtil.intToByteArray(fileId);
+ long resourceID = index.getResourceID();
+ ITreeIndex treeIndex = (ITreeIndex) indexLifecycleManager.getIndex(resourceID);
+ byte[] resourceId = DataUtil.longToByteArray(resourceID);
resourceRepository.registerTransactionalResource(resourceId, treeIndex);
index.initTreeLogger(treeIndex);
}
@@ -241,7 +227,7 @@
primaryIndexes[i].getNodeGroupName());
MetadataManager.INSTANCE.addDataset(mdTxnCtx, new Dataset(primaryIndexes[i].getDataverseName(),
primaryIndexes[i].getIndexedDatasetName(), primaryIndexes[i].getPayloadRecordType().getTypeName(),
- id, DatasetType.INTERNAL));
+ id, DatasetType.INTERNAL, primaryIndexes[i].getDatasetId().getId()));
}
}
@@ -304,33 +290,38 @@
}
- public static void createIndex(IMetadataIndex dataset) throws Exception {
- int fileId = dataset.getFileId();
- ITypeTraits[] typeTraits = dataset.getTypeTraits();
- IBinaryComparatorFactory[] comparatorFactories = dataset.getKeyBinaryComparatorFactory();
- TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
- ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
- ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
+ public static void enlistMetadataDataset(IMetadataIndex index, boolean create) throws Exception {
+ String filePath = metadataStore + index.getFileNameRelativePath();
+ FileReference file = new FileReference(new File(filePath));
+ IInMemoryBufferCache memBufferCache = new InMemoryBufferCache(new HeapBufferAllocator(), DEFAULT_MEM_PAGE_SIZE,
+ DEFAULT_MEM_NUM_PAGES, new TransientFileMapManager());
+ ITypeTraits[] typeTraits = index.getTypeTraits();
+ IBinaryComparatorFactory[] comparatorFactories = index.getKeyBinaryComparatorFactory();
ITreeIndexMetaDataFrameFactory metaDataFrameFactory = new LIFOMetaDataFrameFactory();
- IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, 0, metaDataFrameFactory);
- BTree btree = new BTree(bufferCache, NoOpOperationCallback.INSTANCE, typeTraits.length, comparatorFactories,
- freePageManager, interiorFrameFactory, leafFrameFactory);
- btree.create(fileId);
- btreeRegistry.register(fileId, btree);
- }
-
- public static void enlistMetadataDataset(IMetadataIndex dataset) throws Exception {
- int fileId = dataset.getFileId();
- ITypeTraits[] typeTraits = dataset.getTypeTraits();
- IBinaryComparatorFactory[] comparatorFactories = dataset.getKeyBinaryComparatorFactory();
- TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
- ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
- ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
- ITreeIndexMetaDataFrameFactory metaDataFrameFactory = new LIFOMetaDataFrameFactory();
- IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, 0, metaDataFrameFactory);
- BTree btree = new BTree(bufferCache, NoOpOperationCallback.INSTANCE, typeTraits.length, comparatorFactories,
- freePageManager, interiorFrameFactory, leafFrameFactory);
- btreeRegistry.register(fileId, btree);
+ IInMemoryFreePageManager memFreePageManager = new InMemoryFreePageManager(DEFAULT_MEM_NUM_PAGES,
+ metaDataFrameFactory);
+ LSMBTree lsmBtree = LSMBTreeUtils.createLSMTree(memBufferCache, memFreePageManager, ioManager, file,
+ bufferCache, fileMapProvider, typeTraits, comparatorFactories, runtimeContext.getFlushController(),
+ runtimeContext.getLSMMergePolicy(), runtimeContext.getLSMOperationTracker(),
+ runtimeContext.getLSMIOScheduler());
+ long resourceID = -1;
+ if (create) {
+ lsmBtree.create();
+ resourceID = runtimeContext.getResourceIdFactory().createId();
+ //resourceID = indexArtifactMap.create(file.getFile().getPath(), ioManager.getIODevices());
+ //TODO
+ //replace the transient resource factory provider with the persistent one.
+ ILocalResourceFactoryProvider localResourceFactoryProvider = new TransientLocalResourceFactoryProvider();
+ ILocalResourceFactory localResourceFactory = localResourceFactoryProvider.getLocalResourceFactory();
+ localResourceRepository.insert(localResourceFactory.createLocalResource(resourceID, file.getFile()
+ .getPath()));
+ } else {
+ resourceID = localResourceRepository.getResourceByName(file.getFile().getPath()).getResourceId();
+ }
+ index.setResourceID(resourceID);
+ index.setFile(file);
+ indexLifecycleManager.register(resourceID, lsmBtree);
+ indexLifecycleManager.open(resourceID);
}
public static String getOutputDir() {
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataIndex.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataIndex.java
index 165a605..d1e377e 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataIndex.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataIndex.java
@@ -30,12 +30,13 @@
import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
import edu.uci.ics.asterix.transaction.management.service.logging.DataUtil;
import edu.uci.ics.asterix.transaction.management.service.logging.TreeLogger;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
+import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
/**
@@ -62,15 +63,20 @@
protected final IBinaryComparatorFactory[] bcfs;
// Hash function factories for key fields of btree tuple. Created in c'tor.
protected final IBinaryHashFunctionFactory[] bhffs;
+
+ protected FileReference file;
// Identifier of file BufferCache backing this metadata btree index.
protected int fileId;
+ protected long resourceID;
// Resource id of this index for use in transactions.
protected byte[] indexResourceId;
// Logger for tree indexes.
private TreeLogger treeLogger;
+ // datasetId
+ private final DatasetId datasetId;
public MetadataIndex(String datasetName, String indexName, int numFields, IAType[] keyTypes, String[] keyNames,
- ARecordType payloadType) throws AsterixRuntimeException {
+ ARecordType payloadType, int datasetId) throws AsterixRuntimeException {
// Sanity checks.
if (keyTypes.length != keyNames.length) {
throw new AsterixRuntimeException("Unequal number of key types and names given.");
@@ -116,14 +122,15 @@
// Create binary comparator factories.
bcfs = new IBinaryComparatorFactory[keyTypes.length];
for (int i = 0; i < keyTypes.length; i++) {
- bcfs[i] = AqlBinaryComparatorFactoryProvider.INSTANCE
- .getBinaryComparatorFactory(keyTypes[i], true);
+ bcfs[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyTypes[i], true);
}
// Create binary hash function factories.
bhffs = new IBinaryHashFunctionFactory[keyTypes.length];
for (int i = 0; i < keyTypes.length; i++) {
bhffs[i] = AqlBinaryHashFunctionFactoryProvider.INSTANCE.getBinaryHashFunctionFactory(keyTypes[i]);
}
+
+ this.datasetId = new DatasetId(datasetId);
}
@Override
@@ -224,4 +231,30 @@
public TreeLogger getTreeLogger() {
return treeLogger;
}
+
+ @Override
+ public void setFile(FileReference file) {
+ this.file = file;
+ }
+
+ @Override
+ public FileReference getFile() {
+ return this.file;
+ }
+
+ @Override
+ public void setResourceID(long resourceID) {
+ this.resourceID = resourceID;
+ this.indexResourceId = DataUtil.longToByteArray(resourceID);
+ }
+
+ @Override
+ public long getResourceID() {
+ return resourceID;
+ }
+
+ @Override
+ public DatasetId getDatasetId() {
+ return datasetId;
+ }
}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
index 2a40e73..a81b73e 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
@@ -32,6 +32,16 @@
public static IMetadataIndex NODE_DATASET;
public static IMetadataIndex NODEGROUP_DATASET;
public static IMetadataIndex FUNCTION_DATASET;
+
+ public static final int METADATA_DATASET_ID = 0;
+ public static final int DATAVERSE_DATASET_ID = 1;
+ public static final int DATASET_DATASET_ID = 2;
+ public static final int DATATYPE_DATASET_ID = 3;
+ public static final int INDEX_DATASET_ID = 4;
+ public static final int NODE_DATASET_ID = 5;
+ public static final int NODEGROUP_DATASET_ID = 6;
+ public static final int FUNCTION_DATASET_ID = 7;
+ public static final int FIRST_AVAILABLE_USER_DATASET_ID = 8;
/**
@@ -49,30 +59,30 @@
}
DATAVERSE_DATASET = new MetadataIndex("Dataverse", null, 2, new IAType[] { BuiltinType.ASTRING },
- new String[] { "DataverseName" }, MetadataRecordTypes.DATAVERSE_RECORDTYPE);
+ new String[] { "DataverseName" }, MetadataRecordTypes.DATAVERSE_RECORDTYPE, DATAVERSE_DATASET_ID);
DATASET_DATASET = new MetadataIndex("Dataset", null, 3,
new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING }, new String[] { "DataverseName",
- "DatasetName" }, MetadataRecordTypes.DATASET_RECORDTYPE);
+ "DatasetName" }, MetadataRecordTypes.DATASET_RECORDTYPE, DATASET_DATASET_ID);
DATATYPE_DATASET = new MetadataIndex("Datatype", null, 3, new IAType[] { BuiltinType.ASTRING,
BuiltinType.ASTRING }, new String[] { "DataverseName", "DatatypeName" },
- MetadataRecordTypes.DATATYPE_RECORDTYPE);
+ MetadataRecordTypes.DATATYPE_RECORDTYPE, DATATYPE_DATASET_ID);
INDEX_DATASET = new MetadataIndex("Index", null, 4, new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING,
BuiltinType.ASTRING }, new String[] { "DataverseName", "DatasetName", "IndexName" },
- MetadataRecordTypes.INDEX_RECORDTYPE);
+ MetadataRecordTypes.INDEX_RECORDTYPE, INDEX_DATASET_ID);
NODE_DATASET = new MetadataIndex("Node", null, 2, new IAType[] { BuiltinType.ASTRING },
- new String[] { "NodeName" }, MetadataRecordTypes.NODE_RECORDTYPE);
+ new String[] { "NodeName" }, MetadataRecordTypes.NODE_RECORDTYPE, NODE_DATASET_ID);
NODEGROUP_DATASET = new MetadataIndex("Nodegroup", null, 2, new IAType[] { BuiltinType.ASTRING },
- new String[] { "GroupName" }, MetadataRecordTypes.NODEGROUP_RECORDTYPE);
+ new String[] { "GroupName" }, MetadataRecordTypes.NODEGROUP_RECORDTYPE, NODEGROUP_DATASET_ID);
FUNCTION_DATASET = new MetadataIndex("Function", null, 4,
new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING,
BuiltinType.ASTRING }, new String[] { "DataverseName",
- "FunctionName", "FunctionArity" }, MetadataRecordTypes.FUNCTION_RECORDTYPE);
+ "FunctionName", "FunctionArity" }, MetadataRecordTypes.FUNCTION_RECORDTYPE, FUNCTION_DATASET_ID);
}
}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataRecordTypes.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataRecordTypes.java
index 6936d0d..52e69a0 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataRecordTypes.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataRecordTypes.java
@@ -158,6 +158,7 @@
// Helper constants for accessing fields in an ARecord of type
// DatasetRecordType.
+
public static final int DATASET_ARECORD_DATAVERSENAME_FIELD_INDEX = 0;
public static final int DATASET_ARECORD_DATASETNAME_FIELD_INDEX = 1;
public static final int DATASET_ARECORD_DATATYPENAME_FIELD_INDEX = 2;
@@ -166,11 +167,12 @@
public static final int DATASET_ARECORD_EXTERNALDETAILS_FIELD_INDEX = 5;
public static final int DATASET_ARECORD_FEEDDETAILS_FIELD_INDEX = 6;
public static final int DATASET_ARECORD_TIMESTAMP_FIELD_INDEX = 7;
+ public static final int DATASET_ARECORD_DATASETID_FIELD_INDEX = 8;
private static final ARecordType createDatasetRecordType() {
String[] fieldNames = { "DataverseName", "DatasetName", "DataTypeName",
"DatasetType", "InternalDetails", "ExternalDetails",
- "FeedDetails", "Timestamp" };
+ "FeedDetails", "Timestamp", "DatasetId" };
List<IAType> internalRecordUnionList = new ArrayList<IAType>();
internalRecordUnionList.add(BuiltinType.ANULL);
@@ -191,7 +193,7 @@
IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING,
BuiltinType.ASTRING, BuiltinType.ASTRING, internalRecordUnion,
- externalRecordUnion, feedRecordUnion, BuiltinType.ASTRING };
+ externalRecordUnion, feedRecordUnion, BuiltinType.ASTRING, BuiltinType.AINT32 };
return new ARecordType("DatasetRecordType", fieldNames, fieldTypes,
true);
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataSecondaryIndexes.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataSecondaryIndexes.java
index a23e7d7..f696714 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataSecondaryIndexes.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataSecondaryIndexes.java
@@ -44,14 +44,14 @@
GROUPNAME_ON_DATASET_INDEX = new MetadataIndex("Dataset", "GroupName", 3, new IAType[] { BuiltinType.ASTRING,
BuiltinType.ASTRING, BuiltinType.ASTRING },
- new String[] { "GroupName", "DataverseName", "DatasetName" }, null);
+ new String[] { "GroupName", "DataverseName", "DatasetName" }, null, MetadataPrimaryIndexes.DATASET_DATASET_ID);
DATATYPENAME_ON_DATASET_INDEX = new MetadataIndex("Dataset", "DatatypeName", 3, new IAType[] {
BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING }, new String[] { "DataverseName",
- "DatatypeName", "DatasetName" }, null);
+ "DatatypeName", "DatasetName" }, null, MetadataPrimaryIndexes.DATASET_DATASET_ID);
DATATYPENAME_ON_DATATYPE_INDEX = new MetadataIndex("Datatype", "DatatypeName", 3, new IAType[] {
BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING }, new String[] { "DataverseName",
- "NestedDatatypeName", "TopDatatypeName" }, null);
+ "NestedDatatypeName", "TopDatatypeName" }, null, MetadataPrimaryIndexes.DATATYPE_DATASET_ID);
}
}
\ No newline at end of file
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 ba7c797..6973f28 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
@@ -163,10 +163,6 @@
return ng.getNodeNames();
}
- public String[] getStores(String nodeName) {
- return stores.get(nodeName);
- }
-
public Map<String, String[]> getAllStores() {
return stores;
}
@@ -180,13 +176,13 @@
}
public List<Index> getDatasetIndexes(String dataverseName, String datasetName) throws AlgebricksException {
- try {
+ try {
return metadataManager.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
} catch (MetadataException e) {
throw new AlgebricksException(e);
}
}
-
+
public Index getDatasetPrimaryIndex(String dataverseName, String datasetName) throws AlgebricksException {
try {
return metadataManager.getIndex(mdTxnCtx, dataverseName, datasetName, datasetName);
@@ -202,7 +198,7 @@
throw new AlgebricksException(e);
}
}
-
+
public void setOutputFile(FileSplit outputFile) {
this.outputFile = outputFile;
}
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 b8f0258..6d96d91 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
@@ -20,6 +20,7 @@
import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
import edu.uci.ics.asterix.common.config.GlobalConfig;
+import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
import edu.uci.ics.asterix.common.dataflow.IAsterixApplicationContextInfo;
import edu.uci.ics.asterix.common.parse.IParseFileSplitsDecl;
import edu.uci.ics.asterix.dataflow.data.nontagged.valueproviders.AqlPrimitiveValueProviderFactory;
@@ -45,7 +46,7 @@
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
import edu.uci.ics.asterix.runtime.base.AsterixTupleFilterFactory;
-import edu.uci.ics.asterix.runtime.transaction.TreeIndexInsertUpdateDeleteOperatorDescriptor;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
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;
@@ -69,6 +70,7 @@
import edu.uci.ics.hyracks.algebricks.runtime.operators.std.SinkWriterRuntimeFactory;
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -78,25 +80,30 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
-import edu.uci.ics.hyracks.storage.am.rtree.dataflow.RTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.dataflow.LSMRTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.rtree.dataflow.RTreeSearchOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+import edu.uci.ics.hyracks.storage.am.rtree.linearize.HilbertDoubleComparatorFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.linearize.ZCurveDoubleComparatorFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.linearize.ZCurveIntComparatorFactory;
public class AqlMetadataProvider implements IMetadataProvider<AqlSourceId, String> {
- private final long txnId;
+ private final JobId jobId;
private boolean isWriteTransaction;
private final AqlCompiledMetadataDeclarations metadata;
- public AqlMetadataProvider(long txnId, boolean isWriteTransaction, AqlCompiledMetadataDeclarations metadata) {
- this.txnId = txnId;
+ public AqlMetadataProvider(JobId jobId, boolean isWriteTransaction, AqlCompiledMetadataDeclarations metadata) {
+ this.jobId = jobId;
this.isWriteTransaction = isWriteTransaction;
this.metadata = metadata;
}
@@ -299,12 +306,10 @@
outputVars, keysStartIndex, numKeys, typeEnv, context);
ITypeTraits[] typeTraits = null;
- if (isSecondary) {
- typeTraits = JobGenHelper.variablesToTypeTraits(outputVars, keysStartIndex, numKeys, typeEnv,
- context);
+ if (isSecondary) {
+ typeTraits = JobGenHelper.variablesToTypeTraits(outputVars, keysStartIndex, numKeys, typeEnv, context);
} else {
- typeTraits = JobGenHelper.variablesToTypeTraits(outputVars, keysStartIndex, numKeys + 1, typeEnv,
- context);
+ typeTraits = JobGenHelper.variablesToTypeTraits(outputVars, keysStartIndex, numKeys + 1, typeEnv, context);
}
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
@@ -315,9 +320,11 @@
throw new AlgebricksException(e);
}
BTreeSearchOperatorDescriptor btreeSearchOp = new BTreeSearchOperatorDescriptor(jobSpec, outputRecDesc,
- appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(), spPc.first, typeTraits,
- comparatorFactories, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive,
- new BTreeDataflowHelperFactory(), retainInput, NoOpOperationCallbackProvider.INSTANCE);
+ appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(), spPc.first,
+ typeTraits, comparatorFactories, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive,
+ new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE), retainInput, NoOpOperationCallbackProvider.INSTANCE);
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeSearchOp, spPc.second);
}
@@ -332,6 +339,8 @@
isSecondary = !indexName.equals(primaryIndex.getIndexName());
}
+ IBinaryComparatorFactory[] primaryComparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
+ dataset, recType, context.getBinaryComparatorFactoryProvider());
int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
ISerializerDeserializer[] recordFields;
IBinaryComparatorFactory[] comparatorFactories;
@@ -395,9 +404,12 @@
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,
- NoOpOperationCallbackProvider.INSTANCE);
+ appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(), spPc.first,
+ typeTraits, comparatorFactories, keyFields, new LSMRTreeDataflowHelperFactory(valueProviderFactories,
+ RTreePolicyType.RTREE, primaryComparatorFactories, AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE, proposeLinearizer(nestedKeyType.getTypeTag(),
+ comparatorFactories.length)), false, NoOpOperationCallbackProvider.INSTANCE);
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(rtreeSearchOp, spPc.second);
}
@@ -504,14 +516,16 @@
.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
- appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(),
+ appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(),
splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation,
- GlobalConfig.DEFAULT_BTREE_FILL_FACTOR, new BTreeDataflowHelperFactory(),
+ GlobalConfig.DEFAULT_BTREE_FILL_FACTOR, false, new LSMBTreeDataflowHelperFactory(
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE),
NoOpOperationCallbackProvider.INSTANCE);
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad, splitsAndConstraint.second);
}
- public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertOrDeleteRuntime(IndexOp indexOp,
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertOrDeleteRuntime(IndexOperation indexOp,
IDataSource<AqlSourceId> dataSource, IOperatorSchema propagatedSchema, List<LogicalVariable> keys,
LogicalVariable payload, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec)
throws AlgebricksException {
@@ -545,9 +559,12 @@
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
TreeIndexInsertUpdateDeleteOperatorDescriptor btreeBulkLoad = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
- spec, recordDesc, appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(),
- splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation, indexOp,
- new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackProvider.INSTANCE, txnId);
+ spec, recordDesc, appContext.getStorageManagerInterface(),
+ appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
+ comparatorFactories, fieldPermutation, indexOp, new LSMBTreeDataflowHelperFactory(
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE), null,
+ NoOpOperationCallbackProvider.INSTANCE);
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad, splitsAndConstraint.second);
}
@@ -556,7 +573,7 @@
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,
+ return getInsertOrDeleteRuntime(IndexOperation.INSERT, dataSource, propagatedSchema, keys, payload, recordDesc,
context, spec);
}
@@ -565,11 +582,11 @@
IDataSource<AqlSourceId> dataSource, IOperatorSchema propagatedSchema, List<LogicalVariable> keys,
LogicalVariable payload, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec)
throws AlgebricksException {
- return getInsertOrDeleteRuntime(IndexOp.DELETE, dataSource, propagatedSchema, keys, payload, recordDesc,
+ return getInsertOrDeleteRuntime(IndexOperation.DELETE, dataSource, propagatedSchema, keys, payload, recordDesc,
context, spec);
}
- public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexInsertOrDeleteRuntime(IndexOp indexOp,
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexInsertOrDeleteRuntime(IndexOperation indexOp,
IDataSourceIndex<String, AqlSourceId> dataSourceIndex, IOperatorSchema propagatedSchema,
IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys,
List<LogicalVariable> secondaryKeys, ILogicalExpression filterExpr, RecordDescriptor recordDesc,
@@ -604,7 +621,7 @@
IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys,
List<LogicalVariable> secondaryKeys, ILogicalExpression filterExpr, RecordDescriptor recordDesc,
JobGenContext context, JobSpecification spec) throws AlgebricksException {
- return getIndexInsertOrDeleteRuntime(IndexOp.INSERT, dataSourceIndex, propagatedSchema, inputSchemas, typeEnv,
+ return getIndexInsertOrDeleteRuntime(IndexOperation.INSERT, dataSourceIndex, propagatedSchema, inputSchemas, typeEnv,
primaryKeys, secondaryKeys, filterExpr, recordDesc, context, spec);
}
@@ -614,7 +631,7 @@
IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys,
List<LogicalVariable> secondaryKeys, ILogicalExpression filterExpr, RecordDescriptor recordDesc,
JobGenContext context, JobSpecification spec) throws AlgebricksException {
- return getIndexInsertOrDeleteRuntime(IndexOp.DELETE, dataSourceIndex, propagatedSchema, inputSchemas, typeEnv,
+ return getIndexInsertOrDeleteRuntime(IndexOperation.DELETE, dataSourceIndex, propagatedSchema, inputSchemas, typeEnv,
primaryKeys, secondaryKeys, filterExpr, recordDesc, context, spec);
}
@@ -634,7 +651,7 @@
private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getBTreeDmlRuntime(String datasetName,
String indexName, IOperatorSchema propagatedSchema, List<LogicalVariable> primaryKeys,
List<LogicalVariable> secondaryKeys, AsterixTupleFilterFactory filterFactory, RecordDescriptor recordDesc,
- JobGenContext context, JobSpecification spec, IndexOp indexOp) throws AlgebricksException {
+ JobGenContext context, JobSpecification spec, IndexOperation indexOp) throws AlgebricksException {
int numKeys = primaryKeys.size() + secondaryKeys.size();
// generate field permutations
int[] fieldPermutation = new int[numKeys];
@@ -687,16 +704,19 @@
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
TreeIndexInsertUpdateDeleteOperatorDescriptor btreeBulkLoad = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
- spec, recordDesc, appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(),
- splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation, indexOp,
- new BTreeDataflowHelperFactory(), filterFactory, NoOpOperationCallbackProvider.INSTANCE, txnId);
+ spec, recordDesc, appContext.getStorageManagerInterface(),
+ appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
+ comparatorFactories, fieldPermutation, indexOp, new LSMBTreeDataflowHelperFactory(
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE),
+ filterFactory, NoOpOperationCallbackProvider.INSTANCE);
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad, splitsAndConstraint.second);
}
private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getRTreeDmlRuntime(String datasetName,
String indexName, IOperatorSchema propagatedSchema, List<LogicalVariable> primaryKeys,
List<LogicalVariable> secondaryKeys, AsterixTupleFilterFactory filterFactory, RecordDescriptor recordDesc,
- JobGenContext context, JobSpecification spec, IndexOp indexOp) throws AlgebricksException {
+ JobGenContext context, JobSpecification spec, IndexOperation indexOp) throws AlgebricksException {
Dataset dataset = metadata.findDataset(datasetName);
String itemTypeName = dataset.getItemTypeName();
IAType itemType = metadata.findType(itemTypeName);
@@ -743,26 +763,45 @@
typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
++i;
}
-
+ IBinaryComparatorFactory[] primaryComparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
+ dataset, recType, context.getBinaryComparatorFactoryProvider());
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
TreeIndexInsertUpdateDeleteOperatorDescriptor rtreeUpdate = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
- spec, recordDesc, appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(),
- splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation, indexOp,
- new RTreeDataflowHelperFactory(valueProviderFactories), filterFactory,
- NoOpOperationCallbackProvider.INSTANCE, txnId);
+ spec, recordDesc, appContext.getStorageManagerInterface(),
+ appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
+ comparatorFactories, fieldPermutation, indexOp, new LSMRTreeDataflowHelperFactory(
+ valueProviderFactories, RTreePolicyType.RTREE, primaryComparatorFactories,
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+ proposeLinearizer(nestedKeyType.getTypeTag(), comparatorFactories.length)), filterFactory,
+ NoOpOperationCallbackProvider.INSTANCE);
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(rtreeUpdate, splitsAndConstraint.second);
}
- public long getTxnId() {
- return txnId;
+ public JobId getJobId() {
+ return jobId;
}
public static ITreeIndexFrameFactory createBTreeNSMInteriorFrameFactory(ITypeTraits[] typeTraits) {
return new BTreeNSMInteriorFrameFactory(new TypeAwareTupleWriterFactory(typeTraits));
}
+ public static ILinearizeComparatorFactory proposeLinearizer(ATypeTag keyType, int numKeyFields)
+ throws AlgebricksException {
+ if (numKeyFields / 2 == 2 && (keyType == ATypeTag.DOUBLE)) {
+ return new HilbertDoubleComparatorFactory(2);
+ } else if (keyType == ATypeTag.DOUBLE) {
+ return new ZCurveDoubleComparatorFactory(numKeyFields / 2);
+ } else if (keyType == ATypeTag.INT8 || keyType == ATypeTag.INT16 || keyType == ATypeTag.INT32
+ || keyType == ATypeTag.INT64) {
+ return new ZCurveIntComparatorFactory(numKeyFields / 2);
+ } else {
+ throw new AlgebricksException("Cannot propose linearizer for key with type " + keyType + ".");
+ }
+ }
+
@Override
public IFunctionInfo lookupFunction(FunctionIdentifier fid) {
return AsterixBuiltinFunctions.lookupFunction(fid);
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 d383955..63e837b 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
@@ -34,14 +34,16 @@
private final String itemTypeName;
private final DatasetType datasetType;
private IDatasetDetails datasetDetails;
+ private final int datasetId;
public Dataset(String dataverseName, String datasetName, String itemTypeName, IDatasetDetails datasetDetails,
- DatasetType datasetType) {
+ DatasetType datasetType, int datasetId) {
this.dataverseName = dataverseName;
this.datasetName = datasetName;
this.itemTypeName = itemTypeName;
this.datasetType = datasetType;
this.datasetDetails = datasetDetails;
+ this.datasetId = datasetId;
}
public String getDataverseName() {
@@ -68,6 +70,10 @@
this.datasetDetails = datasetDetails;
}
+ public int getDatasetId() {
+ return datasetId;
+ }
+
@Override
public Object addToCache(MetadataCache cache) {
return cache.addDatasetIfNotExists(this);
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 fbec4b1..e00e9e0 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
@@ -40,10 +40,14 @@
import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails.FileStructure;
import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails.PartitioningStrategy;
+import edu.uci.ics.asterix.om.base.AInt32;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.AMutableString;
import edu.uci.ics.asterix.om.base.AOrderedList;
import edu.uci.ics.asterix.om.base.ARecord;
import edu.uci.ics.asterix.om.base.AString;
import edu.uci.ics.asterix.om.base.IACursor;
+import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -68,9 +72,14 @@
@SuppressWarnings("unchecked")
private ISerializerDeserializer<ARecord> recordSerDes = AqlSerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(MetadataRecordTypes.DATASET_RECORDTYPE);
+ private AMutableInt32 aInt32;
+ protected ISerializerDeserializer<AInt32> aInt32Serde;
- public DatasetTupleTranslator(boolean getTuple) {
+ @SuppressWarnings("unchecked")
+ public DatasetTupleTranslator(boolean getTuple) {
super(getTuple, MetadataPrimaryIndexes.DATASET_DATASET.getFieldCount());
+ aInt32 = new AMutableInt32(-1);
+ aInt32Serde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
}
@Override
@@ -85,6 +94,7 @@
}
private Dataset createDatasetFromARecord(ARecord datasetRecord) {
+
String dataverseName = ((AString) datasetRecord
.getValueByPos(MetadataRecordTypes.DATASET_ARECORD_DATAVERSENAME_FIELD_INDEX)).getStringValue();
String datasetName = ((AString) datasetRecord
@@ -93,6 +103,8 @@
.getValueByPos(MetadataRecordTypes.DATASET_ARECORD_DATATYPENAME_FIELD_INDEX)).getStringValue();
DatasetType datasetType = DatasetType.valueOf(((AString) datasetRecord.getValueByPos(3)).getStringValue());
IDatasetDetails datasetDetails = null;
+ int datasetId = ((AInt32) datasetRecord
+ .getValueByPos(MetadataRecordTypes.DATASET_ARECORD_DATASETID_FIELD_INDEX)).getIntegerValue();
switch (datasetType) {
case FEED:
case INTERNAL: {
@@ -177,7 +189,7 @@
}
datasetDetails = new ExternalDatasetDetails(adapter, properties);
}
- return new Dataset(dataverseName, datasetName, typeName, datasetDetails, datasetType);
+ return new Dataset(dataverseName, datasetName, typeName, datasetDetails, datasetType, datasetId);
}
@Override
@@ -228,7 +240,13 @@
aString.setValue(Calendar.getInstance().getTime().toString());
stringSerde.serialize(aString, fieldValue.getDataOutput());
recordBuilder.addField(MetadataRecordTypes.DATASET_ARECORD_TIMESTAMP_FIELD_INDEX, fieldValue);
-
+
+ // write field 8
+ fieldValue.reset();
+ aInt32.setValue(dataset.getDatasetId());
+ aInt32Serde.serialize(aInt32, fieldValue.getDataOutput());
+ recordBuilder.addField(MetadataRecordTypes.DATASET_ARECORD_DATASETID_FIELD_INDEX, fieldValue);
+
// write record
recordBuilder.write(tupleBuilder.getDataOutput(), true);
tupleBuilder.addFieldEndOffset();
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
index b39a1bf..3931a17 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
@@ -49,6 +49,7 @@
import edu.uci.ics.asterix.om.types.AbstractCollectionType;
import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
@@ -81,11 +82,11 @@
private ISerializerDeserializer<ARecord> recordSerDes = AqlSerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(MetadataRecordTypes.DATATYPE_RECORDTYPE);
private final MetadataNode metadataNode;
- private final long txnId;
+ private final JobId jobId;
- public DatatypeTupleTranslator(long txnId, MetadataNode metadataNode, boolean getTuple) {
+ public DatatypeTupleTranslator(JobId jobId, MetadataNode metadataNode, boolean getTuple) {
super(getTuple, MetadataPrimaryIndexes.DATATYPE_DATASET.getFieldCount());
- this.txnId = txnId;
+ this.jobId = jobId;
this.metadataNode = metadataNode;
}
@@ -182,7 +183,7 @@
IAType type = AsterixBuiltinTypeMap.getBuiltinTypes().get(typeName);
if (type == null) {
try {
- return metadataNode.getDatatype(txnId, dataverseName, typeName).getDatatype();
+ return metadataNode.getDatatype(jobId, dataverseName, typeName).getDatatype();
} catch (RemoteException e) {
throw new MetadataException(e);
}
@@ -417,9 +418,9 @@
MetadataNode mn = MetadataNode.INSTANCE;
if (typeName == null) {
typeName = suggestedTypeName;
- metadataNode.addDatatype(txnId, new Datatype(topLevelType.getDataverseName(), typeName, nestedType, true));
+ metadataNode.addDatatype(jobId, new Datatype(topLevelType.getDataverseName(), typeName, nestedType, true));
try {
- mn.insertIntoDatatypeSecondaryIndex(txnId, topLevelType.getDataverseName(), typeName,
+ mn.insertIntoDatatypeSecondaryIndex(jobId, topLevelType.getDataverseName(), typeName,
topLevelType.getDatatypeName());
} catch (BTreeDuplicateKeyException e) {
// The key may have been inserted by a previous DDL statement or
@@ -427,12 +428,12 @@
}
return typeName;
}
- Datatype dt = mn.getDatatype(txnId, topLevelType.getDataverseName(), typeName);
+ Datatype dt = mn.getDatatype(jobId, topLevelType.getDataverseName(), typeName);
if (dt == null) {
throw new AlgebricksException("There is no datatype with this name " + typeName + ".");
}
try {
- mn.insertIntoDatatypeSecondaryIndex(txnId, topLevelType.getDataverseName(), typeName,
+ mn.insertIntoDatatypeSecondaryIndex(jobId, topLevelType.getDataverseName(), typeName,
topLevelType.getDatatypeName());
} catch (BTreeDuplicateKeyException e) {
// The key may have been inserted by a previous DDL statement or by
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/DatasetNameValueExtractor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/DatasetNameValueExtractor.java
index a6bbb8f..e554643 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/DatasetNameValueExtractor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/DatasetNameValueExtractor.java
@@ -23,6 +23,7 @@
import edu.uci.ics.asterix.metadata.MetadataException;
import edu.uci.ics.asterix.metadata.api.IValueExtractor;
import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -32,7 +33,7 @@
*/
public class DatasetNameValueExtractor implements IValueExtractor<String> {
@Override
- public String getValue(long txnId, ITupleReference tuple) throws MetadataException, HyracksDataException {
+ public String getValue(JobId jobId, ITupleReference tuple) throws MetadataException, HyracksDataException {
byte[] serRecord = tuple.getFieldData(2);
int recordStartOffset = tuple.getFieldStart(2);
int recordLength = tuple.getFieldLength(2);
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/DatatypeNameValueExtractor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/DatatypeNameValueExtractor.java
index 16a501d..2e4dbe4 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/DatatypeNameValueExtractor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/DatatypeNameValueExtractor.java
@@ -25,6 +25,7 @@
import edu.uci.ics.asterix.metadata.MetadataNode;
import edu.uci.ics.asterix.metadata.api.IValueExtractor;
import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -42,7 +43,7 @@
}
@Override
- public String getValue(long txnId, ITupleReference tuple) throws MetadataException, HyracksDataException {
+ public String getValue(JobId jobId, ITupleReference tuple) throws MetadataException, HyracksDataException {
byte[] serRecord = tuple.getFieldData(2);
int recordStartOffset = tuple.getFieldStart(2);
int recordLength = tuple.getFieldLength(2);
@@ -50,10 +51,10 @@
DataInput in = new DataInputStream(stream);
String typeName = ((AString) AObjectSerializerDeserializer.INSTANCE.deserialize(in)).getStringValue();
try {
- if (metadataNode.getDatatype(txnId, dataverseName, typeName).getIsAnonymous()) {
+ if (metadataNode.getDatatype(jobId, dataverseName, typeName).getIsAnonymous()) {
// Get index 0 because it is anonymous type, and it is used in
// only one non-anonymous type.
- typeName = metadataNode.getDatatypeNamesUsingThisDatatype(txnId, dataverseName, typeName).get(0);
+ typeName = metadataNode.getDatatypeNamesUsingThisDatatype(jobId, dataverseName, typeName).get(0);
}
} catch (RemoteException e) {
throw new MetadataException(e);
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/MetadataEntityValueExtractor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/MetadataEntityValueExtractor.java
index 34f9b29..7ae334e 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/MetadataEntityValueExtractor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/MetadataEntityValueExtractor.java
@@ -20,6 +20,7 @@
import edu.uci.ics.asterix.metadata.MetadataException;
import edu.uci.ics.asterix.metadata.api.IMetadataEntityTupleTranslator;
import edu.uci.ics.asterix.metadata.api.IValueExtractor;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -34,7 +35,7 @@
}
@Override
- public T getValue(long txnId, ITupleReference tuple) throws MetadataException, HyracksDataException, IOException {
+ public T getValue(JobId jobId, ITupleReference tuple) throws MetadataException, HyracksDataException, IOException {
return tupleReaderWriter.getMetadataEntytiFromTuple(tuple);
}
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
index 74dabad..d4109cd 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
@@ -23,6 +23,7 @@
import edu.uci.ics.asterix.metadata.MetadataException;
import edu.uci.ics.asterix.metadata.api.IValueExtractor;
import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -40,7 +41,7 @@
}
@Override
- public String getValue(long txnId, ITupleReference tuple) throws MetadataException, HyracksDataException {
+ public String getValue(JobId jobId, ITupleReference tuple) throws MetadataException, HyracksDataException {
byte[] serRecord = tuple.getFieldData(2);
int recordStartOffset = tuple.getFieldStart(2);
int recordLength = tuple.getFieldLength(2);
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/TupleCopyValueExtractor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/TupleCopyValueExtractor.java
index da608e5..03bc963 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/TupleCopyValueExtractor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/TupleCopyValueExtractor.java
@@ -20,6 +20,7 @@
import edu.uci.ics.asterix.metadata.MetadataException;
import edu.uci.ics.asterix.metadata.api.IValueExtractor;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -44,7 +45,7 @@
}
@Override
- public ITupleReference getValue(long txnId, ITupleReference tuple) throws MetadataException, HyracksDataException,
+ public ITupleReference getValue(JobId jobId, ITupleReference tuple) throws MetadataException, HyracksDataException,
IOException {
int numBytes = tupleWriter.bytesRequired(tuple);
tupleBytes = new byte[numBytes];
diff --git a/asterix-om/pom.xml b/asterix-om/pom.xml
index a14f3c2..ff9e143 100644
--- a/asterix-om/pom.xml
+++ b/asterix-om/pom.xml
@@ -1,4 +1,5 @@
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<artifactId>asterix</artifactId>
@@ -32,23 +33,19 @@
</dependency>
<dependency>
<groupId>edu.uci.ics.hyracks</groupId>
- <artifactId>hyracks-storage-am-invertedindex</artifactId>
- <version>0.2.2-SNAPSHOT</version>
- </dependency>
- <dependency>
- <groupId>edu.uci.ics.asterix</groupId>
- <artifactId>asterix-hyracks-glue</artifactId>
- <version>0.0.4-SNAPSHOT</version>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>edu.uci.ics.hyracks</groupId>
<artifactId>hyracks-algebricks-compiler</artifactId>
<version>0.2.2-SNAPSHOT</version>
</dependency>
<dependency>
<groupId>edu.uci.ics.hyracks</groupId>
- <artifactId>hyracks-storage-am-rtree</artifactId>
+ <artifactId>
+ hyracks-storage-am-lsm-invertedindex
+ </artifactId>
+ <version>0.2.2-SNAPSHOT</version>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-storage-am-lsm-rtree</artifactId>
<version>0.2.2-SNAPSHOT</version>
</dependency>
</dependencies>
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AListElementToken.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AListElementToken.java
index cb6838e..82780bb 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AListElementToken.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AListElementToken.java
@@ -3,7 +3,7 @@
import java.io.DataOutput;
import java.io.IOException;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IToken;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
public class AListElementToken implements IToken {
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AListElementTokenFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AListElementTokenFactory.java
index fc46039..1ccf2db 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AListElementTokenFactory.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AListElementTokenFactory.java
@@ -1,7 +1,7 @@
package edu.uci.ics.asterix.dataflow.data.common;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IToken;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
public class AListElementTokenFactory implements ITokenFactory {
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AOrderedListBinaryTokenizer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AOrderedListBinaryTokenizer.java
index 667bfe7..89a3ff7 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AOrderedListBinaryTokenizer.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AOrderedListBinaryTokenizer.java
@@ -5,9 +5,9 @@
import edu.uci.ics.asterix.om.types.ATypeTag;
import edu.uci.ics.asterix.om.types.EnumDeserializer;
import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IToken;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
public class AOrderedListBinaryTokenizer implements IBinaryTokenizer {
@@ -16,13 +16,13 @@
protected int length;
protected int listLength;
protected int itemIndex;
-
+
protected final IToken token;
-
+
public AOrderedListBinaryTokenizer(ITokenFactory tokenFactory) {
token = tokenFactory.createToken();
}
-
+
@Override
public IToken getToken() {
return token;
@@ -58,11 +58,11 @@
this.listLength = getNumberOfItems(data, start);
this.itemIndex = 0;
}
-
+
protected int getItemOffset(byte[] data, int start, int itemIndex) throws AsterixException {
return AOrderedListSerializerDeserializer.getItemOffset(data, start, itemIndex);
}
-
+
protected int getNumberOfItems(byte[] data, int start) {
return AOrderedListSerializerDeserializer.getNumberOfItems(data, start);
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AOrderedListBinaryTokenizerFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AOrderedListBinaryTokenizerFactory.java
index d7712ea..6c5077b 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AOrderedListBinaryTokenizerFactory.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AOrderedListBinaryTokenizerFactory.java
@@ -1,18 +1,18 @@
package edu.uci.ics.asterix.dataflow.data.common;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
public class AOrderedListBinaryTokenizerFactory implements IBinaryTokenizerFactory {
private static final long serialVersionUID = 1L;
private final ITokenFactory tokenFactory;
-
+
public AOrderedListBinaryTokenizerFactory(ITokenFactory tokenFactory) {
this.tokenFactory = tokenFactory;
}
-
+
@Override
public IBinaryTokenizer createTokenizer() {
return new AOrderedListBinaryTokenizer(tokenFactory);
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AUnorderedListBinaryTokenizer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AUnorderedListBinaryTokenizer.java
index 11ab251..7e0ac32 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AUnorderedListBinaryTokenizer.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AUnorderedListBinaryTokenizer.java
@@ -2,14 +2,14 @@
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AUnorderedListSerializerDeserializer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
public class AUnorderedListBinaryTokenizer extends AOrderedListBinaryTokenizer {
public AUnorderedListBinaryTokenizer(ITokenFactory tokenFactory) {
super(tokenFactory);
}
-
+
@Override
protected int getItemOffset(byte[] data, int start, int itemIndex) throws AsterixException {
return AUnorderedListSerializerDeserializer.getItemOffset(data, start, itemIndex);
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AUnorderedListBinaryTokenizerFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AUnorderedListBinaryTokenizerFactory.java
index 54b7692..fc55aa6 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AUnorderedListBinaryTokenizerFactory.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AUnorderedListBinaryTokenizerFactory.java
@@ -1,18 +1,18 @@
package edu.uci.ics.asterix.dataflow.data.common;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
public class AUnorderedListBinaryTokenizerFactory implements IBinaryTokenizerFactory {
-
+
private static final long serialVersionUID = 1L;
private final ITokenFactory tokenFactory;
-
+
public AUnorderedListBinaryTokenizerFactory(ITokenFactory tokenFactory) {
this.tokenFactory = tokenFactory;
}
-
+
@Override
public IBinaryTokenizer createTokenizer() {
return new AUnorderedListBinaryTokenizer(tokenFactory);
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/IBinaryTokenizerFactoryProvider.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/IBinaryTokenizerFactoryProvider.java
index cddb014..b9af9bf 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/IBinaryTokenizerFactoryProvider.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/IBinaryTokenizerFactoryProvider.java
@@ -1,9 +1,11 @@
package edu.uci.ics.asterix.dataflow.data.common;
import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
public interface IBinaryTokenizerFactoryProvider {
public IBinaryTokenizerFactory getWordTokenizerFactory(ATypeTag typeTag, boolean hashedTokens);
- public IBinaryTokenizerFactory getNGramTokenizerFactory(ATypeTag typeTag, int gramLength, boolean usePrePost, boolean hashedTokens);
+
+ public IBinaryTokenizerFactory getNGramTokenizerFactory(ATypeTag typeTag, int gramLength, boolean usePrePost,
+ boolean hashedTokens);
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/ListEditDistanceSearchModifier.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/ListEditDistanceSearchModifier.java
index 2a7fdb5..3d66912 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/ListEditDistanceSearchModifier.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/ListEditDistanceSearchModifier.java
@@ -1,10 +1,6 @@
package edu.uci.ics.asterix.dataflow.data.common;
-import java.util.Collections;
-import java.util.List;
-
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListCursor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
// TODO: Should go into hyracks.
public class ListEditDistanceSearchModifier implements IInvertedIndexSearchModifier {
@@ -15,17 +11,6 @@
this.edThresh = edThresh;
}
- @Override
- public int getOccurrenceThreshold(List<IInvertedListCursor> invListCursors) {
- return invListCursors.size() - edThresh;
- }
-
- @Override
- public int getPrefixLists(List<IInvertedListCursor> invListCursors) {
- Collections.sort(invListCursors);
- return invListCursors.size() - getOccurrenceThreshold(invListCursors) + 1;
- }
-
public int getEdThresh() {
return edThresh;
}
@@ -33,4 +18,14 @@
public void setEdThresh(int edThresh) {
this.edThresh = edThresh;
}
+
+ @Override
+ public int getOccurrenceThreshold(int numQueryTokens) {
+ return numQueryTokens - edThresh;
+ }
+
+ @Override
+ public int getNumPrefixLists(int numQueryTokens) {
+ return numQueryTokens - getOccurrenceThreshold(numQueryTokens) + 1;
+ }
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/ListEditDistanceSearchModifierFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/ListEditDistanceSearchModifierFactory.java
index b117a36..4148c85 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/ListEditDistanceSearchModifierFactory.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/ListEditDistanceSearchModifierFactory.java
@@ -1,7 +1,7 @@
package edu.uci.ics.asterix.dataflow.data.common;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifierFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifierFactory;
// TODO: Should go into hyracks.
public class ListEditDistanceSearchModifierFactory implements IInvertedIndexSearchModifierFactory {
@@ -9,11 +9,11 @@
private static final long serialVersionUID = 1L;
private final int edThresh;
-
+
public ListEditDistanceSearchModifierFactory(int edThresh) {
this.edThresh = edThresh;
}
-
+
@Override
public IInvertedIndexSearchModifier createSearchModifier() {
return new ListEditDistanceSearchModifier(edThresh);
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryTokenizerFactoryProvider.java b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryTokenizerFactoryProvider.java
index 859b0f2..41a921e 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryTokenizerFactoryProvider.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryTokenizerFactoryProvider.java
@@ -1,15 +1,15 @@
package edu.uci.ics.asterix.formats.nontagged;
-import edu.uci.ics.asterix.dataflow.data.common.IBinaryTokenizerFactoryProvider;
import edu.uci.ics.asterix.dataflow.data.common.AListElementTokenFactory;
import edu.uci.ics.asterix.dataflow.data.common.AOrderedListBinaryTokenizerFactory;
import edu.uci.ics.asterix.dataflow.data.common.AUnorderedListBinaryTokenizerFactory;
+import edu.uci.ics.asterix.dataflow.data.common.IBinaryTokenizerFactoryProvider;
import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizerFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.HashedUTF8WordTokenFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.UTF8NGramTokenFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.UTF8WordTokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.HashedUTF8WordTokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.UTF8NGramTokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.UTF8WordTokenFactory;
public class AqlBinaryTokenizerFactoryProvider implements IBinaryTokenizerFactoryProvider {
@@ -17,14 +17,16 @@
private static final IBinaryTokenizerFactory aqlStringTokenizer = new DelimitedUTF8StringBinaryTokenizerFactory(
true, true, new UTF8WordTokenFactory(ATypeTag.STRING.serialize(), ATypeTag.INT32.serialize()));
-
+
private static final IBinaryTokenizerFactory aqlHashingStringTokenizer = new DelimitedUTF8StringBinaryTokenizerFactory(
true, true, new HashedUTF8WordTokenFactory(ATypeTag.INT32.serialize(), ATypeTag.INT32.serialize()));
- private static final IBinaryTokenizerFactory orderedListTokenizer = new AOrderedListBinaryTokenizerFactory(new AListElementTokenFactory());
-
- private static final IBinaryTokenizerFactory unorderedListTokenizer = new AUnorderedListBinaryTokenizerFactory(new AListElementTokenFactory());
-
+ private static final IBinaryTokenizerFactory orderedListTokenizer = new AOrderedListBinaryTokenizerFactory(
+ new AListElementTokenFactory());
+
+ private static final IBinaryTokenizerFactory unorderedListTokenizer = new AUnorderedListBinaryTokenizerFactory(
+ new AListElementTokenFactory());
+
@Override
public IBinaryTokenizerFactory getWordTokenizerFactory(ATypeTag typeTag, boolean hashedTokens) {
switch (typeTag) {
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/NGramUTF8StringBinaryTokenizerFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/NGramUTF8StringBinaryTokenizerFactory.java
index 895dac4..48dde10 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/NGramUTF8StringBinaryTokenizerFactory.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/NGramUTF8StringBinaryTokenizerFactory.java
@@ -19,30 +19,29 @@
package edu.uci.ics.asterix.formats.nontagged;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizer;
// TODO: Should be moved into Hyracks with the rest of the tokenizer code.
-public class NGramUTF8StringBinaryTokenizerFactory implements
- IBinaryTokenizerFactory {
+public class NGramUTF8StringBinaryTokenizerFactory implements IBinaryTokenizerFactory {
- private static final long serialVersionUID = 1L;
- private final int gramLength;
- private final boolean usePrePost;
- private final boolean ignoreTokenCount;
- private final boolean sourceHasTypeTag;
- private final ITokenFactory tokenFactory;
+ private static final long serialVersionUID = 1L;
+ private final int gramLength;
+ private final boolean usePrePost;
+ private final boolean ignoreTokenCount;
+ private final boolean sourceHasTypeTag;
+ private final ITokenFactory tokenFactory;
- public NGramUTF8StringBinaryTokenizerFactory(int gramLength, boolean usePrePost, boolean ignoreTokenCount,
- boolean sourceHasTypeTag, ITokenFactory tokenFactory) {
- this.gramLength = gramLength;
- this.usePrePost = usePrePost;
- this.ignoreTokenCount = ignoreTokenCount;
- this.sourceHasTypeTag = sourceHasTypeTag;
- this.tokenFactory = tokenFactory;
- }
+ public NGramUTF8StringBinaryTokenizerFactory(int gramLength, boolean usePrePost, boolean ignoreTokenCount,
+ boolean sourceHasTypeTag, ITokenFactory tokenFactory) {
+ this.gramLength = gramLength;
+ this.usePrePost = usePrePost;
+ this.ignoreTokenCount = ignoreTokenCount;
+ this.sourceHasTypeTag = sourceHasTypeTag;
+ this.tokenFactory = tokenFactory;
+ }
@Override
public IBinaryTokenizer createTokenizer() {
diff --git a/asterix-runtime/pom.xml b/asterix-runtime/pom.xml
index 77952a6..eb81fe5 100644
--- a/asterix-runtime/pom.xml
+++ b/asterix-runtime/pom.xml
@@ -76,6 +76,11 @@
<type>jar</type>
<scope>compile</scope>
</dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-api</artifactId>
+ <version>0.2.1-SNAPSHOT</version>
+ </dependency>
</dependencies>
</project>
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/GramTokensEvaluator.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/GramTokensEvaluator.java
index 188135c..0660743 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/GramTokensEvaluator.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/GramTokensEvaluator.java
@@ -15,10 +15,10 @@
import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.BooleanSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IToken;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IntArray;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IntArray;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizer;
public class GramTokensEvaluator implements ICopyEvaluator {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/WordTokensEvaluator.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/WordTokensEvaluator.java
index f0f36f5..ac0864c 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/WordTokensEvaluator.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/WordTokensEvaluator.java
@@ -13,9 +13,9 @@
import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IToken;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IntArray;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IntArray;
public class WordTokensEvaluator implements ICopyEvaluator {
protected final DataOutput out;
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CountHashedGramTokensDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CountHashedGramTokensDescriptor.java
index 184d974..942d2ee 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CountHashedGramTokensDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CountHashedGramTokensDescriptor.java
@@ -12,9 +12,9 @@
import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.HashedUTF8NGramTokenFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.HashedUTF8NGramTokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizer;
public class CountHashedGramTokensDescriptor extends AbstractScalarFunctionDynamicDescriptor {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CountHashedWordTokensDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CountHashedWordTokensDescriptor.java
index 7bfbe49..d0a7f4e 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CountHashedWordTokensDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CountHashedWordTokensDescriptor.java
@@ -12,10 +12,10 @@
import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.HashedUTF8WordTokenFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.HashedUTF8WordTokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
public class CountHashedWordTokensDescriptor extends AbstractScalarFunctionDynamicDescriptor {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/GramTokensDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/GramTokensDescriptor.java
index 558ae7e..5d56ae6 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/GramTokensDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/GramTokensDescriptor.java
@@ -12,9 +12,9 @@
import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.UTF8NGramTokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.UTF8NGramTokenFactory;
public class GramTokensDescriptor extends AbstractScalarFunctionDynamicDescriptor {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/HashedGramTokensDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/HashedGramTokensDescriptor.java
index 04088a2..ef5a926 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/HashedGramTokensDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/HashedGramTokensDescriptor.java
@@ -12,9 +12,9 @@
import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.HashedUTF8NGramTokenFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.HashedUTF8NGramTokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizer;
public class HashedGramTokensDescriptor extends AbstractScalarFunctionDynamicDescriptor {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/HashedWordTokensDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/HashedWordTokensDescriptor.java
index dcd5f70..9021233 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/HashedWordTokensDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/HashedWordTokensDescriptor.java
@@ -12,10 +12,10 @@
import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.HashedUTF8WordTokenFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.HashedUTF8WordTokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
public class HashedWordTokensDescriptor extends AbstractScalarFunctionDynamicDescriptor {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/WordTokensDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/WordTokensDescriptor.java
index 5e9065b..61b7009 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/WordTokensDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/WordTokensDescriptor.java
@@ -12,10 +12,10 @@
import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.UTF8WordTokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.UTF8WordTokenFactory;
public class WordTokensDescriptor extends AbstractScalarFunctionDynamicDescriptor {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/job/listener/JobEventListenerFactory.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/job/listener/JobEventListenerFactory.java
index 2d49982..b6f70bc 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/job/listener/JobEventListenerFactory.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/job/listener/JobEventListenerFactory.java
@@ -3,6 +3,7 @@
import edu.uci.ics.asterix.common.context.AsterixAppRuntimeContext;
import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionManager;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext.TransactionType;
import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
@@ -13,11 +14,11 @@
public class JobEventListenerFactory implements IJobletEventListenerFactory {
private static final long serialVersionUID = 1L;
- private final long txnId;
+ private final JobId jobId;
private final boolean transactionalWrite;
- public JobEventListenerFactory(long txnId, boolean transactionalWrite) {
- this.txnId = txnId;
+ public JobEventListenerFactory(JobId jobId, boolean transactionalWrite) {
+ this.jobId = jobId;
this.transactionalWrite = transactionalWrite;
}
@@ -30,7 +31,7 @@
try {
ITransactionManager txnManager = ((AsterixAppRuntimeContext) jobletContext.getApplicationContext()
.getApplicationObject()).getTransactionProvider().getTransactionManager();
- TransactionContext txnContext = txnManager.getTransactionContext(txnId);
+ TransactionContext txnContext = txnManager.getTransactionContext(jobId);
txnContext.setTransactionType(transactionalWrite ? TransactionType.READ_WRITE
: TransactionType.READ);
txnManager.completedTransaction(txnContext, !(jobStatus == JobStatus.FAILURE));
@@ -41,7 +42,12 @@
@Override
public void jobletStart() {
-
+ try {
+ ((AsterixAppRuntimeContext) jobletContext.getApplicationContext().getApplicationObject())
+ .getTransactionProvider().getTransactionManager().getTransactionContext(jobId);
+ } catch (ACIDException e) {
+ throw new Error(e);
+ }
}
};
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/datagen/AdmDataGen.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/datagen/AdmDataGen.java
index 7e4704b..3f6cbff 100644
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/datagen/AdmDataGen.java
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/datagen/AdmDataGen.java
@@ -46,6 +46,7 @@
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.asterix.tools.translator.ADGenDmlTranslator;
import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
import edu.uci.ics.hyracks.algebricks.data.utils.WriteValueTools;
@@ -928,7 +929,7 @@
Query q = (Query) parser.Statement();
aql.close();
// TODO: Need to fix how to use transactions here.
- MetadataTransactionContext mdTxnCtx = new MetadataTransactionContext(-1);
+ MetadataTransactionContext mdTxnCtx = new MetadataTransactionContext(new JobId(-1));
ADGenDmlTranslator dmlt = new ADGenDmlTranslator(mdTxnCtx, q.getPrologDeclList());
dmlt.translate();
AqlCompiledMetadataDeclarations acmd = dmlt.getCompiledDeclarations();
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/DatasetLockInfo.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/DatasetLockInfo.java
new file mode 100644
index 0000000..d5e525a
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/DatasetLockInfo.java
@@ -0,0 +1,504 @@
+package edu.uci.ics.asterix.transaction.management.service.locking;
+
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
+
+public class DatasetLockInfo {
+ private EntityLockInfoManager entityLockInfoManager;
+ private EntityInfoManager entityInfoManager;
+ private LockWaiterManager lockWaiterManager;
+ private PrimitiveIntHashMap entityResourceHT;
+ private int IXCount;
+ private int ISCount;
+ private int XCount;
+ private int SCount;
+ private int lastHolder;
+ private int firstWaiter;
+ private int firstUpgrader;
+
+ public DatasetLockInfo(EntityLockInfoManager entityLockInfoManager, EntityInfoManager entityInfoManager,
+ LockWaiterManager lockWaiterManager) {
+ this.entityLockInfoManager = entityLockInfoManager;
+ this.entityInfoManager = entityInfoManager;
+ this.lockWaiterManager = lockWaiterManager;
+ entityResourceHT = new PrimitiveIntHashMap();
+ lastHolder = -1; //-1 stands for end of list
+ firstWaiter = -1;
+ firstUpgrader = -1;
+ }
+
+ public void increaseLockCount(byte lockMode) {
+ switch (lockMode) {
+ case LockMode.IX:
+ IXCount++;
+ break;
+ case LockMode.IS:
+ ISCount++;
+ break;
+ case LockMode.X:
+ XCount++;
+ break;
+ case LockMode.S:
+ SCount++;
+ break;
+ default:
+ throw new IllegalStateException("Invalid dataset lock mode");
+ }
+ }
+
+ public void decreaseLockCount(byte lockMode) {
+ switch (lockMode) {
+ case LockMode.IX:
+ IXCount--;
+ break;
+ case LockMode.IS:
+ ISCount--;
+ break;
+ case LockMode.X:
+ XCount--;
+ break;
+ case LockMode.S:
+ SCount--;
+ break;
+ default:
+ throw new IllegalStateException("Invalid dataset lock mode");
+ }
+ }
+
+ public void increaseLockCount(byte lockMode, int count) {
+ switch (lockMode) {
+ case LockMode.IX:
+ IXCount += count;
+ break;
+ case LockMode.IS:
+ ISCount += count;
+ break;
+ case LockMode.X:
+ XCount += count;
+ break;
+ case LockMode.S:
+ SCount += count;
+ break;
+ default:
+ throw new IllegalStateException("Invalid dataset lock mode");
+ }
+ }
+
+ public void decreaseLockCount(byte lockMode, int count) {
+ switch (lockMode) {
+ case LockMode.IX:
+ IXCount -= count;
+ break;
+ case LockMode.IS:
+ ISCount -= count;
+ break;
+ case LockMode.X:
+ XCount -= count;
+ break;
+ case LockMode.S:
+ SCount -= count;
+ break;
+ default:
+ throw new IllegalStateException("Invalid dataset lock mode");
+ }
+ }
+
+ public boolean isUpgradeCompatible(byte lockMode, int entityInfo) {
+ switch (lockMode) {
+ //upgrade from IS -> IX
+ //XCount is guaranteed to be 0.
+ //upgrade is allowed if SCount is 0.
+ case LockMode.IX:
+ return SCount == 0;
+
+ //upgrade from S -> X
+ //XCount and IXCount are guaranteed to be 0.
+ //upgrade is allowed if ISCount is 0.
+ case LockMode.X:
+ return ISCount == 0;
+
+ default:
+ throw new IllegalStateException("Invalid upgrade lock mode");
+ }
+ }
+
+ public boolean isCompatible(byte lockMode) {
+ switch (lockMode) {
+ case LockMode.IX:
+ return SCount == 0 && XCount == 0;
+
+ case LockMode.IS:
+ return XCount == 0;
+
+ case LockMode.X:
+ return ISCount == 0 && IXCount == 0 && SCount == 0 && XCount == 0;
+
+ case LockMode.S:
+ return IXCount == 0 && XCount == 0;
+
+ default:
+ throw new IllegalStateException("Invalid upgrade lock mode");
+ }
+ }
+
+ public int findEntityInfoFromHolderList(int jobId, int hashVal) {
+ int entityInfo;
+ int eLockInfo;
+ int waiterObjId;
+ if (hashVal == -1) {//dataset-granule lock
+ entityInfo = lastHolder;
+ while (entityInfo != -1) {
+ if (jobId == entityInfoManager.getJobId(entityInfo)) {
+ return entityInfo;
+ }
+ entityInfo = entityInfoManager.getPrevEntityActor(entityInfo);
+ }
+ return -1;
+ } else { //entity-granule lock
+ eLockInfo = entityResourceHT.get(hashVal);
+ if (eLockInfo == -1) {
+ return -1;
+ }
+ entityInfo = entityLockInfoManager.findEntityInfoFromHolderList(eLockInfo, jobId, hashVal);
+ if (entityInfo == -1) {
+ //find the entityInfo from the waiter list of entityLockInfo.
+ //There is a case where dataset-granule lock is acquired, but entity-granule lock is not acquired yet.
+ //In this case, the waiter of the entityLockInfo represents the holder of the datasetLockInfo.
+ waiterObjId = entityLockInfoManager.findWaiterFromWaiterList(eLockInfo, jobId, hashVal);
+ if (waiterObjId != -1) {
+ entityInfo = lockWaiterManager.getLockWaiter(waiterObjId).getEntityInfoSlot();
+ }
+ }
+ return entityInfo;
+ }
+ }
+
+ public int findWaiterFromWaiterList(int jobId, int hashVal) {
+ int waiterObjId;
+ LockWaiter waiterObj;
+ int entityInfo = 0;
+
+ waiterObjId = firstWaiter;
+ while (waiterObjId != -1) {
+ waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
+ entityInfo = waiterObj.getEntityInfoSlot();
+ if (jobId == entityInfoManager.getJobId(entityInfo)
+ && hashVal == entityInfoManager.getPKHashVal(entityInfo)) {
+ return waiterObjId;
+ }
+ waiterObjId = waiterObj.getNextWaiterObjId();
+ }
+
+ return -1;
+ }
+
+ public int findUpgraderFromUpgraderList(int jobId, int hashVal) {
+ int waiterObjId;
+ LockWaiter waiterObj;
+ int entityInfo = 0;
+
+ waiterObjId = firstUpgrader;
+ while (waiterObjId != -1) {
+ waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
+ entityInfo = waiterObj.getEntityInfoSlot();
+ if (jobId == entityInfoManager.getJobId(entityInfo)
+ && hashVal == entityInfoManager.getPKHashVal(entityInfo)) {
+ return waiterObjId;
+ }
+ waiterObjId = waiterObj.getNextWaiterObjId();
+ }
+
+ return -1;
+ }
+
+ public boolean isNoHolder() {
+ return ISCount == 0 && IXCount == 0 && SCount == 0 && XCount == 0;
+ }
+
+ public void addHolder(int holder) {
+ entityInfoManager.setPrevEntityActor(holder, lastHolder);
+ lastHolder = holder;
+ }
+
+ /**
+ * Remove holder from linked list of Actor.
+ * Also, remove the corresponding resource from linked list of resource
+ * in order to minimize JobInfo's resource link traversal.
+ *
+ * @param holder
+ * @param jobInfo
+ */
+ public void removeHolder(int holder, JobInfo jobInfo) {
+ int prev = lastHolder;
+ int current = -1;
+ int next;
+
+ //remove holder from linked list of Actor
+ while (prev != holder) {
+ if (LockManager.IS_DEBUG_MODE) {
+ if (prev == -1) {
+ //shouldn't occur: debugging purpose
+ try {
+ throw new Exception();
+ } catch (Exception e) {
+ // TODO Auto-generated catch block
+ e.printStackTrace();
+ }
+ }
+ }
+
+ current = prev;
+ prev = entityInfoManager.getPrevEntityActor(current);
+ }
+
+ if (current != -1) {
+ //current->prev = prev->prev
+ entityInfoManager.setPrevEntityActor(current, entityInfoManager.getPrevEntityActor(prev));
+ } else {
+ //lastHolder = prev->prev
+ lastHolder = entityInfoManager.getPrevEntityActor(prev);
+ }
+
+ //Notice!!
+ //remove the corresponding resource from linked list of resource.
+ //it is guaranteed that there is no waiter or upgrader in the JobInfo when this function is called.
+ prev = entityInfoManager.getPrevJobResource(holder);
+ next = entityInfoManager.getNextJobResource(holder);
+
+ if (prev != -1) {
+ entityInfoManager.setNextJobResource(prev, next);
+ }
+
+ if (next != -1) {
+ entityInfoManager.setPrevJobResource(next, prev);
+ } else {
+ //This entityInfo(i.e., holder) is the last resource held by this job.
+ jobInfo.setlastHoldingResource(holder);
+ }
+
+ //jobInfo.decreaseDatasetLockCount(holder);
+ }
+
+ /**
+ * append new waiter to the end of waiters
+ *
+ * @param waiterObjId
+ */
+ public void addWaiter(int waiterObjId) {
+ int lastObjId;
+ LockWaiter lastObj = null;
+
+ if (firstWaiter != -1) {
+ //find the lastWaiter
+ lastObjId = firstWaiter;
+ while (lastObjId != -1) {
+ lastObj = lockWaiterManager.getLockWaiter(lastObjId);
+ lastObjId = lastObj.getNextWaiterObjId();
+ }
+ //last->next = new_waiter
+ lastObj.setNextWaiterObjId(waiterObjId);
+ } else {
+ firstWaiter = waiterObjId;
+ }
+ //new_waiter->next = -1
+ lastObj = lockWaiterManager.getLockWaiter(waiterObjId);
+ lastObj.setNextWaiterObjId(-1);
+
+// if (LockManager.IS_DEBUG_MODE) {
+// System.out.println(printWaiters());
+// }
+ }
+
+ public void removeWaiter(int waiterObjId) {
+ int currentObjId = firstWaiter;
+ LockWaiter currentObj;
+ LockWaiter prevObj = null;
+ int prevObjId = -1;
+ int nextObjId;
+
+ while (currentObjId != waiterObjId) {
+
+ if (LockManager.IS_DEBUG_MODE) {
+ if (currentObjId == -1) {
+ //shouldn't occur: debugging purpose
+ try {
+ throw new Exception();
+ } catch (Exception e) {
+ // TODO Auto-generated catch block
+ e.printStackTrace();
+ }
+ }
+ }
+
+ prevObj = lockWaiterManager.getLockWaiter(currentObjId);
+ prevObjId = currentObjId;
+ currentObjId = prevObj.getNextWaiterObjId();
+ }
+
+ //get current waiter object
+ currentObj = lockWaiterManager.getLockWaiter(currentObjId);
+
+ //get next waiterObjId
+ nextObjId = currentObj.getNextWaiterObjId();
+
+ if (prevObjId != -1) {
+ //prev->next = next
+ prevObj.setNextWaiterObjId(nextObjId);
+ } else {
+ //removed first waiter. firstWaiter = current->next
+ firstWaiter = nextObjId;
+ }
+
+// if (LockManager.IS_DEBUG_MODE) {
+// System.out.println(printWaiters());
+// }
+ }
+
+ public void addUpgrader(int waiterObjId) {
+ int lastObjId;
+ LockWaiter lastObj = null;
+
+ if (firstUpgrader != -1) {
+ //find the lastWaiter
+ lastObjId = firstUpgrader;
+ while (lastObjId != -1) {
+ lastObj = lockWaiterManager.getLockWaiter(lastObjId);
+ lastObjId = lastObj.getNextWaiterObjId();
+ }
+ //last->next = new_waiter
+ lastObj.setNextWaiterObjId(waiterObjId);
+ } else {
+ firstUpgrader = waiterObjId;
+ }
+ //new_waiter->next = -1
+ lastObj = lockWaiterManager.getLockWaiter(waiterObjId);
+ lastObj.setNextWaiterObjId(-1);
+ }
+
+ public void removeUpgrader(int waiterObjId) {
+ int currentObjId = firstUpgrader;
+ LockWaiter currentObj;
+ LockWaiter prevObj = null;
+ int prevObjId = -1;
+ int nextObjId;
+
+ while (currentObjId != waiterObjId) {
+
+ if (LockManager.IS_DEBUG_MODE) {
+ if (currentObjId == -1) {
+ //shouldn't occur: debugging purpose
+ try {
+ throw new Exception();
+ } catch (Exception e) {
+ // TODO Auto-generated catch block
+ e.printStackTrace();
+ }
+ }
+ }
+
+ prevObj = lockWaiterManager.getLockWaiter(currentObjId);
+ prevObjId = currentObjId;
+ currentObjId = prevObj.getNextWaiterObjId();
+ }
+
+ //get current waiter object
+ currentObj = lockWaiterManager.getLockWaiter(currentObjId);
+
+ //get next waiterObjId
+ nextObjId = currentObj.getNextWaiterObjId();
+
+ if (prevObjId != -1) {
+ //prev->next = next
+ prevObj.setNextWaiterObjId(nextObjId);
+ } else {
+ //removed first waiter. firstWaiter = current->next
+ firstUpgrader = nextObjId;
+ }
+ }
+
+ //debugging method
+ public String printWaiters() {
+ StringBuilder s = new StringBuilder();
+ int waiterObjId;
+ LockWaiter waiterObj;
+ int entityInfo;
+
+ s.append("WID\tWCT\tEID\tJID\tDID\tPK\n");
+
+ waiterObjId = firstWaiter;
+ while (waiterObjId != -1) {
+ waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
+ entityInfo = waiterObj.getEntityInfoSlot();
+ s.append(waiterObjId).append("\t").append(waiterObj.getWaiterCount()).append("\t").append(entityInfo)
+ .append("\t").append(entityInfoManager.getJobId(entityInfo)).append("\t")
+ .append(entityInfoManager.getDatasetId(entityInfo)).append("\t")
+ .append(entityInfoManager.getPKHashVal(entityInfo)).append("\n");
+ waiterObjId = waiterObj.getNextWaiterObjId();
+ }
+
+ return s.toString();
+ }
+
+ /////////////////////////////////////////////////////////
+ // set/get method for private variable
+ /////////////////////////////////////////////////////////
+ public void setIXCount(int count) {
+ IXCount = count;
+ }
+
+ public int getIXCount() {
+ return IXCount;
+ }
+
+ public void setISCount(int count) {
+ ISCount = count;
+ }
+
+ public int getISCount() {
+ return ISCount;
+ }
+
+ public void setXCount(int count) {
+ XCount = count;
+ }
+
+ public int getXCount() {
+ return XCount;
+ }
+
+ public void setSCount(int count) {
+ SCount = count;
+ }
+
+ public int getSCount() {
+ return SCount;
+ }
+
+ public void setLastHolder(int holder) {
+ lastHolder = holder;
+ }
+
+ public int getLastHolder() {
+ return lastHolder;
+ }
+
+ public void setFirstWaiter(int waiter) {
+ firstWaiter = waiter;
+ }
+
+ public int getFirstWaiter() {
+ return firstWaiter;
+ }
+
+ public void setFirstUpgrader(int upgrader) {
+ firstUpgrader = upgrader;
+ }
+
+ public int getFirstUpgrader() {
+ return firstUpgrader;
+ }
+
+ public PrimitiveIntHashMap getEntityResourceHT() {
+ return entityResourceHT;
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/DeadlockDetector.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/DeadlockDetector.java
index 695abde..6b6f8e8 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/DeadlockDetector.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/DeadlockDetector.java
@@ -1,67 +1,237 @@
package edu.uci.ics.asterix.transaction.management.service.locking;
-import java.util.ArrayList;
-import java.util.Iterator;
+import java.util.HashMap;
+
+import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
/**
- * @author pouria Performing a DFS search, upon adding each waiter to a waiting
+ * @author pouria, kisskys Performing a BFS search, upon adding each waiter to a waiting
* list to avoid deadlocks this class implements such a loop-detector in
* the wait-for-graph
*/
public class DeadlockDetector {
- LockManager lockMgr;
- ArrayList<Long> grantedList;
- ArrayList<Long> nextTxrIDs;
- ArrayList<Long> visited;
- ArrayList<Long> nextGrantedTxIDs;
+ public static final boolean IS_DEBUG_MODE = true;//false
+ private HashMap<JobId, JobInfo> jobHT;
+ private HashMap<DatasetId, DatasetLockInfo> datasetResourceHT;
+ private EntityLockInfoManager entityLockInfoManager;
+ private EntityInfoManager entityInfoManager;
+ private LockWaiterManager lockWaiterManager;
- public DeadlockDetector(LockManager lm) {
- this.lockMgr = lm;
- this.grantedList = new ArrayList<Long>();
- this.nextTxrIDs = new ArrayList<Long>();
- this.visited = new ArrayList<Long>();
- this.nextGrantedTxIDs = new ArrayList<Long>();
+ private PrimitiveIntHashMap holderList;
+ private PrimitiveIntHashMap nextHolderList;
+ private PrimitiveIntHashMap resourceList;
+ private PrimitiveIntHashMap visitedHolderList;
+ private JobId tempJobIdObj; //temporary object to avoid object creation
+ private DatasetId tempDatasetIdObj; //temporary object to avoid object creation
+
+
+ public DeadlockDetector(HashMap<JobId, JobInfo> jobHT, HashMap<DatasetId, DatasetLockInfo> datasetResourceHT,
+ EntityLockInfoManager entityLockInfoManager, EntityInfoManager entityInfoManager,
+ LockWaiterManager lockWaiterManager) {
+ this.jobHT = jobHT;
+ this.datasetResourceHT = datasetResourceHT;
+ this.entityLockInfoManager = entityLockInfoManager;
+ this.entityInfoManager = entityInfoManager;
+ this.lockWaiterManager = lockWaiterManager;
+ holderList = new PrimitiveIntHashMap(1 << 6, 1 << 3, 180000);
+ nextHolderList = new PrimitiveIntHashMap(1 << 6, 1 << 3, 180000);
+ resourceList = new PrimitiveIntHashMap(1, 1 << 4, 180000);
+ visitedHolderList = new PrimitiveIntHashMap(1 << 6, 1 << 3, 180000);
+ tempJobIdObj = new JobId(0);
+ tempDatasetIdObj = new DatasetId(0);
}
+
+ public boolean isSafeToAdd(DatasetLockInfo dLockInfo, int eLockInfo, int entityInfo, boolean isDatasetLockInfo, boolean isUpgrade) {
+ int holder;
+ int nextHolder;
+ int visitedHolder;
+ int callerId = entityInfoManager.getJobId(entityInfo);
+ int datasetId = entityInfoManager.getDatasetId(entityInfo);
+ int hashValue = entityInfoManager.getPKHashVal(entityInfo);
+ int resource;
+ PrimitiveIntHashMap tempHolderList;
- public synchronized boolean isSafeToAdd(long reqTxId, byte[] resourceId) {
- grantedList.clear();
- lockMgr.getLockInfo(resourceId).getGrantedListTxIDs(grantedList);
- visited.clear();
- while (grantedList.size() > 0) { // Doing a DFS for loop detection
- nextTxrIDs.clear();
- for (long grantee : grantedList) {
- TxrInfo nextTInfoList = lockMgr.getTxrInfo(grantee);
- if (nextTInfoList == null) {
- continue;
- }
- byte[] nextWaitOnRid = nextTInfoList.getWaitOnRid();
- if (nextWaitOnRid == null) {
- continue;
- }
- nextGrantedTxIDs.clear();
- lockMgr.getLockInfo(nextWaitOnRid).getGrantedListTxIDs(nextGrantedTxIDs);
- if (nextGrantedTxIDs.contains(reqTxId)) {
- return false;
- }
- removeVisitedTxIDs();
- nextTxrIDs.addAll(nextGrantedTxIDs);
- visited.add(grantee);
- }
- grantedList.clear();
- grantedList.addAll(nextTxrIDs);
+ holderList.clear(true);
+ visitedHolderList.clear(true);
+
+ //holderlist contains jobId
+ //resourceList contains entityInfo's slot numbers instead of resourceId in order to avoid object creation
+ //since resourceId consists of datasetId and PKHashValue
+
+ //get holder list(jobId list)
+ if (isDatasetLockInfo) {
+ getHolderList(datasetId, -1, holderList);
+ } else {
+ getHolderList(datasetId, hashValue, holderList);
}
+
+ //check whether this caller is upgrader or not
+ //if it is upgrader, then handle it as special case in the following manner
+ //if there is another upgrader or waiter of which lock mode is not compatible with the caller's lock mode,
+ //then this caller's wait causes deadlock.
+ if (holderList.get(callerId) != -1) {
+ if (isUpgrade && dLockInfo.getFirstUpgrader() != -1) {
+ return false;
+ }
+ //there is no case such that while a job is holding any mode of lock on a dataset and waits for the same dataset as an waiter.
+ //But the job may wait for the same dataset as an upgrader.
+ }
+
+ //TODO
+ //check whether when there are multiple resources, the waiter and upgrader should be distinguished or not.
+ //The current logic doesn't distinguish these two types of waiter.
+
+ //while holderList is not empty
+ holderList.beginIterate();
+ holder = holderList.getNextKey();
+ while (holder != -1) {
+
+ nextHolderList.clear(true);
+
+ while (holder != -1) {
+ resourceList.clear(true);
+ getWaitingResourceList(holder, resourceList);
+ resourceList.beginIterate();
+ resource = resourceList.getNextKey();
+
+ while (resource != -1) {
+ //get dataset holder
+ getHolderList(entityInfoManager.getDatasetId(resource), -1, nextHolderList);
+ //get entity holder
+ getHolderList(entityInfoManager.getDatasetId(resource), entityInfoManager.getPKHashVal(resource),
+ nextHolderList);
+ if (nextHolderList.get(callerId) != -1) {
+ return false;
+ }
+ resource = resourceList.getNextKey();
+ }
+
+ visitedHolderList.put(holder, -1);
+ holder = holderList.getNextKey();
+ }
+
+ //remove visitedHolder for nextHolderList;
+ visitedHolderList.beginIterate();
+ visitedHolder = visitedHolderList.getNextKey();
+ while (visitedHolder != -1) {
+ nextHolderList.remove(visitedHolder);
+ visitedHolder = visitedHolderList.getNextKey();
+ }
+
+ //swap holder list
+ //set holderList to nextHolderList and nextHolderList to holderList
+ tempHolderList = holderList;
+ holderList = nextHolderList;
+ nextHolderList = tempHolderList;
+ holderList.beginIterate();
+ holder = holderList.getNextKey();
+ }
+
return true;
}
- private void removeVisitedTxIDs() {
- Iterator<Long> txIdIt = nextGrantedTxIDs.iterator();
- while (txIdIt.hasNext()) {
- if (visited.contains(txIdIt.next())) {
- txIdIt.remove();
+ /**
+ * Get holder list of dataset if hashValue == -1. Get holder list of entity otherwise.
+ * Where, a holder is a jobId, not entityInfo's slotNum
+ * @param datasetId
+ * @param hashValue
+ * @param holderList
+ */
+ private void getHolderList(int datasetId, int hashValue, PrimitiveIntHashMap holderList) {
+ PrimitiveIntHashMap entityHT;
+ DatasetLockInfo dLockInfo;
+ int entityLockInfo;
+ int entityInfo;
+ int waiterObjId;
+ LockWaiter waiterObj;
+
+ //get datasetLockInfo
+ tempDatasetIdObj.setId(datasetId);
+ dLockInfo = datasetResourceHT.get(tempDatasetIdObj);
+ if (dLockInfo == null) {
+ return;
+ }
+
+ if (hashValue == -1) {
+ //get S/X-lock holders of dataset
+ entityInfo = dLockInfo.getLastHolder();
+ while(entityInfo != -1) {
+ holderList.put(entityInfoManager.getJobId(entityInfo), 0);
+ entityInfo = entityInfoManager.getPrevEntityActor(entityInfo);
+ }
+
+ //get IS/IX-lock holders of dataset
+ entityHT = dLockInfo.getEntityResourceHT();
+ entityHT.beginIterate();
+ entityLockInfo = entityHT.getNextValue();
+ while (entityLockInfo != -1) {
+
+ //1. add holder of eLockInfo to holerList
+ entityInfo = entityLockInfoManager.getLastHolder(entityLockInfo);
+ while (entityInfo != -1) {
+ holderList.put(entityInfoManager.getJobId(entityInfo), 0);
+ entityInfo = entityInfoManager.getPrevEntityActor(entityInfo);
+ }
+
+ //2. add waiter of eLockInfo to holderList since waiter of entityLock is a holder of datasetLock
+ //(Upgraders need not to be added since upgraders are also holders)
+ waiterObjId = entityLockInfoManager.getFirstWaiter(entityLockInfo);
+ while (waiterObjId != -1) {
+ waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
+ entityInfo = waiterObj.getEntityInfoSlot();
+ holderList.put(entityInfoManager.getJobId(entityInfo), 0);
+ waiterObjId = waiterObj.getNextWaiterObjId();
+ }
+
+ entityLockInfo = entityHT.getNextValue();
+ }
+ } else {
+ //get S/X-lock holders of entity
+ entityHT = dLockInfo.getEntityResourceHT();
+ entityLockInfo = entityHT.get(hashValue);
+ if (entityLockInfo != -1) {
+ entityInfo = entityLockInfoManager.getLastHolder(entityLockInfo);
+ while (entityInfo != -1) {
+ holderList.put(entityInfoManager.getJobId(entityInfo), 0);
+ entityInfo = entityInfoManager.getPrevEntityActor(entityInfo);
+ }
}
}
+ return;
}
+ /**
+ * Get waiting resource list of jobId, where a resource is represented with entityInfo's slot number
+ * @param jobId
+ * @param resourceList
+ */
+ private void getWaitingResourceList(int jobId, PrimitiveIntHashMap resourceList) {
+ JobInfo jobInfo;
+ int waiterId;
+ LockWaiter waiterObj;
+ int entityInfo;
+
+ //get JobInfo
+ tempJobIdObj.setId(jobId);
+ jobInfo = jobHT.get(tempJobIdObj);
+ if (IS_DEBUG_MODE) {
+ if (jobInfo == null) {
+ System.out.println(Thread.currentThread().getName()+"jobId:"+jobId);
+ }
+ }
+
+ //get WaiterObj
+ waiterId = jobInfo.getFirstWaitingResource();
+ while (waiterId != -1)
+ {
+ waiterObj = lockWaiterManager.getLockWaiter(waiterId);
+ entityInfo = waiterObj.getEntityInfoSlot();
+ resourceList.put(entityInfo, -1);
+ waiterId = waiterObj.getNextWaitingResourceObjId();
+ }
+ return;
+ }
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/EntityInfoManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/EntityInfoManager.java
new file mode 100644
index 0000000..b8820c4
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/EntityInfoManager.java
@@ -0,0 +1,704 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.transaction.management.service.locking;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+
+/**
+ * EntityInfoManager provides EntityInfo arrays backed by ByteBuffer.
+ * The array grows when the slots are overflowed.
+ * Also, the array shrinks according to the following shrink policy
+ * : Shrink when the resource under-utilization lasts for a certain threshold time.
+ *
+ * @author kisskys
+ */
+public class EntityInfoManager {
+
+ public static final int SHRINK_TIMER_THRESHOLD = 120000; //2min
+
+ private ArrayList<ChildEntityInfoArrayManager> pArray;
+ private int allocChild; //used to allocate the next free EntityInfo slot.
+ private long shrinkTimer;
+ private boolean isShrinkTimerOn;
+ private int occupiedSlots;
+
+ // ////////////////////////////////////////////////
+ // // begin of unit test
+ // ////////////////////////////////////////////////
+ //
+ // public static final int SHRINK_TIMER_THRESHOLD = 0; //for unit test
+ //
+ // /**
+ // * @param args
+ // */
+ // public static void main(String[] args) {
+ // final int DataSize = 5000;
+ //
+ // int i, j;
+ // int slots = ChildEntityInfoArrayManager.NUM_OF_SLOTS;
+ // int data[] = new int[DataSize];
+ // EntityInfoManager eiMgr = new EntityInfoManager();
+ //
+ // //allocate: 50
+ // System.out.println("allocate: 50");
+ // for (i = 0; i < 5; i++) {
+ // for (j = i * slots; j < i * slots + slots; j++) {
+ // data[j] = eiMgr.allocate();
+ // }
+ //
+ // System.out.println(eiMgr.prettyPrint());
+ // }
+ //
+ // //deallocate from the last child to the first child
+ // System.out.println("deallocate from the last child to the first child");
+ // for (i = 4; i >= 0; i--) {
+ // for (j = i * slots + slots - 1; j >= i * slots; j--) {
+ // eiMgr.deallocate(data[j]);
+ // }
+ // System.out.println(eiMgr.prettyPrint());
+ // }
+ //
+ // //allocate: 50
+ // System.out.println("allocate: 50");
+ // for (i = 0; i < 5; i++) {
+ // for (j = i * slots; j < i * slots + slots; j++) {
+ // data[j] = eiMgr.allocate();
+ // }
+ //
+ // System.out.println(eiMgr.prettyPrint());
+ // }
+ //
+ // //deallocate from the first child to last child
+ // System.out.println("deallocate from the first child to last child");
+ // for (i = 0; i < 5; i++) {
+ // for (j = i * slots; j < i * slots + slots; j++) {
+ // eiMgr.deallocate(data[j]);
+ // }
+ //
+ // System.out.println(eiMgr.prettyPrint());
+ // }
+ //
+ // //allocate: 50
+ // System.out.println("allocate: 50");
+ // for (i = 0; i < 5; i++) {
+ // for (j = i * slots; j < i * slots + slots; j++) {
+ // data[j] = eiMgr.allocate();
+ // }
+ //
+ // System.out.println(eiMgr.prettyPrint());
+ // }
+ //
+ // //deallocate from the first child to 4th child
+ // System.out.println("deallocate from the first child to 4th child");
+ // for (i = 0; i < 4; i++) {
+ // for (j = i * slots; j < i * slots + slots; j++) {
+ // eiMgr.deallocate(data[j]);
+ // }
+ //
+ // System.out.println(eiMgr.prettyPrint());
+ // }
+ //
+ // //allocate: 40
+ // System.out.println("allocate: 40");
+ // for (i = 0; i < 4; i++) {
+ // for (j = i * slots; j < i * slots + slots; j++) {
+ // data[j] = eiMgr.allocate();
+ // }
+ //
+ // System.out.println(eiMgr.prettyPrint());
+ // }
+ // }
+ //
+ // ////////////////////////////////////////////////
+ // // end of unit test
+ // ////////////////////////////////////////////////
+
+ public EntityInfoManager() {
+ pArray = new ArrayList<ChildEntityInfoArrayManager>();
+ pArray.add(new ChildEntityInfoArrayManager());
+ allocChild = 0;
+ occupiedSlots = 0;
+ isShrinkTimerOn = false;
+ }
+
+ public int allocate(int jobId, int datasetId, int entityHashVal, byte lockMode) {
+ int slotNum = allocate();
+ initEntityInfo(slotNum, jobId, datasetId, entityHashVal, lockMode);
+ return slotNum;
+ }
+
+ public int allocate() {
+ if (pArray.get(allocChild).isFull()) {
+ int size = pArray.size();
+ boolean bAlloc = false;
+ ChildEntityInfoArrayManager child;
+
+ //find a deinitialized child and initialze it
+ for (int i = 0; i < size; i++) {
+ child = pArray.get(i);
+ if (child.isDeinitialized()) {
+ child.initialize();
+ allocChild = i;
+ bAlloc = true;
+ break;
+ }
+ }
+
+ //allocate new child when there is no deinitialized child
+ if (!bAlloc) {
+ pArray.add(new ChildEntityInfoArrayManager());
+ allocChild = pArray.size() - 1;
+ }
+ }
+ occupiedSlots++;
+ return pArray.get(allocChild).allocate() + allocChild * ChildEntityInfoArrayManager.NUM_OF_SLOTS;
+ }
+
+ void deallocate(int slotNum) {
+ pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).deallocate(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
+ occupiedSlots--;
+
+ if (needShrink()) {
+ shrink();
+ }
+ }
+
+ /**
+ * Shrink policy:
+ * Shrink when the resource under-utilization lasts for a certain amount of time.
+ * TODO Need to figure out which of the policies is better
+ * case1.
+ * pArray status : O x x x x x O (O is initialized, x is deinitialized)
+ * In the above status, 'CURRENT' needShrink() returns 'TRUE'
+ * even if there is nothing to shrink or deallocate.
+ * It doesn't distinguish the deinitialized children from initialized children
+ * by calculating totalNumOfSlots = pArray.size() * ChildEntityInfoArrayManager.NUM_OF_SLOTS.
+ * In other words, it doesn't subtract the deinitialized children's slots.
+ * case2.
+ * pArray status : O O x x x x x
+ * However, in the above case, if we subtract the deinitialized children's slots,
+ * needShrink() will return false even if we shrink the pArray at this case.
+ *
+ * @return
+ */
+ private boolean needShrink() {
+ int size = pArray.size();
+ int usedSlots = occupiedSlots;
+ if (usedSlots == 0) {
+ usedSlots = 1;
+ }
+
+ if (size > 1 && size * ChildEntityInfoArrayManager.NUM_OF_SLOTS / usedSlots >= 3) {
+ if (isShrinkTimerOn) {
+ if (System.currentTimeMillis() - shrinkTimer >= SHRINK_TIMER_THRESHOLD) {
+ isShrinkTimerOn = false;
+ return true;
+ }
+ } else {
+ //turn on timer
+ isShrinkTimerOn = true;
+ shrinkTimer = System.currentTimeMillis();
+ }
+ } else {
+ //turn off timer
+ isShrinkTimerOn = false;
+ }
+
+ return false;
+ }
+
+ /**
+ * Shrink() may
+ * deinitialize(:deallocates ByteBuffer of child) Children(s) or
+ * shrink pArray according to the deinitialized children's contiguity status.
+ * It doesn't deinitialze or shrink more than half of children at a time.
+ */
+ private void shrink() {
+ int i;
+ boolean bContiguous = true;
+ int decreaseCount = 0;
+ int size = pArray.size();
+ int maxDecreaseCount = size / 2;
+ ChildEntityInfoArrayManager child;
+ for (i = size - 1; i >= 0; i--) {
+ child = pArray.get(i);
+ if (child.isEmpty() || child.isDeinitialized()) {
+ if (bContiguous) {
+ pArray.remove(i);
+ if (++decreaseCount == maxDecreaseCount) {
+ break;
+ }
+ } else {
+ bContiguous = false;
+ if (child.isEmpty()) {
+ child.deinitialize();
+ if (++decreaseCount == maxDecreaseCount) {
+ break;
+ }
+ }
+ }
+ } else {
+ bContiguous = false;
+ }
+ }
+
+ //reset allocChild when the child is removed or deinitialized.
+ size = pArray.size();
+ if (allocChild >= size || pArray.get(allocChild).isDeinitialized()) {
+ //set allocChild to any initialized one.
+ //It is guaranteed that there is at least one initialized child.
+ for (i = 0; i < size; i++) {
+ if (!pArray.get(i).isDeinitialized()) {
+ allocChild = i;
+ break;
+ }
+ }
+ }
+ }
+
+ public String prettyPrint() {
+ StringBuilder s = new StringBuilder("\n########### EntityInfoManager Status #############\n");
+ int size = pArray.size();
+ ChildEntityInfoArrayManager child;
+
+ for (int i = 0; i < size; i++) {
+ child = pArray.get(i);
+ if (child.isDeinitialized()) {
+ continue;
+ }
+ s.append("child[" + i + "]: occupiedSlots:" + child.getNumOfOccupiedSlots());
+ s.append(" freeSlotNum:" + child.getFreeSlotNum() + "\n");
+ s.append("\tjid\t").append("did\t").append("PK\t").append("DLM\t").append("DLC\t").append("ELM\t")
+ .append("ELC\t").append("NEA\t").append("PJR\t").append("NJR\n");
+ for (int j = 0; j < ChildEntityInfoArrayManager.NUM_OF_SLOTS; j++) {
+ s.append(j).append(": ");
+ s.append("\t" + child.getJobId(j));
+ s.append("\t" + child.getDatasetId(j));
+ s.append("\t" + child.getPKHashVal(j));
+ s.append("\t" + child.getDatasetLockMode(j));
+ s.append("\t" + child.getDatasetLockCount(j));
+ s.append("\t" + child.getEntityLockMode(j));
+ s.append("\t" + child.getEntityLockCount(j));
+ s.append("\t" + child.getNextEntityActor(j));
+ s.append("\t" + child.getPrevJobResource(j));
+ s.append("\t" + child.getNextJobResource(j));
+ //s.append("\t" + child.getNextDatasetActor(j));
+ s.append("\n");
+ }
+ s.append("\n");
+ }
+ return s.toString();
+ }
+
+ public void initEntityInfo(int slotNum, int jobId, int datasetId, int PKHashVal, byte lockMode) {
+ pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).initEntityInfo(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, jobId, datasetId, PKHashVal, lockMode);
+ }
+
+ public boolean compareEntityInfo(int slotNum, int jobId, int datasetId, int PKHashVal) {
+ return getPKHashVal(slotNum) == PKHashVal && getDatasetId(slotNum) == datasetId && getJobId(slotNum) == jobId;
+ }
+
+ public void increaseDatasetLockCount(int slotNum) {
+ setDatasetLockCount(slotNum, (byte) (getDatasetLockCount(slotNum) + 1));
+ }
+
+ public void decreaseDatasetLockCount(int slotNum) {
+ setDatasetLockCount(slotNum, (byte) (getDatasetLockCount(slotNum) - 1));
+ }
+
+ public void increaseEntityLockCount(int slotNum) {
+ setEntityLockCount(slotNum, (byte) (getEntityLockCount(slotNum) + 1));
+ }
+
+ public void decreaseEntityLockCount(int slotNum) {
+ setEntityLockCount(slotNum, (byte) (getEntityLockCount(slotNum) - 1));
+ }
+
+ public void increaseDatasetLockCount(int slotNum, int count) {
+ setDatasetLockCount(slotNum, (byte) (getDatasetLockCount(slotNum) + count));
+ }
+
+ public void decreaseDatasetLockCount(int slotNum, int count) {
+ setDatasetLockCount(slotNum, (byte) (getDatasetLockCount(slotNum) - count));
+ }
+
+ public void increaseEntityLockCount(int slotNum, int count) {
+ setEntityLockCount(slotNum, (byte) (getEntityLockCount(slotNum) + count));
+ }
+
+ public void decreaseEntityLockCount(int slotNum, int count) {
+ setEntityLockCount(slotNum, (byte) (getEntityLockCount(slotNum) - count));
+ }
+
+ //////////////////////////////////////////////////////////////////
+ // set/get method for each field of EntityInfo
+ //////////////////////////////////////////////////////////////////
+
+ public void setJobId(int slotNum, int id) {
+ pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setJobId(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, id);
+ }
+
+ public int getJobId(int slotNum) {
+ return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getJobId(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
+ }
+
+ public void setDatasetId(int slotNum, int id) {
+ pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setDatasetId(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, id);
+ }
+
+ public int getDatasetId(int slotNum) {
+ return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getDatasetId(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
+ }
+
+ public void setPKHashVal(int slotNum, int hashVal) {
+ pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setPKHashVal(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, hashVal);
+ }
+
+ public int getPKHashVal(int slotNum) {
+ return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getPKHashVal(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
+ }
+
+ public void setDatasetLockMode(int slotNum, byte mode) {
+ pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setDatasetLockMode(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, mode);
+ }
+
+ public byte getDatasetLockMode(int slotNum) {
+ return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getDatasetLockMode(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
+ }
+
+ public void setDatasetLockCount(int slotNum, byte count) {
+ pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setDatasetLockCount(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, count);
+ }
+
+ public byte getDatasetLockCount(int slotNum) {
+ return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getDatasetLockCount(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
+ }
+
+ public void setEntityLockMode(int slotNum, byte mode) {
+ pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setEntityLockMode(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, mode);
+ }
+
+ public byte getEntityLockMode(int slotNum) {
+ return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getEntityLockMode(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
+ }
+
+ public void setEntityLockCount(int slotNum, byte count) {
+ pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setEntityLockCount(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, count);
+ }
+
+ public byte getEntityLockCount(int slotNum) {
+ return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getEntityLockCount(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
+ }
+
+ //Used for Waiter/Upgrader
+ public void setNextEntityActor(int slotNum, int nextActorSlotNum) {
+ pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setNextEntityActor(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, nextActorSlotNum);
+ }
+
+ //Used for Waiter/Upgrader
+ public int getNextEntityActor(int slotNum) {
+ return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getNextEntityActor(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
+ }
+
+ //Used for Holder
+ public void setPrevEntityActor(int slotNum, int nextActorSlotNum) {
+ pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setPrevEntityActor(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, nextActorSlotNum);
+ }
+
+ //Used for Holder
+ public int getPrevEntityActor(int slotNum) {
+ return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getPrevEntityActor(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
+ }
+
+ public void setPrevJobResource(int slotNum, int prevResourceSlotNum) {
+ pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setPrevJobResource(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, prevResourceSlotNum);
+ }
+
+ public int getPrevJobResource(int slotNum) {
+ return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getPrevJobResource(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
+ }
+
+ public void setNextJobResource(int slotNum, int nextResourceSlotNum) {
+ pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setNextJobResource(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, nextResourceSlotNum);
+ }
+
+ public int getNextJobResource(int slotNum) {
+ return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getNextJobResource(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
+ }
+
+ // public void setNextDatasetActor(int slotNum, int nextActorSlotNum) {
+ // pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setNextDatasetActor(
+ // slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, nextActorSlotNum);
+ // }
+ //
+ // public int getNextDatasetActor(int slotNum) {
+ // return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getNextDatasetActor(
+ // slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
+ // }
+}
+
+/******************************************
+ * EntityInfo (28 bytes)
+ * ****************************************
+ * int jobId
+ * int datasetId
+ * int PKHashValue
+ * byte datasetLockMode
+ * byte datasetLockCount
+ * byte enitityLockMode
+ * byte entityLockCount
+ * int nextEntityActor : actor can be either holder/waiter/upgrader
+ * int prevJobResource : resource can be either dataset or entity and a job is holding/waiting/upgrading lock(s) on it.
+ * int nextJobResource : resource can be either dataset or entity and a job is holding/waiting/upgrading lock(s) on it.
+ * (int nextDatasetActor : actor can be either holder/waiter/upgrader) --> not used.
+ *******************************************/
+
+class ChildEntityInfoArrayManager {
+ public static final int ENTITY_INFO_SIZE = 28; //28bytes
+ public static final int NUM_OF_SLOTS = 1024; //number of entities in a buffer
+ // public static final int NUM_OF_SLOTS = 10; //for unit test
+ public static final int BUFFER_SIZE = ENTITY_INFO_SIZE * NUM_OF_SLOTS;
+
+ //byte offset of each field of EntityInfo
+ public static final int JOB_ID_OFFSET = 0;
+ public static final int DATASET_ID_OFFSET = 4;
+ public static final int PKHASH_VAL_OFFSET = 8;
+ public static final int DATASET_LOCK_MODE_OFFSET = 12;
+ public static final int DATASET_LOCK_COUNT_OFFSET = 13;
+ public static final int ENTITY_LOCK_MODE_OFFSET = 14;
+ public static final int ENTITY_LOCK_COUNT_OFFSET = 15;
+ public static final int ENTITY_ACTOR_OFFSET = 16;
+ public static final int PREV_JOB_RESOURCE_OFFSET = 20;
+ public static final int NEXT_JOB_RESOURCE_OFFSET = 24;
+ //public static final int DATASET_ACTOR_OFFSET = 28;
+
+ //byte offset of nextFreeSlotNum which shares the same space of JobId
+ //If a slot is in use, the space is used for JobId. Otherwise, it is used for nextFreeSlotNum.
+ public static final int NEXT_FREE_SLOT_OFFSET = 0;
+
+ private ByteBuffer buffer;
+ private int freeSlotNum;
+ private int occupiedSlots; //-1 represents 'deinitialized' state.
+
+ public ChildEntityInfoArrayManager() {
+ initialize();
+ }
+
+ public void initialize() {
+ this.buffer = ByteBuffer.allocate(BUFFER_SIZE);
+ this.freeSlotNum = 0;
+ this.occupiedSlots = 0;
+
+ for (int i = 0; i < NUM_OF_SLOTS - 1; i++) {
+ setNextFreeSlot(i, i + 1);
+ }
+ setNextFreeSlot(NUM_OF_SLOTS - 1, -1); //-1 represents EOL(end of link)
+ }
+
+ public int allocate() {
+ int currentSlot = freeSlotNum;
+ freeSlotNum = getNextFreeSlot(currentSlot);
+ occupiedSlots++;
+ if (LockManager.IS_DEBUG_MODE) {
+ System.out.println(Thread.currentThread().getName()+" entity allocate: "+currentSlot);
+ }
+ return currentSlot;
+ }
+
+ public void deallocate(int slotNum) {
+ setNextFreeSlot(slotNum, freeSlotNum);
+ freeSlotNum = slotNum;
+ occupiedSlots--;
+ if (LockManager.IS_DEBUG_MODE) {
+ System.out.println(Thread.currentThread().getName()+" entity deallocate: "+slotNum);
+ }
+ }
+
+ public void deinitialize() {
+ buffer = null;
+ occupiedSlots = -1;
+ }
+
+ public boolean isDeinitialized() {
+ return occupiedSlots == -1;
+ }
+
+ public boolean isFull() {
+ return occupiedSlots == NUM_OF_SLOTS;
+ }
+
+ public boolean isEmpty() {
+ return occupiedSlots == 0;
+ }
+
+ public int getNumOfOccupiedSlots() {
+ return occupiedSlots;
+ }
+
+ public int getFreeSlotNum() {
+ return freeSlotNum;
+ }
+
+ //////////////////////////////////////////////////////////////////
+ // set/get method for each field of EntityInfo plus freeSlot
+ //////////////////////////////////////////////////////////////////
+ public void initEntityInfo(int slotNum, int jobId, int datasetId, int PKHashVal, byte lockMode) {
+ buffer.putInt(slotNum * ENTITY_INFO_SIZE + JOB_ID_OFFSET, jobId);
+ buffer.putInt(slotNum * ENTITY_INFO_SIZE + DATASET_ID_OFFSET, datasetId);
+ buffer.putInt(slotNum * ENTITY_INFO_SIZE + PKHASH_VAL_OFFSET, PKHashVal);
+ buffer.put(slotNum * ENTITY_INFO_SIZE + DATASET_LOCK_MODE_OFFSET, lockMode);
+ buffer.put(slotNum * ENTITY_INFO_SIZE + DATASET_LOCK_COUNT_OFFSET, (byte) 0);
+ buffer.put(slotNum * ENTITY_INFO_SIZE + ENTITY_LOCK_MODE_OFFSET, lockMode);
+ buffer.put(slotNum * ENTITY_INFO_SIZE + ENTITY_LOCK_COUNT_OFFSET, (byte) 0);
+ buffer.putInt(slotNum * ENTITY_INFO_SIZE + ENTITY_ACTOR_OFFSET, -1);
+ buffer.putInt(slotNum * ENTITY_INFO_SIZE + PREV_JOB_RESOURCE_OFFSET, -1);
+ buffer.putInt(slotNum * ENTITY_INFO_SIZE + NEXT_JOB_RESOURCE_OFFSET, -1);
+ //buffer.putInt(slotNum * ENTITY_INFO_SIZE + DATASET_ACTOR_OFFSET, -1);
+ }
+
+ public void setNextFreeSlot(int slotNum, int nextFreeSlot) {
+ buffer.putInt(slotNum * ENTITY_INFO_SIZE + NEXT_FREE_SLOT_OFFSET, nextFreeSlot);
+ }
+
+ public int getNextFreeSlot(int slotNum) {
+ return buffer.getInt(slotNum * ENTITY_INFO_SIZE + NEXT_FREE_SLOT_OFFSET);
+ }
+
+ public void setJobId(int slotNum, int id) {
+ buffer.putInt(slotNum * ENTITY_INFO_SIZE + JOB_ID_OFFSET, id);
+ }
+
+ public int getJobId(int slotNum) {
+ return buffer.getInt(slotNum * ENTITY_INFO_SIZE + JOB_ID_OFFSET);
+ }
+
+ public void setDatasetId(int slotNum, int id) {
+ buffer.putInt(slotNum * ENTITY_INFO_SIZE + DATASET_ID_OFFSET, id);
+ }
+
+ public int getDatasetId(int slotNum) {
+ return buffer.getInt(slotNum * ENTITY_INFO_SIZE + DATASET_ID_OFFSET);
+ }
+
+ public void setPKHashVal(int slotNum, int hashVal) {
+ buffer.putInt(slotNum * ENTITY_INFO_SIZE + PKHASH_VAL_OFFSET, hashVal);
+ }
+
+ public int getPKHashVal(int slotNum) {
+ return buffer.getInt(slotNum * ENTITY_INFO_SIZE + PKHASH_VAL_OFFSET);
+ }
+
+ public void setDatasetLockMode(int slotNum, byte mode) {
+ buffer.put(slotNum * ENTITY_INFO_SIZE + DATASET_LOCK_MODE_OFFSET, mode);
+ }
+
+ public byte getDatasetLockMode(int slotNum) {
+ return buffer.get(slotNum * ENTITY_INFO_SIZE + DATASET_LOCK_MODE_OFFSET);
+ }
+
+ public void setDatasetLockCount(int slotNum, byte count) {
+ buffer.put(slotNum * ENTITY_INFO_SIZE + DATASET_LOCK_COUNT_OFFSET, count);
+ }
+
+ public byte getDatasetLockCount(int slotNum) {
+ return buffer.get(slotNum * ENTITY_INFO_SIZE + DATASET_LOCK_COUNT_OFFSET);
+ }
+
+ public void setEntityLockMode(int slotNum, byte mode) {
+ buffer.put(slotNum * ENTITY_INFO_SIZE + ENTITY_LOCK_MODE_OFFSET, mode);
+ }
+
+ public byte getEntityLockMode(int slotNum) {
+ return buffer.get(slotNum * ENTITY_INFO_SIZE + ENTITY_LOCK_MODE_OFFSET);
+ }
+
+ public void setEntityLockCount(int slotNum, byte count) {
+ buffer.put(slotNum * ENTITY_INFO_SIZE + ENTITY_LOCK_COUNT_OFFSET, count);
+ }
+
+ public byte getEntityLockCount(int slotNum) {
+ return buffer.get(slotNum * ENTITY_INFO_SIZE + ENTITY_LOCK_COUNT_OFFSET);
+ }
+
+ //Used for Waiter/Upgrader
+ public void setNextEntityActor(int slotNum, int nextActorSlotNum) {
+ buffer.putInt(slotNum * ENTITY_INFO_SIZE + ENTITY_ACTOR_OFFSET, nextActorSlotNum);
+ }
+
+ //Used for Waiter/Upgrader
+ public int getNextEntityActor(int slotNum) {
+ return buffer.getInt(slotNum * ENTITY_INFO_SIZE + ENTITY_ACTOR_OFFSET);
+ }
+
+ //Used for Holder
+ public void setPrevEntityActor(int slotNum, int nextActorSlotNum) {
+ buffer.putInt(slotNum * ENTITY_INFO_SIZE + ENTITY_ACTOR_OFFSET, nextActorSlotNum);
+ }
+
+ //Used for Holder
+ public int getPrevEntityActor(int slotNum) {
+ return buffer.getInt(slotNum * ENTITY_INFO_SIZE + ENTITY_ACTOR_OFFSET);
+ }
+
+ public void setPrevJobResource(int slotNum, int prevResourceSlotNum) {
+ buffer.putInt(slotNum * ENTITY_INFO_SIZE + PREV_JOB_RESOURCE_OFFSET, prevResourceSlotNum);
+ }
+
+ public int getPrevJobResource(int slotNum) {
+ return buffer.getInt(slotNum * ENTITY_INFO_SIZE + PREV_JOB_RESOURCE_OFFSET);
+ }
+
+ public void setNextJobResource(int slotNum, int prevResourceSlotNum) {
+ buffer.putInt(slotNum * ENTITY_INFO_SIZE + NEXT_JOB_RESOURCE_OFFSET, prevResourceSlotNum);
+ }
+
+ public int getNextJobResource(int slotNum) {
+ return buffer.getInt(slotNum * ENTITY_INFO_SIZE + NEXT_JOB_RESOURCE_OFFSET);
+ }
+
+ // public void setNextDatasetActor(int slotNum, int nextActorSlotNum) {
+ // buffer.putInt(slotNum * ENTITY_INFO_SIZE + DATASET_ACTOR_OFFSET, nextActorSlotNum);
+ // }
+ //
+ // public int getNextDatasetActor(int slotNum) {
+ // return buffer.getInt(slotNum * ENTITY_INFO_SIZE + DATASET_ACTOR_OFFSET);
+ // }
+}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/EntityLockInfoManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/EntityLockInfoManager.java
new file mode 100644
index 0000000..2542157
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/EntityLockInfoManager.java
@@ -0,0 +1,802 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.transaction.management.service.locking;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
+
+/**
+ * EntityLockInfoManager provides EntityLockInfo arrays backed by ByteBuffer.
+ * The array grows when the slots are overflowed.
+ * Also, the array shrinks according to the following shrink policy
+ * : Shrink when the resource under-utilization lasts for a certain threshold time.
+ *
+ * @author kisskys
+ */
+public class EntityLockInfoManager {
+
+ public static final int SHRINK_TIMER_THRESHOLD = 120000; //2min
+
+ private ArrayList<ChildEntityLockInfoArrayManager> pArray;
+ private int allocChild; //used to allocate the next free EntityInfo slot.
+ private long shrinkTimer;
+ private boolean isShrinkTimerOn;
+ private int occupiedSlots;
+ private EntityInfoManager entityInfoManager;
+ LockWaiterManager lockWaiterManager;
+
+ // ////////////////////////////////////////////////
+ // // begin of unit test
+ // ////////////////////////////////////////////////
+ //
+ // public static final int SHRINK_TIMER_THRESHOLD = 0; //for unit test
+ //
+ // /**
+ // * @param args
+ // */
+ // public static void main(String[] args) {
+ // final int DataSize = 5000;
+ //
+ // int i, j;
+ // int slots = ChildEntityLockInfoArrayManager.NUM_OF_SLOTS;
+ // int data[] = new int[DataSize];
+ // EntityLockInfoManager eliMgr = new EntityLockInfoManager();
+ //
+ // //allocate: 50
+ // System.out.println("allocate: 50");
+ // for (i = 0; i < 5; i++) {
+ // for (j = i * slots; j < i * slots + slots; j++) {
+ // data[j] = eliMgr.allocate();
+ // }
+ //
+ // System.out.println(eliMgr.prettyPrint());
+ // }
+ //
+ // //deallocate from the last child to the first child
+ // System.out.println("deallocate from the last child to the first child");
+ // for (i = 4; i >= 0; i--) {
+ // for (j = i * slots + slots - 1; j >= i * slots; j--) {
+ // eliMgr.deallocate(data[j]);
+ // }
+ // System.out.println(eliMgr.prettyPrint());
+ // }
+ //
+ // //allocate: 50
+ // System.out.println("allocate: 50");
+ // for (i = 0; i < 5; i++) {
+ // for (j = i * slots; j < i * slots + slots; j++) {
+ // data[j] = eliMgr.allocate();
+ // }
+ //
+ // System.out.println(eliMgr.prettyPrint());
+ // }
+ //
+ // //deallocate from the first child to last child
+ // System.out.println("deallocate from the first child to last child");
+ // for (i = 0; i < 5; i++) {
+ // for (j = i * slots; j < i * slots + slots; j++) {
+ // eliMgr.deallocate(data[j]);
+ // }
+ //
+ // System.out.println(eliMgr.prettyPrint());
+ // }
+ //
+ // //allocate: 50
+ // System.out.println("allocate: 50");
+ // for (i = 0; i < 5; i++) {
+ // for (j = i * slots; j < i * slots + slots; j++) {
+ // data[j] = eliMgr.allocate();
+ // }
+ //
+ // System.out.println(eliMgr.prettyPrint());
+ // }
+ //
+ // //deallocate from the first child to 4th child
+ // System.out.println("deallocate from the first child to 4th child");
+ // for (i = 0; i < 4; i++) {
+ // for (j = i * slots; j < i * slots + slots; j++) {
+ // eliMgr.deallocate(data[j]);
+ // }
+ //
+ // System.out.println(eliMgr.prettyPrint());
+ // }
+ //
+ // //allocate: 40
+ // System.out.println("allocate: 40");
+ // for (i = 0; i < 4; i++) {
+ // for (j = i * slots; j < i * slots + slots; j++) {
+ // data[j] = eliMgr.allocate();
+ // }
+ //
+ // System.out.println(eliMgr.prettyPrint());
+ // }
+ // }
+ //
+ // ////////////////////////////////////////////////
+ // // end of unit test
+ // ////////////////////////////////////////////////
+
+ public EntityLockInfoManager(EntityInfoManager entityInfoManager, LockWaiterManager lockWaiterManager) {
+ pArray = new ArrayList<ChildEntityLockInfoArrayManager>();
+ pArray.add(new ChildEntityLockInfoArrayManager());
+ allocChild = 0;
+ occupiedSlots = 0;
+ isShrinkTimerOn = false;
+ this.entityInfoManager = entityInfoManager;
+ this.lockWaiterManager = lockWaiterManager;
+ }
+
+ public int allocate() {
+ if (pArray.get(allocChild).isFull()) {
+ int size = pArray.size();
+ boolean bAlloc = false;
+ ChildEntityLockInfoArrayManager child;
+
+ //find a deinitialized child and initialze it
+ for (int i = 0; i < size; i++) {
+ child = pArray.get(i);
+ if (child.isDeinitialized()) {
+ child.initialize();
+ allocChild = i;
+ bAlloc = true;
+ break;
+ }
+ }
+
+ //allocate new child when there is no deinitialized child
+ if (!bAlloc) {
+ pArray.add(new ChildEntityLockInfoArrayManager());
+ allocChild = pArray.size() - 1;
+ }
+ }
+ occupiedSlots++;
+ return pArray.get(allocChild).allocate() + allocChild * ChildEntityLockInfoArrayManager.NUM_OF_SLOTS;
+ }
+
+ void deallocate(int slotNum) {
+ pArray.get(slotNum / ChildEntityLockInfoArrayManager.NUM_OF_SLOTS).deallocate(
+ slotNum % ChildEntityLockInfoArrayManager.NUM_OF_SLOTS);
+ occupiedSlots--;
+
+ if (needShrink()) {
+ shrink();
+ }
+ }
+
+ /**
+ * Shrink policy:
+ * Shrink when the resource under-utilization lasts for a certain amount of time.
+ * TODO Need to figure out which of the policies is better
+ * case1.
+ * pArray status : O x x x x x O (O is initialized, x is deinitialized)
+ * In the above status, 'CURRENT' needShrink() returns 'TRUE'
+ * even if there is nothing to shrink or deallocate.
+ * It doesn't distinguish the deinitialized children from initialized children
+ * by calculating totalNumOfSlots = pArray.size() * ChildEntityLockInfoArrayManager.NUM_OF_SLOTS.
+ * In other words, it doesn't subtract the deinitialized children's slots.
+ * case2.
+ * pArray status : O O x x x x x
+ * However, in the above case, if we subtract the deinitialized children's slots,
+ * needShrink() will return false even if we shrink the pArray at this case.
+ *
+ * @return
+ */
+ private boolean needShrink() {
+ int size = pArray.size();
+ int usedSlots = occupiedSlots;
+ if (usedSlots == 0) {
+ usedSlots = 1;
+ }
+
+ if (size > 1 && size * ChildEntityLockInfoArrayManager.NUM_OF_SLOTS / usedSlots >= 3) {
+ if (isShrinkTimerOn) {
+ if (System.currentTimeMillis() - shrinkTimer >= SHRINK_TIMER_THRESHOLD) {
+ isShrinkTimerOn = false;
+ return true;
+ }
+ } else {
+ //turn on timer
+ isShrinkTimerOn = true;
+ shrinkTimer = System.currentTimeMillis();
+ }
+ } else {
+ //turn off timer
+ isShrinkTimerOn = false;
+ }
+
+ return false;
+ }
+
+ /**
+ * Shrink() may
+ * deinitialize(:deallocates ByteBuffer of child) Children(s) or
+ * shrink pArray according to the deinitialized children's contiguity status.
+ * It doesn't deinitialze or shrink more than half of children at a time.
+ */
+ private void shrink() {
+ int i;
+ boolean bContiguous = true;
+ int decreaseCount = 0;
+ int size = pArray.size();
+ int maxDecreaseCount = size / 2;
+ ChildEntityLockInfoArrayManager child;
+ for (i = size - 1; i >= 0; i--) {
+ child = pArray.get(i);
+ if (child.isEmpty() || child.isDeinitialized()) {
+ if (bContiguous) {
+ pArray.remove(i);
+ if (++decreaseCount == maxDecreaseCount) {
+ break;
+ }
+ } else {
+ bContiguous = false;
+ if (child.isEmpty()) {
+ child.deinitialize();
+ if (++decreaseCount == maxDecreaseCount) {
+ break;
+ }
+ }
+ }
+ } else {
+ bContiguous = false;
+ }
+ }
+
+ //reset allocChild when the child is removed or deinitialized.
+ size = pArray.size();
+ if (allocChild >= size || pArray.get(allocChild).isDeinitialized()) {
+ //set allocChild to any initialized one.
+ //It is guaranteed that there is at least one initialized child.
+ for (i = 0; i < size; i++) {
+ if (!pArray.get(i).isDeinitialized()) {
+ allocChild = i;
+ break;
+ }
+ }
+ }
+ }
+
+ public String prettyPrint() {
+ StringBuilder s = new StringBuilder("\n########### EntityLockInfoManager Status #############\n");
+ int size = pArray.size();
+ ChildEntityLockInfoArrayManager child;
+
+ for (int i = 0; i < size; i++) {
+ child = pArray.get(i);
+ if (child.isDeinitialized()) {
+ continue;
+ }
+ s.append("child[" + i + "]: occupiedSlots:" + child.getNumOfOccupiedSlots());
+ s.append(" freeSlotNum:" + child.getFreeSlotNum() + "\n");
+ s.append("\tX\t").append("S\t").append("LH\t").append("FW\t").append("UP\n");
+ for (int j = 0; j < ChildEntityLockInfoArrayManager.NUM_OF_SLOTS; j++) {
+ s.append(j).append(": ");
+ s.append("\t" + child.getXCount(j));
+ s.append("\t" + child.getSCount(j));
+ s.append("\t" + child.getLastHolder(j));
+ s.append("\t" + child.getFirstWaiter(j));
+ s.append("\t" + child.getUpgrader(j));
+ s.append("\n");
+ }
+ s.append("\n");
+ }
+ return s.toString();
+ }
+
+ //debugging method
+ public String printWaiters(int slotNum) {
+ StringBuilder s = new StringBuilder();
+ int waiterObjId;
+ LockWaiter waiterObj;
+ int entityInfo;
+
+ s.append("WID\tWCT\tEID\tJID\tDID\tPK\n");
+
+ waiterObjId = getFirstWaiter(slotNum);
+ while (waiterObjId != -1) {
+ waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
+ entityInfo = waiterObj.getEntityInfoSlot();
+ s.append(waiterObjId).append("\t").append(waiterObj.getWaiterCount()).append("\t").append(entityInfo)
+ .append("\t").append(entityInfoManager.getJobId(entityInfo)).append("\t")
+ .append(entityInfoManager.getDatasetId(entityInfo)).append("\t")
+ .append(entityInfoManager.getPKHashVal(entityInfo)).append("\n");
+ waiterObjId = waiterObj.getNextWaiterObjId();
+ }
+
+ return s.toString();
+ }
+
+ public void addHolder(int slotNum, int holder) {
+ entityInfoManager.setPrevEntityActor(holder, getLastHolder(slotNum));
+ setLastHolder(slotNum, holder);
+ }
+
+ /**
+ * Remove holder from linked list of Actor.
+ * Also, remove the corresponding resource from linked list of resource
+ * in order to minimize JobInfo's resource link traversal.
+ *
+ * @param slotNum
+ * @param holder
+ * @param jobInfo
+ */
+ public void removeHolder(int slotNum, int holder, JobInfo jobInfo) {
+ int prev = getLastHolder(slotNum);
+ int current = -1;
+ int next;
+
+ //remove holder from linked list of Actor
+ while (prev != holder) {
+ if (LockManager.IS_DEBUG_MODE) {
+ if (prev == -1) {
+ //shouldn't occur: debugging purpose
+ try {
+ throw new Exception();
+ } catch (Exception e) {
+ // TODO Auto-generated catch block
+ e.printStackTrace();
+ }
+ }
+ }
+
+ current = prev;
+ prev = entityInfoManager.getPrevEntityActor(current);
+ }
+
+ if (current != -1) {
+ //current->prev = prev->prev
+ entityInfoManager.setPrevEntityActor(current, entityInfoManager.getPrevEntityActor(prev));
+ } else {
+ //lastHolder = prev->prev
+ setLastHolder(slotNum, entityInfoManager.getPrevEntityActor(prev));
+ }
+
+ //Notice!!
+ //remove the corresponding resource from linked list of resource.
+ prev = entityInfoManager.getPrevJobResource(holder);
+ next = entityInfoManager.getNextJobResource(holder);
+
+ if (prev != -1) {
+ entityInfoManager.setNextJobResource(prev, next);
+ }
+
+ if (next != -1) {
+ entityInfoManager.setPrevJobResource(next, prev);
+ } else {
+ //This entityInfo(i.e., holder) is the last resource held by this job.
+ jobInfo.setlastHoldingResource(prev);
+ }
+
+ //jobInfo.decreaseDatasetLockCount(holder);
+ }
+
+ public void addWaiter(int slotNum, int waiterObjId) {
+ int lastObjId;
+ LockWaiter lastObj = null;
+ int firstWaiter = getFirstWaiter(slotNum);
+
+ if (firstWaiter != -1) {
+ //find the lastWaiter
+ lastObjId = firstWaiter;
+ while (lastObjId != -1) {
+ lastObj = lockWaiterManager.getLockWaiter(lastObjId);
+ lastObjId = lastObj.getNextWaiterObjId();
+ }
+ //last->next = new_waiter
+ lastObj.setNextWaiterObjId(waiterObjId);
+ } else {
+ setFirstWaiter(slotNum, waiterObjId);
+ }
+ //new_waiter->next = -1
+ lastObj = lockWaiterManager.getLockWaiter(waiterObjId);
+ lastObj.setNextWaiterObjId(-1);
+ }
+
+ public void removeWaiter(int slotNum, int waiterObjId) {
+ int currentObjId = getFirstWaiter(slotNum);
+ LockWaiter currentObj;
+ LockWaiter prevObj = null;
+ int prevObjId = -1;
+ int nextObjId;
+
+ while (currentObjId != waiterObjId) {
+
+ if (LockManager.IS_DEBUG_MODE) {
+ if (currentObjId == -1) {
+ //shouldn't occur: debugging purpose
+ try {
+ throw new Exception();
+ } catch (Exception e) {
+ // TODO Auto-generated catch block
+ e.printStackTrace();
+ }
+ }
+ }
+
+ prevObj = lockWaiterManager.getLockWaiter(currentObjId);
+ prevObjId = currentObjId;
+ currentObjId = prevObj.getNextWaiterObjId();
+ }
+
+ //get current waiter object
+ currentObj = lockWaiterManager.getLockWaiter(currentObjId);
+
+ //get next waiterObjId
+ nextObjId = currentObj.getNextWaiterObjId();
+
+ if (prevObjId != -1) {
+ //prev->next = next
+ prevObj.setNextWaiterObjId(nextObjId);
+ } else {
+ //removed first waiter. firstWaiter = current->next
+ setFirstWaiter(slotNum, nextObjId);
+ }
+ }
+
+ public void addUpgrader(int slotNum, int waiterObjId) {
+ //[Notice]
+ //Even if there are multiple threads in a job try to upgrade lock mode on same resource which is entity-granule,
+ //while the first upgrader is waiting, all the incoming upgrade requests from other threads should be rejected by aborting them.
+ //Therefore, there is no actual "ADD" upgrader method. Instead, it only has "SET" upgrader method.
+ if (LockManager.IS_DEBUG_MODE) {
+ if (getUpgrader(slotNum) != -1) {
+ throw new IllegalStateException("Invalid lock upgrade request. This call should be handled as deadlock");
+ }
+ }
+
+ setUpgrader(slotNum, waiterObjId);
+ }
+
+ public void removeUpgrader(int slotNum, int waiterObjId) {
+ setUpgrader(slotNum, -1);
+ }
+
+ public boolean isUpgradeCompatible(int slotNum, byte lockMode, int entityInfo) {
+ switch (lockMode) {
+ case LockMode.X:
+ return getSCount(slotNum) - entityInfoManager.getEntityLockCount(entityInfo) == 0;
+
+ default:
+ throw new IllegalStateException("Invalid upgrade lock mode");
+ }
+ }
+
+ public boolean isCompatible(int slotNum, byte lockMode) {
+ switch (lockMode) {
+ case LockMode.X:
+ return getSCount(slotNum) == 0 && getXCount(slotNum) == 0;
+
+ case LockMode.S:
+ return getXCount(slotNum) == 0;
+
+ default:
+ throw new IllegalStateException("Invalid upgrade lock mode");
+ }
+ }
+
+ public int findEntityInfoFromHolderList(int eLockInfo, int jobId, int hashVal) {
+ int entityInfo = getLastHolder(eLockInfo);
+
+ while (entityInfo != -1) {
+ if (jobId == entityInfoManager.getJobId(entityInfo)
+ && hashVal == entityInfoManager.getPKHashVal(entityInfo)) {
+ return entityInfo;
+ }
+// if (LockManager.IS_DEBUG_MODE) {
+// System.out.println("eLockInfo(" + eLockInfo + "),entityInfo(" + entityInfo + "), Request[" + jobId
+// + "," + hashVal + "]:Result[" + entityInfoManager.getJobId(entityInfo) + ","
+// + entityInfoManager.getPKHashVal(entityInfo) + "]");
+// }
+ entityInfo = entityInfoManager.getPrevEntityActor(entityInfo);
+ }
+
+ return -1;
+ }
+
+ public int findWaiterFromWaiterList(int eLockInfo, int jobId, int hashVal) {
+ int waiterObjId = getFirstWaiter(eLockInfo);
+ LockWaiter waiterObj;
+ int entityInfo;
+
+ while (waiterObjId != -1) {
+ waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
+ entityInfo = waiterObj.getEntityInfoSlot();
+ if (jobId == entityInfoManager.getJobId(entityInfo)
+ && hashVal == entityInfoManager.getPKHashVal(entityInfo)) {
+ return waiterObjId;
+ }
+ waiterObjId = waiterObj.getNextWaiterObjId();
+ }
+
+ return -1;
+ }
+
+ public int findUpgraderFromUpgraderList(int eLockInfo, int jobId, int hashVal) {
+ int waiterObjId = getUpgrader(eLockInfo);
+ LockWaiter waiterObj;
+ int entityInfo;
+
+ if (waiterObjId != -1) {
+ waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
+ entityInfo = waiterObj.getEntityInfoSlot();
+ if (jobId == entityInfoManager.getJobId(entityInfo)
+ && hashVal == entityInfoManager.getPKHashVal(entityInfo)) {
+ return waiterObjId;
+ }
+ }
+
+ return -1;
+ }
+
+ public void increaseLockCount(int slotNum, byte lockMode) {
+ switch (lockMode) {
+ case LockMode.X:
+ setXCount(slotNum, (short) (getXCount(slotNum) + 1));
+ break;
+ case LockMode.S:
+ setSCount(slotNum, (short) (getSCount(slotNum) + 1));
+ break;
+ default:
+ throw new IllegalStateException("Invalid entity lock mode " + lockMode);
+ }
+ }
+
+ public void decreaseLockCount(int slotNum, byte lockMode) {
+ switch (lockMode) {
+ case LockMode.X:
+ setXCount(slotNum, (short) (getXCount(slotNum) - 1));
+ break;
+ case LockMode.S:
+ setSCount(slotNum, (short) (getSCount(slotNum) - 1));
+ break;
+ default:
+ throw new IllegalStateException("Invalid entity lock mode " + lockMode);
+ }
+ }
+
+ public void increaseLockCount(int slotNum, byte lockMode, short count) {
+ switch (lockMode) {
+ case LockMode.X:
+ setXCount(slotNum, (short) (getXCount(slotNum) + count));
+ break;
+ case LockMode.S:
+ setSCount(slotNum, (short) (getSCount(slotNum) + count));
+ break;
+ default:
+ throw new IllegalStateException("Invalid entity lock mode " + lockMode);
+ }
+ }
+
+ public void decreaseLockCount(int slotNum, byte lockMode, short count) {
+ switch (lockMode) {
+ case LockMode.X:
+ setXCount(slotNum, (short) (getXCount(slotNum) - count));
+ break;
+ case LockMode.S:
+ setSCount(slotNum, (short) (getSCount(slotNum) - count));
+ break;
+ default:
+ throw new IllegalStateException("Invalid entity lock mode " + lockMode);
+ }
+ }
+
+ //////////////////////////////////////////////////////////////////
+ // set/get method for each field of EntityLockInfo
+ //////////////////////////////////////////////////////////////////
+
+ public void setXCount(int slotNum, short count) {
+ pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setXCount(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, count);
+ }
+
+ public short getXCount(int slotNum) {
+ return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getXCount(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
+ }
+
+ public void setSCount(int slotNum, short count) {
+ pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setSCount(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, count);
+ }
+
+ public short getSCount(int slotNum) {
+ return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getSCount(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
+ }
+
+ public void setLastHolder(int slotNum, int holder) {
+ pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setLastHolder(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, holder);
+ }
+
+ public int getLastHolder(int slotNum) {
+ return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getLastHolder(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
+ }
+
+ public void setFirstWaiter(int slotNum, int waiter) {
+ pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setFirstWaiter(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, waiter);
+ }
+
+ public int getFirstWaiter(int slotNum) {
+ return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getFirstWaiter(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
+ }
+
+ public void setUpgrader(int slotNum, int upgrader) {
+ pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setUpgrader(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, upgrader);
+ }
+
+ public int getUpgrader(int slotNum) {
+ return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getUpgrader(
+ slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
+ }
+
+}
+
+/******************************************
+ * EntityLockInfo (16 bytes)
+ * ****************************************
+ * short XCount : used to represent the count of X mode lock if it is allocated. Otherwise, it represents next free slot.
+ * short SCount
+ * int lastHolder
+ * int firstWaiter
+ * int upgrader : may exist only one since there are only S and X mode lock in Entity-level
+ *******************************************/
+
+class ChildEntityLockInfoArrayManager {
+ public static final int ENTITY_LOCK_INFO_SIZE = 16; //16bytes
+ public static final int NUM_OF_SLOTS = 1024; //number of entityLockInfos in a buffer
+ //public static final int NUM_OF_SLOTS = 10; //for unit test
+ public static final int BUFFER_SIZE = ENTITY_LOCK_INFO_SIZE * NUM_OF_SLOTS;
+
+ //byte offset of each field of EntityLockInfo
+ public static final int XCOUNT_OFFSET = 0;
+ public static final int SCOUNT_OFFSET = 2;
+ public static final int LAST_HOLDER_OFFSET = 4;
+ public static final int FIRST_WAITER_OFFSET = 8;
+ public static final int UPGRADER_OFFSET = 12;
+
+ //byte offset of nextFreeSlotNum which shares the same space with LastHolder field
+ //If a slot is in use, the space is used for LastHolder. Otherwise, it is used for nextFreeSlotNum.
+ public static final int NEXT_FREE_SLOT_OFFSET = 4;
+
+ private ByteBuffer buffer;
+ private int freeSlotNum;
+ private int occupiedSlots; //-1 represents 'deinitialized' state.
+
+ public ChildEntityLockInfoArrayManager() {
+ initialize();
+ }
+
+ public void initialize() {
+ this.buffer = ByteBuffer.allocate(BUFFER_SIZE);
+ this.freeSlotNum = 0;
+ this.occupiedSlots = 0;
+
+ for (int i = 0; i < NUM_OF_SLOTS - 1; i++) {
+ setNextFreeSlot(i, i + 1);
+ }
+ setNextFreeSlot(NUM_OF_SLOTS - 1, -1); //-1 represents EOL(end of link)
+ }
+
+ public int allocate() {
+ int currentSlot = freeSlotNum;
+ freeSlotNum = getNextFreeSlot(currentSlot);
+ //initialize values
+ setXCount(currentSlot, (short) 0);
+ setSCount(currentSlot, (short) 0);
+ setLastHolder(currentSlot, -1);
+ setFirstWaiter(currentSlot, -1);
+ setUpgrader(currentSlot, -1);
+ occupiedSlots++;
+ if (LockManager.IS_DEBUG_MODE) {
+ System.out.println(Thread.currentThread().getName() + " Allocated ELockInfo[" + currentSlot + "]");
+ }
+ return currentSlot;
+ }
+
+ public void deallocate(int slotNum) {
+ setNextFreeSlot(slotNum, freeSlotNum);
+ freeSlotNum = slotNum;
+ occupiedSlots--;
+ if (LockManager.IS_DEBUG_MODE) {
+ System.out.println(Thread.currentThread().getName() + " Deallocated ELockInfo[" + slotNum + "]");
+ }
+ }
+
+ public void deinitialize() {
+ buffer = null;
+ occupiedSlots = -1;
+ }
+
+ public boolean isDeinitialized() {
+ return occupiedSlots == -1;
+ }
+
+ public boolean isFull() {
+ return occupiedSlots == NUM_OF_SLOTS;
+ }
+
+ public boolean isEmpty() {
+ return occupiedSlots == 0;
+ }
+
+ public int getNumOfOccupiedSlots() {
+ return occupiedSlots;
+ }
+
+ public int getFreeSlotNum() {
+ return freeSlotNum;
+ }
+
+ //////////////////////////////////////////////////////////////////
+ // set/get method for each field of EntityLockInfo plus freeSlot
+ //////////////////////////////////////////////////////////////////
+
+ public void setNextFreeSlot(int slotNum, int nextFreeSlot) {
+ buffer.putInt(slotNum * ENTITY_LOCK_INFO_SIZE + NEXT_FREE_SLOT_OFFSET, nextFreeSlot);
+ }
+
+ public int getNextFreeSlot(int slotNum) {
+ return buffer.getInt(slotNum * ENTITY_LOCK_INFO_SIZE + NEXT_FREE_SLOT_OFFSET);
+ }
+
+ public void setXCount(int slotNum, short count) {
+ buffer.putShort(slotNum * ENTITY_LOCK_INFO_SIZE + XCOUNT_OFFSET, count);
+ }
+
+ public short getXCount(int slotNum) {
+ return buffer.getShort(slotNum * ENTITY_LOCK_INFO_SIZE + XCOUNT_OFFSET);
+ }
+
+ public void setSCount(int slotNum, short count) {
+ buffer.putShort(slotNum * ENTITY_LOCK_INFO_SIZE + SCOUNT_OFFSET, count);
+ }
+
+ public short getSCount(int slotNum) {
+ return buffer.getShort(slotNum * ENTITY_LOCK_INFO_SIZE + SCOUNT_OFFSET);
+ }
+
+ public void setLastHolder(int slotNum, int holder) {
+ buffer.putInt(slotNum * ENTITY_LOCK_INFO_SIZE + LAST_HOLDER_OFFSET, holder);
+ }
+
+ public int getLastHolder(int slotNum) {
+ return buffer.getInt(slotNum * ENTITY_LOCK_INFO_SIZE + LAST_HOLDER_OFFSET);
+ }
+
+ public void setFirstWaiter(int slotNum, int waiter) {
+ buffer.putInt(slotNum * ENTITY_LOCK_INFO_SIZE + FIRST_WAITER_OFFSET, waiter);
+ }
+
+ public int getFirstWaiter(int slotNum) {
+ return buffer.getInt(slotNum * ENTITY_LOCK_INFO_SIZE + FIRST_WAITER_OFFSET);
+ }
+
+ public void setUpgrader(int slotNum, int upgrader) {
+ buffer.putInt(slotNum * ENTITY_LOCK_INFO_SIZE + UPGRADER_OFFSET, upgrader);
+ }
+
+ public int getUpgrader(int slotNum) {
+ return buffer.getInt(slotNum * ENTITY_LOCK_INFO_SIZE + UPGRADER_OFFSET);
+ }
+}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/ILockManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/ILockManager.java
index b8c0e17..207ee58 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/ILockManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/ILockManager.java
@@ -15,12 +15,16 @@
package edu.uci.ics.asterix.transaction.management.service.locking;
import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
/**
- * @author pouria Interface for the lockManager
+ * Interface for the lockManager
+ *
+ * @author pouria
+ * @author kisskys
+ *
*/
-
public interface ILockManager {
/**
@@ -36,13 +40,13 @@
* has a "weaker" lock, then the request would be interpreted as a convert
* request
* Waiting transaction would eventually garb the lock, or get timed-out
- *
- * @param resourceID
- * @param mode
- * @return
+ * @param datasetId
+ * @param entityHashValue
+ * @param lockMode
+ * @param txnContext
* @throws ACIDException
*/
- public boolean lock(TransactionContext context, byte[] resourceID, int mode) throws ACIDException;
+ public void lock(DatasetId datasetId, int entityHashValue, byte lockMode, TransactionContext txnContext) throws ACIDException;
/**
* The method releases "All" the locks taken/waiting-on by a specific
@@ -50,58 +54,63 @@
* potential waiters, which can be waken up based on their requested lock
* mode and the waiting policy would be waken up
*
- * @param context
- * @return
+ * @param txnContext
* @throws ACIDException
*/
- public Boolean releaseLocks(TransactionContext context) throws ACIDException;
+ public void releaseLocks(TransactionContext txnContext) throws ACIDException;
/**
- * Releases "All" the locks by a transaction on a "single specific" resource
- * Upon releasing, potential waiters, which can be waken up based on their
- * requested lock mode and the waiting policy would be waken up
*
- * @param resourceID
- * @return
- * @throws ACIDException
+ * @param datasetId
+ * @param entityHashValue
+ * @param txnContext
+ * @throws ACIDException TODO
*/
- public boolean unlock(TransactionContext context, byte[] resourceID) throws ACIDException;
-
- /**
- * Request to convert granted lockMode of a transaction on a specific
- * resource. Requesting transaction would either grab the lock, or sent to
- * waiting based on the type of the request, and current mask on the
- * resource and possible set of waiting converters
- * - If the transaction does not have any lock on the resource, then an
- * exception is thrown - If the transaction already has a stronger lock,
- * then no operation is taken
- *
- * @param context
- * @param resourceID
- * @param mode
- * @return
- * @throws ACIDException
- */
- public boolean convertLock(TransactionContext context, byte[] resourceID, int mode) throws ACIDException;
+ public void unlock(DatasetId datasetId, int entityHashValue, TransactionContext txnContext) throws ACIDException;
/**
* Call to lock and unlock a specific resource in a specific lock mode
- *
+ * @param datasetId
+ * @param entityHashValue
+ * @param lockMode TODO
* @param context
- * @param resourceID
- * @param mode
- * @param timeout
+ *
* @return
* @throws ACIDException
*/
- public boolean getInstantlock(TransactionContext context, byte[] resourceID, int mode) throws ACIDException;
+ public void instantLock(DatasetId datasetId, int entityHashValue, byte lockMode, TransactionContext context) throws ACIDException;
+
+ /**
+ *
+ * @param datasetId
+ * @param entityHashValue
+ * @param lockMode
+ * @param context
+ * @return
+ * @throws ACIDException
+ */
+ public boolean tryLock(DatasetId datasetId, int entityHashValue, byte lockMode, TransactionContext context) throws ACIDException;
+
+ /**
+ *
+ * @param datasetId
+ * @param entityHashValue
+ * @param lockMode
+ * @param txnContext
+ * @return
+ * @throws ACIDException
+ */
+ boolean instantTryLock(DatasetId datasetId, int entityHashValue, byte lockMode, TransactionContext txnContext)
+ throws ACIDException;
/**
* Prints out the contents of the transactions' table in a readable fashion
*
* @return
* @throws ACIDException
*/
- public String getDebugLockStatus() throws ACIDException;
+ public String prettyPrint() throws ACIDException;
+
+
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/JobInfo.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/JobInfo.java
new file mode 100644
index 0000000..1a3ac03
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/JobInfo.java
@@ -0,0 +1,292 @@
+package edu.uci.ics.asterix.transaction.management.service.locking;
+
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
+
+public class JobInfo {
+ private EntityInfoManager entityInfoManager;
+ private LockWaiterManager lockWaiterManager;
+ private TransactionContext jobCtx;
+ private int lastHoldingResource; //resource(entity or dataset) which is held by this job lastly
+ private int firstWaitingResource; //resource(entity or dataset) which this job is waiting for
+ private int upgradingResource; //resource(entity or dataset) which this job is waiting for to upgrade
+
+ //private PrimitiveIntHashMap dLockHT; //used for keeping dataset-granule-lock's count acquired by this job.
+
+ public JobInfo(EntityInfoManager entityInfoManager, LockWaiterManager lockWaiterManager, TransactionContext txnCtx) {
+ this.entityInfoManager = entityInfoManager;
+ this.lockWaiterManager = lockWaiterManager;
+ this.jobCtx = txnCtx;
+ this.lastHoldingResource = -1;
+ this.firstWaitingResource = -1;
+ this.upgradingResource = -1;
+ //this.dLockHT = new PrimitiveIntHashMap(1<<6, 1<<3, 180000);
+ }
+
+ public void addHoldingResource(int resource) {
+
+ if (LockManager.IS_DEBUG_MODE) {
+ if (entityInfoManager.getJobId(resource) != jobCtx.getJobId().getId()) {
+ throw new IllegalStateException("JobInfo(" + jobCtx.getJobId().getId() + ") has diffrent Job(JID:"
+ + entityInfoManager.getJobId(resource) + "'s resource!!!");
+ }
+ //System.out.println(Thread.currentThread().getName()+"\tJobInfo_AddHolder:"+ resource);
+ }
+
+ if (lastHoldingResource != -1) {
+ entityInfoManager.setNextJobResource(lastHoldingResource, resource);
+ }
+ entityInfoManager.setPrevJobResource(resource, lastHoldingResource);
+ entityInfoManager.setNextJobResource(resource, -1);
+ lastHoldingResource = resource;
+
+ //increaseDatasetLockCount(resource);
+ }
+
+ public void removeHoldingResource(int resource) {
+ int current = lastHoldingResource;
+ int prev;
+ int next;
+
+ if (LockManager.IS_DEBUG_MODE) {
+ if (entityInfoManager.getJobId(resource) != jobCtx.getJobId().getId()) {
+ throw new IllegalStateException("JobInfo(" + jobCtx.getJobId().getId() + ") has diffrent Job(JID:"
+ + entityInfoManager.getJobId(resource) + "'s resource!!!");
+ }
+ //System.out.println(Thread.currentThread().getName()+"\tJobInfo_RemoveHolder:"+ resource);
+ }
+
+ while (current != resource) {
+
+ if (LockManager.IS_DEBUG_MODE) {
+ if (current == -1) {
+ //shouldn't occur: debugging purpose
+ try {
+ throw new Exception();
+ } catch (Exception e) {
+ // TODO Auto-generated catch block
+ e.printStackTrace();
+ }
+ }
+ }
+
+ current = entityInfoManager.getPrevJobResource(current);
+ }
+
+ prev = entityInfoManager.getPrevJobResource(current);
+ next = entityInfoManager.getNextJobResource(current);
+ //update prev->next = next
+ if (prev != -1) {
+ entityInfoManager.setNextJobResource(prev, next);
+ }
+ if (next != -1) {
+ entityInfoManager.setPrevJobResource(next, prev);
+ }
+ if (lastHoldingResource == resource) {
+ lastHoldingResource = prev;
+ }
+
+ //decreaseDatasetLockCount(resource);
+ }
+
+ public void addWaitingResource(int waiterObjId) {
+ int lastObjId;
+ LockWaiter lastObj = null;
+
+ if (firstWaitingResource != -1) {
+ //find the lastWaiter
+ lastObjId = firstWaitingResource;
+ while (lastObjId != -1) {
+ lastObj = lockWaiterManager.getLockWaiter(lastObjId);
+ if (LockManager.IS_DEBUG_MODE) {
+ int entityInfo = lastObj.getEntityInfoSlot();
+ if (entityInfoManager.getJobId(entityInfo) != jobCtx.getJobId().getId()) {
+ throw new IllegalStateException("JobInfo(" + jobCtx.getJobId().getId()
+ + ") has diffrent Job(JID:" + entityInfoManager.getJobId(entityInfo) + "'s resource!!!");
+ }
+ }
+ lastObjId = lastObj.getNextWaitingResourceObjId();
+ }
+ //last->next = new_waiter
+ lastObj.setNextWaitingResourceObjId(waiterObjId);
+ } else {
+ firstWaitingResource = waiterObjId;
+ }
+ //new_waiter->next = -1
+ lastObj = lockWaiterManager.getLockWaiter(waiterObjId);
+ if (LockManager.IS_DEBUG_MODE) {
+ int entityInfo = lastObj.getEntityInfoSlot();
+ if (entityInfoManager.getJobId(entityInfo) != jobCtx.getJobId().getId()) {
+ throw new IllegalStateException("JobInfo(" + jobCtx.getJobId().getId() + ") has diffrent Job(JID:"
+ + entityInfoManager.getJobId(entityInfo) + "'s resource!!!");
+ }
+ }
+ lastObj.setNextWaitingResourceObjId(-1);
+
+ // if (LockManager.IS_DEBUG_MODE) {
+ // System.out.println(Thread.currentThread().getName()+"\tJobInfo_AddWaiter:"+ waiterObjId + ", FirstWaiter:"+firstWaitingResource);
+ // }
+ }
+
+ public void removeWaitingResource(int waiterObjId) {
+ int currentObjId = firstWaitingResource;
+ LockWaiter currentObj;
+ LockWaiter prevObj = null;
+ int prevObjId = -1;
+ int nextObjId;
+
+ while (currentObjId != waiterObjId) {
+
+ if (LockManager.IS_DEBUG_MODE) {
+ if (currentObjId == -1) {
+ //shouldn't occur: debugging purpose
+ try {
+ throw new Exception();
+ } catch (Exception e) {
+ // TODO Auto-generated catch block
+ e.printStackTrace();
+ }
+ }
+ }
+
+ prevObj = lockWaiterManager.getLockWaiter(currentObjId);
+ prevObjId = currentObjId;
+ currentObjId = prevObj.getNextWaitingResourceObjId();
+ }
+
+ //get current waiter object
+ currentObj = lockWaiterManager.getLockWaiter(currentObjId);
+
+ if (LockManager.IS_DEBUG_MODE) {
+ int entityInfo = currentObj.getEntityInfoSlot();
+ if (entityInfoManager.getJobId(entityInfo) != jobCtx.getJobId().getId()) {
+ throw new IllegalStateException("JobInfo(" + jobCtx.getJobId().getId() + ") has diffrent Job(JID:"
+ + entityInfoManager.getJobId(entityInfo) + "'s resource!!!");
+ }
+ }
+
+ //get next waiterObjId
+ nextObjId = currentObj.getNextWaitingResourceObjId();
+
+ if (prevObjId != -1) {
+ //prev->next = next
+ prevObj.setNextWaitingResourceObjId(nextObjId);
+ } else {
+ //removed first waiter. firstWaiter = current->next
+ firstWaitingResource = nextObjId;
+ }
+
+ // if (LockManager.IS_DEBUG_MODE) {
+ // System.out.println(Thread.currentThread().getName()+"\tJobInfo_RemoveWaiter:"+ waiterObjId + ", FirstWaiter:"+firstWaitingResource);
+ // }
+ }
+
+ /**********************************************************************************
+ * public void increaseDatasetLockCount(int entityInfo) {
+ * int datasetId = entityInfoManager.getDatasetId(entityInfo);
+ * int count = dLockHT.get(datasetId);
+ * if (count == -1) {
+ * dLockHT.upsert(datasetId, 1);
+ * } else {
+ * dLockHT.upsert(datasetId, count+1);
+ * }
+ * }
+ * public void decreaseDatasetLockCount(int entityInfo) {
+ * int datasetId = entityInfoManager.getDatasetId(entityInfo);
+ * int count = dLockHT.get(datasetId);
+ * if (count > 1) {
+ * dLockHT.upsert(datasetId, count-1);
+ * } else if (count == 1) {
+ * dLockHT.remove(datasetId);
+ * } else if (count <= 0 ) {
+ * throw new IllegalStateException("Illegal state of datasetLock count in JobInfo's dLockHT");
+ * }
+ * }
+ * public boolean isDatasetLockGranted(int datasetId) {
+ * return dLockHT.get(datasetId) == -1 ? false : true;
+ * }
+ **********************************************************************************/
+
+ public boolean isDatasetLockGranted(int datasetId, byte lockMode) {
+ int entityInfo = lastHoldingResource;
+ byte datasetLockMode;
+
+ while (entityInfo != -1) {
+ datasetLockMode = entityInfoManager.getDatasetLockMode(entityInfo);
+ datasetLockMode = entityInfoManager.getPKHashVal(entityInfo) == -1 ? datasetLockMode
+ : datasetLockMode == LockMode.S ? LockMode.IS : LockMode.IX;
+ if (entityInfoManager.getDatasetId(entityInfo) == datasetId
+ && isStrongerOrEqualToLockMode(datasetLockMode, lockMode)) {
+ return true;
+ }
+ entityInfo = entityInfoManager.getPrevJobResource(entityInfo);
+ }
+ return false;
+ }
+
+ //check whether LockMode modeA is stronger than or equal to LockMode modeB
+ private boolean isStrongerOrEqualToLockMode(byte modeA, byte modeB) {
+ switch (modeB) {
+ case LockMode.X:
+ return modeA == LockMode.X;
+
+ case LockMode.IX:
+ return modeA == LockMode.IX || modeA == LockMode.X;
+
+ case LockMode.S:
+ return modeA == LockMode.S || modeA == LockMode.X;
+
+ case LockMode.IS:
+ return true;
+
+ default:
+ throw new IllegalStateException("Unsupported dataset lock mode.");
+ }
+ }
+
+ public String printHoldingResource () {
+ StringBuilder s = new StringBuilder();
+ int entityInfo = lastHoldingResource;
+
+ while (entityInfo != -1) {
+ s.append("entityInfo[").append(entityInfo).append("] ");
+ s.append(entityInfoManager.getJobId(entityInfo)).append(" ");
+ s.append(entityInfoManager.getDatasetId(entityInfo)).append(" ");
+ s.append(entityInfoManager.getPKHashVal(entityInfo)).append(" ");
+ s.append(entityInfoManager.getDatasetLockMode(entityInfo)).append(" ");
+ s.append(entityInfoManager.getDatasetLockCount(entityInfo)).append(" ");
+ s.append(entityInfoManager.getEntityLockCount(entityInfo)).append(" ");
+ s.append(entityInfoManager.getEntityLockMode(entityInfo)).append(" ");
+ s.append("\n");
+ entityInfo = entityInfoManager.getPrevJobResource(entityInfo);
+ }
+ return s.toString();
+ }
+
+ /////////////////////////////////////////////////////////
+ // set/get method for private variable
+ /////////////////////////////////////////////////////////
+ public void setlastHoldingResource(int resource) {
+ lastHoldingResource = resource;
+ }
+
+ public int getLastHoldingResource() {
+ return lastHoldingResource;
+ }
+
+ public void setFirstWaitingResource(int resource) {
+ firstWaitingResource = resource;
+ }
+
+ public int getFirstWaitingResource() {
+ return firstWaitingResource;
+ }
+
+ public void setUpgradingResource(int resource) {
+ upgradingResource = resource;
+ }
+
+ public int getUpgradingResource() {
+ return upgradingResource;
+ }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockInfo.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockInfo.java
deleted file mode 100644
index f3bd6f3..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockInfo.java
+++ /dev/null
@@ -1,621 +0,0 @@
-package edu.uci.ics.asterix.transaction.management.service.locking;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-
-/**
- * @author pouria An instance shows information on a "single resource" about
- * 1) current granted locks on the resource to all transactions 2) locks
- * that are being waiting on by all converting transactions 3) locks
- * that are being waiting on by all regular waiting transactions
- * Each lock mode is interpreted as an integer, and it has a
- * corresponding bit in the mask variable mask variable should be
- * interpreted as a sequence of bits, where the i-th bit is 1, if and
- * only if some transaction(s) have a lock of mode i on this resource
- * counter is an array which has an entry for each lock mode, and its
- * i-th entry shows the total number of locks of mode i, granted to all
- * transactions
- */
-
-public class LockInfo {
- final int NUM_OF_LOCK_MODES = 32;
- final int TX_ARRAY_SIZE = 50;
- final int EOL = -1;
-
- public static final int NOT_FOUND = -2;
- public static final int UNKNOWN_IX = -3;
- public static final int ANY_LOCK_MODE = -4;
- public static final int UNKNOWN_LOCK_MODE = -5;
-
- private int mask; // i-th bit corresponds to the i-th lock mode
- private int[] counter; // i-th entry shows total num of granted locks of
- // mode i
-
- private ArrayList<Integer> grantedList; // (contains index of entries, in
- // the txId, mode and counter lists)
- private ArrayList<WaitingInfo> convertList; // Waiting Converters
- private ArrayList<WaitingInfo> waitList; // Regular Waiters
-
- int nextFreeIx; // Head of free entries lists
- private ArrayList<long[]> txIdList; // i-th entry shows the id of the
- // granted/waiting transactions
- private ArrayList<int[]> modeList; // i-th entry shows the mode of the
- // granted/waiting-on lock
- private ArrayList<int[]> counterList; // i-th entry shows the number of
- // locks (with the defined mode) a
- // transaction has taken (In a free
- // entry is used as the next ptr (next
- // free entry))
-
- public LockInfo() {
- this.mask = 0;
- this.counter = new int[NUM_OF_LOCK_MODES];
- this.grantedList = new ArrayList<Integer>();
- this.waitList = new ArrayList<WaitingInfo>();
- this.convertList = new ArrayList<WaitingInfo>();
- nextFreeIx = 0;
- this.txIdList = new ArrayList<long[]>();
- txIdList.add(new long[TX_ARRAY_SIZE]);
- this.modeList = new ArrayList<int[]>();
- modeList.add(new int[TX_ARRAY_SIZE]);
- this.counterList = new ArrayList<int[]>();
- counterList.add(initArray(0));
- }
-
- private int[] initArray(int ixToStart) {
- int[] n = new int[TX_ARRAY_SIZE];
- for (int i = 0; i < TX_ARRAY_SIZE - 1; i++) { // Initializing a new set
- // of entries, attaching
- // them
- n[i] = (++ixToStart); // to the chain of free entries
- }
- n[TX_ARRAY_SIZE - 1] = EOL;
- return n;
- }
-
- /**
- * @param txId
- * @param lMode
- * @return the index of the entry corresponding to the transaction with the
- * specified granted lock
- * @throws ACIDException
- */
- public int findInGrantedList(long txId, int lMode) throws ACIDException {
- for (int i : grantedList) {
- if ((getTxId(i) == txId) && ((lMode == ANY_LOCK_MODE) || (lMode == getLockMode(i)))) {
- return i;
- }
- }
- return NOT_FOUND;
- }
-
- /**
- * @param txId
- * @param lMode
- * @return the index of the entry corresponding to the transaction which is
- * waiting (as a converter) for the specified lock
- * @throws ACIDException
- */
- public int findInConvertList(long txId, int lMode) throws ACIDException {
- for (WaitingInfo wi : convertList) {
- int i = wi.getWaitingEntry().getIX();
- if ((getTxId(i) == txId) && ((lMode == ANY_LOCK_MODE) || (lMode == getLockMode(i)))) {
- return i;
- }
- }
- return NOT_FOUND;
- }
-
- /**
- * @param txId
- * @param lMode
- * @return the index of the entry corresponding to the transaction which is
- * waiting (as a regular waiter) for the specified lock
- * @throws ACIDException
- */
- public int findInWaitList(long txId, int lMode) throws ACIDException {
- for (WaitingInfo wi : waitList) {
- int i = wi.getWaitingEntry().getIX();
- if ((getTxId(i) == txId) && ((lMode == ANY_LOCK_MODE) || (lMode == getLockMode(i)))) {
- return i;
- }
- }
- return NOT_FOUND;
- }
-
- /**
- * @param txId
- * @param lMode
- * @return the object, on which the specified transaction is waiting for the
- * specified lock
- * @throws ACIDException
- */
- public WaitingInfo getWaitingOnObject(long txId, int lMode) throws ACIDException {
- WaitingInfo wObj = null;
- Iterator<WaitingInfo> cIt = convertList.iterator();
- while (cIt.hasNext()) {
- wObj = cIt.next();
- int ix = wObj.getWaitingEntry().getIX();
- if ((getTxId(ix) == txId) && ((lMode == ANY_LOCK_MODE) || (lMode == getLockMode(ix)))) {
- return wObj;
- }
- }
-
- Iterator<WaitingInfo> wIt = waitList.iterator();
- while (wIt.hasNext()) {
- wObj = wIt.next();
- int ix = wObj.getWaitingEntry().getIX();
- if ((getTxId(ix) == txId) && ((lMode == ANY_LOCK_MODE) || (lMode == getLockMode(ix)))) {
- return wObj;
- }
- }
- throw new ACIDException("Waiting Entry for transaction " + txId + " Could not be found");
- }
-
- public Iterator<WaitingInfo> getIteratorOnConverter() {
- return (convertList.iterator());
- }
-
- public Iterator<WaitingInfo> getIteratorOnWaiters() {
- return (waitList.iterator());
- }
-
- /**
- * @param txId
- * @param lMode
- * @param eix
- * index of the entry corresponding to the transaction, its
- * granted lock and its counter
- * @throws ACIDException
- */
- public void addToGranted(long txId, int lMode, int eix) throws ACIDException {
- if (eix == UNKNOWN_IX) {
- eix = findInGrantedList(txId, lMode);
- }
- if (eix == NOT_FOUND) { // new lock of mode lMode for Txr
- int ix = allocateEntryForRequest();
- grantedList.add(ix);
- setTxId(txId, ix);
- setLockMode(lMode, ix);
- setReqCount(1, ix);
- mask |= (0x01 << lMode);
- } else { // Redundant lock of mode lMode for Txr
- incReqCount(eix);
- }
- counter[lMode]++;
- }
-
- /**
- * @param txId
- * @param lMode
- * @param eix
- * index of the entry corresponding to the transaction, its
- * granted lock and its counter
- * @throws ACIDException
- */
- public void removeFromGranted(long txId, int lMode, int eix) throws ACIDException {
- removeFromGranted(txId, lMode, true, eix);
- }
-
- /**
- * @param txId
- * @param lMode
- * @param forced
- * whether to remove all the locks, with the given mode, grabbed
- * by the transaction or consider the counter (removing just one
- * lock in case the transaction has several locks with the
- * specified mode)
- * @param eix
- * index of the entry corresponding to the transaction, its
- * granted lock and its counter
- * @throws ACIDException
- */
- private void removeFromGranted(long txId, int lMode, boolean forced, int eix) throws ACIDException {
- if (eix == UNKNOWN_IX) {
- eix = findInGrantedList(txId, lMode);
- if (eix == NOT_FOUND) {
- return;
- }
- }
-
- if (lMode == ANY_LOCK_MODE) {
- lMode = getLockMode(eix);
- }
-
- int count = getReqCount(eix);
- if (!forced) {
- if (count > 1) {
- setReqCount((count - 1), eix);
- counter[lMode]--;
- return;
- }
- }
- // forced or count is 1
- grantedList.remove((new Integer(eix)));
- freeEntry(eix);
- counter[lMode] -= count;
- if (counter[lMode] == 0) { // No one else has lock with this mode
- mask &= (~(0x00 | (0x01 << lMode)));
- }
- }
-
- /**
- * @param txId
- * @param lMode
- * @param entry
- * the object, specified transaction is going to wait on
- * @throws ACIDException
- */
- public void addToConvert(long txId, int lMode, WaitEntry entry) throws ACIDException {
- int eix = findInConvertList(txId, lMode);
- if (eix == NOT_FOUND) {
- int ix = allocateEntryForRequest();
- entry.setIx(ix);
- entry.setForWait();
- convertList.add(new WaitingInfo(entry));
- setTxId(txId, ix);
- setLockMode(lMode, ix);
- setReqCount(1, ix);
- } else {
- throw new ACIDException("Adding an already existing converter");
- }
- }
-
- /**
- * @param txId
- * @param lMode
- * @param eix
- * index of the entry corresponding to the transaction in the
- * converters list
- * @throws ACIDException
- */
- public void prepareToRemoveFromConverters(long txId, int lMode, int eix) throws ACIDException {
- prepareToRemoveFromConverters(txId, lMode, true, eix);
- }
-
- /**
- * @param txId
- * @param lMode
- * @param forced
- * whether to ignore the counter and remove the transaction from
- * the converters list or consider the request counter
- * @param eix
- * index of the entry corresponding to the transaction in the
- * converters list
- * @throws ACIDException
- */
- private void prepareToRemoveFromConverters(long txId, int lMode, boolean forced, int eix) throws ACIDException {
- if (eix == UNKNOWN_IX) {
- eix = findInConvertList(txId, lMode);
- if (eix == NOT_FOUND) {
- throw new ACIDException("Lock entry not found in the waiting list");
- }
- }
- freeEntry(eix);
- }
-
- /**
- * @param txId
- * @return the object specified transaction is waiting on for conversion
- * @throws ACIDException
- */
- public WaitEntry removeFromConverters(long txId) throws ACIDException {
- Iterator<WaitingInfo> it = convertList.iterator();
- while (it.hasNext()) {
- WaitingInfo next = it.next();
- if (getTxId(next.getWaitingEntry().getIX()) == txId) {
- it.remove();
- return next.getWaitingEntry();
- }
- }
- return null;
- }
-
- /**
- * @param txId
- * @param lMode
- * @param entry
- * @throws ACIDException
- */
- public void addToWaiters(long txId, int lMode, WaitEntry entry) throws ACIDException {
- int ix = allocateEntryForRequest();
- entry.setIx(ix);
- entry.setForWait();
- waitList.add(new WaitingInfo(entry));
- setTxId(txId, ix);
- setLockMode(lMode, ix);
- setReqCount(1, ix);
- }
-
- public void prepareToRemoveFromWaiters(long txId, int lMode, int eix) throws ACIDException {
- prepareToRemoveFromWaiters(txId, lMode, true, eix);
- }
-
- /**
- * Removes and recycles the entry containing the information about the
- * transaction, its lock mode and the counter
- *
- * @param txId
- * @param lMode
- * @param forced
- * @param eix
- * index of the entry, needs to be freed
- * @throws ACIDException
- */
- private void prepareToRemoveFromWaiters(long txId, int lMode, boolean forced, int eix) throws ACIDException {
- if (eix == UNKNOWN_IX) {
- eix = findInWaitList(txId, lMode);
- if (eix == NOT_FOUND) {
- throw new ACIDException("Lock entry not found in the waiting list");
- }
- }
- freeEntry(eix);
- }
-
- /**
- * @param txId
- * @return the object the transaction is waiting on (as a regular waiter)
- * @throws ACIDException
- */
- public WaitEntry removeFromWaiters(long txId) throws ACIDException {
- Iterator<WaitingInfo> it = waitList.iterator();
- while (it.hasNext()) {
- WaitingInfo next = it.next();
- if (getTxId(next.getWaitingEntry().getIX()) == txId) {
- it.remove();
- return next.getWaitingEntry();
- }
- }
- return null;
- }
-
- /**
- * @param lMode
- * @param eix
- * index of the entry corresponding to the transaction's lock and
- * its counter
- */
- public void grantRedundantLock(int lMode, int eix) {
- incReqCount(eix);
- counter[lMode]++;
- }
-
- /**
- * @param txId
- * @param eix
- * index of the entry corresponding to the transaction
- * @return the actual lock mode, granted to the specified transaction
- * @throws ACIDException
- */
- public int getGrantedLockMode(long txId, int eix) throws ACIDException {
- if (eix != UNKNOWN_IX) {
- return getLockMode(eix);
- }
- int ix = findInGrantedList(txId, ANY_LOCK_MODE);
- if (ix == NOT_FOUND) {
- return UNKNOWN_LOCK_MODE;
- }
- return getLockMode(ix);
- }
-
- /**
- * @param txId
- * @param eix
- * index of the entry corresponding to the transaction
- * @return the actual lock mode, the specified transaction is waiting to
- * convert to
- * @throws ACIDException
- */
- public int getConvertLockMode(long txId, int eix) throws ACIDException {
- if (eix != UNKNOWN_IX) {
- return getLockMode(eix);
- }
- int ix = findInConvertList(txId, ANY_LOCK_MODE);
- if (ix == NOT_FOUND) {
- return UNKNOWN_LOCK_MODE;
- }
- return getLockMode(ix);
- }
-
- /**
- * @param txId
- * @param eix
- * index of the entry corresponding to the transaction
- * @return the actual lock mode, the specified transaction is waiting to
- * grab
- * @throws ACIDException
- */
- public int getWaitLockMode(long txId, int eix) throws ACIDException {
- if (eix != UNKNOWN_IX) {
- return getLockMode(eix);
- }
- int ix = findInWaitList(txId, ANY_LOCK_MODE);
- if (ix == NOT_FOUND) {
- return UNKNOWN_LOCK_MODE;
- }
- return getLockMode(ix);
- }
-
- public boolean isConvertListEmpty() {
- return (!(convertList.size() > 0));
- }
-
- public int getMask() {
- return mask;
- }
-
- /**
- * @param txId
- * @param lMode
- * @param eix
- * index of the entry corresponding to the transaction's
- * currently grabbed lock
- * @return the updated as if the granted lock to the specified transaction
- * gets removed from it (Mainly used to exclude self-conflicts when
- * checking for conversions)
- * @throws ACIDException
- */
- public int getUpdatedMask(long txId, int lMode, int eix) throws ACIDException {
- if (eix == UNKNOWN_IX) {
- eix = findInGrantedList(txId, lMode);
- }
- if (eix == NOT_FOUND) {
- return mask;
- }
-
- int txCount = getReqCount(eix);
- int totalCount = getLockAggCounter(lMode);
-
- if (totalCount == txCount) { // txId is the only lock-holder with this
- // mode
- return (mask & (~(0x00 | (0x01 << lMode))));
- }
-
- return mask;
- }
-
- /**
- * @param lmix
- * @return the total number of locks of the specified mode, grabbed on this
- * resource (by all transactions)
- */
- private int getLockAggCounter(int lmix) {
- return counter[lmix];
- }
-
- /**
- * Populates the grantedIDs list with the ids of all transactions in the
- * granted list
- *
- * @param grantedIDs
- */
- public void getGrantedListTxIDs(ArrayList<Long> grantedIDs) {
- Iterator<Integer> gIt = grantedList.iterator();
- while (gIt.hasNext()) {
- grantedIDs.add(getTxId(gIt.next()));
- }
- }
-
- /**
- * @return the index of an entry that can be used to capture one transaction
- * and its requested/granted lock mode and the counter
- */
- private int allocateEntryForRequest() {
- if (nextFreeIx == EOL) {
- nextFreeIx = txIdList.size() * TX_ARRAY_SIZE;
- txIdList.add(new long[TX_ARRAY_SIZE]);
- modeList.add(new int[TX_ARRAY_SIZE]);
- counterList.add(initArray(nextFreeIx));
- }
- int ixToRet = nextFreeIx;
- nextFreeIx = getReqCount(nextFreeIx);
- return ixToRet;
- }
-
- /**
- * @param ix
- * index of the entry, to be recycled
- */
- private void freeEntry(int ix) {
- setReqCount(nextFreeIx, ix); // counter holds ptr to next free entry in
- // free entries
- nextFreeIx = ix;
- }
-
- /**
- * @param ix
- * index of the entry that captures the transaction id
- * @return id of the transaction whose info is captured in the specified
- * index
- */
- public long getTxId(int ix) {
- return (txIdList.get(ix / TX_ARRAY_SIZE)[ix % TX_ARRAY_SIZE]);
- }
-
- /**
- * @param txId
- * @param ix
- * index of the entry that will capture the transaction id
- */
- private void setTxId(long txId, int ix) {
- txIdList.get(ix / TX_ARRAY_SIZE)[ix % TX_ARRAY_SIZE] = txId;
- }
-
- /**
- * @param ix
- * index of the entry that captures the lock mode
- * requested/grabbed by the specified transaction
- * @return
- */
- public int getLockMode(int ix) {
- return (modeList.get(ix / TX_ARRAY_SIZE)[ix % TX_ARRAY_SIZE]);
- }
-
- /**
- * @param lMode
- * @param index
- * of the entry that will capture the lock mode requested/grabbed
- * by the specified transaction
- */
- private void setLockMode(int lMode, int ix) {
- modeList.get(ix / TX_ARRAY_SIZE)[ix % TX_ARRAY_SIZE] = lMode;
- }
-
- /**
- * @param ix
- * @return index of the entry that captures the counter of locks
- */
- public int getReqCount(int ix) {
- return (counterList.get(ix / TX_ARRAY_SIZE)[ix % TX_ARRAY_SIZE]);
- }
-
- /**
- * @param count
- * @param ix
- * index of the entry that captures the counter of locks
- */
- private void setReqCount(int count, int ix) {
- counterList.get(ix / TX_ARRAY_SIZE)[ix % TX_ARRAY_SIZE] = count;
- }
-
- /**
- * @param ix
- * index of the counter, needed to be incremented on behalf of a
- * transaction
- */
- private void incReqCount(int ix) {
- counterList.get(ix / TX_ARRAY_SIZE)[ix % TX_ARRAY_SIZE]++;
- }
-}
-
-class WaitingInfo {
- /**
- * An object of this class captures the information corresponding to a
- * regular or converter waiter
- */
-
- private boolean isVictim; // Whether the corresponding transaction is an
- // Victim or it can be waken up safely
- private WaitEntry waitEntry; // The object, on which the waiter is waiting.
- // This object is mainly used to notify the
- // waiter, to be waken up
-
- public WaitingInfo(WaitEntry waitEntry) {
- this.waitEntry = waitEntry;
- this.isVictim = false;
- }
-
- public boolean isVictim() {
- return isVictim;
- }
-
- public void setAsVictim() {
- this.isVictim = true;
- }
-
- public WaitEntry getWaitingEntry() {
- return this.waitEntry;
- }
-}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java
index 22cab54..0d2f3f6 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java
@@ -1,982 +1,1698 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package edu.uci.ics.asterix.transaction.management.service.locking;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Hashtable;
+import java.io.Serializable;
+import java.util.HashMap;
import java.util.Iterator;
-import java.util.Properties;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.logging.Level;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.logging.Logger;
import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
+import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionManager.TransactionState;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
/**
- * @author pouria An implementation of the ILockManager interface for the
- * specific case of locking protocol with two lock modes: (S) and (X),
- * where S lock mode is shown by 0, and X lock mode is shown by 1.
- * @see ILockManager, DeadlockDetector, TimeOutDetector, ILockMatrix,
- * LockMatrix, TxrInfo and LockInfo
+ * An implementation of the ILockManager interface for the
+ * specific case of locking protocol with two lock modes: (S) and (X),
+ * where S lock mode is shown by 0, and X lock mode is shown by 1.
+ *
+ * @author pouria, kisskys
*/
public class LockManager implements ILockManager {
private static final Logger LOGGER = Logger.getLogger(LockManager.class.getName());
+ private static final int LOCK_MANAGER_INITIAL_HASH_TABLE_SIZE = 50;// do we need this?
+ public static final boolean IS_DEBUG_MODE = false;//true
- final int INIT_TABLE_SIZE = 50;
- private LMTables lmTables;
- ILockMatrix lMtx;
+ private TransactionProvider txnProvider;
- WaitObjectManager woManager;
- TimeOutDetector toutDetector;
- DeadlockDetector deadlockDetector;
+ //all threads accessing to LockManager's tables such as jobHT and datasetResourceHT
+ //are serialized through LockTableLatch. All threads waiting the latch will be fairly served
+ //in FIFO manner when the latch is available.
+ private final ReadWriteLock lockTableLatch;
+ private final ReadWriteLock waiterLatch;
+ private HashMap<JobId, JobInfo> jobHT;
+ private HashMap<DatasetId, DatasetLockInfo> datasetResourceHT;
- public LockManager(TransactionProvider factory) throws ACIDException {
- Properties p = new Properties();
- InputStream is = null;
- ILockMatrix lockMatrix = null;
- int[] confTab = null;
- int[] convTab = null;
+ private EntityLockInfoManager entityLockInfoManager;
+ private EntityInfoManager entityInfoManager;
+ private LockWaiterManager lockWaiterManager;
- try {
- File file = new File(TransactionManagementConstants.LockManagerConstants.LOCK_CONF_DIR + File.separator
- + TransactionManagementConstants.LockManagerConstants.LOCK_CONF_FILE);
- if (file.exists()) {
- is = new FileInputStream(TransactionManagementConstants.LockManagerConstants.LOCK_CONF_DIR
- + File.separator + TransactionManagementConstants.LockManagerConstants.LOCK_CONF_FILE);
- p.load(is);
- confTab = getConfTab(p);
- convTab = getConvTab(p);
- } else {
- if (LOGGER.isLoggable(Level.SEVERE)) {
- LOGGER.severe("Lock configuration file not found, using defaults !");
- }
- confTab = TransactionManagementConstants.LockManagerConstants.LOCK_CONFLICT_MATRIX;
- convTab = TransactionManagementConstants.LockManagerConstants.LOCK_CONVERT_MATRIX;
- }
- lockMatrix = new LockMatrix(confTab, convTab);
- initialize(lockMatrix);
- } catch (IOException ioe) {
- if (is != null) {
- try {
- is.close();
- } catch (IOException e) {
- throw new ACIDException("unable to close input stream ", e);
- }
- }
- throw new ACIDException(" unable to create LockManager", ioe);
- }
- }
+ private DeadlockDetector deadlockDetector;
+ private TimeOutDetector toutDetector;
+ private DatasetId tempDatasetIdObj; //temporary object to avoid object creation
- private static int[] getConfTab(Properties properties) {
- return null;
- }
+ private int tryLockDatasetGranuleRevertOperation;
- private static int[] getConvTab(Properties properties) {
- return null;
- }
+ private LockRequestTracker lockRequestTracker; //for debugging
+ private ConsecutiveWakeupContext consecutiveWakeupContext;
- private void initialize(ILockMatrix matrix) throws ACIDException {
- this.lmTables = new LMTables(INIT_TABLE_SIZE);
- this.lMtx = matrix;
- woManager = new WaitObjectManager();
- this.deadlockDetector = new DeadlockDetector(this);
+ public LockManager(TransactionProvider txnProvider) throws ACIDException {
+ this.txnProvider = txnProvider;
+ this.lockTableLatch = new ReentrantReadWriteLock(true);
+ this.waiterLatch = new ReentrantReadWriteLock(true);
+ this.jobHT = new HashMap<JobId, JobInfo>();
+ this.datasetResourceHT = new HashMap<DatasetId, DatasetLockInfo>();
+ this.entityInfoManager = new EntityInfoManager();
+ this.lockWaiterManager = new LockWaiterManager();
+ this.entityLockInfoManager = new EntityLockInfoManager(entityInfoManager, lockWaiterManager);
+ this.deadlockDetector = new DeadlockDetector(jobHT, datasetResourceHT, entityLockInfoManager,
+ entityInfoManager, lockWaiterManager);
this.toutDetector = new TimeOutDetector(this);
+ this.tempDatasetIdObj = new DatasetId(0);
+ this.consecutiveWakeupContext = new ConsecutiveWakeupContext();
+
+ if (IS_DEBUG_MODE) {
+ this.lockRequestTracker = new LockRequestTracker();
+ }
}
@Override
- public boolean lock(TransactionContext context, byte[] resourceID, int mode) throws ACIDException {
- long txId = context.getTransactionID();
- TxrInfo txrInfo = null;
- WaitEntry waitObj = null;
- Boolean isConverting = false;
- int grantedMode = -1;
- LockInfo lInfo = null;
- boolean shouldAbort = false;
+ public void lock(DatasetId datasetId, int entityHashValue, byte lockMode, TransactionContext txnContext)
+ throws ACIDException {
+ internalLock(datasetId, entityHashValue, lockMode, txnContext);
+ }
- synchronized (lmTables) {
- txrInfo = lmTables.getTxrInfo(txId);
- if (txrInfo == null) {
- txrInfo = new TxrInfo(context);
- lmTables.putTxrInfo(txId, txrInfo);
- }
+ private void internalLock(DatasetId datasetId, int entityHashValue, byte lockMode, TransactionContext txnContext)
+ throws ACIDException {
- lInfo = lmTables.getLockInfo(resourceID);
- if (lInfo == null) { // First lock on the resource, grant it
- lInfo = new LockInfo();
- lInfo.addToGranted(txId, mode, LockInfo.NOT_FOUND);
- lmTables.putLockInfo(resourceID, lInfo);
- txrInfo.addGrantedLock(resourceID, mode);
- return true;
- }
+ JobId jobId = txnContext.getJobId();
+ int jId = jobId.getId(); //int-type jobId
+ int dId = datasetId.getId(); //int-type datasetId
+ int entityInfo;
+ int eLockInfo = -1;
+ DatasetLockInfo dLockInfo = null;
+ JobInfo jobInfo;
+ byte datasetLockMode = entityHashValue == -1 ? lockMode : lockMode == LockMode.S ? LockMode.IS : LockMode.IX;
- int eix = lInfo.findInGrantedList(txId, LockInfo.ANY_LOCK_MODE);
- if (eix == LockInfo.NOT_FOUND) { // First lock by this Txr on the
- // resource
- if (!lInfo.isConvertListEmpty()) { // If Some converter(s)
- // is(are) waiting, Txr needs
- // to wait for fairness
+ latchLockTable();
+ validateJob(txnContext);
- // ----Deadlock Detection ---
- if (!isDeadlockFree(txId, resourceID)) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("DEADLOCK DETECTED FOR TRANSACTION " + txId);
- }
- context.setStatus(TransactionContext.TIMED_OUT_SATUS);
- shouldAbort = true;
- }
- // ---------------------------
-
- else { // Safe to wait
- waitObj = woManager.allocate();
- if (waitObj == null) {
- throw new ACIDException("Invalid (null) object allocated as the WaitEntry for Txr " + txId);
- }
- lInfo.addToWaiters(txId, mode, waitObj);
- txrInfo.setWaitOnRid(resourceID);
- context.setStartWaitTime(System.currentTimeMillis());
-
- }
- } else { // No converter(s) waiting
- int mask = lInfo.getMask();
- if (lMtx.conflicts(mask, mode)) { // If There is conflict,
- // Txr needs to wait
-
- // ----Deadlock Detection ---
- if (!isDeadlockFree(txId, resourceID)) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("DEADLOCK DETECTED FOR TRANSACTION " + txId);
- }
- context.setStatus(TransactionContext.TIMED_OUT_SATUS);
- shouldAbort = true;
- }
- // ---------------------------
- else { // Safe to wait
- waitObj = woManager.allocate();
- if (waitObj == null) {
- throw new ACIDException("Invalid (null) object allocated as the WaitEntry for Txr "
- + txId);
- }
- lInfo.addToWaiters(txId, mode, waitObj);
- txrInfo.setWaitOnRid(resourceID);
- context.setStartWaitTime(System.currentTimeMillis());
- }
- } else { // No conflicts with the current mask, just grant
- // it
- lInfo.addToGranted(txId, mode, LockInfo.NOT_FOUND);
- txrInfo.addGrantedLock(resourceID, mode);
- return true;
- }
- }
- }
-
- else { // Redundant or Conversion
- grantedMode = lInfo.getGrantedLockMode(txId, eix);
- if (grantedMode == mode) {
- lInfo.grantRedundantLock(mode, eix);
- return true; // No need to update tInfo
- } else {
- if (lMtx.isConversion(grantedMode, mode)) {
- isConverting = true;
- } else {
- return true; // Txr already has a stronger lock on the
- // resource
- }
-
- }
- }
+ if (IS_DEBUG_MODE) {
+ trackLockRequest("Requested", RequestType.LOCK, datasetId, entityHashValue, lockMode, txnContext,
+ dLockInfo, eLockInfo);
}
- if (isConverting) {
+ dLockInfo = datasetResourceHT.get(datasetId);
+ jobInfo = jobHT.get(jobId);
+
+ //#. if the datasetLockInfo doesn't exist in datasetResourceHT
+ if (dLockInfo == null || dLockInfo.isNoHolder()) {
+ if (dLockInfo == null) {
+ dLockInfo = new DatasetLockInfo(entityLockInfoManager, entityInfoManager, lockWaiterManager);
+ datasetResourceHT.put(new DatasetId(dId), dLockInfo); //datsetId obj should be created
+ }
+ entityInfo = entityInfoManager.allocate(jId, dId, entityHashValue, lockMode);
+
+ //if dataset-granule lock
+ if (entityHashValue == -1) { //-1 stands for dataset-granule
+ entityInfoManager.increaseDatasetLockCount(entityInfo);
+ dLockInfo.increaseLockCount(datasetLockMode);
+ dLockInfo.addHolder(entityInfo);
+ } else {
+ entityInfoManager.increaseDatasetLockCount(entityInfo);
+ dLockInfo.increaseLockCount(datasetLockMode);
+ //add entityLockInfo
+ eLockInfo = entityLockInfoManager.allocate();
+ dLockInfo.getEntityResourceHT().put(entityHashValue, eLockInfo);
+ entityInfoManager.increaseEntityLockCount(entityInfo);
+ entityLockInfoManager.increaseLockCount(eLockInfo, lockMode);
+ entityLockInfoManager.addHolder(eLockInfo, entityInfo);
+ }
+
+ if (jobInfo == null) {
+ jobInfo = new JobInfo(entityInfoManager, lockWaiterManager, txnContext);
+ jobHT.put(jobId, jobInfo); //jobId obj doesn't have to be created
+ }
+ jobInfo.addHoldingResource(entityInfo);
+
+ if (IS_DEBUG_MODE) {
+ trackLockRequest("Granted", RequestType.LOCK, datasetId, entityHashValue, lockMode, txnContext,
+ dLockInfo, eLockInfo);
+ }
+
+ unlatchLockTable();
+ return;
+ }
+
+ //#. the datasetLockInfo exists in datasetResourceHT.
+ //1. handle dataset-granule lock
+ entityInfo = lockDatasetGranule(datasetId, entityHashValue, lockMode, txnContext);
+
+ //2. handle entity-granule lock
+ if (entityHashValue != -1) {
+ lockEntityGranule(datasetId, entityHashValue, lockMode, entityInfo, txnContext);
+ }
+
+ if (IS_DEBUG_MODE) {
+ trackLockRequest("Granted", RequestType.LOCK, datasetId, entityHashValue, lockMode, txnContext, dLockInfo,
+ eLockInfo);
+ }
+ unlatchLockTable();
+ return;
+ }
+
+ private void validateJob(TransactionContext txnContext) throws ACIDException {
+ if (txnContext.getTxnState() == TransactionState.ABORTED) {
+ unlatchLockTable();
+ throw new ACIDException("" + txnContext.getJobId() + " is in ABORTED state.");
+ } else if (txnContext.getStatus() == TransactionContext.TIMED_OUT_STATUS) {
try {
- return convertLockForNewTransaction(context, resourceID, grantedMode, mode);
- } catch (InterruptedException e) {
- e.printStackTrace();
+ requestAbort(txnContext);
+ } finally {
+ unlatchLockTable();
}
}
+ }
- if (shouldAbort) {
- requestTxrAbort(context);
- return false;
- }
+ private int lockDatasetGranule(DatasetId datasetId, int entityHashValue, byte lockMode,
+ TransactionContext txnContext) throws ACIDException {
+ JobId jobId = txnContext.getJobId();
+ int jId = jobId.getId(); //int-type jobId
+ int dId = datasetId.getId(); //int-type datasetId
+ int waiterObjId;
+ int entityInfo = -1;
+ DatasetLockInfo dLockInfo;
+ JobInfo jobInfo;
+ boolean isUpgrade = false;
+ int weakerModeLockCount;
+ int waiterCount = 0;
+ byte datasetLockMode = entityHashValue == -1 ? lockMode : lockMode == LockMode.S ? LockMode.IS : LockMode.IX;
- // Txr needs to wait and it is safe to wait
- synchronized (waitObj) {
- while (waitObj.needWait()) {
- try {
- waitObj.wait();
- } catch (InterruptedException e) {
- e.printStackTrace();
+ dLockInfo = datasetResourceHT.get(datasetId);
+ jobInfo = jobHT.get(jobId);
+
+ //check duplicated call
+
+ //1. lock request causing duplicated upgrading requests from different threads in a same job
+ waiterObjId = dLockInfo.findUpgraderFromUpgraderList(jId, entityHashValue);
+ if (waiterObjId != -1) {
+ //make the caller wait on the same LockWaiter object
+ entityInfo = lockWaiterManager.getLockWaiter(waiterObjId).getEntityInfoSlot();
+ waiterCount = handleLockWaiter(dLockInfo, -1, entityInfo, true, true, txnContext, jobInfo, waiterObjId);
+
+ //Only for the first-get-up thread, the waiterCount will be more than 0 and
+ //the thread updates lock count on behalf of the all other waiting threads.
+ //Therefore, all the next-get-up threads will not update any lock count.
+ if (waiterCount > 0) {
+ //add ((the number of waiting upgrader) - 1) to entityInfo's dataset lock count and datasetLockInfo's lock count
+ //where -1 is for not counting the first upgrader's request since the lock count for the first upgrader's request
+ //is already counted.
+ weakerModeLockCount = entityInfoManager.getDatasetLockCount(entityInfo);
+ entityInfoManager.setDatasetLockMode(entityInfo, lockMode);
+ entityInfoManager.increaseDatasetLockCount(entityInfo, waiterCount - 1);
+
+ if (entityHashValue == -1) { //dataset-granule lock
+ dLockInfo.increaseLockCount(LockMode.X, weakerModeLockCount + waiterCount - 1);//new lock mode
+ dLockInfo.decreaseLockCount(LockMode.S, weakerModeLockCount);//current lock mode
+ } else {
+ dLockInfo.increaseLockCount(LockMode.IX, weakerModeLockCount + waiterCount - 1);
+ dLockInfo.decreaseLockCount(LockMode.IS, weakerModeLockCount);
}
}
- // Txr just woke up
- woManager.deAllocate(waitObj);
- if (context.getStatus() == TransactionContext.TIMED_OUT_SATUS) { // selected
- // as
- // a
- // victim
- requestTxrAbort(context);
- return false;
+
+ return entityInfo;
+ }
+
+ //2. lock request causing duplicated waiting requests from different threads in a same job
+ waiterObjId = dLockInfo.findWaiterFromWaiterList(jId, entityHashValue);
+ if (waiterObjId != -1) {
+ //make the caller wait on the same LockWaiter object
+ entityInfo = lockWaiterManager.getLockWaiter(waiterObjId).getEntityInfoSlot();
+ waiterCount = handleLockWaiter(dLockInfo, -1, entityInfo, false, true, txnContext, jobInfo, waiterObjId);
+
+ if (waiterCount > 0) {
+ entityInfoManager.increaseDatasetLockCount(entityInfo, waiterCount);
+ if (entityHashValue == -1) {
+ dLockInfo.increaseLockCount(datasetLockMode, waiterCount);
+ dLockInfo.addHolder(entityInfo);
+ } else {
+ dLockInfo.increaseLockCount(datasetLockMode, waiterCount);
+ //IS and IX holders are implicitly handled.
+ }
+ //add entityInfo to JobInfo's holding-resource list
+ jobInfo.addHoldingResource(entityInfo);
+ }
+
+ return entityInfo;
+ }
+
+ //3. lock request causing duplicated holding requests from different threads or a single thread in a same job
+ entityInfo = dLockInfo.findEntityInfoFromHolderList(jId, entityHashValue);
+ if (entityInfo == -1) {
+
+ entityInfo = entityInfoManager.allocate(jId, dId, entityHashValue, lockMode);
+ if (jobInfo == null) {
+ jobInfo = new JobInfo(entityInfoManager, lockWaiterManager, txnContext);
+ jobHT.put(jobId, jobInfo);
+ }
+
+ //wait if any upgrader exists or upgrading lock mode is not compatible
+ if (dLockInfo.getFirstUpgrader() != -1 || dLockInfo.getFirstWaiter() != -1
+ || !dLockInfo.isCompatible(datasetLockMode)) {
+
+ /////////////////////////////////////////////////////////////////////////////////////////////
+ //[Notice]
+ //There has been no same caller as (jId, dId, entityHashValue) triplet.
+ //But there could be the same caller as (jId, dId) pair.
+ //For example, two requests (J1, D1, E1) and (J1, D1, E2) are considered as duplicated call in dataset-granule perspective.
+ //Therefore, the above duplicated call case is covered in the following code.
+ //find the same dataset-granule lock request, that is, (J1, D1) pair in the above example.
+ //if (jobInfo.isDatasetLockGranted(dId, datasetLockMode)) {
+ if (jobInfo.isDatasetLockGranted(dId, LockMode.IS)) {
+ if (dLockInfo.isCompatible(datasetLockMode)) {
+ //this is duplicated call
+ entityInfoManager.increaseDatasetLockCount(entityInfo);
+ if (entityHashValue == -1) {
+ dLockInfo.increaseLockCount(datasetLockMode);
+ dLockInfo.addHolder(entityInfo);
+ } else {
+ dLockInfo.increaseLockCount(datasetLockMode);
+ //IS and IX holders are implicitly handled.
+ }
+ //add entityInfo to JobInfo's holding-resource list
+ jobInfo.addHoldingResource(entityInfo);
+
+ return entityInfo;
+ }
+ else {
+ //considered as upgrader
+ waiterCount = handleLockWaiter(dLockInfo, -1, entityInfo, true, true, txnContext, jobInfo, -1);
+ if (waiterCount > 0) {
+ entityInfoManager.increaseDatasetLockCount(entityInfo);
+ if (entityHashValue == -1) {
+ dLockInfo.increaseLockCount(datasetLockMode);
+ dLockInfo.addHolder(entityInfo);
+ } else {
+ dLockInfo.increaseLockCount(datasetLockMode);
+ //IS and IX holders are implicitly handled.
+ }
+ //add entityInfo to JobInfo's holding-resource list
+ jobInfo.addHoldingResource(entityInfo);
+ }
+ return entityInfo;
+ }
+ }
+ /////////////////////////////////////////////////////////////////////////////////////////////
+
+ waiterCount = handleLockWaiter(dLockInfo, -1, entityInfo, false, true, txnContext, jobInfo, -1);
+ } else {
+ waiterCount = 1;
+ }
+
+ if (waiterCount > 0) {
+ entityInfoManager.increaseDatasetLockCount(entityInfo);
+ if (entityHashValue == -1) {
+ dLockInfo.increaseLockCount(datasetLockMode);
+ dLockInfo.addHolder(entityInfo);
+ } else {
+ dLockInfo.increaseLockCount(datasetLockMode);
+ //IS and IX holders are implicitly handled.
+ }
+ //add entityInfo to JobInfo's holding-resource list
+ jobInfo.addHoldingResource(entityInfo);
+ }
+ } else {
+ isUpgrade = isLockUpgrade(entityInfoManager.getDatasetLockMode(entityInfo), lockMode);
+ if (isUpgrade) { //upgrade call
+ //wait if any upgrader exists or upgrading lock mode is not compatible
+ if (dLockInfo.getFirstUpgrader() != -1 || !dLockInfo.isUpgradeCompatible(datasetLockMode, entityInfo)) {
+ waiterCount = handleLockWaiter(dLockInfo, -1, entityInfo, true, true, txnContext, jobInfo, -1);
+ } else {
+ waiterCount = 1;
+ }
+
+ if (waiterCount > 0) {
+ //add ((the number of waiting upgrader) - 1) to entityInfo's dataset lock count and datasetLockInfo's lock count
+ //where -1 is for not counting the first upgrader's request since the lock count for the first upgrader's request
+ //is already counted.
+ weakerModeLockCount = entityInfoManager.getDatasetLockCount(entityInfo);
+ entityInfoManager.setDatasetLockMode(entityInfo, lockMode);
+ entityInfoManager.increaseDatasetLockCount(entityInfo, waiterCount - 1);
+
+ if (entityHashValue == -1) { //dataset-granule lock
+ dLockInfo.increaseLockCount(LockMode.X, weakerModeLockCount + waiterCount - 1);//new lock mode
+ dLockInfo.decreaseLockCount(LockMode.S, weakerModeLockCount);//current lock mode
+ } else {
+ dLockInfo.increaseLockCount(LockMode.IX, weakerModeLockCount + waiterCount - 1);
+ dLockInfo.decreaseLockCount(LockMode.IS, weakerModeLockCount);
+ }
+ }
+ } else { //duplicated call
+ entityInfoManager.increaseDatasetLockCount(entityInfo);
+ datasetLockMode = entityInfoManager.getDatasetLockMode(entityInfo);
+
+ if (entityHashValue == -1) { //dataset-granule
+ dLockInfo.increaseLockCount(datasetLockMode);
+ } else { //entity-granule
+ datasetLockMode = datasetLockMode == LockMode.S? LockMode.IS: LockMode.IX;
+ dLockInfo.increaseLockCount(datasetLockMode);
+ }
}
}
- synchronized (context) {
- context.setStatus(TransactionContext.ACTIVE_STATUS);
- context.setStartWaitTime(TransactionContext.INVALID_TIME);
- }
+ return entityInfo;
+ }
- synchronized (lmTables) {
- txrInfo = lmTables.getTxrInfo(txId);
- if (txrInfo == null) {
- throw new ACIDException("Transaction " + txId + " removed from Txr Table Unexpectedlly");
+ private void lockEntityGranule(DatasetId datasetId, int entityHashValue, byte lockMode,
+ int entityInfoFromDLockInfo, TransactionContext txnContext) throws ACIDException {
+ JobId jobId = txnContext.getJobId();
+ int jId = jobId.getId(); //int-type jobId
+ int waiterObjId;
+ int eLockInfo = -1;
+ int entityInfo;
+ DatasetLockInfo dLockInfo;
+ JobInfo jobInfo;
+ boolean isUpgrade = false;
+ int waiterCount = 0;
+ int weakerModeLockCount;
+
+ dLockInfo = datasetResourceHT.get(datasetId);
+ jobInfo = jobHT.get(jobId);
+ eLockInfo = dLockInfo.getEntityResourceHT().get(entityHashValue);
+
+ if (eLockInfo != -1) {
+ //check duplicated call
+
+ //1. lock request causing duplicated upgrading requests from different threads in a same job
+ waiterObjId = entityLockInfoManager.findUpgraderFromUpgraderList(eLockInfo, jId, entityHashValue);
+ if (waiterObjId != -1) {
+ entityInfo = lockWaiterManager.getLockWaiter(waiterObjId).getEntityInfoSlot();
+ waiterCount = handleLockWaiter(dLockInfo, eLockInfo, -1, true, false, txnContext, jobInfo, waiterObjId);
+
+ if (waiterCount > 0) {
+ weakerModeLockCount = entityInfoManager.getEntityLockCount(entityInfo);
+ entityInfoManager.setEntityLockMode(entityInfo, LockMode.X);
+ entityInfoManager.increaseEntityLockCount(entityInfo, waiterCount - 1);
+
+ entityLockInfoManager.increaseLockCount(eLockInfo, LockMode.X, (short) (weakerModeLockCount
+ + waiterCount - 1));//new lock mode
+ entityLockInfoManager.decreaseLockCount(eLockInfo, LockMode.S, (short) weakerModeLockCount);//old lock mode
+ }
+ return;
}
- txrInfo.addGrantedLock(resourceID, mode);
- txrInfo.setWaitOnRid(null);
- }
- return true; // Arriving here when Txr wakes up and it successfully
- // locks the resource
+ //2. lock request causing duplicated waiting requests from different threads in a same job
+ waiterObjId = entityLockInfoManager.findWaiterFromWaiterList(eLockInfo, jId, entityHashValue);
+ if (waiterObjId != -1) {
+ entityInfo = lockWaiterManager.getLockWaiter(waiterObjId).getEntityInfoSlot();
+ waiterCount = handleLockWaiter(dLockInfo, eLockInfo, -1, false, false, txnContext, jobInfo, waiterObjId);
+
+ if (waiterCount > 0) {
+ entityInfoManager.increaseEntityLockCount(entityInfo, waiterCount);
+ entityLockInfoManager.increaseLockCount(eLockInfo, lockMode, (short) waiterCount);
+ entityLockInfoManager.addHolder(eLockInfo, entityInfo);
+ }
+ return;
+ }
+
+ //3. lock request causing duplicated holding requests from different threads or a single thread in a same job
+ entityInfo = entityLockInfoManager.findEntityInfoFromHolderList(eLockInfo, jId, entityHashValue);
+ if (entityInfo != -1) {//duplicated call or upgrader
+
+ isUpgrade = isLockUpgrade(entityInfoManager.getEntityLockMode(entityInfo), lockMode);
+ if (isUpgrade) {//upgrade call
+ //wait if any upgrader exists or upgrading lock mode is not compatible
+ if (entityLockInfoManager.getUpgrader(eLockInfo) != -1
+ || !entityLockInfoManager.isUpgradeCompatible(eLockInfo, lockMode, entityInfo)) {
+ waiterCount = handleLockWaiter(dLockInfo, eLockInfo, entityInfo, true, false, txnContext, jobInfo,
+ -1);
+ } else {
+ waiterCount = 1;
+ }
+
+ if (waiterCount > 0) {
+ weakerModeLockCount = entityInfoManager.getEntityLockCount(entityInfo);
+ entityInfoManager.setEntityLockMode(entityInfo, lockMode);
+ entityInfoManager.increaseEntityLockCount(entityInfo, waiterCount - 1);
+
+ entityLockInfoManager.increaseLockCount(eLockInfo, LockMode.X, (short) (weakerModeLockCount
+ + waiterCount - 1));//new lock mode
+ entityLockInfoManager.decreaseLockCount(eLockInfo, LockMode.S, (short) weakerModeLockCount);//old lock mode
+ }
+
+ } else {//duplicated call
+ entityInfoManager.increaseEntityLockCount(entityInfo);
+ entityLockInfoManager.increaseLockCount(eLockInfo, entityInfoManager.getEntityLockMode(entityInfo));
+ }
+ } else {//new call from this job, but still eLockInfo exists since other threads hold it or wait on it
+ entityInfo = entityInfoFromDLockInfo;
+ if (entityLockInfoManager.getUpgrader(eLockInfo) != -1
+ || entityLockInfoManager.getFirstWaiter(eLockInfo) != -1
+ || !entityLockInfoManager.isCompatible(eLockInfo, lockMode)) {
+ waiterCount = handleLockWaiter(dLockInfo, eLockInfo, entityInfo, false, false, txnContext, jobInfo, -1);
+ } else {
+ waiterCount = 1;
+ }
+
+ if (waiterCount > 0) {
+ entityInfoManager.increaseEntityLockCount(entityInfo, waiterCount);
+ entityLockInfoManager.increaseLockCount(eLockInfo, lockMode, (short) waiterCount);
+ entityLockInfoManager.addHolder(eLockInfo, entityInfo);
+ }
+ }
+ } else {//eLockInfo doesn't exist, so this lock request is the first request and can be granted without waiting.
+ eLockInfo = entityLockInfoManager.allocate();
+ dLockInfo.getEntityResourceHT().put(entityHashValue, eLockInfo);
+ entityInfoManager.increaseEntityLockCount(entityInfoFromDLockInfo);
+ entityLockInfoManager.increaseLockCount(eLockInfo, lockMode);
+ entityLockInfoManager.addHolder(eLockInfo, entityInfoFromDLockInfo);
+ }
}
@Override
- public boolean convertLock(TransactionContext context, byte[] resourceID, int mode) throws ACIDException {
- long txId = context.getTransactionID();
- int curMode = -1;
- TxrInfo txrInfo = null;
- LockInfo lInfo = null;
- synchronized (lmTables) {
- txrInfo = lmTables.getTxrInfo(txId);
+ public void unlock(DatasetId datasetId, int entityHashValue, TransactionContext txnContext) throws ACIDException {
+ JobId jobId = txnContext.getJobId();
+ int eLockInfo = -1;
+ DatasetLockInfo dLockInfo = null;
+ JobInfo jobInfo;
+ int entityInfo = -1;
- if (txrInfo == null) {
- throw new ACIDException("No lock is granted to the transaction, to convert");
+ if (IS_DEBUG_MODE) {
+ if (entityHashValue == -1) {
+ throw new UnsupportedOperationException(
+ "Unsupported unlock request: dataset-granule unlock is not supported");
}
+ }
- TInfo tInfo = txrInfo.getTxrInfo(resourceID, LockInfo.ANY_LOCK_MODE, TxrInfo.NOT_KNOWN_IX);
- if (tInfo == null) {
- throw new ACIDException("No lock is granted to the transaction on the resource, to convert");
+ latchLockTable();
+ validateJob(txnContext);
+
+ if (IS_DEBUG_MODE) {
+ trackLockRequest("Requested", RequestType.UNLOCK, datasetId, entityHashValue, (byte) 0, txnContext,
+ dLockInfo, eLockInfo);
+ }
+
+ //find the resource to be unlocked
+ dLockInfo = datasetResourceHT.get(datasetId);
+ jobInfo = jobHT.get(jobId);
+ if (IS_DEBUG_MODE) {
+ if (dLockInfo == null || jobInfo == null) {
+ throw new IllegalStateException("Invalid unlock request: Corresponding lock info doesn't exist.");
}
-
- curMode = tInfo.getMode();
- if (mode == curMode) { // Redundant
- return true; // We do not increment the counter, because it is a
- // conversion
+ }
+ eLockInfo = dLockInfo.getEntityResourceHT().get(entityHashValue);
+ if (IS_DEBUG_MODE) {
+ if (eLockInfo == -1) {
+ throw new IllegalStateException("Invalid unlock request: Corresponding lock info doesn't exist.");
}
+ }
- if (!lMtx.isConversion(curMode, mode)) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Transaction " + txId + " already has grabbed a stronger mode (" + curMode + ") than "
- + mode);
+ //find the corresponding entityInfo
+ entityInfo = entityLockInfoManager.findEntityInfoFromHolderList(eLockInfo, jobId.getId(), entityHashValue);
+ if (IS_DEBUG_MODE) {
+ if (entityInfo == -1) {
+ throw new IllegalStateException("Invalid unlock request[" + jobId.getId() + "," + datasetId.getId()
+ + "," + entityHashValue + "]: Corresponding lock info doesn't exist.");
+ }
+ }
+
+ //decrease the corresponding count of dLockInfo/eLockInfo/entityInfo
+ dLockInfo.decreaseLockCount(entityInfoManager.getDatasetLockMode(entityInfo) == LockMode.S ? LockMode.IS
+ : LockMode.IX);
+ entityLockInfoManager.decreaseLockCount(eLockInfo, entityInfoManager.getEntityLockMode(entityInfo));
+ entityInfoManager.decreaseDatasetLockCount(entityInfo);
+ entityInfoManager.decreaseEntityLockCount(entityInfo);
+
+ if (entityInfoManager.getEntityLockCount(entityInfo) == 0
+ && entityInfoManager.getDatasetLockCount(entityInfo) == 0) {
+ int threadCount = 0; //number of threads(in the same job) waiting on the same resource
+ int waiterObjId = jobInfo.getFirstWaitingResource();
+ int waitingEntityInfo;
+ LockWaiter waiterObj;
+
+ //1) wake up waiters and remove holder
+ //wake up waiters of dataset-granule lock
+ wakeUpDatasetLockWaiters(dLockInfo);
+ //wake up waiters of entity-granule lock
+ wakeUpEntityLockWaiters(eLockInfo);
+ //remove the holder from eLockInfo's holder list and remove the holding resource from jobInfo's holding resource list
+ //this can be done in the following single function call.
+ entityLockInfoManager.removeHolder(eLockInfo, entityInfo, jobInfo);
+
+ //2) if
+ // there is no waiting thread on the same resource (this can be checked through jobInfo)
+ // then
+ // a) delete the corresponding entityInfo
+ // b) write commit log for the unlocked resource(which is a committed txn).
+ while (waiterObjId != -1) {
+ waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
+ waitingEntityInfo = waiterObj.getEntityInfoSlot();
+ if (entityInfoManager.getDatasetId(waitingEntityInfo) == datasetId.getId()
+ && entityInfoManager.getPKHashVal(waitingEntityInfo) == entityHashValue) {
+ threadCount++;
+ break;
}
-
- return true;
+ waiterObjId = waiterObj.getNextWaiterObjId();
}
-
- lInfo = lmTables.getLockInfo(resourceID);
- if (lInfo == null) {
- throw new ACIDException("No lock on the resource, to convert");
+ if (threadCount == 0) {
+ if (entityInfoManager.getEntityLockMode(entityInfo) == LockMode.X) {
+ //TODO
+ //write a commit log for the unlocked resource
+ //need to figure out that instantLock() also needs to write a commit log.
+ }
+ entityInfoManager.deallocate(entityInfo);
}
}
- try {
- return convertLockForNewTransaction(context, resourceID, curMode, mode);
- } catch (InterruptedException e) {
- e.printStackTrace();
+ //deallocate entityLockInfo's slot if there is no txn referring to the entityLockInfo.
+ if (entityLockInfoManager.getFirstWaiter(eLockInfo) == -1
+ && entityLockInfoManager.getLastHolder(eLockInfo) == -1
+ && entityLockInfoManager.getUpgrader(eLockInfo) == -1) {
+ dLockInfo.getEntityResourceHT().remove(entityHashValue);
+ entityLockInfoManager.deallocate(eLockInfo);
}
- throw new ACIDException("Problem in Lock Converting for Transaction " + txId
- + " (We unexpectedly returned from convert lock for new transaction)");
+ //we don't deallocate datasetLockInfo even if there is no txn referring to the datasetLockInfo
+ //since the datasetLockInfo is likely to be referred to again.
+
+ if (IS_DEBUG_MODE) {
+ trackLockRequest("Granted", RequestType.UNLOCK, datasetId, entityHashValue, (byte) 0, txnContext,
+ dLockInfo, eLockInfo);
+ }
+ unlatchLockTable();
}
- private boolean convertLockForNewTransaction(TransactionContext context, byte[] resourceId, int curMode, int reqMode)
- throws ACIDException, InterruptedException {
- long txId = context.getTransactionID();
- WaitEntry waitObj = null;
- boolean shouldAbort = false;
- TxrInfo txrInfo = null;
- synchronized (lmTables) {
- LockInfo lInfo = lmTables.getLockInfo(resourceId);
- txrInfo = lmTables.getTxrInfo(txId);
- // ---Check if the conversion is already done---
- int eix = lInfo.findInGrantedList(txId, reqMode);
- if (eix != LockInfo.NOT_FOUND) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Conversion already done for Transaction " + txId + " for lock " + reqMode
- + " on resource ");
+ @Override
+ public void releaseLocks(TransactionContext txnContext) throws ACIDException {
+ LockWaiter waiterObj;
+ int entityInfo;
+ int prevEntityInfo;
+ int entityHashValue;
+ DatasetLockInfo dLockInfo = null;
+ int eLockInfo = -1;
+ int did;//int-type dataset id
+ int datasetLockCount;
+ int entityLockCount;
+ byte lockMode;
+ boolean existWaiter = false;
+
+ JobId jobId = txnContext.getJobId();
+
+ latchLockTable();
+
+ if (IS_DEBUG_MODE) {
+ trackLockRequest("Requested", RequestType.RELEASE_LOCKS, new DatasetId(0), 0, (byte) 0, txnContext,
+ dLockInfo, eLockInfo);
+ }
+
+ JobInfo jobInfo = jobHT.get(jobId);
+ if (jobInfo == null) {
+ unlatchLockTable();
+ return ;
+ }
+
+ //remove waiterObj of JobInfo
+ //[Notice]
+ //waiterObjs may exist if aborted thread is the caller of this function.
+ //Even if there are the waiterObjs, there is no waiting thread on the objects.
+ //If the caller of this function is an aborted thread, it is guaranteed that there is no waiting threads
+ //on the waiterObjs since when the aborted caller thread is waken up, all other waiting threads are
+ //also waken up at the same time through 'notifyAll()' call.
+ //In contrast, if the caller of this function is not an aborted thread, then there is no waiting object.
+ int waiterObjId = jobInfo.getFirstWaitingResource();
+ int nextWaiterObjId;
+ while (waiterObjId != -1) {
+ existWaiter = true;
+ waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
+ nextWaiterObjId = waiterObj.getNextWaitingResourceObjId();
+ entityInfo = waiterObj.getEntityInfoSlot();
+ if (IS_DEBUG_MODE) {
+ if (jobId.getId() != entityInfoManager.getJobId(entityInfo)) {
+ throw new IllegalStateException("JobInfo(" + jobId + ") has diffrent Job(JID:"
+ + entityInfoManager.getJobId(entityInfo) + "'s lock request!!!");
}
- return true;
}
- // --------------------------------------------
-
- int updatedMask = lInfo.getUpdatedMask(txId, curMode, LockInfo.UNKNOWN_IX);
- if (lMtx.conflicts(updatedMask, reqMode)) { // if Conflicting, Txr
- // needs to wait
-
- // ---- Deadlock Detection ---
- if (!isDeadlockFree(txId, resourceId)) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("DEADLOCK DETECTED IN CONVERSION FOR TRANSACTION ");
- }
- context.setStatus(TransactionContext.TIMED_OUT_SATUS);
- shouldAbort = true;
+
+ //1. remove from waiter(or upgrader)'s list of dLockInfo or eLockInfo.
+ did = entityInfoManager.getDatasetId(entityInfo);
+ tempDatasetIdObj.setId(did);
+ dLockInfo = datasetResourceHT.get(tempDatasetIdObj);
+
+ if (waiterObj.isWaitingOnEntityLock()) {
+ entityHashValue = entityInfoManager.getPKHashVal(entityInfo);
+ eLockInfo = dLockInfo.getEntityResourceHT().get(entityHashValue);
+ if (waiterObj.isWaiter()) {
+ entityLockInfoManager.removeWaiter(eLockInfo, waiterObjId);
+ } else {
+ entityLockInfoManager.removeUpgrader(eLockInfo, waiterObjId);
}
- // ---------------------------
+ } else {
+ if (waiterObj.isWaiter()) {
+ dLockInfo.removeWaiter(waiterObjId);
+ } else {
+ dLockInfo.removeUpgrader(waiterObjId);
+ }
+ }
+
+ //2. wake-up waiters
+ latchWaitNotify();
+ synchronized (waiterObj) {
+ unlatchWaitNotify();
+ waiterObj.setWait(false);
+ if (IS_DEBUG_MODE) {
+ System.out.println("" + Thread.currentThread().getName() + "\twake-up(D): WID(" + waiterObjId
+ + "),EID(" + waiterObj.getEntityInfoSlot() + ")");
+ }
+ waiterObj.notifyAll();
+ }
+
+ //3. deallocate waiterObj
+ lockWaiterManager.deallocate(waiterObjId);
- else {
- waitObj = woManager.allocate();
- if (waitObj == null) {
- throw new ACIDException("Invalid (null) object allocated as the WaitEntry for Txr " + txId);
+ //4. deallocate entityInfo only if this waiter is not an upgrader
+ if (entityInfoManager.getDatasetLockCount(entityInfo) == 0
+ && entityInfoManager.getEntityLockCount(entityInfo) == 0) {
+ entityInfoManager.deallocate(entityInfo);
+ }
+ waiterObjId = nextWaiterObjId;
+ }
+
+ //release holding resources
+ entityInfo = jobInfo.getLastHoldingResource();
+ while (entityInfo != -1) {
+ prevEntityInfo = entityInfoManager.getPrevJobResource(entityInfo);
+
+ //decrease lock count of datasetLock and entityLock
+ did = entityInfoManager.getDatasetId(entityInfo);
+ tempDatasetIdObj.setId(did);
+ dLockInfo = datasetResourceHT.get(tempDatasetIdObj);
+ entityHashValue = entityInfoManager.getPKHashVal(entityInfo);
+
+ if (entityHashValue == -1) {
+ //decrease datasetLockCount
+ lockMode = entityInfoManager.getDatasetLockMode(entityInfo);
+ datasetLockCount = entityInfoManager.getDatasetLockCount(entityInfo);
+ if (datasetLockCount != 0) {
+ dLockInfo.decreaseLockCount(lockMode, datasetLockCount);
+
+ //wakeup waiters of datasetLock and remove holder from datasetLockInfo
+ wakeUpDatasetLockWaiters(dLockInfo);
+
+ //remove the holder from datasetLockInfo only if the lock is dataset-granule lock.
+ //--> this also removes the holding resource from jobInfo
+ //(Because the IX and IS lock's holders are handled implicitly,
+ //those are not in the holder list of datasetLockInfo.)
+ dLockInfo.removeHolder(entityInfo, jobInfo);
+ }
+ } else {
+ //decrease datasetLockCount
+ lockMode = entityInfoManager.getDatasetLockMode(entityInfo);
+ lockMode = lockMode == LockMode.S ? LockMode.IS : LockMode.IX;
+ datasetLockCount = entityInfoManager.getDatasetLockCount(entityInfo);
+
+ if (datasetLockCount != 0) {
+ dLockInfo.decreaseLockCount(lockMode, datasetLockCount);
+ }
+
+ //decrease entityLockCount
+ lockMode = entityInfoManager.getEntityLockMode(entityInfo);
+ entityLockCount = entityInfoManager.getEntityLockCount(entityInfo);
+ eLockInfo = dLockInfo.getEntityResourceHT().get(entityHashValue);
+ if (IS_DEBUG_MODE) {
+ if (eLockInfo < 0) {
+ System.out.println("eLockInfo:" + eLockInfo);
}
- lInfo.addToConvert(txId, reqMode, waitObj);
- txrInfo.setWaitOnRid(resourceId);
- context.setStartWaitTime(System.currentTimeMillis());
+ }
+
+ if (entityLockCount != 0) {
+ entityLockInfoManager.decreaseLockCount(eLockInfo, lockMode, (short) entityLockCount);
+ }
+
+ if (datasetLockCount != 0) {
+ //wakeup waiters of datasetLock and don't remove holder from datasetLockInfo
+ wakeUpDatasetLockWaiters(dLockInfo);
+ }
+
+ if (entityLockCount != 0) {
+ //wakeup waiters of entityLock
+ wakeUpEntityLockWaiters(eLockInfo);
+
+ //remove the holder from entityLockInfo
+ //--> this also removes the holding resource from jobInfo
+ entityLockInfoManager.removeHolder(eLockInfo, entityInfo, jobInfo);
+ }
+
+ //deallocate entityLockInfo if there is no holder and waiter.
+ if (entityLockInfoManager.getLastHolder(eLockInfo) == -1
+ && entityLockInfoManager.getFirstWaiter(eLockInfo) == -1
+ && entityLockInfoManager.getUpgrader(eLockInfo) == -1) {
+ dLockInfo.getEntityResourceHT().remove(entityHashValue);
+ entityLockInfoManager.deallocate(eLockInfo);
+ // if (IS_DEBUG_MODE) {
+ // System.out.println("removed PK["+entityHashValue+"]");
+ // }
}
}
- else { // no conflicts, grant it
- lInfo.removeFromGranted(txId, curMode, LockInfo.UNKNOWN_IX);
- lInfo.addToGranted(txId, reqMode, LockInfo.NOT_FOUND);
- txrInfo.removeLock(resourceId, curMode, TxrInfo.NOT_KNOWN_IX);
- txrInfo.addGrantedLock(resourceId, reqMode);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Transaction " + txId + " could convert to " + reqMode + " lock on resource ");
+ //deallocate entityInfo
+ entityInfoManager.deallocate(entityInfo);
+ // if (IS_DEBUG_MODE) {
+ // System.out.println("dellocate EntityInfo["+entityInfo+"]");
+ // }
+
+ entityInfo = prevEntityInfo;
+ }
+
+ //remove JobInfo
+ jobHT.remove(jobId);
+
+ if (existWaiter) {
+ txnContext.setStatus(TransactionContext.TIMED_OUT_STATUS);
+ txnContext.setTxnState(TransactionState.ABORTED);
+ }
+
+ if (IS_DEBUG_MODE) {
+ trackLockRequest("Granted", RequestType.RELEASE_LOCKS, new DatasetId(0), 0, (byte) 0, txnContext,
+ dLockInfo, eLockInfo);
+ }
+ unlatchLockTable();
+ }
+
+ @Override
+ public void instantLock(DatasetId datasetId, int entityHashValue, byte lockMode, TransactionContext txnContext)
+ throws ACIDException {
+
+ // try {
+ // internalLock(datasetId, entityHashValue, lockMode, txnContext);
+ // return;
+ // } finally {
+ // unlock(datasetId, entityHashValue, txnContext);
+ // }
+ internalLock(datasetId, entityHashValue, lockMode, txnContext);
+ unlock(datasetId, entityHashValue, txnContext);
+ }
+
+ @Override
+ public boolean tryLock(DatasetId datasetId, int entityHashValue, byte lockMode, TransactionContext txnContext)
+ throws ACIDException {
+ return internalTryLock(datasetId, entityHashValue, lockMode, txnContext);
+ }
+
+ @Override
+ public boolean instantTryLock(DatasetId datasetId, int entityHashValue, byte lockMode, TransactionContext txnContext)
+ throws ACIDException {
+ boolean isGranted = false;
+ // try {
+ // isGranted = internalTryLock(datasetId, entityHashValue, lockMode, txnContext);
+ // return isGranted;
+ // } finally {
+ // if (isGranted) {
+ // unlock(datasetId, entityHashValue, txnContext);
+ // }
+ // }
+ isGranted = internalTryLock(datasetId, entityHashValue, lockMode, txnContext);
+ if (isGranted) {
+ unlock(datasetId, entityHashValue, txnContext);
+ }
+ return isGranted;
+ }
+
+ private boolean internalTryLock(DatasetId datasetId, int entityHashValue, byte lockMode,
+ TransactionContext txnContext) throws ACIDException {
+ JobId jobId = txnContext.getJobId();
+ int jId = jobId.getId(); //int-type jobId
+ int dId = datasetId.getId(); //int-type datasetId
+ int entityInfo;
+ int eLockInfo = -1;
+ DatasetLockInfo dLockInfo = null;
+ JobInfo jobInfo;
+ byte datasetLockMode = entityHashValue == -1 ? lockMode : lockMode == LockMode.S ? LockMode.IS : LockMode.IX;
+ boolean isSuccess = false;
+
+ latchLockTable();
+ validateJob(txnContext);
+
+ if (IS_DEBUG_MODE) {
+ trackLockRequest("Requested", RequestType.TRY_LOCK, datasetId, entityHashValue, lockMode, txnContext,
+ dLockInfo, eLockInfo);
+ }
+
+ dLockInfo = datasetResourceHT.get(datasetId);
+ jobInfo = jobHT.get(jobId);
+
+ //#. if the datasetLockInfo doesn't exist in datasetResourceHT
+ if (dLockInfo == null || dLockInfo.isNoHolder()) {
+ if (dLockInfo == null) {
+ dLockInfo = new DatasetLockInfo(entityLockInfoManager, entityInfoManager, lockWaiterManager);
+ datasetResourceHT.put(new DatasetId(dId), dLockInfo); //datsetId obj should be created
+ }
+ entityInfo = entityInfoManager.allocate(jId, dId, entityHashValue, lockMode);
+
+ //if dataset-granule lock
+ if (entityHashValue == -1) { //-1 stands for dataset-granule
+ entityInfoManager.increaseDatasetLockCount(entityInfo);
+ dLockInfo.increaseLockCount(datasetLockMode);
+ dLockInfo.addHolder(entityInfo);
+ } else {
+ entityInfoManager.increaseDatasetLockCount(entityInfo);
+ dLockInfo.increaseLockCount(datasetLockMode);
+ //add entityLockInfo
+ eLockInfo = entityLockInfoManager.allocate();
+ dLockInfo.getEntityResourceHT().put(entityHashValue, eLockInfo);
+ entityInfoManager.increaseEntityLockCount(entityInfo);
+ entityLockInfoManager.increaseLockCount(eLockInfo, lockMode);
+ entityLockInfoManager.addHolder(eLockInfo, entityInfo);
+ }
+
+ if (jobInfo == null) {
+ jobInfo = new JobInfo(entityInfoManager, lockWaiterManager, txnContext);
+ jobHT.put(jobId, jobInfo); //jobId obj doesn't have to be created
+ }
+ jobInfo.addHoldingResource(entityInfo);
+
+ if (IS_DEBUG_MODE) {
+ trackLockRequest("Granted", RequestType.TRY_LOCK, datasetId, entityHashValue, lockMode, txnContext,
+ dLockInfo, eLockInfo);
+ }
+
+ unlatchLockTable();
+ return true;
+ }
+
+ //#. the datasetLockInfo exists in datasetResourceHT.
+ //1. handle dataset-granule lock
+ tryLockDatasetGranuleRevertOperation = 0;
+ entityInfo = tryLockDatasetGranule(datasetId, entityHashValue, lockMode, txnContext);
+ if (entityInfo == -2) {//-2 represents fail
+ isSuccess = false;
+ } else {
+ //2. handle entity-granule lock
+ if (entityHashValue != -1) {
+ isSuccess = tryLockEntityGranule(datasetId, entityHashValue, lockMode, entityInfo, txnContext);
+ if (!isSuccess) {
+ revertTryLockDatasetGranuleOperation(datasetId, entityHashValue, lockMode, entityInfo, txnContext);
}
- return true;
}
}
- if (shouldAbort) {
- requestTxrAbort(context);
- return false;
+ if (IS_DEBUG_MODE) {
+ if (isSuccess) {
+ trackLockRequest("Granted", RequestType.TRY_LOCK, datasetId, entityHashValue, lockMode, txnContext,
+ dLockInfo, eLockInfo);
+ } else {
+ trackLockRequest("Failed", RequestType.TRY_LOCK, datasetId, entityHashValue, lockMode, txnContext,
+ dLockInfo, eLockInfo);
+ }
}
- // Txr needs to wait, and it is safe
- synchronized (waitObj) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Transaction " + txId + " needs to wait for convert " + reqMode);
- }
- while (waitObj.needWait()) {
+ unlatchLockTable();
- waitObj.wait();
- }
+ return isSuccess;
+ }
- if (context.getStatus() == TransactionContext.TIMED_OUT_SATUS) { // selected
- // as
- // a
- // victim
- requestTxrAbort(context);
- woManager.deAllocate(waitObj);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Transaction " + txId + " wakes up and victimied for convert " + reqMode);
+ private void trackLockRequest(String msg, int requestType, DatasetId datasetIdObj, int entityHashValue,
+ byte lockMode, TransactionContext txnContext, DatasetLockInfo dLockInfo, int eLockInfo) {
+ StringBuilder s = new StringBuilder();
+ LockRequest request = new LockRequest(Thread.currentThread().getName(), requestType, datasetIdObj,
+ entityHashValue, lockMode, txnContext);
+ s.append(msg);
+ if (msg.equals("Granted")) {
+ if (dLockInfo != null) {
+ s.append("\t|D| ");
+ s.append(dLockInfo.getIXCount()).append(",");
+ s.append(dLockInfo.getISCount()).append(",");
+ s.append(dLockInfo.getXCount()).append(",");
+ s.append(dLockInfo.getSCount()).append(",");
+ if (dLockInfo.getFirstUpgrader() != -1) {
+ s.append("+");
+ } else {
+ s.append("-");
}
+ s.append(",");
+ if (dLockInfo.getFirstWaiter() != -1) {
+ s.append("+");
+ } else {
+ s.append("-");
+ }
+ }
+
+ if (eLockInfo != -1) {
+ s.append("\t|E| ");
+ s.append(entityLockInfoManager.getXCount(eLockInfo)).append(",");
+ s.append(entityLockInfoManager.getSCount(eLockInfo)).append(",");
+ if (entityLockInfoManager.getUpgrader(eLockInfo) != -1) {
+ s.append("+");
+ } else {
+ s.append("-");
+ }
+ s.append(",");
+ if (entityLockInfoManager.getFirstWaiter(eLockInfo) != -1) {
+ s.append("+");
+ } else {
+ s.append("-");
+ }
+ }
+ }
+
+ lockRequestTracker.addEvent(s.toString(), request);
+ if (msg.equals("Requested")) {
+ lockRequestTracker.addRequest(request);
+ }
+ System.out.println(request.prettyPrint() + "--> " + s.toString());
+ }
+
+ public String getHistoryForAllJobs() {
+ if (IS_DEBUG_MODE) {
+ return lockRequestTracker.getHistoryForAllJobs();
+ }
+ return null;
+ }
+
+ public String getHistoryPerJob() {
+ if (IS_DEBUG_MODE) {
+ return lockRequestTracker.getHistoryPerJob();
+ }
+ return null;
+ }
+
+ public String getRequestHistoryForAllJobs() {
+ if (IS_DEBUG_MODE) {
+ return lockRequestTracker.getRequestHistoryForAllJobs();
+ }
+ return null;
+ }
+
+ private void revertTryLockDatasetGranuleOperation(DatasetId datasetId, int entityHashValue, byte lockMode,
+ int entityInfo, TransactionContext txnContext) {
+ JobId jobId = txnContext.getJobId();
+ DatasetLockInfo dLockInfo;
+ JobInfo jobInfo;
+ int lockCount;
+ byte datasetLockMode = entityHashValue == -1 ? lockMode : lockMode == LockMode.S ? LockMode.IS : LockMode.IX;
+
+ dLockInfo = datasetResourceHT.get(datasetId);
+ jobInfo = jobHT.get(jobId);
+
+ //see tryLockDatasetGranule() function to know the revert operation
+ switch (tryLockDatasetGranuleRevertOperation) {
+
+ case 1://[revertOperation1]: reverting 'adding a holder'
+
+ if (entityHashValue == -1) {
+ dLockInfo.decreaseLockCount(datasetLockMode);
+ dLockInfo.removeHolder(entityInfo, jobInfo); //--> this call removes entityInfo from JobInfo's holding-resource-list as well.
+ } else {
+ dLockInfo.decreaseLockCount(datasetLockMode);
+ jobInfo.removeHoldingResource(entityInfo);
+ }
+ entityInfoManager.decreaseDatasetLockCount(entityInfo);
+ if (jobInfo.getLastHoldingResource() == -1 && jobInfo.getFirstWaitingResource() == -1) {
+ jobHT.remove(jobId);
+ }
+ entityInfoManager.deallocate(entityInfo);
+ break;
+
+ case 2://[revertOperation2]: reverting 'adding an upgrader'
+ lockCount = entityInfoManager.getDatasetLockCount(entityInfo);
+ if (entityHashValue == -1) { //dataset-granule lock
+ dLockInfo.decreaseLockCount(LockMode.X, lockCount);
+ dLockInfo.increaseLockCount(LockMode.S, lockCount);
+ } else {
+ dLockInfo.decreaseLockCount(LockMode.IX, lockCount);
+ dLockInfo.increaseLockCount(LockMode.IS, lockCount);
+ }
+ entityInfoManager.setDatasetLockMode(entityInfo, LockMode.S);
+ break;
+
+ case 3://[revertOperation3]: reverting 'adding a duplicated call'
+ entityInfoManager.decreaseDatasetLockCount(entityInfo);
+ datasetLockMode = entityInfoManager.getDatasetLockMode(entityInfo);
+ if (entityHashValue == -1) { //dataset-granule
+ dLockInfo.decreaseLockCount(datasetLockMode);
+ } else { //entity-granule
+ datasetLockMode = datasetLockMode == LockMode.S? LockMode.IS: LockMode.IX;
+ dLockInfo.decreaseLockCount(datasetLockMode);
+ }
+
+ break;
+ default:
+ //do nothing;
+ }
+ }
+
+ private int tryLockDatasetGranule(DatasetId datasetId, int entityHashValue, byte lockMode,
+ TransactionContext txnContext) throws ACIDException {
+ JobId jobId = txnContext.getJobId();
+ int jId = jobId.getId(); //int-type jobId
+ int dId = datasetId.getId(); //int-type datasetId
+ int waiterObjId;
+ int entityInfo = -1;
+ DatasetLockInfo dLockInfo;
+ JobInfo jobInfo;
+ boolean isUpgrade = false;
+ int weakerModeLockCount;
+ byte datasetLockMode = entityHashValue == -1 ? lockMode : lockMode == LockMode.S ? LockMode.IS : LockMode.IX;
+
+ dLockInfo = datasetResourceHT.get(datasetId);
+ jobInfo = jobHT.get(jobId);
+
+ //check duplicated call
+
+ //1. lock request causing duplicated upgrading requests from different threads in a same job
+ waiterObjId = dLockInfo.findUpgraderFromUpgraderList(jId, entityHashValue);
+ if (waiterObjId != -1) {
+ return -2;
+ }
+
+ //2. lock request causing duplicated waiting requests from different threads in a same job
+ waiterObjId = dLockInfo.findWaiterFromWaiterList(jId, entityHashValue);
+ if (waiterObjId != -1) {
+ return -2;
+ }
+
+ //3. lock request causing duplicated holding requests from different threads or a single thread in a same job
+ entityInfo = dLockInfo.findEntityInfoFromHolderList(jId, entityHashValue);
+ if (entityInfo == -1) { //new call from this job -> doesn't mean that eLockInfo doesn't exist since another thread might have create the eLockInfo already.
+
+ //////////////////////////////////////////////////////////////////////////////////////
+ //[part of revertOperation1]
+ entityInfo = entityInfoManager.allocate(jId, dId, entityHashValue, lockMode);
+ if (jobInfo == null) {
+ jobInfo = new JobInfo(entityInfoManager, lockWaiterManager, txnContext);
+ jobHT.put(jobId, jobInfo);
+ }
+ //////////////////////////////////////////////////////////////////////////////////////
+
+ //return fail if any upgrader exists or upgrading lock mode is not compatible
+ if (dLockInfo.getFirstUpgrader() != -1 || dLockInfo.getFirstWaiter() != -1
+ || !dLockInfo.isCompatible(datasetLockMode)) {
+
+ //[Notice]
+ //There has been no same caller as (jId, dId, entityHashValue) triplet.
+ //But there could be the same caller as (jId, dId) pair.
+ //For example, two requests (J1, D1, E1) and (J1, D1, E2) are considered as duplicated call in dataset-granule perspective.
+ //Therefore, the above duplicated call case is covered in the following code.
+ //find the same dataset-granule lock request, that is, (J1, D1) pair in the above example.
+ if (jobInfo.isDatasetLockGranted(dId, LockMode.IS)) {
+ if (dLockInfo.isCompatible(datasetLockMode)) {
+ //this is duplicated call
+ entityInfoManager.increaseDatasetLockCount(entityInfo);
+ if (entityHashValue == -1) {
+ dLockInfo.increaseLockCount(datasetLockMode);
+ dLockInfo.addHolder(entityInfo);
+ } else {
+ dLockInfo.increaseLockCount(datasetLockMode);
+ //IS and IX holders are implicitly handled.
+ }
+ //add entityInfo to JobInfo's holding-resource list
+ jobInfo.addHoldingResource(entityInfo);
+
+ tryLockDatasetGranuleRevertOperation = 1;
+
+ return entityInfo;
+ }
+ }
+
+ //revert [part of revertOperation1] before return
+ if (jobInfo.getLastHoldingResource() == -1 && jobInfo.getFirstWaitingResource() == -1) {
+ jobHT.remove(jobId);
+ }
+ entityInfoManager.deallocate(entityInfo);
+
+ return -2;
+ }
+
+ //////////////////////////////////////////////////////////////////////////////////////
+ //revert the following operations if the caller thread has to wait during this call.
+ //[revertOperation1]
+ entityInfoManager.increaseDatasetLockCount(entityInfo);
+ if (entityHashValue == -1) {
+ dLockInfo.increaseLockCount(datasetLockMode);
+ dLockInfo.addHolder(entityInfo);
+ } else {
+ dLockInfo.increaseLockCount(datasetLockMode);
+ //IS and IX holders are implicitly handled.
+ }
+ //add entityInfo to JobInfo's holding-resource list
+ jobInfo.addHoldingResource(entityInfo);
+
+ //set revert operation to be reverted when tryLock() fails
+ tryLockDatasetGranuleRevertOperation = 1;
+ //////////////////////////////////////////////////////////////////////////////////////
+
+ } else {
+ isUpgrade = isLockUpgrade(entityInfoManager.getDatasetLockMode(entityInfo), lockMode);
+ if (isUpgrade) { //upgrade call
+ //return fail if any upgrader exists or upgrading lock mode is not compatible
+ if (dLockInfo.getFirstUpgrader() != -1 || !dLockInfo.isUpgradeCompatible(datasetLockMode, entityInfo)) {
+ return -2;
+ }
+
+ //update entityInfo's dataset lock count and datasetLockInfo's lock count
+ weakerModeLockCount = entityInfoManager.getDatasetLockCount(entityInfo);
+
+ //////////////////////////////////////////////////////////////////////////////////////
+ //revert the following operations if the caller thread has to wait during this call.
+ //[revertOperation2]
+ entityInfoManager.setDatasetLockMode(entityInfo, lockMode);
+
+ if (entityHashValue == -1) { //dataset-granule lock
+ dLockInfo.increaseLockCount(LockMode.X, weakerModeLockCount);//new lock mode
+ dLockInfo.decreaseLockCount(LockMode.S, weakerModeLockCount);//current lock mode
+ } else {
+ dLockInfo.increaseLockCount(LockMode.IX, weakerModeLockCount);
+ dLockInfo.decreaseLockCount(LockMode.IS, weakerModeLockCount);
+ }
+ tryLockDatasetGranuleRevertOperation = 2;
+ //////////////////////////////////////////////////////////////////////////////////////
+
+ } else { //duplicated call
+
+ //////////////////////////////////////////////////////////////////////////////////////
+ //revert the following operations if the caller thread has to wait during this call.
+ //[revertOperation3]
+ entityInfoManager.increaseDatasetLockCount(entityInfo);
+ datasetLockMode = entityInfoManager.getDatasetLockMode(entityInfo);
+
+ if (entityHashValue == -1) { //dataset-granule
+ dLockInfo.increaseLockCount(datasetLockMode);
+ } else { //entity-granule
+ datasetLockMode = datasetLockMode == LockMode.S? LockMode.IS: LockMode.IX;
+ dLockInfo.increaseLockCount(datasetLockMode);
+ }
+
+ tryLockDatasetGranuleRevertOperation = 3;
+ //////////////////////////////////////////////////////////////////////////////////////
+
+ }
+ }
+
+ return entityInfo;
+ }
+
+ private boolean tryLockEntityGranule(DatasetId datasetId, int entityHashValue, byte lockMode,
+ int entityInfoFromDLockInfo, TransactionContext txnContext) throws ACIDException {
+ JobId jobId = txnContext.getJobId();
+ int jId = jobId.getId(); //int-type jobId
+ int waiterObjId;
+ int eLockInfo = -1;
+ int entityInfo;
+ DatasetLockInfo dLockInfo;
+ boolean isUpgrade = false;
+ int weakerModeLockCount;
+
+ dLockInfo = datasetResourceHT.get(datasetId);
+ eLockInfo = dLockInfo.getEntityResourceHT().get(entityHashValue);
+
+ if (eLockInfo != -1) {
+ //check duplicated call
+
+ //1. lock request causing duplicated upgrading requests from different threads in a same job
+ waiterObjId = entityLockInfoManager.findUpgraderFromUpgraderList(eLockInfo, jId, entityHashValue);
+ if (waiterObjId != -1) {
return false;
}
- }
- synchronized (context) {
- context.setStatus(TransactionContext.ACTIVE_STATUS);
- context.setStartWaitTime(TransactionContext.INVALID_TIME);
- }
-
- synchronized (lmTables) {
- txrInfo = lmTables.getTxrInfo(txId);
- if (txrInfo == null) {
- throw new ACIDException("Transaction " + txId + " removed from Txr Table Unexpectedlly");
+ //2. lock request causing duplicated waiting requests from different threads in a same job
+ waiterObjId = entityLockInfoManager.findWaiterFromWaiterList(eLockInfo, jId, entityHashValue);
+ if (waiterObjId != -1) {
+ return false;
}
- txrInfo.removeLock(resourceId, curMode, TxrInfo.NOT_KNOWN_IX);
- txrInfo.addGrantedLock(resourceId, reqMode);
- txrInfo.setWaitOnRid(null);
+
+ //3. lock request causing duplicated holding requests from different threads or a single thread in a same job
+ entityInfo = entityLockInfoManager.findEntityInfoFromHolderList(eLockInfo, jId, entityHashValue);
+ if (entityInfo != -1) {//duplicated call or upgrader
+
+ isUpgrade = isLockUpgrade(entityInfoManager.getEntityLockMode(entityInfo), lockMode);
+ if (isUpgrade) {//upgrade call
+ //wait if any upgrader exists or upgrading lock mode is not compatible
+ if (entityLockInfoManager.getUpgrader(eLockInfo) != -1
+ || !entityLockInfoManager.isUpgradeCompatible(eLockInfo, lockMode, entityInfo)) {
+ return false;
+ }
+
+ weakerModeLockCount = entityInfoManager.getEntityLockCount(entityInfo);
+ entityInfoManager.setEntityLockMode(entityInfo, lockMode);
+
+ entityLockInfoManager.increaseLockCount(eLockInfo, LockMode.X, (short) weakerModeLockCount);//new lock mode
+ entityLockInfoManager.decreaseLockCount(eLockInfo, LockMode.S, (short) weakerModeLockCount);//old lock mode
+
+ } else {//duplicated call
+ entityInfoManager.increaseEntityLockCount(entityInfo);
+ entityLockInfoManager.increaseLockCount(eLockInfo, entityInfoManager.getEntityLockMode(entityInfo));
+ }
+ } else {//new call from this job, but still eLockInfo exists since other threads hold it or wait on it
+ entityInfo = entityInfoFromDLockInfo;
+ if (entityLockInfoManager.getUpgrader(eLockInfo) != -1
+ || entityLockInfoManager.getFirstWaiter(eLockInfo) != -1
+ || !entityLockInfoManager.isCompatible(eLockInfo, lockMode)) {
+ return false;
+ }
+
+ entityInfoManager.increaseEntityLockCount(entityInfo);
+ entityLockInfoManager.increaseLockCount(eLockInfo, lockMode);
+ entityLockInfoManager.addHolder(eLockInfo, entityInfo);
+ }
+ } else {//eLockInfo doesn't exist, so this lock request is the first request and can be granted without waiting.
+ eLockInfo = entityLockInfoManager.allocate();
+ dLockInfo.getEntityResourceHT().put(entityHashValue, eLockInfo);
+ entityInfoManager.increaseEntityLockCount(entityInfoFromDLockInfo);
+ entityLockInfoManager.increaseLockCount(eLockInfo, lockMode);
+ entityLockInfoManager.addHolder(eLockInfo, entityInfoFromDLockInfo);
}
- woManager.deAllocate(waitObj);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Transaction " + txId + " wakes up and convert to " + reqMode);
- }
return true;
}
- @Override
- public boolean unlock(TransactionContext context, byte[] resourceID) throws ACIDException {
- long txId = context.getTransactionID();
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Transaction " + txId + " wants to unlock on ");
- }
- synchronized (lmTables) {
- TxrInfo txrInfo = lmTables.getTxrInfo(txId);
- if (txrInfo == null) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Transaction " + txId + " has no locks on resource ");
- }
- return true;
- }
-
- TInfo transactionInfo = txrInfo.getTxrInfo(resourceID, LockInfo.ANY_LOCK_MODE, TxrInfo.NOT_KNOWN_IX);
- if (transactionInfo == null) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Transaction " + txId + " has no locks on resource ");
- }
- return true;
- }
-
- int lockMode = transactionInfo.getMode();
-
- LockInfo lInfo = (LockInfo) lmTables.getLockInfo(resourceID);
- if (lInfo == null) {
- throw new ACIDException("Trying to unlock() a lock, on a non-existing resource");
- }
- txrInfo.removeLock(resourceID, lockMode, TxrInfo.NOT_KNOWN_IX);
- lInfo.removeFromGranted(txId, lockMode, LockInfo.UNKNOWN_IX);
-
- Iterator<WaitingInfo> convIt = lInfo.getIteratorOnConverter();
- while (convIt.hasNext()) {
- WaitingInfo nextConvInfo = convIt.next();
- if (nextConvInfo.isVictim()) {
- continue;
- }
- WaitEntry nextConv = nextConvInfo.getWaitingEntry();
- synchronized (nextConv) {
- int reqIx = nextConv.getIX(); // entry ix for the (new)
- // requested lock
- long convIx = lInfo.getTxId(reqIx);
- long convTxId = lInfo.getTxId(reqIx);
- int curConvMode = lInfo.getGrantedLockMode(convTxId, LockInfo.UNKNOWN_IX);
- int reqConvMode = lInfo.getLockMode(reqIx);
- int updatedMask = lInfo.getUpdatedMask(convIx, curConvMode, LockInfo.UNKNOWN_IX);
- if (lMtx.conflicts(updatedMask, reqConvMode)) { // We found
- // conflict,
- // no more
- // transactions
- // need to
- // be waken
- // up
- context.setStartWaitTime(TransactionContext.INVALID_TIME);
- if (txrInfo.getSize() == 0) {
- lmTables.removeTxrInfo(txId);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Entry for Transaction " + txId + " removed from Txr Table (in unlock)");
- }
- }
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Transaction " + txId + " unlocked its lock");
- }
- return true;
- }
- // Converter is ready to wake up
- lmTables.getTxrInfo(convTxId).getContext().setStatus(TransactionContext.ACTIVE_STATUS);
- lInfo.removeFromGranted(convTxId, curConvMode, LockInfo.UNKNOWN_IX /* curIx */);
- lInfo.addToGranted(convTxId, reqConvMode, LockInfo.NOT_FOUND);
- lInfo.prepareToRemoveFromConverters(convTxId, reqConvMode, reqIx);
- nextConv.wakeUp();
- convIt.remove();
- nextConv.notifyAll();
- }
- }
-
- Iterator<WaitingInfo> waitIt = lInfo.getIteratorOnWaiters();
- while (waitIt.hasNext()) {
- WaitingInfo nextWaiterInfo = waitIt.next();
- if (nextWaiterInfo.isVictim()) {
- continue;
- }
- WaitEntry nextWaiter = nextWaiterInfo.getWaitingEntry();
- synchronized (nextWaiter) {
- int waitIx = nextWaiter.getIX();
- long waitTxId = lInfo.getTxId(waitIx);
- int reqLock = lInfo.getLockMode(waitIx);
- int mask = lInfo.getMask();
- if (lMtx.conflicts(mask, reqLock)) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Transaction " + txId + " unlocked its lock on ");
- }
-
- context.setStartWaitTime(TransactionContext.INVALID_TIME);
- if (txrInfo.getSize() == 0) {
- lmTables.removeTxrInfo(txId);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Entry for Transaction " + txId + " removed from Txr Table (in unlock)");
- }
- }
- return true;
- }
- lInfo.addToGranted(waitTxId, reqLock, LockInfo.NOT_FOUND);
- lInfo.prepareToRemoveFromWaiters(waitTxId, reqLock, waitIx);
- nextWaiter.wakeUp();
- waitIt.remove();
- nextWaiter.notifyAll();
- }
- }
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Transaction " + txId + " unlocked its lock");
- }
- context.setStartWaitTime(TransactionContext.INVALID_TIME);
- if (txrInfo.getSize() == 0) {
- lmTables.removeTxrInfo(txId);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Entry for Transaction " + txId + " removed from Txr Table (in unlock)");
- }
- }
- return true;
- }
+ private void latchLockTable() {
+ lockTableLatch.writeLock().lock();
}
- @Override
- public boolean getInstantlock(TransactionContext context, byte[] resourceID, int mode) throws ACIDException {
- throw new ACIDException("Instant Locking is not supported");
+ private void unlatchLockTable() {
+ lockTableLatch.writeLock().unlock();
}
- public Iterator<Long> getTxrInfoIterator() {
- return lmTables.getIteratorOnTxrs();
+ private void latchWaitNotify() {
+ waiterLatch.writeLock().lock();
}
- @Override
- public synchronized Boolean releaseLocks(TransactionContext context) throws ACIDException {
- long txId = context.getTransactionID();
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Entry for Transaction " + txId + " removed from Txr Table (in unlock)");
- }
- synchronized (lmTables) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Transaction " + txId + " started releasing its locks !");
- }
- TxrInfo txrInfo = lmTables.getTxrInfo(txId);
- if (txrInfo == null) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Transaction with ID " + txId
- + " has no locks to release. (Returning from Release Locks)");
- }
- return true;
- }
- // First Remove from the waiting list (if waiting)
- byte[] waitOnRid = txrInfo.getWaitOnRid();
- if (waitOnRid != null) {
- LockInfo lInfo = (LockInfo) lmTables.getLockInfo(waitOnRid);
- if ((lInfo.removeFromConverters(txId)) == null) {
- if ((lInfo.removeFromWaiters(txId)) == null) {
- throw new ACIDException("Transaction " + txId
- + " Not Found in the convert/wait list of the resource, it should have waited for");
- }
- }
- }
+ private void unlatchWaitNotify() {
+ waiterLatch.writeLock().unlock();
+ }
- Iterator<TInfo> tInfoIt = txrInfo.getIterator();
+ private int handleLockWaiter(DatasetLockInfo dLockInfo, int eLockInfo, int entityInfo, boolean isUpgrade,
+ boolean isDatasetLockInfo, TransactionContext txnContext, JobInfo jobInfo, int duplicatedWaiterObjId)
+ throws ACIDException {
+ int waiterId = -1;
+ LockWaiter waiter;
+ int waiterCount = 0;
+ boolean isInterruptedExceptionOccurred = false;
- while (tInfoIt.hasNext()) {
- TInfo nextInfo = tInfoIt.next();
- byte[] nextRid = nextInfo.getResourceId();
- int nextLockMode = nextInfo.getMode();
- LockInfo lInfo = lmTables.getLockInfo(nextRid);
- lInfo.removeFromGranted(txId, nextLockMode, LockInfo.UNKNOWN_IX); // Remove
- // transaction's
- // granted
- // lock
- // Now lets try to wake up Waiting Transactions
- // First go through the ConvertList
- Iterator<WaitingInfo> convIt = lInfo.getIteratorOnConverter();
- boolean checkWaiters = true;
- while (convIt.hasNext()) {
- WaitingInfo nextConvInfo = convIt.next();
- if (nextConvInfo.isVictim()) {
- continue;
- }
- WaitEntry nextConv = nextConvInfo.getWaitingEntry();
- synchronized (nextConv) {
- int reqIx = nextConv.getIX();
- long convIx = lInfo.getTxId(reqIx);
- int curIx = lInfo.findInGrantedList(convIx, LockInfo.ANY_LOCK_MODE); // index
- // of
- // the
- // entry
- // for
- // the
- // (old)
- // already
- // granted
- // lock
- long convTxId = lInfo.getTxId(reqIx);
- int curConvMode = lInfo.getGrantedLockMode(convTxId, curIx);
- int reqConvMode = lInfo.getLockMode(reqIx);
- int updatedMask = lInfo.getUpdatedMask(convIx, curConvMode, curIx);
- if (lMtx.conflicts(updatedMask, reqConvMode)) {
- checkWaiters = false;
- break;
- }
- lInfo.removeFromGranted(convTxId, curConvMode, curIx);
- lInfo.addToGranted(convTxId, reqConvMode, LockInfo.NOT_FOUND);
- lInfo.prepareToRemoveFromConverters(convTxId, reqConvMode, reqIx);
- lmTables.getTxrInfo(convTxId).getContext().setStartWaitTime(TransactionContext.INVALID_TIME);
- nextConv.wakeUp();
- convIt.remove();
- nextConv.notifyAll();
- }
- }
-
- if (checkWaiters) {
- // Going through the WaitList
- Iterator<WaitingInfo> waitIt = lInfo.getIteratorOnWaiters();
- while (waitIt.hasNext()) {
- WaitingInfo nextWaiterInfo = waitIt.next();
- if (nextWaiterInfo.isVictim()) {
- continue;
- }
- WaitEntry nextWaiter = nextWaiterInfo.getWaitingEntry();
- synchronized (nextWaiter) {
- int waitIx = nextWaiter.getIX();
- long waitTxId = lInfo.getTxId(waitIx);
- int reqLock = lInfo.getLockMode(waitIx);
- int mask = lInfo.getMask();
- if (lMtx.conflicts(mask, reqLock)) {
- break;
- }
- lInfo.addToGranted(waitTxId, reqLock, LockInfo.NOT_FOUND);
- lInfo.prepareToRemoveFromWaiters(waitTxId, reqLock, waitIx);
- lmTables.getTxrInfo(waitTxId).getContext()
- .setStartWaitTime(TransactionContext.INVALID_TIME);
- nextWaiter.wakeUp();
- waitIt.remove();
- nextWaiter.notifyAll();
- }
- }
- }
- }
-
- context.setStartWaitTime(TransactionContext.INVALID_TIME);
- if ((lmTables.removeTxrInfo(txId)) == null) { // Remove Txr's entry
- // from the
- // transactions' table
- throw new ACIDException("Transaction " + txId + " Not found in transactions table for removal");
+ if (duplicatedWaiterObjId != -1 || isDeadlockFree(dLockInfo, eLockInfo, entityInfo, isDatasetLockInfo, isUpgrade)) {//deadlock free -> wait
+ if (duplicatedWaiterObjId == -1) {
+ waiterId = lockWaiterManager.allocate(); //initial value of waiterObj: wait = true, victim = false
+ waiter = lockWaiterManager.getLockWaiter(waiterId);
+ waiter.setEntityInfoSlot(entityInfo);
+ jobInfo.addWaitingResource(waiterId);
+ waiter.setBeginWaitTime(System.currentTimeMillis());
} else {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Entry for Transaction " + txId + " removed from Txr Table (in release locks)");
+ waiterId = duplicatedWaiterObjId;
+ waiter = lockWaiterManager.getLockWaiter(waiterId);
+ }
+
+ if (duplicatedWaiterObjId == -1) {
+ //add actor properly
+ if (isDatasetLockInfo) {
+ waiter.setWaitingOnEntityLock(false);
+ if (isUpgrade) {
+ dLockInfo.addUpgrader(waiterId);
+ waiter.setWaiter(false);
+ } else {
+ dLockInfo.addWaiter(waiterId);
+ waiter.setWaiter(true);
+ }
+ } else {
+ waiter.setWaitingOnEntityLock(true);
+ if (isUpgrade) {
+ waiter.setWaiter(false);
+ entityLockInfoManager.addUpgrader(eLockInfo, waiterId);
+ } else {
+ waiter.setWaiter(true);
+ entityLockInfoManager.addWaiter(eLockInfo, waiterId);
+ }
}
}
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Transaction " + txId + " released its locks successfully !");
+ waiter.increaseWaiterCount();
+ waiter.setFirstGetUp(true);
+
+ latchWaitNotify();
+ unlatchLockTable();
+ synchronized (waiter) {
+ unlatchWaitNotify();
+ while (waiter.needWait()) {
+ try {
+ if (IS_DEBUG_MODE) {
+ System.out.println("" + Thread.currentThread().getName() + "\twaits("
+ + waiter.getWaiterCount() + "): WID(" + waiterId + "),EID("
+ + waiter.getEntityInfoSlot() + ")");
+ }
+ waiter.wait();
+ } catch (InterruptedException e) {
+ //TODO figure-out what is the appropriate way to handle this exception
+ e.printStackTrace();
+ isInterruptedExceptionOccurred = true;
+ waiter.setWait(false);
+ }
+ }
+ }
+
+ if (isInterruptedExceptionOccurred) {
+ throw new ACIDException("InterruptedException is caught");
}
- return true;
+ //waiter woke up -> remove/deallocate waiter object and abort if timeout
+ latchLockTable();
+
+ if (txnContext.getStatus() == TransactionContext.TIMED_OUT_STATUS || waiter.isVictim()) {
+ try {
+ requestAbort(txnContext);
+ } finally {
+ unlatchLockTable();
+ }
+ }
+
+ if (waiter.isFirstGetUp()) {
+ waiter.setFirstGetUp(false);
+ waiterCount = waiter.getWaiterCount();
+ } else {
+ waiterCount = 0;
+ }
+
+ waiter.decreaseWaiterCount();
+ if (IS_DEBUG_MODE) {
+ System.out.println("" + Thread.currentThread().getName() + "\tgot-up!(" + waiter.getWaiterCount()
+ + "): WID(" + waiterId + "),EID(" + waiter.getEntityInfoSlot() + ")");
+ }
+ if (waiter.getWaiterCount() == 0) {
+ //remove actor properly
+ if (isDatasetLockInfo) {
+ if (isUpgrade) {
+ dLockInfo.removeUpgrader(waiterId);
+ } else {
+ dLockInfo.removeWaiter(waiterId);
+ }
+ } else {
+ if (isUpgrade) {
+ entityLockInfoManager.removeUpgrader(eLockInfo, waiterId);
+ } else {
+ entityLockInfoManager.removeWaiter(eLockInfo, waiterId);
+ }
+ }
+
+ //if (!isUpgrade && isDatasetLockInfo) {
+ jobInfo.removeWaitingResource(waiterId);
+ //}
+ lockWaiterManager.deallocate(waiterId);
+ }
+
+ } else { //deadlock -> abort
+ //[Notice]
+ //Before requesting abort, the entityInfo for waiting datasetLock request is deallocated.
+ if (!isUpgrade && isDatasetLockInfo) {
+ //deallocate the entityInfo
+ entityInfoManager.deallocate(entityInfo);
+ }
+ try {
+ requestAbort(txnContext);
+ } finally {
+ unlatchLockTable();
+ }
}
+
+ return waiterCount;
}
- private boolean isDeadlockFree(long txId, byte[] resourceId) {
- return deadlockDetector.isSafeToAdd(txId, resourceId);
+ private boolean isDeadlockFree(DatasetLockInfo dLockInfo, int eLockInfo, int entityInfo, boolean isDatasetLockInfo, boolean isUpgrade) {
+ return deadlockDetector.isSafeToAdd(dLockInfo, eLockInfo, entityInfo, isDatasetLockInfo, isUpgrade);
}
- private void requestTxrAbort(TransactionContext context) throws ACIDException {
- context.setStartWaitTime(TransactionContext.INVALID_TIME);
- throw new ACIDException("Transaction " + context.getTransactionID()
+ private void requestAbort(TransactionContext txnContext) throws ACIDException {
+ txnContext.setStatus(TransactionContext.TIMED_OUT_STATUS);
+ txnContext.setStartWaitTime(TransactionContext.INVALID_TIME);
+ throw new ACIDException("Transaction " + txnContext.getJobId()
+ " should abort (requested by the Lock Manager)");
}
+
+
+ /**
+ * For now, upgrading lock granule from entity-granule to dataset-granule is not supported!!
+ *
+ * @param fromLockMode
+ * @param toLockMode
+ * @return
+ */
+ private boolean isLockUpgrade(byte fromLockMode, byte toLockMode) {
+ return fromLockMode == LockMode.S && toLockMode == LockMode.X;
+ }
+
+ /**
+ * wake up upgraders first, then waiters.
+ * Criteria to wake up upgraders: if the upgrading lock mode is compatible, then wake up the upgrader.
+ */
+ private void wakeUpDatasetLockWaiters(DatasetLockInfo dLockInfo) {
+ int waiterObjId = dLockInfo.getFirstUpgrader();
+ int entityInfo;
+ LockWaiter waiterObj;
+ byte datasetLockMode;
+ byte lockMode;
+ boolean areAllUpgradersAwaken = true;
+
+ consecutiveWakeupContext.reset();
+ while (waiterObjId != -1) {
+ //wake up upgraders
+ waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
+ entityInfo = waiterObj.getEntityInfoSlot();
+ datasetLockMode = entityInfoManager.getPKHashVal(entityInfo) == -1 ? LockMode.X : LockMode.IX;
+ if (dLockInfo.isUpgradeCompatible(datasetLockMode, entityInfo)
+ && consecutiveWakeupContext.isCompatible(datasetLockMode)) {
+ consecutiveWakeupContext.setLockMode(datasetLockMode);
+ //compatible upgrader is waken up
+ latchWaitNotify();
+ synchronized (waiterObj) {
+ unlatchWaitNotify();
+ waiterObj.setWait(false);
+ if (IS_DEBUG_MODE) {
+ System.out.println("" + Thread.currentThread().getName() + "\twake-up(D): WID(" + waiterObjId
+ + "),EID(" + waiterObj.getEntityInfoSlot() + ")");
+ }
+ waiterObj.notifyAll();
+ }
+ waiterObjId = waiterObj.getNextWaiterObjId();
+ } else {
+ areAllUpgradersAwaken = false;
+ break;
+ }
+ }
+
+ if (areAllUpgradersAwaken) {
+ //wake up waiters
+ waiterObjId = dLockInfo.getFirstWaiter();
+ while (waiterObjId != -1) {
+ waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
+ entityInfo = waiterObj.getEntityInfoSlot();
+ lockMode = entityInfoManager.getDatasetLockMode(entityInfo);
+ datasetLockMode = entityInfoManager.getPKHashVal(entityInfo) == -1 ? lockMode
+ : lockMode == LockMode.S ? LockMode.IS : LockMode.IX;
+ if (dLockInfo.isCompatible(datasetLockMode) && consecutiveWakeupContext.isCompatible(datasetLockMode)) {
+ consecutiveWakeupContext.setLockMode(datasetLockMode);
+ //compatible waiter is waken up
+ latchWaitNotify();
+ synchronized (waiterObj) {
+ unlatchWaitNotify();
+ waiterObj.setWait(false);
+ if (IS_DEBUG_MODE) {
+ System.out.println("" + Thread.currentThread().getName() + "\twake-up(D): WID("
+ + waiterObjId + "),EID(" + waiterObj.getEntityInfoSlot() + ")");
+ }
+ waiterObj.notifyAll();
+ }
+ waiterObjId = waiterObj.getNextWaiterObjId();
+ } else {
+ break;
+ }
+ }
+ }
+ }
+
+ private void wakeUpEntityLockWaiters(int eLockInfo) {
+ boolean areAllUpgradersAwaken = true;
+ int waiterObjId = entityLockInfoManager.getUpgrader(eLockInfo);
+ int entityInfo;
+ LockWaiter waiterObj;
+ byte entityLockMode;
+
+ consecutiveWakeupContext.reset();
+ while (waiterObjId != -1) {
+ //wake up upgraders
+ waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
+ entityInfo = waiterObj.getEntityInfoSlot();
+ if (entityLockInfoManager.isUpgradeCompatible(eLockInfo, LockMode.X, entityInfo) && consecutiveWakeupContext.isCompatible(LockMode.X)) {
+ consecutiveWakeupContext.setLockMode(LockMode.X);
+ latchWaitNotify();
+ synchronized (waiterObj) {
+ unlatchWaitNotify();
+ waiterObj.setWait(false);
+ if (IS_DEBUG_MODE) {
+ System.out.println("" + Thread.currentThread().getName() + "\twake-up(E): WID(" + waiterObjId
+ + "),EID(" + waiterObj.getEntityInfoSlot() + ")");
+ }
+ waiterObj.notifyAll();
+ }
+ waiterObjId = waiterObj.getNextWaiterObjId();
+ } else {
+ areAllUpgradersAwaken = false;
+ break;
+ }
+ }
+
+ if (areAllUpgradersAwaken) {
+ //wake up waiters
+ waiterObjId = entityLockInfoManager.getFirstWaiter(eLockInfo);
+ while (waiterObjId != -1) {
+ waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
+ entityInfo = waiterObj.getEntityInfoSlot();
+ entityLockMode = entityInfoManager.getEntityLockMode(entityInfo);
+ if (entityLockInfoManager.isCompatible(eLockInfo, entityLockMode) && consecutiveWakeupContext.isCompatible(entityLockMode)) {
+ consecutiveWakeupContext.setLockMode(entityLockMode);
+ //compatible waiter is waken up
+ latchWaitNotify();
+ synchronized (waiterObj) {
+ unlatchWaitNotify();
+ waiterObj.setWait(false);
+ if (IS_DEBUG_MODE) {
+ System.out.println("" + Thread.currentThread().getName() + "\twake-up(E): WID("
+ + waiterObjId + "),EID(" + waiterObj.getEntityInfoSlot() + ")");
+ }
+ waiterObj.notifyAll();
+ }
+ } else {
+ break;
+ }
+ waiterObjId = waiterObj.getNextWaiterObjId();
+ }
+ }
+ }
@Override
- public String getDebugLockStatus() throws ACIDException {
- String s = "\nLock Status (For Debug Purpose):\n";
- synchronized (lmTables) {
- Iterator<Long> txIdIt = getTxrInfoIterator();
- while (txIdIt.hasNext()) {
- long nextTxId = txIdIt.next();
- TxrInfo nextInfoList = lmTables.getTxrInfo(nextTxId);
- byte[] nextWaitOnRid = nextInfoList.getWaitOnRid();
- String status = (nextWaitOnRid == null ? " ACTIVE" : " WAITING");
- if ((nextWaitOnRid != null)) {
- LockInfo lInfo = (LockInfo) lmTables.getLockInfo(nextWaitOnRid);
- int wlModeIx = lInfo.findInConvertList(nextTxId, LockInfo.ANY_LOCK_MODE);
- if (wlModeIx == LockInfo.NOT_FOUND) {
- wlModeIx = lInfo.findInWaitList(nextTxId, LockInfo.ANY_LOCK_MODE);
- }
- int wlMode = lInfo.getLockMode(wlModeIx);
- String wLModeRep = (wlMode == 0 ? "S" : "X");
- status += " for " + wLModeRep + " lock";
- }
-
- String lockModes = "";
- Iterator<TInfo> tInfoIt = nextInfoList.getIterator();
- while (tInfoIt.hasNext()) {
- TInfo next = tInfoIt.next();
- int nextLockMode = next.getMode();
- lockModes += (nextLockMode == 0 ? "S" : "X");
- lockModes += ", ";
- }
- s += "Transaction: " + nextTxId + "\t- (Status: " + status + ") --> Granted Locks List: ( " + lockModes
- + " )\n";
- }
-
- }
+ public String prettyPrint() throws ACIDException {
+ StringBuilder s = new StringBuilder("\n########### LockManager Status #############\n");
return s + "\n";
}
public void sweepForTimeout() throws ACIDException {
- synchronized (lmTables) {
- Iterator<Long> txrIt = lmTables.getIteratorOnTxrs();
- while (txrIt.hasNext()) {
- long nextTxrID = txrIt.next();
- TxrInfo nextTxrInfo = lmTables.getTxrInfo(nextTxrID);
- if (toutDetector.isVictim(nextTxrInfo)) {
- nextTxrInfo.getContext().setStatus(TransactionContext.TIMED_OUT_SATUS);
- LockInfo nextLockInfo = lmTables.getLockInfo(nextTxrInfo.getWaitOnRid());
- synchronized (nextLockInfo) {
- WaitingInfo nextVictim = nextLockInfo.getWaitingOnObject(nextTxrID, LockInfo.ANY_LOCK_MODE);
- nextVictim.setAsVictim();
- toutDetector.addToVictimsList(nextVictim.getWaitingEntry());
- }
- }
+ JobInfo jobInfo;
+ int waiterObjId;
+ LockWaiter waiterObj;
+
+ latchLockTable();
+
+ Iterator<Entry<JobId, JobInfo>> iter = jobHT.entrySet().iterator();
+ while (iter.hasNext()) {
+ Map.Entry<JobId, JobInfo> pair = (Map.Entry<JobId, JobInfo>) iter.next();
+ jobInfo = pair.getValue();
+ waiterObjId = jobInfo.getFirstWaitingResource();
+ while (waiterObjId != -1) {
+ waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
+ toutDetector.checkAndSetVictim(waiterObj);
+ waiterObjId = waiterObj.getNextWaiterObjId();
}
}
- }
- public LockInfo getLockInfo(byte[] resourceID) {
- return lmTables.getLockInfo(resourceID);
- }
-
- public TxrInfo getTxrInfo(long txrId) {
- return lmTables.getTxrInfo(txrId);
+ unlatchLockTable();
}
}
-class LMTables {
- /**
- * An instance of this class mainly manages and synchronizes the access to
- * the lock manager hash tables
- */
+class ConsecutiveWakeupContext {
+ private boolean IS;
+ private boolean IX;
+ private boolean S;
+ private boolean X;
- private ILockHashTable<byte[], LockInfo> resToLInfo; // mapping from
- // resourceID to
- // information about
- // the locks on it
- private ILockHashTable<Long, TxrInfo> txToTxrInfo; // mapping from
- // transactionID to
- // informations about its
- // lock(s)
-
- public LMTables(int initialSize) {
- resToLInfo = new ResourcesHT(initialSize);
- txToTxrInfo = new TransactionsHT(initialSize);
+ public void reset() {
+ IS = false;
+ IX = false;
+ S = false;
+ X = false;
}
- public LockInfo getLockInfo(byte[] resourceId) {
- return resToLInfo.get(resourceId);
- }
+ public boolean isCompatible(byte lockMode) {
+ switch (lockMode) {
+ case LockMode.IX:
+ return !S && !X;
- public void putLockInfo(byte[] resourceID, LockInfo lInfo) {
- resToLInfo.put(resourceID, lInfo);
- }
+ case LockMode.IS:
+ return !X;
- public TxrInfo getTxrInfo(long txrId) {
- return txToTxrInfo.get(txrId);
- }
+ case LockMode.X:
+ return !IS && !IX && !S && !X;
- public void putTxrInfo(long txrId, TxrInfo txrInfo) {
- txToTxrInfo.put(txrId, txrInfo);
- }
+ case LockMode.S:
+ return !IX && !X;
- public TxrInfo removeTxrInfo(long txId) {
- return txToTxrInfo.remove(txId);
- }
-
- public int getTxrTableSize() {
- return txToTxrInfo.getKeysetSize();
- }
-
- public Iterator<Long> getIteratorOnTxrs() {
- return ((TransactionsHT) txToTxrInfo).getIteratorOnTxs();
- }
-
-}
-
-class ResourcesHT implements ILockHashTable<byte[], LockInfo> {
-
- private Hashtable<LockTag, LockInfo> table;
- private LockTag tag;
-
- public ResourcesHT(int initCapacity) {
- this.table = new Hashtable<LockTag, LockInfo>(initCapacity);
- this.tag = new LockTag(null);
- }
-
- @Override
- public synchronized void put(byte[] rsId, LockInfo info) {
- table.put(new LockTag(rsId), (LockInfo) info);
- }
-
- @Override
- public synchronized LockInfo get(byte[] rsId) {
- tag.setRsId(rsId);
- return (table.get(tag));
- }
-
- @Override
- public LockInfo remove(byte[] rsId) {
- tag.setRsId(rsId);
- return (table.remove(tag));
- }
-
- @Override
- public int getKeysetSize() {
- return table.size();
- }
-
-}
-
-class TransactionsHT implements ILockHashTable<Long, TxrInfo> {
-
- private Hashtable<Long, TxrInfo> table;
-
- public TransactionsHT(int initCapacity) {
- this.table = new Hashtable<Long, TxrInfo>(initCapacity);
- }
-
- @Override
- public synchronized void put(Long key, TxrInfo value) {
- table.put(key, value);
-
- }
-
- @Override
- public synchronized TxrInfo get(Long key) {
- return (table.get(key));
- }
-
- public Iterator<Long> getIteratorOnTxs() {
- return table.keySet().iterator();
- }
-
- @Override
- public TxrInfo remove(Long key) {
- return table.remove(key);
- }
-
- @Override
- public int getKeysetSize() {
- return table.size();
- }
-
-}
-
-class LockTag {
- /**
- * Used as a wrapper around byte[], which is used as the key for the
- * hashtables
- */
-
- byte[] rsId;
-
- public LockTag(byte[] rsId) {
- setRsId(rsId);
- }
-
- public void setRsId(byte[] rsId) {
- this.rsId = rsId;
- }
-
- @Override
- public int hashCode() {
- return Arrays.hashCode(rsId);
- }
-
- @Override
- public boolean equals(Object o) {
- if ((o == null) || !(o instanceof LockTag)) {
- return false;
- }
- return Arrays.equals(((LockTag) o).rsId, this.rsId);
- }
-}
-
-class WaitObjectManager {
- /**
- * Manages the set of waiting objects (objects used to manage waiters) to
- * avoid object/garbage creation as much as possible
- */
- final int EOL = -1;
- ArrayList<WaitEntry> list;
- AtomicInteger max;
- int nextFree;
-
- public WaitObjectManager() {
- list = new ArrayList<WaitEntry>();
- nextFree = EOL;
- max = new AtomicInteger(0);
- }
-
- public WaitEntry allocate() throws ACIDException {
- WaitEntry o = null;
- synchronized (list) {
- if (nextFree == EOL) {
- o = new WaitEntry(max.getAndIncrement(), LockInfo.UNKNOWN_IX, EOL);
- list.add(o);
- return o;
- }
- o = list.get(nextFree);
- nextFree = o.getNext();
- o.setNext(EOL);
- }
- return o;
- }
-
- public void deAllocate(Object o) {
- synchronized (list) {
- ((WaitEntry) o).setNext(nextFree);
- nextFree = ((WaitEntry) o).getId();
+ default:
+ throw new IllegalStateException("Invalid upgrade lock mode");
}
}
+ public void setLockMode(byte lockMode) {
+ switch (lockMode) {
+ case LockMode.IX:
+ IX = true;
+ return;
+
+ case LockMode.IS:
+ IS = true;
+ return;
+
+ case LockMode.X:
+ X = true;
+ return;
+
+ case LockMode.S:
+ S = true;
+ return;
+
+ default:
+ throw new IllegalStateException("Invalid lock mode");
+ }
+
+ }
+
}
-class WaitEntry {
- /**
- * Captures the information about a waiting transaction
- */
+/******************************************
+ * datasetResourceHT
+ ******************************************/
+/*
+class DatasetId implements Serializable {
+ int id;
- private int id; // ID of this object (used for managing the waiting objects
- // and recycling them)
- private int eix; // index of the entry corresponding to the waiting
- // transaction
- private boolean shouldWait; // whether the waiter needs to continue its
- // waiting or not
- private int next; // The next waitEntry in the chain of wait Entries (used
- // for managing the waiting objects and recycling them)
-
- public WaitEntry(int id, int eix, int next) {
+ public DatasetId(int id) {
this.id = id;
- this.eix = eix;
- shouldWait = true;
- this.next = next;
}
- public int getIX() {
- return eix;
- }
-
- public void setIx(int eix) {
- this.eix = eix;
+ public void setId(int id) {
+ this.id = id;
}
public int getId() {
return id;
}
- public void setNext(int n) {
- next = n;
+ @Override
+ public int hashCode() {
+ return id;
}
- public int getNext() {
- return next;
+ @Override
+ public boolean equals(Object o) {
+ if ((o == null) || !(o instanceof DatasetId)) {
+ return false;
+ }
+ return ((DatasetId) o).id == this.id;
}
-
- public boolean needWait() {
- return shouldWait;
- }
-
- public void wakeUp() {
- this.shouldWait = false;
- }
-
- public void setForWait() {
- this.shouldWait = true;
- }
-}
\ No newline at end of file
+};
+*/
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManagerDeterministicUnitTest.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManagerDeterministicUnitTest.java
new file mode 100644
index 0000000..063194b
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManagerDeterministicUnitTest.java
@@ -0,0 +1,617 @@
+package edu.uci.ics.asterix.transaction.management.service.locking;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.NoSuchElementException;
+import java.util.Scanner;
+
+import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
+import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionManager.TransactionState;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
+
+public class LockManagerDeterministicUnitTest {
+
+ public static void main(String args[]) throws ACIDException, IOException {
+ //initialize controller thread
+ String requestFileName = new String(
+ "src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockRequestFile");
+ Thread t = new Thread(new LockRequestController(requestFileName));
+ t.start();
+ }
+}
+
+class LockRequestController implements Runnable {
+
+ public static final boolean IS_DEBUG_MODE = false;
+ TransactionProvider txnProvider;
+ WorkerReadyQueue workerReadyQueue;
+ ArrayList<LockRequest> requestList;
+ ArrayList<ArrayList<Integer>> expectedResultList;
+ int resultListIndex;
+ LockManager lockMgr;
+ String requestFileName;
+ long defaultWaitTime;
+
+ public LockRequestController(String requestFileName) throws ACIDException {
+ this.txnProvider = new TransactionProvider("LockManagerPredefinedUnitTest");;
+ this.workerReadyQueue = new WorkerReadyQueue();
+ this.requestList = new ArrayList<LockRequest>();
+ this.expectedResultList = new ArrayList<ArrayList<Integer>>();
+ this.lockMgr = (LockManager) txnProvider.getLockManager();
+ this.requestFileName = new String(requestFileName);
+ this.resultListIndex = 0;
+ this.defaultWaitTime = 10;
+ }
+
+ @Override
+ public void run() {
+ Thread.currentThread().setName("Thread-0");
+ HashMap<String, Thread> threadMap = new HashMap<String, Thread>();
+ Thread t = null;
+ LockRequest lockRequest = null;
+ boolean isSuccess = true;
+
+ try {
+ readRequest();
+ } catch (IOException e) {
+ e.printStackTrace();
+ System.exit(-1);
+ } catch (ACIDException e) {
+ e.printStackTrace();
+ System.exit(-1);
+ }
+
+ //initialize workerThread
+ int size = requestList.size();
+ for (int i = 0; i < size; i++) {
+ lockRequest = requestList.get(i);
+ if (lockRequest.threadName.equals("Thread-0")) {
+ //Thread-0 is controller thread.
+ continue;
+ }
+ t = threadMap.get(lockRequest.threadName);
+ if (t == null) {
+ t = new Thread(new LockRequestWorker(txnProvider, workerReadyQueue, lockRequest.threadName),
+ lockRequest.threadName);
+ threadMap.put(lockRequest.threadName, t);
+ t.start();
+ log("Created " + lockRequest.threadName);
+ }
+ }
+
+ //wait for all workerThreads to be ready
+ try {
+ log("waiting for all workerThreads to complete initialization ...");
+ Thread.sleep(5);
+ } catch (InterruptedException e1) {
+ e1.printStackTrace();
+ }
+ while (workerReadyQueue.size() != threadMap.size()) {
+ try {
+ log(" .");
+ Thread.sleep(5);
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ }
+
+ //make workerThread work
+ while (requestList.size() != 0) {
+ lockRequest = requestList.remove(0);
+ log("Processing: " + lockRequest.prettyPrint());
+ try {
+ if (!handleRequest(lockRequest)) {
+ log("\n*** Test Failed ***");
+ isSuccess = false;
+ break;
+ } else {
+ log("Processed: "+lockRequest.prettyPrint());
+ }
+ } catch (ACIDException e) {
+ e.printStackTrace();
+ break;
+ }
+ }
+
+ if (isSuccess) {
+ log("\n*** Test Passed ***");
+ }
+ }
+
+ public boolean handleRequest(LockRequest request) throws ACIDException {
+ LockRequestWorker worker = null;
+ int i = 0;
+
+ if (request.requestType == RequestType.CHECK_SEQUENCE) {
+ return validateExpectedResult(true);
+ } else if (request.requestType == RequestType.CHECK_SET) {
+ return validateExpectedResult(false);
+ } else if (request.requestType == RequestType.WAIT) {
+ try {
+ Thread.sleep((long)request.entityHashValue);
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ return false;
+ }
+ } else if (request.requestType == RequestType.END) {
+ worker = workerReadyQueue.pop(request.threadName);
+ while (worker == null) {
+ if (!IS_DEBUG_MODE) {
+ log(request.threadName + " is not in the workerReadyQueue");
+ return false;
+ }
+ log(Thread.currentThread().getName() + " waiting for "+request.threadName+" to be in the workerReadyQueue["+ i++ +"].");
+ try {
+ Thread.sleep((long)10);
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ return false;
+ }
+ worker = workerReadyQueue.pop(request.threadName);
+ }
+ synchronized (worker) {
+ worker.setDone(true);
+ worker.setWait(false);
+ worker.notify();
+ }
+ try {
+ Thread.sleep((long) defaultWaitTime);
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ } else {
+ worker = workerReadyQueue.pop(request.threadName);
+ while (worker == null) {
+ if (!IS_DEBUG_MODE) {
+ log(request.threadName + " is not in the workerReadyQueue");
+ return false;
+ }
+ log(Thread.currentThread().getName() + " waiting for "+request.threadName+" to be in the workerReadyQueue["+ i++ +"].");
+ try {
+ Thread.sleep((long)10);
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ worker = workerReadyQueue.pop(request.threadName);
+ }
+
+ synchronized (worker) {
+ worker.setLockRequest(request);
+ worker.setWait(false);
+ worker.notify();
+ }
+
+ try {
+ Thread.sleep((long) defaultWaitTime);
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ }
+
+ return true;
+ }
+
+ public boolean validateExpectedResult(boolean isSequence) {
+
+ if (isSequence) {
+ return workerReadyQueue.checkSequence(expectedResultList.get(resultListIndex++));
+ } else {
+ return workerReadyQueue.checkSet(expectedResultList.get(resultListIndex++));
+ }
+
+ }
+
+ public void readRequest() throws IOException, ACIDException {
+ int i = 0;
+ LockRequest lockRequest = null;
+ TransactionContext txnContext = null;
+ HashMap<Integer, TransactionContext> jobMap = new HashMap<Integer, TransactionContext>();
+
+ int threadId;
+ String requestType;
+ int jobId;
+ int datasetId;
+ int PKHashVal;
+ int waitTime;
+ ArrayList<Integer> list = null;
+ String lockMode;
+
+ Scanner scanner = new Scanner(new FileInputStream(requestFileName));
+ while (scanner.hasNextLine()) {
+ try {
+ threadId = Integer.parseInt(scanner.next().substring(1));
+ requestType = scanner.next();
+ if (requestType.equals("CSQ") || requestType.equals("CST") || requestType.equals("END")) {
+ log("LockRequest[" + i++ + "]:T" + threadId + "," + requestType);
+ lockRequest = new LockRequest("Thread-" + threadId, getRequestType(requestType));
+ if (requestType.equals("CSQ") || requestType.equals("CST")) {
+ list = new ArrayList<Integer>();
+ while (scanner.hasNextInt()) {
+ threadId = scanner.nextInt();
+ if (threadId < 0) {
+ break;
+ }
+ list.add(threadId);
+ }
+ expectedResultList.add(list);
+ }
+ } else if (requestType.equals("DW")) {
+ defaultWaitTime = scanner.nextInt();
+ log("LockRequest[" + i++ + "]:T" + threadId + "," + requestType + "," + defaultWaitTime);
+ continue;
+ } else if (requestType.equals("W")) {
+ waitTime = scanner.nextInt();
+ log("LockRequest[" + i++ + "]:T" + threadId + "," + requestType);
+ lockRequest = new LockRequest("Thread-" + threadId, getRequestType(requestType), waitTime);
+ } else {
+ jobId = Integer.parseInt(scanner.next().substring(1));
+ datasetId = Integer.parseInt(scanner.next().substring(1));
+ PKHashVal = Integer.parseInt(scanner.next().substring(1));
+ lockMode = scanner.next();
+ txnContext = jobMap.get(jobId);
+ if (txnContext == null) {
+ txnContext = new TransactionContext(new JobId(jobId), txnProvider);
+ jobMap.put(jobId, txnContext);
+ }
+ log("LockRequest[" + i++ + "]:T" + threadId + "," + requestType + ",J" + jobId + ",D" + datasetId + ",E"
+ + PKHashVal + "," + lockMode);
+ lockRequest = new LockRequest("Thread-" + threadId, getRequestType(requestType), new DatasetId(
+ datasetId), PKHashVal, getLockMode(lockMode), txnContext);
+ }
+
+ requestList.add(lockRequest);
+ } catch (NoSuchElementException e) {
+ scanner.close();
+ break;
+ }
+ }
+ }
+
+ public void log(String s) {
+ System.out.println(s);
+ }
+
+ private int getRequestType(String s) {
+ if (s.equals("L")) {
+ return RequestType.LOCK;
+ }
+
+ if (s.equals("TL")) {
+ return RequestType.TRY_LOCK;
+ }
+
+ if (s.equals("IL")) {
+ return RequestType.INSTANT_LOCK;
+ }
+
+ if (s.equals("ITL")) {
+ return RequestType.INSTANT_TRY_LOCK;
+ }
+
+ if (s.equals("UL")) {
+ return RequestType.UNLOCK;
+ }
+
+ if (s.equals("RL")) {
+ return RequestType.RELEASE_LOCKS;
+ }
+
+ if (s.equals("CSQ")) {
+ return RequestType.CHECK_SEQUENCE;
+ }
+
+ if (s.equals("CST")) {
+ return RequestType.CHECK_SET;
+ }
+
+ if (s.equals("END")) {
+ return RequestType.END;
+ }
+
+ if (s.equals("W")) {
+ return RequestType.WAIT;
+ }
+
+ try {
+ throw new UnsupportedOperationException("Invalid request type:" + s);
+ } catch (Exception e) {
+ e.printStackTrace();
+ } finally {
+ System.exit(0);
+ }
+
+ return -1;
+
+ }
+
+ private byte getLockMode(String s) {
+ if (s.equals("S")) {
+ return LockMode.S;
+ }
+
+ if (s.equals("X")) {
+ return LockMode.X;
+ }
+
+ try {
+ throw new UnsupportedOperationException("Invalid lock mode type:" + s);
+ } catch (Exception e) {
+ e.printStackTrace();
+ } finally {
+ System.exit(0);
+ }
+
+ return -1;
+ }
+}
+
+class LockRequestWorker implements Runnable {
+
+ String threadName;
+ TransactionProvider txnProvider;
+ ILockManager lockMgr;
+ WorkerReadyQueue workerReadyQueue;
+ LockRequest lockRequest;
+ boolean needWait;
+ boolean isAwaken;
+ boolean isDone;
+
+ public LockRequestWorker(TransactionProvider txnProvider, WorkerReadyQueue workerReadyQueue, String threadName) {
+ this.txnProvider = txnProvider;
+ this.lockMgr = txnProvider.getLockManager();
+ this.workerReadyQueue = workerReadyQueue;
+ this.threadName = new String(threadName);
+ this.lockRequest = null;
+ needWait = true;
+ isDone = false;
+ isAwaken = false;
+ }
+
+ public boolean isAwaken() {
+ return isAwaken;
+ }
+
+ @Override
+ public void run() {
+ //initial wait
+ needWait = true;
+ isAwaken = false;
+
+
+ while (!isDone) {
+ while (needWait) {
+ synchronized(this) {
+ workerReadyQueue.push(this);
+ try {
+ this.wait();
+ isAwaken = true;
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ }
+ }
+
+ if (isDone) {
+ break;
+ }
+
+ try {
+ sendRequest(lockRequest);
+ } catch (ACIDException e) {
+ if (lockRequest.txnContext.getStatus() == TransactionContext.TIMED_OUT_STATUS) {
+ if (lockRequest.txnContext.getTxnState() != TransactionState.ABORTED) {
+ lockRequest.txnContext.setTxnState(TransactionState.ABORTED);
+ log("*** "+ lockRequest.txnContext.getJobId()+ " lock request causing deadlock ***");
+ log("Abort --> Releasing all locks acquired by "+ lockRequest.txnContext.getJobId());
+ try {
+ lockMgr.releaseLocks(lockRequest.txnContext);
+ } catch (ACIDException e1) {
+ e1.printStackTrace();
+ }
+ log("Abort --> Released all locks acquired by "+ lockRequest.txnContext.getJobId());
+ }
+ isDone = true;
+ } else {
+ e.printStackTrace();
+ System.exit(-1);
+ }
+ }
+
+ try {
+ Thread.sleep(1);
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+
+ needWait = true;
+ isAwaken = false;
+ }
+ }
+
+ public void sendRequest(LockRequest request) throws ACIDException {
+
+ switch (request.requestType) {
+ case RequestType.LOCK:
+ lockMgr.lock(request.datasetIdObj, request.entityHashValue, request.lockMode, request.txnContext);
+ break;
+ case RequestType.INSTANT_LOCK:
+ lockMgr.instantLock(request.datasetIdObj, request.entityHashValue, request.lockMode, request.txnContext);
+ break;
+ case RequestType.TRY_LOCK:
+ request.isTryLockFailed = !lockMgr.tryLock(request.datasetIdObj, request.entityHashValue,
+ request.lockMode, request.txnContext);
+ break;
+ case RequestType.INSTANT_TRY_LOCK:
+ lockMgr.instantTryLock(request.datasetIdObj, request.entityHashValue, request.lockMode,
+ request.txnContext);
+ break;
+ case RequestType.UNLOCK:
+ lockMgr.unlock(request.datasetIdObj, request.entityHashValue, request.txnContext);
+ break;
+ case RequestType.RELEASE_LOCKS:
+ lockMgr.releaseLocks(request.txnContext);
+ break;
+ default:
+ throw new UnsupportedOperationException("Unsupported lock method");
+ }
+ }
+
+ public void setLockRequest(LockRequest request) {
+ this.lockRequest = request;
+ }
+
+ public void setWait(boolean wait) {
+ needWait = wait;
+ }
+
+ public void setDone(boolean done) {
+ isDone = done;
+ }
+
+ public String getThreadName() {
+ return threadName;
+ }
+
+ public void log(String s) {
+ System.out.println(s);
+ }
+}
+
+class WorkerReadyQueue {
+ ArrayList<LockRequestWorker> workerReadyQueue;
+
+ public WorkerReadyQueue() {
+ workerReadyQueue = new ArrayList<LockRequestWorker>();
+ }
+
+ public synchronized void push(LockRequestWorker worker) {
+ workerReadyQueue.add(worker);
+ }
+
+ public synchronized LockRequestWorker pop(String threadName) {
+ int i;
+ LockRequestWorker worker = null;
+ int size = workerReadyQueue.size();
+ for (i = 0; i < size; i++) {
+ worker = workerReadyQueue.get(i);
+ if (worker.getThreadName().equals(threadName)) {
+ workerReadyQueue.remove(i);
+ break;
+ }
+ }
+
+ if (i == size) {
+ return null;
+ } else {
+ return worker;
+ }
+ }
+
+ public synchronized int size() {
+ return workerReadyQueue.size();
+ }
+
+ public boolean checkSet(ArrayList<Integer> threadIdList) {
+ int i;
+ int j;
+ StringBuilder s = new StringBuilder();
+ LockRequestWorker worker = null;
+ int resultListSize = 0;
+ int queueSize = workerReadyQueue.size();
+ int listSize = threadIdList.size();
+
+ s.append("ExpectedList(Set):\t");
+ for (i=0; i < listSize; i++) {
+ s.append(threadIdList.get(i)).append(" ");
+ }
+ s.append("\n");
+
+ while (queueSize < listSize) {
+ //wait until workers finish its task
+ try {
+ Thread.sleep(1);
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ log(Thread.currentThread().getName() + " waiting for worker to finish its task...");
+ queueSize = workerReadyQueue.size();
+ }
+
+ if (listSize != queueSize) {
+ log("listSize:"+listSize +", queueSize:"+queueSize);
+ return false;
+ }
+
+ s.append("ResultList(Set):\t");
+ for (i = 0; i < listSize; i++) {
+ for (j = 0; j < queueSize; j++) {
+ worker = workerReadyQueue.get(j);
+ if (worker.getThreadName().equals("Thread-" + threadIdList.get(i))) {
+ s.append(threadIdList.get(i)).append(" ");
+ resultListSize++;
+ break;
+ }
+ }
+ }
+
+ log(s.toString());
+ if (listSize != resultListSize) {
+ return false;
+ }
+
+ return true;
+ }
+
+ public boolean checkSequence(ArrayList<Integer> threadIdList) {
+ int i;
+ StringBuilder s = new StringBuilder();
+ LockRequestWorker worker = null;
+ int queueSize = workerReadyQueue.size();
+ int listSize = threadIdList.size();
+
+ s.append("ExpectedList(Sequence):\t");
+ for (i=0; i < listSize; i++) {
+ s.append(threadIdList.get(i)).append(" ");
+ }
+ s.append("\n");
+
+ while (queueSize < listSize) {
+ //wait until workers finish its task
+ try {
+ Thread.sleep(1);
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ log(Thread.currentThread().getName() + "Waiting for worker to finish its task...");
+ queueSize = workerReadyQueue.size();
+ }
+
+ if (queueSize != listSize) {
+ return false;
+ }
+
+ s.append("ResultList(Sequence):\t");
+ for (i = 0; i < listSize; i++) {
+ worker = workerReadyQueue.get(i);
+ if (!worker.getThreadName().equals("Thread-" + threadIdList.get(i))) {
+ log(s.toString());
+ return false;
+ } else {
+ s.append(threadIdList.get(i)).append(" ");
+ }
+ }
+
+ log(s.toString());
+ return true;
+ }
+
+ public void log(String s) {
+ System.out.println(s);
+ }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManagerRandomUnitTest.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManagerRandomUnitTest.java
new file mode 100644
index 0000000..7add782
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManagerRandomUnitTest.java
@@ -0,0 +1,610 @@
+package edu.uci.ics.asterix.transaction.management.service.locking;
+
+import java.util.ArrayList;
+import java.util.Random;
+
+import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
+import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionManager.TransactionState;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
+
+/**
+ * LockManagerUnitTest: unit test of LockManager
+ *
+ * @author kisskys
+ */
+
+public class LockManagerRandomUnitTest {
+
+ private static final int MAX_NUM_OF_UPGRADE_JOB = 2;//2
+ private static final int MAX_NUM_OF_ENTITY_LOCK_JOB = 8;//8
+ private static final int MAX_NUM_OF_DATASET_LOCK_JOB = 2;//2
+ private static final int MAX_NUM_OF_THREAD_IN_A_JOB = 2; //4
+ private static int jobId = 0;
+ private static Random rand;
+
+ public static void main(String args[]) throws ACIDException {
+ int i;
+ TransactionProvider txnProvider = new TransactionProvider("LockManagerRandomUnitTest");
+ rand = new Random(System.currentTimeMillis());
+ for (i = 0; i < MAX_NUM_OF_ENTITY_LOCK_JOB; i++) {
+ System.out.println("Creating " + i + "th EntityLockJob..");
+ generateEntityLockThread(txnProvider);
+ }
+
+ for (i = 0; i < MAX_NUM_OF_DATASET_LOCK_JOB; i++) {
+ System.out.println("Creating " + i + "th DatasetLockJob..");
+ generateDatasetLockThread(txnProvider);
+ }
+
+ for (i = 0; i < MAX_NUM_OF_UPGRADE_JOB; i++) {
+ System.out.println("Creating " + i + "th EntityLockUpgradeJob..");
+ generateEntityLockUpgradeThread(txnProvider);
+ }
+ }
+
+ private static void generateEntityLockThread(TransactionProvider txnProvider) {
+ Thread t;
+ int childCount = rand.nextInt(MAX_NUM_OF_THREAD_IN_A_JOB);
+ if (MAX_NUM_OF_THREAD_IN_A_JOB != 0 && childCount == 0) {
+ childCount = 1;
+ }
+ TransactionContext txnContext = generateTxnContext(txnProvider);
+
+ for (int i = 0; i < childCount; i++) {
+ System.out.println("Creating " + txnContext.getJobId() + "," + i+ "th EntityLockThread..");
+ t = new Thread(new LockRequestProducer(txnProvider.getLockManager(), txnContext, false, false, false));
+ t.start();
+ }
+ }
+
+ private static void generateDatasetLockThread(TransactionProvider txnProvider) {
+ Thread t;
+// int childCount = rand.nextInt(MAX_NUM_OF_THREAD_IN_A_JOB);
+// if (MAX_NUM_OF_THREAD_IN_A_JOB != 0 && childCount == 0) {
+// childCount = 1;
+// }
+ int childCount = 1;
+
+ TransactionContext txnContext = generateTxnContext(txnProvider);
+
+ for (int i = 0; i < childCount; i++) {
+ System.out.println("Creating " + txnContext.getJobId() + "," + i + "th DatasetLockThread..");
+ t = new Thread(new LockRequestProducer(txnProvider.getLockManager(), txnContext, true, false, false));
+ t.start();
+ }
+ }
+
+ private static void generateEntityLockUpgradeThread(TransactionProvider txnProvider) {
+ int i;
+ Thread t;
+ int childCount = MAX_NUM_OF_THREAD_IN_A_JOB;
+ if (MAX_NUM_OF_THREAD_IN_A_JOB != 0 && childCount == 0) {
+ childCount = 1;
+ }
+ TransactionContext txnContext = generateTxnContext(txnProvider);
+
+ for (i = 0; i < childCount - 1; i++) {
+ System.out.println("Creating " + txnContext.getJobId() + "," + i + "th EntityLockUpgradeThread(false)..");
+ t = new Thread(new LockRequestProducer(txnProvider.getLockManager(), txnContext, false, true, false));
+ t.start();
+ }
+ System.out.println("Creating " + txnContext.getJobId() + "," + i + "th EntityLockUpgradeThread(true)..");
+ t = new Thread(new LockRequestProducer(txnProvider.getLockManager(), txnContext, false, true, true));
+ t.start();
+ }
+
+ private static TransactionContext generateTxnContext(TransactionProvider txnProvider) {
+ try {
+ return new TransactionContext(new JobId(jobId++), txnProvider);
+ } catch (ACIDException e) {
+ e.printStackTrace();
+ return null;
+ }
+ }
+
+}
+
+class LockRequestProducer implements Runnable {
+
+ private static final long serialVersionUID = -3191274684985609965L;
+ private static final int MAX_DATASET_NUM = 10;//10
+ private static final int MAX_ENTITY_NUM = 30;//30
+ private static final int MAX_LOCK_MODE_NUM = 2;
+ private static final long DATASET_LOCK_THREAD_SLEEP_TIME = 1000;
+ private static final int MAX_LOCK_REQUEST_TYPE_NUM = 4;
+
+ private ILockManager lockMgr;
+ private TransactionContext txnContext;
+ private Random rand;
+ private boolean isDatasetLock; //dataset or entity
+ private ArrayList<LockRequest> requestQueue;
+ private StringBuilder requestHistory;
+ private int unlockIndex;
+ private int upgradeIndex;
+ private boolean isUpgradeThread;
+ private boolean isUpgradeThreadJob;
+ private boolean isDone;
+
+ public LockRequestProducer(ILockManager lockMgr, TransactionContext txnContext, boolean isDatasetLock,
+ boolean isUpgradeThreadJob, boolean isUpgradeThread) {
+ this.lockMgr = lockMgr;
+ this.txnContext = txnContext;
+ this.isDatasetLock = isDatasetLock;
+ this.isUpgradeThreadJob = isUpgradeThreadJob;
+ this.isUpgradeThread = isUpgradeThread;
+
+ this.rand = new Random(System.currentTimeMillis());
+ requestQueue = new ArrayList<LockRequest>();
+ requestHistory = new StringBuilder();
+ unlockIndex = 0;
+ upgradeIndex = 0;
+ isDone = false;
+ }
+
+ @Override
+ public void run() {
+ try {
+ if (isDatasetLock) {
+ System.out.println("DatasetLockThread(" + Thread.currentThread().getName() + ") is running...");
+ runDatasetLockTask();
+ } else {
+ System.out.println("EntityLockThread(" + Thread.currentThread().getName() + "," + isUpgradeThreadJob
+ + "," + isUpgradeThread + ") is running...");
+ runEntityLockTask();
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ return;
+ } finally {
+
+
+ /*
+ System.out.println("" + Thread.currentThread().getName() + "\n" + requestHistory.toString() + ""
+ + Thread.currentThread().getName() + "\n");
+ System.out.println("RequestHistoryPerJobId\n" + ((LockManager) lockMgr).getLocalRequestHistory());
+ System.out.println("");
+ System.out.println("GlobalRequestHistory\n" + ((LockManager) lockMgr).getGlobalRequestHistory());
+ System.out.println("");
+ */
+ }
+ }
+
+ private void runDatasetLockTask() {
+ try {
+ produceDatasetLockRequest();
+ if (isDone) {
+ return;
+ }
+ } catch (ACIDException e) {
+ e.printStackTrace();
+ return;
+ }
+
+ try {
+ Thread.sleep(DATASET_LOCK_THREAD_SLEEP_TIME);
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+
+ try {
+ produceDatasetUnlockRequest();
+ if (isDone) {
+ return;
+ }
+ } catch (ACIDException e) {
+ e.printStackTrace();
+ return;
+ }
+ }
+
+ private void runEntityLockTask() {
+ int i;
+ byte lockMode;
+ int lockCount;
+ int upgradeCount;
+ int releaseCount;
+ boolean mayRelease = false;
+
+ lockCount = 1 + rand.nextInt(20);
+ if (isUpgradeThreadJob) {
+ if (isUpgradeThread) {
+ upgradeCount = 1; //rand.nextInt(4) + 1;
+ if (upgradeCount > lockCount) {
+ upgradeCount = lockCount;
+ }
+ } else {
+ upgradeCount = 0;
+ }
+ lockMode = LockMode.S;
+ } else {
+ upgradeCount = 0;
+ lockMode = (byte) (this.txnContext.getJobId().getId() % 2);
+ }
+ releaseCount = rand.nextInt(5) % 3 == 0 ? 1 : 0;
+
+ //lock
+ for (i = 0; i < lockCount; i++) {
+ try {
+ produceEntityLockRequest(lockMode);
+ if (isDone) {
+ return;
+ }
+ } catch (ACIDException e) {
+ e.printStackTrace();
+ return;
+ }
+ }
+
+ //upgrade
+ for (i = 0; i < upgradeCount; i++) {
+ try {
+ produceEntityLockUpgradeRequest();
+ if (isDone) {
+ return;
+ }
+ } catch (ACIDException e) {
+ e.printStackTrace();
+ return;
+ }
+ }
+
+ //unlock or releaseLocks
+ if (releaseCount == 0) {
+ //unlock
+ for (i = 0; i < lockCount; i++) {
+ try {
+ produceEntityUnlockRequest();
+ if (isDone) {
+ return;
+ }
+ } catch (ACIDException e) {
+ e.printStackTrace();
+ return;
+ }
+ }
+ } else {
+ try {
+ synchronized (txnContext) {
+ if (txnContext.getTxnState() != TransactionState.ABORTED) {
+ txnContext.setTxnState(TransactionState.ABORTED);
+ mayRelease = true;
+ }
+ }
+ if (mayRelease) {
+ produceEntityReleaseLocksRequest();
+ }
+ } catch (ACIDException e) {
+ e.printStackTrace();
+ return;
+ }
+ }
+ }
+
+ private void produceDatasetLockRequest() throws ACIDException {
+ int requestType = RequestType.LOCK;
+ int datasetId = rand.nextInt(MAX_DATASET_NUM);
+ int entityHashValue = -1;
+ byte lockMode = (byte) (rand.nextInt(MAX_LOCK_MODE_NUM));
+ LockRequest request = new LockRequest(Thread.currentThread().getName(), requestType, new DatasetId(datasetId), entityHashValue, lockMode,
+ txnContext);
+ requestQueue.add(request);
+ requestHistory.append(request.prettyPrint());
+ sendRequest(request);
+ }
+
+ private void produceDatasetUnlockRequest() throws ACIDException {
+ LockRequest lockRequest = requestQueue.get(0);
+
+ int requestType = RequestType.RELEASE_LOCKS;
+ int datasetId = lockRequest.datasetIdObj.getId();
+ int entityHashValue = -1;
+ byte lockMode = LockMode.S;//lockMode is not used for unlock() call.
+ LockRequest request = new LockRequest(Thread.currentThread().getName(), requestType, new DatasetId(datasetId), entityHashValue, lockMode,
+ txnContext);
+ requestQueue.add(request);
+ requestHistory.append(request.prettyPrint());
+ sendRequest(request);
+ }
+
+ private void produceEntityLockRequest(byte lockMode) throws ACIDException {
+ int requestType = rand.nextInt(MAX_LOCK_REQUEST_TYPE_NUM);
+ int datasetId = rand.nextInt(MAX_DATASET_NUM);
+ int entityHashValue = rand.nextInt(MAX_ENTITY_NUM);
+ LockRequest request = new LockRequest(Thread.currentThread().getName(), requestType, new DatasetId(datasetId), entityHashValue, lockMode,
+ txnContext);
+ requestQueue.add(request);
+ requestHistory.append(request.prettyPrint());
+ sendRequest(request);
+ }
+
+ private void produceEntityLockUpgradeRequest() throws ACIDException {
+ LockRequest lockRequest = null;
+ int size = requestQueue.size();
+ boolean existLockRequest = false;
+
+ while (upgradeIndex < size) {
+ lockRequest = requestQueue.get(upgradeIndex++);
+ if (lockRequest.isUpgrade || lockRequest.isTryLockFailed) {
+ continue;
+ }
+ if (lockRequest.requestType == RequestType.UNLOCK || lockRequest.requestType == RequestType.RELEASE_LOCKS
+ || lockRequest.requestType == RequestType.INSTANT_LOCK
+ || lockRequest.requestType == RequestType.INSTANT_TRY_LOCK) {
+ continue;
+ }
+ if (lockRequest.lockMode == LockMode.X) {
+ continue;
+ }
+ existLockRequest = true;
+ break;
+ }
+
+ if (existLockRequest) {
+ int requestType = lockRequest.requestType;
+ int datasetId = lockRequest.datasetIdObj.getId();
+ int entityHashValue = lockRequest.entityHashValue;
+ byte lockMode = LockMode.X;
+ LockRequest request = new LockRequest(Thread.currentThread().getName(), requestType, new DatasetId(datasetId), entityHashValue, lockMode,
+ txnContext);
+ request.isUpgrade = true;
+ requestQueue.add(request);
+ requestHistory.append(request.prettyPrint());
+ sendRequest(request);
+ }
+ }
+
+ private void produceEntityUnlockRequest() throws ACIDException {
+ LockRequest lockRequest = null;
+ int size = requestQueue.size();
+ boolean existLockRequest = false;
+
+ while (unlockIndex < size) {
+ lockRequest = requestQueue.get(unlockIndex++);
+ if (lockRequest.isUpgrade || lockRequest.isTryLockFailed) {
+ continue;
+ }
+ if (lockRequest.requestType == RequestType.UNLOCK || lockRequest.requestType == RequestType.RELEASE_LOCKS
+ || lockRequest.requestType == RequestType.INSTANT_LOCK
+ || lockRequest.requestType == RequestType.INSTANT_TRY_LOCK) {
+ continue;
+ }
+ existLockRequest = true;
+ break;
+ }
+
+ if (existLockRequest) {
+ int requestType = RequestType.UNLOCK;
+ int datasetId = lockRequest.datasetIdObj.getId();
+ int entityHashValue = lockRequest.entityHashValue;
+ byte lockMode = lockRequest.lockMode;
+ LockRequest request = new LockRequest(Thread.currentThread().getName(), requestType, new DatasetId(datasetId), entityHashValue, lockMode,
+ txnContext);
+ requestQueue.add(request);
+ requestHistory.append(request.prettyPrint());
+ sendRequest(request);
+ }
+ }
+
+ private void produceEntityReleaseLocksRequest() throws ACIDException {
+ LockRequest lockRequest = null;
+ int size = requestQueue.size();
+ boolean existLockRequest = false;
+
+ while (unlockIndex < size) {
+ lockRequest = requestQueue.get(unlockIndex++);
+ if (lockRequest.isUpgrade || lockRequest.isTryLockFailed) {
+ continue;
+ }
+ if (lockRequest.requestType == RequestType.UNLOCK || lockRequest.requestType == RequestType.RELEASE_LOCKS
+ || lockRequest.requestType == RequestType.INSTANT_LOCK
+ || lockRequest.requestType == RequestType.INSTANT_TRY_LOCK) {
+ continue;
+ }
+ existLockRequest = true;
+ break;
+ }
+
+ if (existLockRequest) {
+ int requestType = RequestType.RELEASE_LOCKS;
+ int datasetId = lockRequest.datasetIdObj.getId();
+ int entityHashValue = lockRequest.entityHashValue;
+ byte lockMode = lockRequest.lockMode;
+ LockRequest request = new LockRequest(Thread.currentThread().getName(), requestType, new DatasetId(datasetId), entityHashValue, lockMode,
+ txnContext);
+ requestQueue.add(request);
+ requestHistory.append(request.prettyPrint());
+ sendRequest(request);
+ }
+ }
+
+ private void sendRequest(LockRequest request) throws ACIDException {
+
+ switch (request.requestType) {
+ case RequestType.LOCK:
+ try {
+ lockMgr.lock(request.datasetIdObj, request.entityHashValue, request.lockMode, request.txnContext);
+ } catch (ACIDException e) {
+ if (request.txnContext.getStatus() == TransactionContext.TIMED_OUT_STATUS) {
+ if (request.txnContext.getTxnState() != TransactionState.ABORTED) {
+ request.txnContext.setTxnState(TransactionState.ABORTED);
+ log("*** "+ request.txnContext.getJobId()+ " lock request causing deadlock ***");
+ log("Abort --> Releasing all locks acquired by "+ request.txnContext.getJobId());
+ try {
+ lockMgr.releaseLocks(request.txnContext);
+ } catch (ACIDException e1) {
+ e1.printStackTrace();
+ }
+ log("Abort --> Released all locks acquired by "+ request.txnContext.getJobId());
+ }
+ isDone = true;
+ } else {
+ throw e;
+ }
+ }
+ break;
+ case RequestType.INSTANT_LOCK:
+ try {
+ lockMgr.instantLock(request.datasetIdObj, request.entityHashValue, request.lockMode, request.txnContext);
+ } catch (ACIDException e) {
+ if (request.txnContext.getStatus() == TransactionContext.TIMED_OUT_STATUS) {
+ if (request.txnContext.getTxnState() != TransactionState.ABORTED) {
+ request.txnContext.setTxnState(TransactionState.ABORTED);
+ log("*** "+ request.txnContext.getJobId()+ " lock request causing deadlock ***");
+ log("Abort --> Releasing all locks acquired by "+ request.txnContext.getJobId());
+ try {
+ lockMgr.releaseLocks(request.txnContext);
+ } catch (ACIDException e1) {
+ e1.printStackTrace();
+ }
+ log("Abort --> Released all locks acquired by "+ request.txnContext.getJobId());
+ }
+ isDone = true;
+ } else {
+ throw e;
+ }
+ }
+ break;
+ case RequestType.TRY_LOCK:
+ request.isTryLockFailed = !lockMgr.tryLock(request.datasetIdObj, request.entityHashValue,
+ request.lockMode, request.txnContext);
+ break;
+ case RequestType.INSTANT_TRY_LOCK:
+ lockMgr.instantTryLock(request.datasetIdObj, request.entityHashValue, request.lockMode,
+ request.txnContext);
+ break;
+ case RequestType.UNLOCK:
+ lockMgr.unlock(request.datasetIdObj, request.entityHashValue, request.txnContext);
+ break;
+ case RequestType.RELEASE_LOCKS:
+ lockMgr.releaseLocks(request.txnContext);
+ break;
+ default:
+ throw new UnsupportedOperationException("Unsupported lock method");
+ }
+ try {
+ Thread.sleep((long)0);
+ } catch (InterruptedException e) {
+ // TODO Auto-generated catch block
+ e.printStackTrace();
+ }
+ }
+
+ private void log(String s) {
+ System.out.println(s);
+ }
+}
+
+class LockRequest {
+ public int requestType;
+ public DatasetId datasetIdObj;
+ public int entityHashValue;
+ public byte lockMode;
+ public TransactionContext txnContext;
+ public boolean isUpgrade;
+ public boolean isTryLockFailed;
+ public long requestTime;
+ public String threadName;
+ public LockRequest(String threadName, int requestType, DatasetId datasetIdObj, int entityHashValue, byte lockMode,
+ TransactionContext txnContext) {
+ this.requestType = requestType;
+ this.datasetIdObj = datasetIdObj;
+ this.entityHashValue = entityHashValue;
+ this.lockMode = lockMode;
+ this.txnContext = txnContext;
+ this.requestTime = System.currentTimeMillis();
+ this.threadName = new String(threadName);
+ isUpgrade = false;
+ isTryLockFailed = false;//used for TryLock request not to call Unlock when the tryLock failed.
+ }
+
+ public LockRequest(String threadName, int requestType) {
+ this.requestType = requestType;
+ this.requestTime = System.currentTimeMillis();
+ this.threadName = new String(threadName);
+ }
+
+ //used for "W" request type
+ public LockRequest(String threadName, int requestType, int waitTime) {
+ this.requestType = requestType;
+ this.requestTime = System.currentTimeMillis();
+ this.threadName = new String(threadName);
+ this.entityHashValue = waitTime;
+ }
+
+ public String prettyPrint() {
+ StringBuilder s = new StringBuilder();
+ //s.append(threadName.charAt(7)).append("\t").append("\t");
+ s.append("T").append(threadName.substring(7)).append("\t");
+ switch (requestType) {
+ case RequestType.LOCK:
+ s.append("L");
+ break;
+ case RequestType.TRY_LOCK:
+ s.append("TL");
+ break;
+ case RequestType.INSTANT_LOCK:
+ s.append("IL");
+ break;
+ case RequestType.INSTANT_TRY_LOCK:
+ s.append("ITL");
+ break;
+ case RequestType.UNLOCK:
+ s.append("UL");
+ break;
+ case RequestType.RELEASE_LOCKS:
+ s.append("RL");
+ break;
+ case RequestType.CHECK_SEQUENCE:
+ s.append("CSQ");
+ return s.toString();
+ case RequestType.CHECK_SET:
+ s.append("CST");
+ return s.toString();
+ case RequestType.END:
+ s.append("END");
+ return s.toString();
+ case RequestType.WAIT:
+ s.append("W").append("\t").append(entityHashValue);
+ return s.toString();
+ default:
+ throw new UnsupportedOperationException("Unsupported method");
+ }
+ s.append("\tJ").append(txnContext.getJobId().getId()).append("\tD").append(datasetIdObj.getId()).append("\tE")
+ .append(entityHashValue).append("\t");
+ switch (lockMode) {
+ case LockMode.S:
+ s.append("S");
+ break;
+ case LockMode.X:
+ s.append("X");
+ break;
+ case LockMode.IS:
+ s.append("IS");
+ break;
+ case LockMode.IX:
+ s.append("IX");
+ break;
+ default:
+ throw new UnsupportedOperationException("Unsupported lock mode");
+ }
+ s.append("\n");
+ return s.toString();
+ }
+}
+
+class RequestType {
+ public static final int LOCK = 0;
+ public static final int TRY_LOCK = 1;
+ public static final int INSTANT_LOCK = 2;
+ public static final int INSTANT_TRY_LOCK = 3;
+ public static final int UNLOCK = 4;
+ public static final int RELEASE_LOCKS = 5;
+ public static final int CHECK_SEQUENCE = 6;
+ public static final int CHECK_SET = 7;
+ public static final int END = 8;
+ public static final int WAIT = 9;
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockMatrix.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockMatrix.java
deleted file mode 100644
index 651909d..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockMatrix.java
+++ /dev/null
@@ -1,35 +0,0 @@
-package edu.uci.ics.asterix.transaction.management.service.locking;
-
-/**
- * @author pouria An implementation of the ILockMatrix Each lock mode is shown
- * as an integer. More specifically:
- * - i-th entry of the conflictTable corresponds to the i-th lock mode
- * and it shows the conflicting mask of that mode. j-th bit of the i-th
- * entry is 1 if and only if i-th lock mode conflicts with the j-th lock
- * mode.
- * - i-th entry of the conversionTable corresponds to the i-th lock mode
- * and it shows whether going from that mode to a new mode is actually a
- * conversion or not. j-th bit of the i-th entry is 1 if and only if
- * j-th lock mode is "stronger" than the i-th mode, i.e. lock changing
- * from i-th mode to the j-th mode is actually a conversion.
- */
-public class LockMatrix implements ILockMatrix {
-
- int[] conflictTable;
- int[] conversionTable;
-
- public LockMatrix(int[] confTab, int[] convTab) {
- this.conflictTable = confTab;
- this.conversionTable = convTab;
- }
-
- @Override
- public boolean conflicts(int reqMask, int lockMode) {
- return ((reqMask & conflictTable[lockMode]) != 0);
- }
-
- @Override
- public boolean isConversion(int currentLockMode, int reqLockMode) {
- return ((conversionTable[currentLockMode] & (0x01 << reqLockMode)) != 0);
- }
-}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockRequestFile b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockRequestFile
new file mode 100644
index 0000000..fc2a883
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockRequestFile
@@ -0,0 +1,20 @@
+T1 L J1 D1 E1 S
+T3 L J3 D1 E-1 S
+T2 L J2 D1 E-1 X
+T4 L J4 D1 E1 S
+T0 CST 1 3 -1
+T1 L J1 D1 E2 X
+T0 CST 3 -1
+T3 RL J3 D1 E-1 S
+T0 CST 1 3 -1
+T1 UL J1 D1 E1 S
+T0 CST 1 3 -1
+T1 UL J1 D1 E2 X
+T0 CST 1 2 3 -1
+T3 END
+T1 END
+T2 RL J2 D1 E-1 X
+T2 END
+T0 CST 4 -1
+T4 UL J4 D1 E1 S
+T4 END
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockRequestTracker.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockRequestTracker.java
new file mode 100644
index 0000000..ba47b5a
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockRequestTracker.java
@@ -0,0 +1,57 @@
+package edu.uci.ics.asterix.transaction.management.service.locking;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+public class LockRequestTracker {
+ HashMap<Integer, StringBuilder> historyPerJob; //per job
+ StringBuilder historyForAllJobs;
+ StringBuilder requestHistoryForAllJobs; //request only
+
+ public LockRequestTracker() {
+ historyForAllJobs = new StringBuilder();
+ historyPerJob = new HashMap<Integer, StringBuilder>();
+ requestHistoryForAllJobs = new StringBuilder();
+ }
+
+ public void addEvent(String msg, LockRequest request) {
+ int jobId = request.txnContext.getJobId().getId();
+ StringBuilder jobHistory = historyPerJob.get(jobId);
+
+ //update jobHistory
+ if (jobHistory == null) {
+ jobHistory = new StringBuilder();
+ }
+ jobHistory.append(request.prettyPrint()).append("--> ").append(msg).append("\n");
+ historyPerJob.put(jobId, jobHistory);
+
+ //handle global request queue
+ historyForAllJobs.append(request.prettyPrint()).append("--> ").append(msg).append("\n");
+ }
+
+ public void addRequest(LockRequest request) {
+ requestHistoryForAllJobs.append(request.prettyPrint());
+ }
+
+ public String getHistoryForAllJobs() {
+ return historyForAllJobs.toString();
+ }
+
+ public String getHistoryPerJob() {
+ StringBuilder history = new StringBuilder();
+ Set<Entry<Integer, StringBuilder>> s = historyPerJob.entrySet();
+ Iterator<Entry<Integer, StringBuilder>> iter = s.iterator();
+ while (iter.hasNext()) {
+ Map.Entry<Integer, StringBuilder> entry = (Map.Entry<Integer, StringBuilder>) iter.next();
+ history.append(entry.getValue().toString());
+ }
+ return history.toString();
+ }
+
+ public String getRequestHistoryForAllJobs() {
+ return requestHistoryForAllJobs.toString();
+ }
+}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockWaiter.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockWaiter.java
new file mode 100644
index 0000000..2015aec
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockWaiter.java
@@ -0,0 +1,145 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.transaction.management.service.locking;
+
+/**
+ * LockWaiter object is used for keeping a lock waiter or a lock upgrader information on a certain resource.
+ * The resource can be a dataset or an entity.
+ * @author kisskys
+ *
+ */
+public class LockWaiter {
+ /**
+ * entityInfoSlotNum:
+ * If this LockWaiter object is used, this variable is used to indicate the corresponding EntityInfoSlotNum.
+ * Otherwise, the variable is used for nextFreeSlot Which indicates the next free waiter object.
+ */
+ private int entityInfoSlotNum;
+ private boolean wait;
+ private boolean victim;
+ private byte waiterCount;
+ private boolean firstGetUp;
+ private int nextWaiterObjId; //used for DatasetLockInfo and EntityLockInfo
+ private int nextWaitingResourceObjId; //used for JobInfo
+ private long beginWaitTime;
+ private boolean isWaiter; //is upgrader or waiter
+ private boolean isWaitingOnEntityLock; //is waiting on datasetLock or entityLock
+
+ public LockWaiter() {
+ this.victim = false;
+ this.wait = true;
+ waiterCount = 0;
+ nextWaiterObjId = -1;
+ nextWaitingResourceObjId = -1;
+ }
+
+ public void setEntityInfoSlot(int slotNum) {
+ this.entityInfoSlotNum = slotNum;
+ }
+
+ public int getEntityInfoSlot() {
+ return this.entityInfoSlotNum;
+ }
+
+ public void setNextFreeSlot(int slotNum) {
+ this.entityInfoSlotNum = slotNum;
+ }
+
+ public int getNextFreeSlot() {
+ return this.entityInfoSlotNum;
+ }
+
+ public void setWait(boolean wait) {
+ this.wait = wait;
+ }
+
+ public boolean needWait() {
+ return this.wait;
+ }
+
+ public void setVictim(boolean victim) {
+ this.victim = victim;
+ }
+
+ public boolean isVictim() {
+ return this.victim;
+ }
+
+ public void increaseWaiterCount() {
+ waiterCount++;
+ }
+
+ public void decreaseWaiterCount() {
+ waiterCount--;
+ }
+
+ public byte getWaiterCount() {
+ return waiterCount;
+ }
+
+ public void setWaiterCount(byte count) {
+ waiterCount = count;
+ }
+
+ public void setFirstGetUp(boolean isFirst) {
+ firstGetUp = isFirst;
+ }
+
+ public boolean isFirstGetUp() {
+ return firstGetUp;
+ }
+
+ public void setNextWaiterObjId(int next) {
+ nextWaiterObjId = next;
+ }
+
+ public int getNextWaiterObjId() {
+ return nextWaiterObjId;
+ }
+
+ public void setNextWaitingResourceObjId(int next) {
+ nextWaitingResourceObjId = next;
+ }
+
+ public int getNextWaitingResourceObjId() {
+ return nextWaitingResourceObjId;
+ }
+
+ public void setBeginWaitTime(long time) {
+ this.beginWaitTime = time;
+ }
+
+ public long getBeginWaitTime() {
+ return beginWaitTime;
+ }
+
+ public boolean isWaiter() {
+ return isWaiter;
+ }
+
+ public void setWaiter(boolean isWaiter) {
+ this.isWaiter = isWaiter;
+ }
+
+ public boolean isWaitingOnEntityLock() {
+ return isWaitingOnEntityLock;
+ }
+
+ public void setWaitingOnEntityLock(boolean isWaitingOnEntityLock) {
+ this.isWaitingOnEntityLock = isWaitingOnEntityLock;
+ }
+
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockWaiterManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockWaiterManager.java
new file mode 100644
index 0000000..dbe76ff
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockWaiterManager.java
@@ -0,0 +1,377 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.transaction.management.service.locking;
+
+import java.util.ArrayList;
+
+/**
+ * LockWaiterManager manages LockWaiter objects array.
+ * The array grows when the slots are overflowed.
+ * Also, the array shrinks according to the following shrink policy
+ * : Shrink when the resource under-utilization lasts for a certain threshold time.
+ *
+ * @author kisskys
+ */
+public class LockWaiterManager {
+
+ public static final int SHRINK_TIMER_THRESHOLD = 120000; //2min
+
+ private ArrayList<ChildLockWaiterArrayManager> pArray; //parent array
+ private int allocChild; //used to allocate the next free LockWaiter object.
+ private long shrinkTimer;
+ private boolean isShrinkTimerOn;
+ private int occupiedSlots;
+
+// ////////////////////////////////////////////////
+// // begin of unit test
+// ////////////////////////////////////////////////
+//
+// public static final int SHRINK_TIMER_THRESHOLD = 0; //for unit test
+//
+// /**
+// * @param args
+// */
+// public static void main(String[] args) {
+// final int DataSize = 5000;
+//
+// int i, j;
+// int slots = ChildLockWaiterArrayManager.NUM_OF_SLOTS;
+// int data[] = new int[DataSize];
+// LockWaiterManager lwMgr = new LockWaiterManager();
+//
+// //allocate: 50
+// System.out.println("allocate: 50");
+// for (i = 0; i < 5; i++) {
+// for (j = i * slots; j < i * slots + slots; j++) {
+// data[j] = lwMgr.allocate();
+// }
+//
+// System.out.println(lwMgr.prettyPrint());
+// }
+//
+// //deallocate from the last child to the first child
+// System.out.println("deallocate from the last child to the first child");
+// for (i = 4; i >= 0; i--) {
+// for (j = i * slots + slots - 1; j >= i * slots; j--) {
+// lwMgr.deallocate(data[j]);
+// }
+// System.out.println(lwMgr.prettyPrint());
+// }
+//
+// //allocate: 50
+// System.out.println("allocate: 50");
+// for (i = 0; i < 5; i++) {
+// for (j = i * slots; j < i * slots + slots; j++) {
+// data[j] = lwMgr.allocate();
+// }
+//
+// System.out.println(lwMgr.prettyPrint());
+// }
+//
+// //deallocate from the first child to last child
+// System.out.println("deallocate from the first child to last child");
+// for (i = 0; i < 5; i++) {
+// for (j = i * slots; j < i * slots + slots; j++) {
+// lwMgr.deallocate(data[j]);
+// }
+//
+// System.out.println(lwMgr.prettyPrint());
+// }
+//
+// //allocate: 50
+// System.out.println("allocate: 50");
+// for (i = 0; i < 5; i++) {
+// for (j = i * slots; j < i * slots + slots; j++) {
+// data[j] = lwMgr.allocate();
+// }
+//
+// System.out.println(lwMgr.prettyPrint());
+// }
+//
+// //deallocate from the first child to 4th child
+// System.out.println("deallocate from the first child to 4th child");
+// for (i = 0; i < 4; i++) {
+// for (j = i * slots; j < i * slots + slots; j++) {
+// lwMgr.deallocate(data[j]);
+// }
+//
+// System.out.println(lwMgr.prettyPrint());
+// }
+//
+// //allocate: 40
+// System.out.println("allocate: 40");
+// for (i = 0; i < 4; i++) {
+// for (j = i * slots; j < i * slots + slots; j++) {
+// data[j] = lwMgr.allocate();
+// }
+//
+// System.out.println(lwMgr.prettyPrint());
+// }
+// }
+//
+// ////////////////////////////////////////////////
+// // end of unit test
+// ////////////////////////////////////////////////
+
+ public LockWaiterManager() {
+ pArray = new ArrayList<ChildLockWaiterArrayManager>();
+ pArray.add(new ChildLockWaiterArrayManager());
+ allocChild = 0;
+ occupiedSlots = 0;
+ isShrinkTimerOn = false;
+ }
+
+ public int allocate() {
+ if (pArray.get(allocChild).isFull()) {
+ int size = pArray.size();
+ boolean bAlloc = false;
+ ChildLockWaiterArrayManager child;
+
+ //find a deinitialized child and initialize it
+ for (int i = 0; i < size; i++) {
+ child = pArray.get(i);
+ if (child.isDeinitialized()) {
+ child.initialize();
+ allocChild = i;
+ bAlloc = true;
+ break;
+ }
+ }
+
+ //allocate new child when there is no deinitialized child
+ if (!bAlloc) {
+ pArray.add(new ChildLockWaiterArrayManager());
+ allocChild = pArray.size() - 1;
+ }
+ }
+ occupiedSlots++;
+ return pArray.get(allocChild).allocate() + allocChild * ChildLockWaiterArrayManager.NUM_OF_SLOTS;
+ }
+
+ void deallocate(int slotNum) {
+ pArray.get(slotNum / ChildLockWaiterArrayManager.NUM_OF_SLOTS).deallocate(
+ slotNum % ChildLockWaiterArrayManager.NUM_OF_SLOTS);
+ occupiedSlots--;
+
+ if (needShrink()) {
+ shrink();
+ }
+ }
+
+ /**
+ * Shrink policy:
+ * Shrink when the resource under-utilization lasts for a certain amount of time.
+ * TODO Need to figure out which of the policies is better
+ * case1.
+ * pArray status : O x x x x x O (O is initialized, x is deinitialized)
+ * In the above status, 'CURRENT' needShrink() returns 'TRUE'
+ * even if there is nothing to shrink or deallocate.
+ * It doesn't distinguish the deinitialized children from initialized children
+ * by calculating totalNumOfSlots = pArray.size() * ChildLockWaiterArrayManager.NUM_OF_SLOTS.
+ * In other words, it doesn't subtract the deinitialized children's slots.
+ * case2.
+ * pArray status : O O x x x x x
+ * However, in the above case, if we subtract the deinitialized children's slots,
+ * needShrink() will return false even if we shrink the pArray at this case.
+ *
+ * @return
+ */
+ private boolean needShrink() {
+ int size = pArray.size();
+ int usedSlots = occupiedSlots;
+ if (usedSlots == 0) {
+ usedSlots = 1;
+ }
+
+ if (size > 1 && size * ChildLockWaiterArrayManager.NUM_OF_SLOTS / usedSlots >= 3) {
+ if (isShrinkTimerOn) {
+ if (System.currentTimeMillis() - shrinkTimer >= SHRINK_TIMER_THRESHOLD) {
+ isShrinkTimerOn = false;
+ return true;
+ }
+ } else {
+ //turn on timer
+ isShrinkTimerOn = true;
+ shrinkTimer = System.currentTimeMillis();
+ }
+ } else {
+ //turn off timer
+ isShrinkTimerOn = false;
+ }
+
+ return false;
+ }
+
+ /**
+ * Shrink() may
+ * deinitialize(:deallocates array of LockWaiter objects in a child) Children(s) or
+ * shrink pArray according to the deinitialized children's contiguity status.
+ * It doesn't deinitialize or shrink more than half of children at a time.
+ */
+ private void shrink() {
+ int i;
+ boolean bContiguous = true;
+ int decreaseCount = 0;
+ int size = pArray.size();
+ int maxDecreaseCount = size / 2;
+ ChildLockWaiterArrayManager child;
+ for (i = size - 1; i >= 0; i--) {
+ child = pArray.get(i);
+ if (child.isEmpty() || child.isDeinitialized()) {
+ if (bContiguous) {
+ pArray.remove(i);
+ if (++decreaseCount == maxDecreaseCount) {
+ break;
+ }
+ } else {
+ bContiguous = false;
+ if (child.isEmpty()) {
+ child.deinitialize();
+ if (++decreaseCount == maxDecreaseCount) {
+ break;
+ }
+ }
+ }
+ } else {
+ bContiguous = false;
+ }
+ }
+
+ //reset allocChild when the child is removed or deinitialized.
+ size = pArray.size();
+ if (allocChild >= size || pArray.get(allocChild).isDeinitialized()) {
+ //set allocChild to any initialized one.
+ //It is guaranteed that there is at least one initialized child.
+ for (i = 0; i < size; i++) {
+ if (!pArray.get(i).isDeinitialized()) {
+ allocChild = i;
+ break;
+ }
+ }
+ }
+ }
+
+ public String prettyPrint() {
+ StringBuilder s = new StringBuilder("\n########### LockWaiterManager Status #############\n");
+ int size = pArray.size();
+ ChildLockWaiterArrayManager child;
+ LockWaiter waiter;
+
+ for (int i = 0; i < size; i++) {
+ child = pArray.get(i);
+ if (child.isDeinitialized()) {
+ continue;
+ }
+ s.append("child[" + i + "]: occupiedSlots:" + child.getNumOfOccupiedSlots());
+ s.append(" freeSlotNum:" + child.getFreeSlotNum() + "\n");
+ for (int j = 0; j < ChildLockWaiterArrayManager.NUM_OF_SLOTS; j++) {
+ waiter = child.getLockWaiter(j);
+ s.append(j).append(": ");
+ s.append("\t" + waiter.getEntityInfoSlot());
+ s.append("\t" + waiter.needWait());
+ s.append("\t" + waiter.isVictim());
+ s.append("\n");
+ }
+ s.append("\n");
+ }
+ return s.toString();
+ }
+
+ public LockWaiter getLockWaiter(int slotNum) {
+ return pArray.get(slotNum / ChildLockWaiterArrayManager.NUM_OF_SLOTS).getLockWaiter(
+ slotNum % ChildLockWaiterArrayManager.NUM_OF_SLOTS);
+ }
+}
+
+class ChildLockWaiterArrayManager {
+ public static final int NUM_OF_SLOTS = 100; //number of LockWaiter objects in 'childArray'.
+// public static final int NUM_OF_SLOTS = 10; //for unit test
+
+ private int freeSlotNum;
+ private int occupiedSlots; //-1 represents 'deinitialized' state.
+ LockWaiter childArray[];//childArray
+
+ public ChildLockWaiterArrayManager() {
+ initialize();
+ }
+
+ public void initialize() {
+ this.childArray = new LockWaiter[NUM_OF_SLOTS];
+ this.freeSlotNum = 0;
+ this.occupiedSlots = 0;
+
+ for (int i = 0; i < NUM_OF_SLOTS - 1; i++) {
+ childArray[i] = new LockWaiter();
+ childArray[i].setNextFreeSlot(i + 1);
+ }
+ childArray[NUM_OF_SLOTS - 1] = new LockWaiter();
+ childArray[NUM_OF_SLOTS - 1].setNextFreeSlot(-1); //-1 represents EOL(end of link)
+ }
+
+ public LockWaiter getLockWaiter(int slotNum) {
+ return childArray[slotNum];
+ }
+
+ public int allocate() {
+ int currentSlot = freeSlotNum;
+ freeSlotNum = childArray[currentSlot].getNextFreeSlot();
+ childArray[currentSlot].setWait(true);
+ childArray[currentSlot].setVictim(false);
+ childArray[currentSlot].setWaiterCount((byte)0);
+ childArray[currentSlot].setNextWaiterObjId(-1);
+ childArray[currentSlot].setNextWaitingResourceObjId(-1);
+ childArray[currentSlot].setBeginWaitTime(-1l);
+ occupiedSlots++;
+ if (LockManager.IS_DEBUG_MODE) {
+ System.out.println(Thread.currentThread().getName()+" Alloc LockWaiterId("+currentSlot+")");
+ }
+ return currentSlot;
+ }
+
+ public void deallocate(int slotNum) {
+ childArray[slotNum].setNextFreeSlot(freeSlotNum);
+ freeSlotNum = slotNum;
+ occupiedSlots--;
+ if (LockManager.IS_DEBUG_MODE) {
+ System.out.println(Thread.currentThread().getName()+" Dealloc LockWaiterId("+slotNum+")");
+ }
+ }
+
+ public void deinitialize() {
+ childArray = null;
+ occupiedSlots = -1;
+ }
+
+ public boolean isDeinitialized() {
+ return occupiedSlots == -1;
+ }
+
+ public boolean isFull() {
+ return occupiedSlots == NUM_OF_SLOTS;
+ }
+
+ public boolean isEmpty() {
+ return occupiedSlots == 0;
+ }
+
+ public int getNumOfOccupiedSlots() {
+ return occupiedSlots;
+ }
+
+ public int getFreeSlotNum() {
+ return freeSlotNum;
+ }
+}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/PrimitiveIntHashMap.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/PrimitiveIntHashMap.java
new file mode 100644
index 0000000..be9c080
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/PrimitiveIntHashMap.java
@@ -0,0 +1,592 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.transaction.management.service.locking;
+
+import java.util.ArrayList;
+
+/**
+ * PrimitiveIntHashMap supports primitive int type as key and value.
+ * The hash map grows when the available slots in a bucket are overflowed.
+ * Also, the hash map shrinks according to the following shrink policy.
+ * : Shrink when the resource under-utilization lasts for a certain threshold time.
+ *
+ * @author kisskys
+ *
+ */
+public class PrimitiveIntHashMap {
+ private final int CHILD_BUCKETS; //INIT_NUM_OF_BUCKETS;
+ private final int NUM_OF_SLOTS; //NUM_OF_SLOTS_IN_A_BUCKET;
+ private final int SHRINK_TIMER_THRESHOLD;
+
+ private int occupiedSlots;
+ private ArrayList<ChildIntArrayManager> pArray; //parent array
+ private int hashMod;
+ private long shrinkTimer;
+ private boolean isShrinkTimerOn;
+ private int iterBucketIndex;
+ private int iterSlotIndex;
+ private int iterChildIndex;
+ private KeyValuePair iterPair;
+
+// ////////////////////////////////////////////////
+// // begin of unit test
+// ////////////////////////////////////////////////
+//
+// /**
+// * @param args
+// */
+// public static void main(String[] args) {
+// int i, j;
+// int k = 0;
+// int num = 5;
+// int key[] = new int[500];
+// int val[] = new int[500];
+// KeyValuePair pair;
+// PrimitiveIntHashMap map = new PrimitiveIntHashMap(1<<4, 1<<3, 5);
+//
+// for (j=0; j < num; j++) {
+//
+// k += 100;
+// //generate data
+// for (i=0; i < k; i++) {
+// key[i] = i;
+// val[i] = i;
+// }
+//
+// //put data to map
+// for (i=0; i < k-30; i++) {
+// map.put(key[i], val[i]);
+// }
+//
+// //put data to map
+// for (i=0; i < k-30; i++) {
+// map.put(key[i], val[i]);
+// }
+//
+// map.beginIterate();
+// pair = map.getNextKeyValue();
+// i = 0;
+// while (pair != null) {
+// i++;
+// System.out.println("["+i+"] key:"+ pair.key + ", val:"+ pair.value);
+// pair = map.getNextKeyValue();
+// }
+//
+// //System.out.println(map.prettyPrint());
+//
+// for (i=k-20; i< k; i++) { //skip X70~X79
+// map.put(key[i], val[i]);
+// }
+//
+// System.out.println(map.prettyPrint());
+//
+// //remove data to map
+// for (i=0; i < k-10; i++) {
+// map.remove(key[i]);
+// try {
+// Thread.currentThread().sleep(1);
+// } catch (InterruptedException e) {
+// e.printStackTrace();
+// }
+// }
+//
+// map.beginIterate();
+// pair = map.getNextKeyValue();
+// i = 0;
+// while (pair != null) {
+// i++;
+// System.out.println("["+i+"] key:"+ pair.key + ", val:"+ pair.value);
+// pair = map.getNextKeyValue();
+// }
+//
+// //remove data to map
+// for (i=0; i < k-10; i++) {
+// map.remove(key[i]);
+// try {
+// Thread.currentThread().sleep(1);
+// } catch (InterruptedException e) {
+// // TODO Auto-generated catch block
+// e.printStackTrace();
+// }
+// }
+//
+// System.out.println(map.prettyPrint());
+//
+// //get data from map
+// for (i=0; i < k; i++) {
+// System.out.println(""+i+"=> key:"+ key[i] + ", val:"+val[i] +", result: " + map.get(key[i]));
+// }
+// }
+//
+// map.beginIterate();
+// pair = map.getNextKeyValue();
+// i = 0;
+// while (pair != null) {
+// i++;
+// System.out.println("["+i+"] key:"+ pair.key + ", val:"+ pair.value);
+// pair = map.getNextKeyValue();
+// }
+// }
+//
+// ////////////////////////////////////////////////
+// // end of unit test
+// ////////////////////////////////////////////////
+
+ public PrimitiveIntHashMap() {
+ CHILD_BUCKETS = 1<<9; //INIT_NUM_OF_BUCKETS;
+ NUM_OF_SLOTS = 1<<3; //NUM_OF_SLOTS_IN_A_BUCKET;
+ SHRINK_TIMER_THRESHOLD = 120000; //2min
+ pArray = new ArrayList<ChildIntArrayManager>();
+ pArray.add(new ChildIntArrayManager(this));
+ hashMod = CHILD_BUCKETS;
+ occupiedSlots = 0;
+ iterPair = new KeyValuePair();
+ }
+
+ public PrimitiveIntHashMap(int childBuckets, int numOfSlots, int shrinkTimerThreshold) {
+ CHILD_BUCKETS = childBuckets;
+ NUM_OF_SLOTS = numOfSlots;
+ SHRINK_TIMER_THRESHOLD = shrinkTimerThreshold;
+ pArray = new ArrayList<ChildIntArrayManager>();
+ pArray.add(new ChildIntArrayManager(this));
+ hashMod = CHILD_BUCKETS;
+ occupiedSlots = 0;
+ iterPair = new KeyValuePair();
+ }
+
+ public void put(int key, int value) {
+ int growCount = 0;
+ int bucketNum = hash(key);
+ ChildIntArrayManager child = pArray.get(bucketNum/CHILD_BUCKETS);
+ while (child.isFull(bucketNum%CHILD_BUCKETS)) {
+ growHashMap();
+ bucketNum = hash(key);
+ child = pArray.get(bucketNum/CHILD_BUCKETS);
+ if (growCount > 2) {
+ //changeHashFunc();
+ }
+ growCount++;
+ }
+ occupiedSlots += child.put(bucketNum%CHILD_BUCKETS, key, value, false);
+ }
+
+ public void upsert (int key, int value) {
+ int growCount = 0;
+ int bucketNum = hash(key);
+ ChildIntArrayManager child = pArray.get(bucketNum/CHILD_BUCKETS);
+ while (child.isFull(bucketNum%CHILD_BUCKETS)) {
+ growHashMap();
+ bucketNum = hash(key);
+ child = pArray.get(bucketNum/CHILD_BUCKETS);
+ if (growCount > 2) {
+ //changeHashFunc();
+ }
+ growCount++;
+ }
+ occupiedSlots += child.put(bucketNum%CHILD_BUCKETS, key, value, true);
+ }
+
+ private int hash(int key) {
+ return key%hashMod;
+ }
+
+ private void growHashMap() {
+ int size = pArray.size();
+ int i;
+
+ //grow buckets by adding more child
+ for (i=0; i<size; i++) {
+ pArray.add(new ChildIntArrayManager(this));
+ }
+
+ //increase hashMod
+ hashMod *= 2;
+
+ //re-hash
+ rehash(0, size, hashMod/2);
+ }
+
+ private void shrinkHashMap() {
+ int size = pArray.size();
+ int i;
+
+ //decrease hashMod
+ hashMod /= 2;
+
+ //re-hash
+ rehash(size/2, size, hashMod*2);
+
+ //shrink buckets by removing child(s)
+ for (i=size-1; i>=size/2;i--) {
+ pArray.remove(i);
+ }
+ }
+
+ private void rehash(int begin, int end, int oldHashMod) {
+ int i, j, k;
+ int key, value;
+ ChildIntArrayManager child;
+
+ //re-hash
+ for (i=begin; i<end; i++) {
+ child = pArray.get(i);
+ for (j=0; j<CHILD_BUCKETS; j++) {
+ if (child.cArray[j][0] == 0) {
+ continue;
+ }
+ for (k=1; k<NUM_OF_SLOTS; k++) {
+ //if the hashValue of the key is different, then re-hash it.
+ key = child.cArray[j][k*2];
+ if (hash(key) != key%oldHashMod) {
+ value = child.cArray[j][k*2+1];
+ //remove existing key and value
+ //Notice! To avoid bucket iteration, child.remove() is not used.
+ child.cArray[j][k*2] = -1;
+ child.cArray[j][0]--;
+ //re-hash it
+ pArray.get(hash(key)/CHILD_BUCKETS).put(hash(key)%CHILD_BUCKETS, key, value, false);
+ }
+ }
+ }
+ }
+ }
+
+// private void changeHashFunc() {
+// //TODO need to implement.
+// throw new UnsupportedOperationException("changeHashFunc() not implemented");
+// }
+
+ public int get(int key) {
+ int bucketNum = hash(key);
+ return pArray.get(bucketNum/CHILD_BUCKETS).get(bucketNum%CHILD_BUCKETS, key);
+ }
+
+ public void remove(int key) {
+ int bucketNum = hash(key);
+ occupiedSlots -= pArray.get(bucketNum/CHILD_BUCKETS).remove(bucketNum%CHILD_BUCKETS, key);
+
+ if (needShrink()) {
+ shrinkHashMap();
+ }
+ }
+
+ /**
+ * Shrink policy:
+ * Shrink when the resource under-utilization lasts for a certain amount of time.
+ * @return
+ */
+ private boolean needShrink() {
+ int size = pArray.size();
+ int usedSlots = occupiedSlots;
+ if (usedSlots == 0) {
+ usedSlots = 1;
+ }
+ if (size > 1 && size*CHILD_BUCKETS*NUM_OF_SLOTS/usedSlots >= 3 && isSafeToShrink()) {
+ if (isShrinkTimerOn) {
+ if (System.currentTimeMillis() - shrinkTimer >= SHRINK_TIMER_THRESHOLD) {
+ isShrinkTimerOn = false;
+ return true;
+ }
+ } else {
+ //turn on timer
+ isShrinkTimerOn = true;
+ shrinkTimer = System.currentTimeMillis();
+ }
+ } else {
+ //turn off timer
+ isShrinkTimerOn = false;
+ }
+ return false;
+ }
+
+ private boolean isSafeToShrink() {
+ int i, j;
+ int size = pArray.size();
+ //Child: 0, 1, 2, 3, 4, 5, 6, 7
+ //[HChild(Head Child):0 and TChild(Tail Child): 4], [1(H),5(T)], [2(H),6(T)] and so on.
+ //When the map shrinks, the sum of occupied slots in H/TChild should not exceed the NUM_OF_SLOTS-1.
+ //Then it is safe to shrink. Otherwise, unsafe.
+ ChildIntArrayManager HChild, TChild;
+
+ for (i=0; i<size/2; i++){
+ HChild = pArray.get(i);
+ TChild = pArray.get(size/2+i);
+ for (j=0; j<CHILD_BUCKETS; j++) {
+ if (HChild.cArray[j][0] + TChild.cArray[j][0] > NUM_OF_SLOTS-1) {
+ return false;
+ }
+ }
+ }
+ return true;
+ }
+
+ public String prettyPrint() {
+ StringBuilder s = new StringBuilder("\n########### PrimitiveIntHashMap Status #############\n");
+ ChildIntArrayManager child;
+ int i, j, k;
+ int size = pArray.size();
+ for (i=0; i<size;i++) {
+ child = pArray.get(i);
+ s.append("child[").append(i).append("]\n");
+ for (j=0; j<CHILD_BUCKETS;j++) {
+ s.append(j).append(" ");
+ for (k=0; k<NUM_OF_SLOTS;k++) {
+ s.append("[").append(child.cArray[j][k*2]).append(",").append(child.cArray[j][k*2+1]).append("] ");
+ }
+ s.append("\n");
+ }
+ }
+ return s.toString();
+ }
+
+ public int getNumOfSlots() {
+ return NUM_OF_SLOTS;
+ }
+
+ public int getNumOfChildBuckets() {
+ return CHILD_BUCKETS;
+ }
+
+ public void clear(boolean needShrink) {
+ int size = pArray.size();
+ for (int i=size-1; i >= 0; i--) {
+ if (needShrink && i != 0) {
+ pArray.remove(i);
+ } else {
+ pArray.get(i).clear();
+ }
+ }
+ occupiedSlots = 0;
+ }
+
+ ///////////////////////////////////////
+ // iterate method
+ ///////////////////////////////////////
+
+ public void beginIterate() {
+ iterChildIndex = 0;
+ iterBucketIndex = 0;
+ iterSlotIndex = 1;
+ }
+
+ public KeyValuePair getNextKeyValue() {
+ for (; iterChildIndex < pArray.size(); iterChildIndex++, iterBucketIndex = 0) {
+ for (; iterBucketIndex < CHILD_BUCKETS; iterBucketIndex++, iterSlotIndex = 1) {
+ if (iterSlotIndex ==1 && pArray.get(iterChildIndex).cArray[iterBucketIndex][0] == 0) {
+ continue;
+ }
+ for (; iterSlotIndex < NUM_OF_SLOTS; iterSlotIndex++) {
+ iterPair.key = pArray.get(iterChildIndex).cArray[iterBucketIndex][iterSlotIndex*2];
+ if (iterPair.key == -1) {
+ continue;
+ }
+ iterPair.value = pArray.get(iterChildIndex).cArray[iterBucketIndex][iterSlotIndex*2+1];
+ iterSlotIndex++;
+ return iterPair;
+ }
+ }
+ }
+ return null;
+ }
+
+ public int getNextKey() {
+ for (; iterChildIndex < pArray.size(); iterChildIndex++, iterBucketIndex = 0) {
+ for (; iterBucketIndex < CHILD_BUCKETS; iterBucketIndex++, iterSlotIndex = 1) {
+ if (iterSlotIndex ==1 && pArray.get(iterChildIndex).cArray[iterBucketIndex][0] == 0) {
+ continue;
+ }
+ for (; iterSlotIndex < NUM_OF_SLOTS; iterSlotIndex++) {
+ iterPair.key = pArray.get(iterChildIndex).cArray[iterBucketIndex][iterSlotIndex*2];
+ if (iterPair.key == -1) {
+ continue;
+ }
+ iterSlotIndex++;
+ return iterPair.key;
+ }
+ }
+ }
+ return -1;
+ }
+
+ public int getNextValue() {
+ for (; iterChildIndex < pArray.size(); iterChildIndex++, iterBucketIndex = 0) {
+ for (; iterBucketIndex < CHILD_BUCKETS; iterBucketIndex++, iterSlotIndex = 1) {
+ if (iterSlotIndex ==1 && pArray.get(iterChildIndex).cArray[iterBucketIndex][0] == 0) {
+ continue;
+ }
+ for (; iterSlotIndex < NUM_OF_SLOTS; iterSlotIndex++) {
+ iterPair.key = pArray.get(iterChildIndex).cArray[iterBucketIndex][iterSlotIndex*2];
+ if (iterPair.key == -1) {
+ continue;
+ }
+ iterPair.value = pArray.get(iterChildIndex).cArray[iterBucketIndex][iterSlotIndex*2+1];
+ iterSlotIndex++;
+ return iterPair.value;
+ }
+ }
+ }
+ return -1;
+ }
+
+ public static class KeyValuePair {
+ public int key;
+ public int value;
+ }
+}
+
+class ChildIntArrayManager {
+ private final int DIM1_SIZE;
+ private final int DIM2_SIZE;
+ private final int NUM_OF_SLOTS;
+ public int[][] cArray; //child array
+
+ public ChildIntArrayManager(PrimitiveIntHashMap parentHashMap) {
+ DIM1_SIZE = parentHashMap.getNumOfChildBuckets();
+ DIM2_SIZE = parentHashMap.getNumOfSlots() * 2; //2: Array of [key, value] pair
+ NUM_OF_SLOTS = parentHashMap.getNumOfSlots() ;
+ initialize();
+ }
+
+ private void initialize() {
+ cArray = new int[DIM1_SIZE][DIM2_SIZE];
+ int i, j;
+ for (i = 0; i < DIM1_SIZE; i++) {
+ //cArray[i][0] is used as a counter to count how many slots are used in this bucket.
+ //cArray[i][1] is not used.
+ cArray[i][0] = 0;
+ for (j = 1; j < NUM_OF_SLOTS; j++) {
+ cArray[i][j*2] = -1; // -1 represent that the slot is empty
+ }
+ }
+ }
+
+ public void clear() {
+ int i, j;
+ for (i = 0; i < DIM1_SIZE; i++) {
+ //cArray[i][0] is used as a counter to count how many slots are used in this bucket.
+ //cArray[i][1] is not used.
+ if (cArray[i][0] == 0) {
+ continue;
+ }
+ cArray[i][0] = 0;
+ for (j = 1; j < NUM_OF_SLOTS; j++) {
+ cArray[i][j*2] = -1; // -1 represent that the slot is empty
+ }
+ }
+ }
+
+ public void deinitialize() {
+ cArray = null;
+ }
+
+ public void allocate() {
+ initialize();
+ }
+
+ public boolean isFull(int bucketNum) {
+ return cArray[bucketNum][0] == NUM_OF_SLOTS-1;
+ }
+
+ public boolean isEmpty(int bucketNum) {
+ return cArray[bucketNum][0] == 0;
+ }
+
+ /**
+ * Put key,value into a slot in the bucket if the key doesn't exist.
+ * Update value if the key exists and if isUpsert is true
+ * No need to call get() to check the existence of the key before calling put().
+ * Notice! Caller should make sure that there is an available slot.
+ *
+ * @param bucketNum
+ * @param key
+ * @param value
+ * @param isUpsert
+ * @return 1 for new insertion, 0 for key duplication
+ */
+ public int put(int bucketNum, int key, int value, boolean isUpsert) {
+ int i;
+ int emptySlot=-1;
+
+ if (cArray[bucketNum][0] == 0) {
+ cArray[bucketNum][2] = key;
+ cArray[bucketNum][3] = value;
+ cArray[bucketNum][0]++;
+ return 1;
+ }
+
+ for (i = 1; i < NUM_OF_SLOTS; i++) {
+ if (cArray[bucketNum][i*2] == key) {
+ if (isUpsert) {
+ cArray[bucketNum][emptySlot*2+1] = value;
+ }
+ return 0;
+ }
+ else if (cArray[bucketNum][i*2] == -1) {
+ emptySlot = i;
+ }
+ }
+
+ if (emptySlot == -1) {
+ throw new UnsupportedOperationException("error");
+ }
+
+ cArray[bucketNum][emptySlot*2] = key;
+ cArray[bucketNum][emptySlot*2+1] = value;
+ cArray[bucketNum][0]++;
+ return 1;
+ }
+
+ public int get(int bucketNum, int key) {
+ int i;
+
+ if (cArray[bucketNum][0] == 0) {
+ return -1;
+ }
+
+ for (i = 1; i < NUM_OF_SLOTS; i++) {
+ if (cArray[bucketNum][i*2] == key) {
+ return cArray[bucketNum][i*2+1];
+ }
+ }
+ return -1;
+ }
+
+ /**
+ * remove key if it exists. Otherwise, ignore it.
+ * @param bucketNum
+ * @param key
+ * @return 1 for success, 0 if the key doesn't exist
+ */
+ public int remove(int bucketNum, int key) {
+ int i;
+
+ if (cArray[bucketNum][0] == 0) {
+ return 0;
+ }
+
+ for (i = 1; i < NUM_OF_SLOTS; i++) {
+ if (cArray[bucketNum][i*2] == key) {
+ cArray[bucketNum][i*2] = -1;
+ cArray[bucketNum][0]--;
+ return 1;
+ }
+ }
+
+ return 0;
+ }
+}
+
+
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/TimeOutDetector.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/TimeOutDetector.java
index 6c391f4..699fd74 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/TimeOutDetector.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/TimeOutDetector.java
@@ -6,7 +6,8 @@
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
/**
- * @author pouria Any transaction which has been waiting for a lock for more
+ * @author pouria, kisskys
+ * Any transaction which has been waiting for a lock for more
* than the predefined time-out threshold is considered to be deadlocked
* (this can happen in distributed case for example) An instance of this
* class triggers scanning (sweeping) lock manager's transactions table
@@ -15,14 +16,14 @@
public class TimeOutDetector {
static final long TIME_OUT_THRESHOLD = 60000;
- static final long SWEEP_PERIOD = 120000;
+ static final long SWEEP_PERIOD = 10000;//120000;
LockManager lockMgr;
Thread trigger;
- LinkedList<WaitEntry> victimsWObjs;
+ LinkedList<LockWaiter> victimList;
public TimeOutDetector(LockManager lockMgr) {
- this.victimsWObjs = new LinkedList<WaitEntry>();
+ this.victimList = new LinkedList<LockWaiter>();
this.lockMgr = lockMgr;
this.trigger = new Thread(new TimeoutTrigger(this));
trigger.setDaemon(true);
@@ -30,33 +31,29 @@
}
public void sweep() throws ACIDException {
- victimsWObjs.clear();
- lockMgr.sweepForTimeout(); // Initiates the time-out sweeping process
- // from the lockManager
+ victimList.clear();
+ // Initiates the time-out sweeping process
+ // from the lockManager
+ lockMgr.sweepForTimeout();
notifyVictims();
}
- public boolean isVictim(TxrInfo txrInfo) {
- long sWTime = txrInfo.getStartWaitTime();
- int status = txrInfo.getContext().getStatus();
- return (status != TransactionContext.TIMED_OUT_SATUS && sWTime != TransactionContext.INVALID_TIME && (System
- .currentTimeMillis() - sWTime) >= TIME_OUT_THRESHOLD);
- }
-
- public void addToVictimsList(WaitEntry wEntry) {
- victimsWObjs.add(wEntry);
+ public void checkAndSetVictim(LockWaiter waiterObj) {
+ if (System.currentTimeMillis() - waiterObj.getBeginWaitTime() >= TIME_OUT_THRESHOLD) {
+ waiterObj.setVictim(true);
+ waiterObj.setWait(false);
+ victimList.add(waiterObj);
+ }
}
private void notifyVictims() {
- for (WaitEntry w : victimsWObjs) {
- synchronized (w) {
- w.wakeUp();
- w.notifyAll();
+ for (LockWaiter waiterObj : victimList) {
+ synchronized (waiterObj) {
+ waiterObj.notifyAll();
}
}
- victimsWObjs.clear();
+ victimList.clear();
}
-
}
class TimeoutTrigger implements Runnable {
@@ -79,9 +76,6 @@
} catch (ACIDException e) {
throw new IllegalStateException(e);
}
-
}
-
}
-
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/TxrInfo.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/TxrInfo.java
deleted file mode 100644
index 097c3ec..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/TxrInfo.java
+++ /dev/null
@@ -1,130 +0,0 @@
-package edu.uci.ics.asterix.transaction.management.service.locking;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Iterator;
-
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
-
-/**
- * @author pouria An instance shows information about all the locks a specific
- * transaction is holding and/or is waiting on (whether for conversion
- * or as a regular waiter) (Each TInfo instance in the infoList captures
- * information about one lock on one resource)
- * If the transaction is waiting for a lock on a specific resource, the
- * ID of that resource is captured in waitingOnRid
- */
-
-public class TxrInfo {
- public static final int NOT_FOUND = -2;
- public static final int NOT_KNOWN_IX = -3;
-
- private ArrayList<TInfo> infoList;
- private byte[] waitingOnRid;
- private TransactionContext context;
-
- public TxrInfo(TransactionContext context) {
- this.context = context;
- this.infoList = new ArrayList<TInfo>();
- this.waitingOnRid = null;
- }
-
- public TInfo getTxrInfo(byte[] resourceId, int lMode, int eix) {
- if (eix == NOT_KNOWN_IX) {
- eix = findInList(resourceId, lMode);
- }
-
- if (eix != NOT_FOUND) {
- return infoList.get(eix);
- }
- return null;
- }
-
- public void addGrantedLock(byte[] resourceId, int lMode) {
- int eix = findInList(resourceId, lMode);
- if (eix == NOT_FOUND) { // We do not add a redundant lock here
- infoList.add(new TInfo(resourceId, lMode));
- }
- }
-
- public void removeLock(byte[] resourceId, int lMode, int eix) {
- if (eix == NOT_KNOWN_IX) {
- eix = findInList(resourceId, lMode);
- }
- if (eix != NOT_FOUND) {
- infoList.remove(eix);
- }
- }
-
- public TransactionContext getContext() {
- return context;
- }
-
- public void setWaitOnRid(byte[] resourceId) {
- this.waitingOnRid = null;
- if (resourceId != null) {
- this.waitingOnRid = Arrays.copyOf(resourceId, resourceId.length);
- }
-
- }
-
- public byte[] getWaitOnRid() {
- return this.waitingOnRid;
- }
-
- public long getStartWaitTime() {
- return this.context.getStartWaitTime();
- }
-
- public int getSize() {
- return infoList.size();
- }
-
- public int findInList(byte[] resourceId, int lMode) {
- for (int i = 0; i < infoList.size(); i++) {
- TInfo ti = infoList.get(i);
- if (((lMode == LockInfo.ANY_LOCK_MODE) || (lMode == ti.getMode()))
- && Arrays.equals(ti.getResourceId(), resourceId)) {
- return i;
- }
- }
- return NOT_FOUND;
- }
-
- public Iterator<TInfo> getIterator() { // TODO change the direct way of
- // accessing
- return infoList.iterator();
- }
-}
-
-class TInfo {
- private byte[] resourceId; // The resource on which the lock is held or is
- // waiting to be held
- private int lockMode; // The granted/waiting-for lockMode
-
- public TInfo(byte[] rId, int lMode) {
- this.resourceId = rId;
- this.lockMode = lMode;
- }
-
- public byte[] getResourceId() {
- return this.resourceId;
- }
-
- public int getMode() {
- return lockMode;
- }
-
- public void setMode(int mode) {
- lockMode = mode;
- }
-
- @Override
- public boolean equals(Object o) {
- if (o == null || !(o instanceof TInfo)) {
- return false;
- }
- TInfo t = (TInfo) o;
- return ((t.lockMode == lockMode) && (Arrays.equals(t.resourceId, resourceId)));
- }
-}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/DataUtil.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/DataUtil.java
index 41a2a52..4b8d62f 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/DataUtil.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/DataUtil.java
@@ -47,4 +47,17 @@
return bytes;
}
+ public static byte[] longToByteArray(long value) {
+ byte[] bytes = new byte[8];
+ bytes[0] = (byte) ((value >>> 56) & 0xFF);
+ bytes[1] = (byte) ((value >>> 48) & 0xFF);
+ bytes[2] = (byte) ((value >>> 40) & 0xFF);
+ bytes[3] = (byte) ((value >>> 32) & 0xFF);
+ bytes[4] = (byte) ((value >>> 24) & 0xFF);
+ bytes[5] = (byte) ((value >>> 16) & 0xFF);
+ bytes[6] = (byte) ((value >>> 8) & 0xFF);
+ bytes[7] = (byte) ((value >>> 0) & 0xFF);
+ return bytes;
+ }
+
}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecordHelper.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecordHelper.java
index e5cf1af..7575b45 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecordHelper.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecordHelper.java
@@ -30,10 +30,16 @@
private final int BEGIN_ACTION_TYPE_POS = 9;
private final int BEGIN_TIMESTAMP_POS = 10;
private final int BEGIN_TRANSACTION_ID_POS = 18;
+ /*
private final int BEGIN_RESOURCE_MGR_ID_POS = 26;
private final int BEGIN_PAGE_ID_POS = 27;
private final int BEGIN_PREV_LSN_POS = 35;
+ */
+ private final int BEGIN_RESOURCE_MGR_ID_POS = 22;
+ private final int BEGIN_PAGE_ID_POS = 23;
+ private final int BEGIN_PREV_LSN_POS = 31;
+
private ILogManager logManager;
public LogRecordHelper(ILogManager logManager) {
@@ -62,7 +68,7 @@
}
public long getLogTransactionId(LogicalLogLocator logicalLogLocator) {
- return (logicalLogLocator.getBuffer()).readLong(logicalLogLocator.getMemoryOffset() + BEGIN_TRANSACTION_ID_POS);
+ return (logicalLogLocator.getBuffer()).readInt(logicalLogLocator.getMemoryOffset() + BEGIN_TRANSACTION_ID_POS);
}
public byte getResourceMgrId(LogicalLogLocator logicalLogLocator) {
@@ -144,8 +150,8 @@
(logicalLogLocator.getBuffer()).writeLong(logicalLogLocator.getMemoryOffset() + BEGIN_TIMESTAMP_POS, timestamp);
/* transaction id */
- (logicalLogLocator.getBuffer()).writeLong(logicalLogLocator.getMemoryOffset() + BEGIN_TRANSACTION_ID_POS,
- context.getTransactionID());
+ (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + BEGIN_TRANSACTION_ID_POS,
+ context.getJobId().getId());
/* resource Mgr id */
(logicalLogLocator.getBuffer()).put(logicalLogLocator.getMemoryOffset() + BEGIN_RESOURCE_MGR_ID_POS,
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeLogger.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeLogger.java
index 3b923dd..b742ada 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeLogger.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeLogger.java
@@ -25,7 +25,7 @@
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
/**
* Represents a utility class for generating log records corresponding to
@@ -58,10 +58,10 @@
class TxnThreadState {
private ITupleReference tuple;
- private IndexOp indexOperation;
+ private IndexOperation indexOperation;
private LogicalLogLocator logicalLogLocator;
- public TxnThreadState(LogicalLogLocator logicalLogLocator, IndexOp indexOperation, ITupleReference tupleReference) {
+ public TxnThreadState(LogicalLogLocator logicalLogLocator, IndexOperation indexOperation, ITupleReference tupleReference) {
this.tuple = tupleReference;
this.indexOperation = indexOperation;
this.logicalLogLocator = logicalLogLocator;
@@ -75,11 +75,11 @@
this.tuple = tuple;
}
- public synchronized IndexOp getIndexOperation() {
+ public synchronized IndexOperation getIndexOperation() {
return indexOperation;
}
- public synchronized void setIndexOperation(IndexOp indexOperation) {
+ public synchronized void setIndexOperation(IndexOperation indexOperation) {
this.indexOperation = indexOperation;
}
@@ -122,32 +122,32 @@
}
public synchronized void close(TransactionContext context) {
- TransactionState txnState = (TransactionState) arguments.get(context.getTransactionID());
+ TransactionState txnState = (TransactionState) arguments.get(context.getJobId());
txnState.remove(Thread.currentThread().getId());
- arguments.remove(context.getTransactionID());
+ arguments.remove(context.getJobId());
}
- public void generateLogRecord(TransactionProvider provider, TransactionContext context, IndexOp operation,
+ public void generateLogRecord(TransactionProvider provider, TransactionContext context, IndexOperation operation,
ITupleReference tuple) throws ACIDException {
context.addCloseableResource(this); // the close method would be called
// on this TreeLogger instance at
// the time of transaction
// commit/abort.
- if (operation != IndexOp.INSERT && operation != IndexOp.DELETE) {
+ if (operation != IndexOperation.INSERT && operation != IndexOperation.DELETE) {
throw new ACIDException("Loging for Operation " + operation + " not supported");
}
TxnThreadState txnThreadState = null;
TransactionState txnState;
- txnState = (TransactionState) arguments.get(context.getTransactionID());
+ txnState = (TransactionState) arguments.get(context.getJobId());
if (txnState == null) {
synchronized (context) { // threads belonging to different
// transaction do not need to
// synchronize amongst them.
if (txnState == null) {
txnState = new TransactionState();
- arguments.put(context.getTransactionID(), txnState);
+ arguments.put(context.getJobId(), txnState);
}
}
}
@@ -172,7 +172,7 @@
@Override
public void log(TransactionContext context, LogicalLogLocator logicalLogLocator, int logRecordSize,
Map<Object, Object> loggerArguments) throws ACIDException {
- TransactionState txnState = (TransactionState) loggerArguments.get(context.getTransactionID());
+ TransactionState txnState = (TransactionState) loggerArguments.get(context.getJobId());
TxnThreadState state = (TxnThreadState) txnState.getTransactionThreadState(Thread.currentThread().getId());
int count = 0;
byte[] logBuffer = logicalLogLocator.getBuffer().getArray();
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeResourceManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeResourceManager.java
index 7c6d05a..4e6ad80 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeResourceManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeResourceManager.java
@@ -20,6 +20,7 @@
import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
public class TreeResourceManager implements IResourceManager {
@@ -36,7 +37,6 @@
}
public void undo(ILogRecordHelper logRecordHelper, LogicalLogLocator logLocator) throws ACIDException {
-
int logContentBeginPos = logRecordHelper.getLogContentBeginPos(logLocator);
byte[] logBufferContent = logLocator.getBuffer().getArray();
// read the length of resource id byte array
@@ -58,7 +58,8 @@
tupleReference.setFieldCount(tupleReference.getFieldCount());
tupleReference.resetByTupleOffset(logLocator.getBuffer().getByteBuffer(), tupleBeginPos);
byte operation = logBufferContent[operationOffset];
- IIndexAccessor treeIndexAccessor = treeIndex.createAccessor();
+ IIndexAccessor treeIndexAccessor = treeIndex.createAccessor(NoOpOperationCallback.INSTANCE,
+ NoOpOperationCallback.INSTANCE);
try {
switch (operation) {
case TreeLogger.BTreeOperationCodes.INSERT:
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
index 9c78b95..e96cc73 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
@@ -166,7 +166,7 @@
/**
* Rollback a transaction (non-Javadoc)
*
- * @see edu.uci.ics.transaction.management.service.recovery.IRecoveryManager# rollbackTransaction (edu.uci.ics.transaction.management.service.transaction .TransactionContext)
+ * @see edu.uci.ics.transaction.management.service.recovery.IRecoveryManager# rollbackTransaction (edu.uci.ics.TransactionContext.management.service.transaction .TransactionContext)
*/
@Override
public void rollbackTransaction(TransactionContext txnContext) throws ACIDException {
@@ -183,7 +183,7 @@
if (lsn.getLsn() == TransactionManagementConstants.LogManagerConstants.TERMINAL_LSN) {
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info(" no need to roll back as there were no operations by the transaction "
- + txnContext.getTransactionID());
+ + txnContext.getJobId());
}
return;
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/DatasetId.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/DatasetId.java
new file mode 100644
index 0000000..9aded2a
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/DatasetId.java
@@ -0,0 +1,32 @@
+package edu.uci.ics.asterix.transaction.management.service.transaction;
+
+import java.io.Serializable;
+
+public class DatasetId implements Serializable {
+ int id;
+
+ public DatasetId(int id) {
+ this.id = id;
+ }
+
+ public void setId(int id) {
+ this.id = id;
+ }
+
+ public int getId() {
+ return id;
+ }
+
+ @Override
+ public int hashCode() {
+ return id;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if ((o == null) || !(o instanceof DatasetId)) {
+ return false;
+ }
+ return ((DatasetId) o).id == this.id;
+ }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/DatasetIdFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/DatasetIdFactory.java
new file mode 100644
index 0000000..65512ec
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/DatasetIdFactory.java
@@ -0,0 +1,15 @@
+package edu.uci.ics.asterix.transaction.management.service.transaction;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class DatasetIdFactory {
+ private static AtomicInteger id = new AtomicInteger();
+
+ public static void initialize(int initialId) {
+ id.set(initialId);
+ }
+
+ public static int generateDatasetId() {
+ return id.incrementAndGet();
+ }
+}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/ITransactionManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/ITransactionManager.java
index d7078bd..a876bb2 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/ITransactionManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/ITransactionManager.java
@@ -40,24 +40,24 @@
* Begins a transaction identified by a transaction id and returns the
* associated transaction context.
*
- * @param transactionId
+ * @param jobId
* a unique value for the transaction id.
* @return the transaction context associated with the initiated transaction
* @see TransactionContext
* @throws ACIDException
*/
- public TransactionContext beginTransaction(long transactionId) throws ACIDException;
+ public TransactionContext beginTransaction(JobId jobId) throws ACIDException;
/**
* Returns the transaction context of an active transaction given the
* transaction id.
*
- * @param transactionId
+ * @param jobId
* a unique value for the transaction id.
* @return
* @throws ACIDException
*/
- public TransactionContext getTransactionContext(long transactionId) throws ACIDException;
+ public TransactionContext getTransactionContext(JobId jobId) throws ACIDException;
/**
* Commits a transaction.
@@ -65,7 +65,7 @@
* @param txnContext
* the transaction context associated with the transaction
* @throws ACIDException
- * @see transactionContext
+ * @see transactionContextimport edu.uci.ics.hyracks.api.job.JobId;
* @see ACIDException
*/
public void commitTransaction(TransactionContext txnContext) throws ACIDException;
@@ -96,21 +96,6 @@
public void completedTransaction(TransactionContext txnContext, boolean success) throws ACIDException;
/**
- * Associates a resource manager with a transaction. In a distributed
- * transaction multiple resource managers can join a transaction and
- * participate in a two phase commit protocol. This method is not used
- * currently as we do not support distributed transactions.
- *
- * @param txnContext
- * the transaction context associated with the transaction
- * @param resourceMgrID
- * a unique identifier for the resource manager.
- * @see IResourceManager
- * @throws ACIDException
- */
- public void joinTransaction(TransactionContext txnContext, byte[] resourceMgrID) throws ACIDException;
-
- /**
* Returns the Transaction Provider for the transaction eco-system. A
* transaction eco-system consists of a Log Manager, a Recovery Manager, a
* Transaction Manager and a Lock Manager.
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/JobId.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/JobId.java
new file mode 100644
index 0000000..d306670
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/JobId.java
@@ -0,0 +1,42 @@
+package edu.uci.ics.asterix.transaction.management.service.transaction;
+
+import java.io.Serializable;
+
+public class JobId implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ private int id;
+
+ public JobId(int id) {
+ this.id = id;
+ }
+
+ public int getId() {
+ return id;
+ }
+
+ @Override
+ public int hashCode() {
+ return id;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (o == this) {
+ return true;
+ }
+ if (!(o instanceof JobId)) {
+ return false;
+ }
+ return ((JobId) o).id == id;
+ }
+
+ @Override
+ public String toString() {
+ return "JID:" + id;
+ }
+
+ public void setId(int jobId) {
+ id = jobId;
+ }
+}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionIDFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/JobIdFactory.java
similarity index 77%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionIDFactory.java
rename to asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/JobIdFactory.java
index e5475f4..da86199 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionIDFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/JobIdFactory.java
@@ -14,15 +14,15 @@
*/
package edu.uci.ics.asterix.transaction.management.service.transaction;
-import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicInteger;
/**
* Represents a factory to generate unique transaction IDs.
*/
-public class TransactionIDFactory {
- private static final AtomicLong ID = new AtomicLong();
+public class JobIdFactory {
+ private static final AtomicInteger Id = new AtomicInteger();
- public static long generateTransactionId() {
- return ID.incrementAndGet();
+ public static JobId generateJobId() {
+ return new JobId(Id.incrementAndGet());
}
}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java
index e37f892..587f358 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java
@@ -36,7 +36,7 @@
public static final long INVALID_TIME = -1l; // used for showing a
// transaction is not waiting.
public static final int ACTIVE_STATUS = 0;
- public static final int TIMED_OUT_SATUS = 1;
+ public static final int TIMED_OUT_STATUS = 1;
public enum TransactionType {
READ,
@@ -45,13 +45,13 @@
private static final long serialVersionUID = -6105616785783310111L;
private TransactionProvider transactionProvider;
- private long transactionID;
private LogicalLogLocator lastLogLocator;
private TransactionState txnState;
private long startWaitTime;
private int status;
private Set<ICloseable> resources = new HashSet<ICloseable>();
private TransactionType transactionType = TransactionType.READ;
+ private JobId jobId;
public void setTransactionType(TransactionType transactionType) {
this.transactionType = transactionType;
@@ -65,8 +65,8 @@
resources.add(resource);
}
- public TransactionContext(long transactionId, TransactionProvider transactionProvider) throws ACIDException {
- this.transactionID = transactionId;
+ public TransactionContext(JobId jobId, TransactionProvider transactionProvider) throws ACIDException {
+ this.jobId = jobId;
this.transactionProvider = transactionProvider;
init();
}
@@ -86,8 +86,8 @@
this.lastLogLocator = lastLogLocator;
}
- public long getTransactionID() {
- return transactionID;
+ public JobId getJobId() {
+ return jobId;
}
public void setStartWaitTime(long time) {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManagementConstants.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManagementConstants.java
index ad65973..cb71e9f 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManagementConstants.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManagementConstants.java
@@ -41,8 +41,10 @@
public static final int[] LOCK_CONVERT_MATRIX = new int[] { 2, 0 };
public static class LockMode {
- public static final int SHARED = 0;
- public static final int EXCLUSIVE = 1;
+ public static final byte S = 0;
+ public static final byte X = 1;
+ public static final byte IS = 2;
+ public static final byte IX = 3;
}
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManager.java
index 8e7f26a..63deb13 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManager.java
@@ -30,7 +30,7 @@
public class TransactionManager implements ITransactionManager {
private static final Logger LOGGER = Logger.getLogger(TransactionManager.class.getName());
private final TransactionProvider transactionProvider;
- private Map<Long, TransactionContext> transactionContextRepository = new HashMap<Long, TransactionContext>();
+ private Map<JobId, TransactionContext> transactionContextRepository = new HashMap<JobId, TransactionContext>();
public TransactionManager(TransactionProvider provider) {
this.transactionProvider = provider;
@@ -54,29 +54,29 @@
} finally {
txnContext.releaseResources();
transactionProvider.getLockManager().releaseLocks(txnContext);
- transactionContextRepository.remove(txnContext.getTransactionID());
+ transactionContextRepository.remove(txnContext.getJobId());
txnContext.setTxnState(TransactionState.ABORTED);
}
}
}
@Override
- public TransactionContext beginTransaction(long transactionId) throws ACIDException {
- TransactionContext txnContext = new TransactionContext(transactionId, transactionProvider);
+ public TransactionContext beginTransaction(JobId jobId) throws ACIDException {
+ TransactionContext txnContext = new TransactionContext(jobId, transactionProvider);
synchronized (this) {
- transactionContextRepository.put(transactionId, txnContext);
+ transactionContextRepository.put(jobId, txnContext);
}
return txnContext;
}
@Override
- public TransactionContext getTransactionContext(long transactionId) throws ACIDException {
+ public TransactionContext getTransactionContext(JobId jobId) throws ACIDException {
synchronized (transactionContextRepository) {
- TransactionContext context = transactionContextRepository.get(transactionId);
+ TransactionContext context = transactionContextRepository.get(jobId);
if (context == null) {
- context = transactionContextRepository.get(transactionId);
- context = new TransactionContext(transactionId, transactionProvider);
- transactionContextRepository.put(transactionId, context);
+ context = transactionContextRepository.get(jobId);
+ context = new TransactionContext(jobId, transactionProvider);
+ transactionContextRepository.put(jobId, context);
}
return context;
}
@@ -100,27 +100,19 @@
}
} catch (ACIDException ae) {
if (LOGGER.isLoggable(Level.SEVERE)) {
- LOGGER.severe(" caused exception in commit !" + txnContext.getTransactionID());
+ LOGGER.severe(" caused exception in commit !" + txnContext.getJobId());
}
throw ae;
} finally {
txnContext.releaseResources();
transactionProvider.getLockManager().releaseLocks(txnContext); // release
- transactionContextRepository.remove(txnContext.getTransactionID());
+ transactionContextRepository.remove(txnContext.getJobId());
txnContext.setTxnState(TransactionState.COMMITTED);
}
}
}
@Override
- public void joinTransaction(TransactionContext txnContext, byte[] resourceMgrID) throws ACIDException {
- throw new UnsupportedOperationException();
- // TODO this method will be implemented as part of support for
- // distributed transactions
-
- }
-
- @Override
public void completedTransaction(TransactionContext txnContext, boolean success) throws ACIDException {
if (!success) {
abortTransaction(txnContext);
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/TransactionWorkloadSimulator.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/TransactionWorkloadSimulator.java
index 496260d..cc266fe 100644
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/TransactionWorkloadSimulator.java
+++ b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/TransactionWorkloadSimulator.java
@@ -20,7 +20,6 @@
import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
import edu.uci.ics.asterix.transaction.management.logging.BasicLogger;
-import edu.uci.ics.asterix.transaction.management.resource.TransactionalResourceRepository;
import edu.uci.ics.asterix.transaction.management.service.locking.ILockManager;
import edu.uci.ics.asterix.transaction.management.service.logging.ILogManager;
import edu.uci.ics.asterix.transaction.management.service.logging.ILogRecordHelper;
@@ -28,9 +27,10 @@
import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
import edu.uci.ics.asterix.transaction.management.service.logging.LogUtil;
import edu.uci.ics.asterix.transaction.management.service.logging.LogicalLogLocator;
+import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobIdFactory;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionIDFactory;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
public class TransactionWorkloadSimulator {
@@ -99,7 +99,7 @@
public static TransactionContext getContext(TransactionProvider provider) throws ACIDException {
if (context == null) {
- context = new TransactionContext(TransactionIDFactory.generateTransactionId(), provider);
+ context = new TransactionContext(JobIdFactory.generateJobId(), provider);
}
return context;
}
@@ -108,7 +108,7 @@
class MultipleTransactionContextFactory {
public static TransactionContext getContext(TransactionProvider provider) throws ACIDException {
- return new TransactionContext(TransactionIDFactory.generateTransactionId(), provider);
+ return new TransactionContext(JobIdFactory.generateJobId(), provider);
}
}
@@ -121,10 +121,12 @@
LogicalLogLocator memLSN;
String name;
TransactionContext context;
- private byte[] resourceID = new byte[1];
+ //private byte[] resourceID = new byte[1];
+ private int resourceID;
private int myLogCount = 0;
private TransactionProvider transactionProvider;
private ILogManager logManager;
+ private DatasetId tempDatasetId = new DatasetId(-1);
public Transaction(TransactionProvider provider, String name, boolean singleTransaction) throws ACIDException {
this.name = name;
@@ -157,12 +159,12 @@
}
if (TransactionWorkloadSimulator.workload.singleResource) {
int choice = random.nextInt(2);
- resourceID[0] = (byte) (choice % 2);
+ resourceID = (byte) (choice % 2);
} else {
- random.nextBytes(resourceID);
+ random.nextInt(resourceID);
}
boolean retry = false;
- int lockMode = -1;
+ byte lockMode = -1;
try {
for (int i = 0; i < numLogs - 1; i++) {
int logSize = TransactionWorkloadSimulator.workload.minLogSize
@@ -174,13 +176,10 @@
byte logActionType = LogActionType.REDO_UNDO;
long pageId = 0;
if (!retry) {
- lockMode = random.nextInt(2);
+ lockMode = (byte)(random.nextInt(2));
}
- boolean lockGranted = TransactionWorkloadSimulator.lockManager.lock(context, resourceID, lockMode);
- if (!lockGranted) {
- retry = true;
- continue;
- }
+ tempDatasetId.setId(resourceID);
+ TransactionWorkloadSimulator.lockManager.lock(tempDatasetId, -1, lockMode, context);
TransactionWorkloadSimulator.logManager.log(memLSN, context, ResourceMgrInfo.BTreeResourceMgrId,
pageId, logType, logActionType, logSize, logger, null);
retry = false;
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionSimulator.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionSimulator.java
index ffa5af6..4476955 100644
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionSimulator.java
+++ b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionSimulator.java
@@ -31,8 +31,9 @@
import edu.uci.ics.asterix.transaction.management.service.recovery.IRecoveryManager.SystemState;
import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager;
import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionManager;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionIDFactory;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobIdFactory;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
public class TransactionSimulator {
@@ -63,8 +64,8 @@
}
public TransactionContext beginTransaction() throws ACIDException {
- long transactionId = TransactionIDFactory.generateTransactionId();
- return transactionManager.beginTransaction(transactionId);
+ JobId jobId = JobIdFactory.generateJobId();
+ return transactionManager.beginTransaction(jobId);
}
public void executeTransactionOperation(TransactionContext txnContext, FileResource.CounterOperation operation)
diff --git a/pom.xml b/pom.xml
index 95f8103..c5a3959 100644
--- a/pom.xml
+++ b/pom.xml
@@ -74,7 +74,6 @@
<module>asterix-runtime</module>
<module>asterix-om</module>
<module>asterix-aql</module>
- <module>asterix-hyracks-glue</module>
<module>asterix-external-data</module>
<module>asterix-metadata</module>
<module>asterix-dist</module>