merged r1017:1055 asterix_stabilization
git-svn-id: https://asterixdb.googlecode.com/svn/branches/asterix_lsm_stabilization@1056 eaa15691-b419-025a-1212-ee371bd00084
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/CommitOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/CommitOperator.java
new file mode 100644
index 0000000..98774d2
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/CommitOperator.java
@@ -0,0 +1,66 @@
+/*
+ * 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.algebra.operators;
+
+import java.util.Collection;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractExtensibleLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorExtension;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+
+public class CommitOperator extends AbstractExtensibleLogicalOperator {
+
+ private final List<LogicalVariable> primaryKeyLogicalVars;
+
+ public CommitOperator(List<LogicalVariable> primaryKeyLogicalVars) {
+ this.primaryKeyLogicalVars = primaryKeyLogicalVars;
+ }
+
+ @Override
+ public boolean isMap() {
+ // TODO Auto-generated method stub
+ return false;
+ }
+
+ @Override
+ public IOperatorExtension newInstance() {
+ return new CommitOperator(primaryKeyLogicalVars);
+ }
+
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform transform) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return false;
+ }
+
+ @Override
+ public String toString() {
+ return "commit";
+ }
+
+ @Override
+ public void getUsedVariables(Collection<LogicalVariable> usedVars) {
+ usedVars.addAll(primaryKeyLogicalVars);
+ }
+
+ @Override
+ public void getProducedVariables(Collection<LogicalVariable> producedVars) {
+ // No produced variables.
+ }
+}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitPOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitPOperator.java
new file mode 100644
index 0000000..dcf601c
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitPOperator.java
@@ -0,0 +1,100 @@
+/*
+ * 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.algebra.operators.physical;
+
+import java.util.List;
+
+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.core.algebra.base.IHyracksJobBuilder;
+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.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AbstractPhysicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public class CommitPOperator extends AbstractPhysicalOperator {
+
+ private final List<LogicalVariable> primaryKeyLogicalVars;
+ private final JobId jobId;
+ private final int datasetId;
+ private final boolean isWriteTransaction;
+
+ public CommitPOperator(JobId jobId, int datasetId, List<LogicalVariable> primaryKeyLogicalVars,
+ boolean isWriteTransaction) {
+ this.jobId = jobId;
+ this.datasetId = datasetId;
+ this.primaryKeyLogicalVars = primaryKeyLogicalVars;
+ this.isWriteTransaction = isWriteTransaction;
+ }
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.EXTENSION_OPERATOR;
+ }
+
+ @Override
+ public String toString() {
+ return "COMMIT";
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ return emptyUnaryRequirements();
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context)
+ throws AlgebricksException {
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+ deliveredProperties = (StructuralPropertiesVector) op2.getDeliveredPhysicalProperties().clone();
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+
+ RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema,
+ context);
+ int[] primaryKeyFields = JobGenHelper.variablesToFieldIndexes(primaryKeyLogicalVars, inputSchemas[0]);
+ IVariableTypeEnvironment varTypeEnv = context.getTypeEnvironment(op.getInputs().get(0).getValue());
+ IBinaryHashFunctionFactory[] binaryHashFunctionFactories = JobGenHelper.variablesToBinaryHashFunctionFactories(
+ primaryKeyLogicalVars, varTypeEnv, context);
+
+ CommitRuntimeFactory runtime = new CommitRuntimeFactory(jobId, datasetId, primaryKeyFields,
+ binaryHashFunctionFactories, isWriteTransaction);
+ builder.contributeMicroOperator(op, runtime, recDesc);
+ ILogicalOperator src = op.getInputs().get(0).getValue();
+ builder.contributeGraphEdge(src, 0, op, 0);
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return true;
+ }
+}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntime.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntime.java
new file mode 100644
index 0000000..98dbb5b
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntime.java
@@ -0,0 +1,123 @@
+/*
+ * 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.algebra.operators.physical;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+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.DatasetId;
+import edu.uci.ics.asterix.transaction.management.service.transaction.FieldsHashValueGenerator;
+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.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntime;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.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.data.accessors.FrameTupleReference;
+
+public class CommitRuntime implements IPushRuntime {
+
+ private final IHyracksTaskContext hyracksTaskCtx;
+ private final ITransactionManager transactionManager;
+ private final JobId jobId;
+ private final DatasetId datasetId;
+ private final int[] primaryKeyFields;
+ private final IBinaryHashFunction[] primaryKeyHashFunctions;
+ private final boolean isWriteTransaction;
+
+ private TransactionContext transactionContext;
+ private RecordDescriptor inputRecordDesc;
+ private FrameTupleAccessor frameTupleAccessor;
+ private FrameTupleReference frameTupleReference;
+
+ public CommitRuntime(IHyracksTaskContext ctx, JobId jobId, int datasetId, int[] primaryKeyFields,
+ IBinaryHashFunctionFactory[] binaryHashFunctionFactories, boolean isWriteTransaction) {
+ this.hyracksTaskCtx = ctx;
+ AsterixAppRuntimeContext runtimeCtx = (AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext()
+ .getApplicationObject();
+ this.transactionManager = runtimeCtx.getTransactionSubsystem().getTransactionManager();
+ this.jobId = jobId;
+ this.datasetId = new DatasetId(datasetId);
+ this.primaryKeyFields = primaryKeyFields;
+ primaryKeyHashFunctions = new IBinaryHashFunction[binaryHashFunctionFactories.length];
+ for (int i = 0; i < binaryHashFunctionFactories.length; ++i) {
+ this.primaryKeyHashFunctions[i] = binaryHashFunctionFactories[i].createBinaryHashFunction();
+ }
+ this.frameTupleReference = new FrameTupleReference();
+ this.isWriteTransaction = isWriteTransaction;
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ try {
+ transactionContext = transactionManager.getTransactionContext(jobId);
+ transactionContext.setTransactionType(isWriteTransaction ? TransactionType.READ_WRITE
+ : TransactionType.READ);
+ } catch (ACIDException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ int pkHash = 0;
+ frameTupleAccessor.reset(buffer);
+ int nTuple = frameTupleAccessor.getTupleCount();
+ for (int t = 0; t < nTuple; t++) {
+ frameTupleReference.reset(frameTupleAccessor, t);
+ pkHash = FieldsHashValueGenerator.computeFieldsHashValue(frameTupleReference, primaryKeyFields,
+ primaryKeyHashFunctions);
+ try {
+ transactionManager.commitTransaction(transactionContext, datasetId, pkHash);
+ } catch (ACIDException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ // TODO Auto-generated method stub
+
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ // TODO Auto-generated method stub
+ }
+
+ @Override
+ public void setFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+ throw new IllegalStateException();
+ }
+
+ @Override
+ public void setInputRecordDescriptor(int index, RecordDescriptor recordDescriptor) {
+ this.inputRecordDesc = recordDescriptor;
+ this.frameTupleAccessor = new FrameTupleAccessor(hyracksTaskCtx.getFrameSize(), recordDescriptor);
+ }
+}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntimeFactory.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntimeFactory.java
new file mode 100644
index 0000000..4012bbb
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntimeFactory.java
@@ -0,0 +1,53 @@
+/*
+ * 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.algebra.operators.physical;
+
+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.runtime.base.IPushRuntime;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+
+public class CommitRuntimeFactory implements IPushRuntimeFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private final JobId jobId;
+ private final int datasetId;
+ private final int[] primaryKeyFields;
+ IBinaryHashFunctionFactory[] binaryHashFunctionFactories;
+ private final boolean isWriteTransaction;
+
+ public CommitRuntimeFactory(JobId jobId, int datasetId, int[] primaryKeyFields,
+ IBinaryHashFunctionFactory[] binaryHashFunctionFactories, boolean isWriteTransaction) {
+ this.jobId = jobId;
+ this.datasetId = datasetId;
+ this.primaryKeyFields = primaryKeyFields;
+ this.binaryHashFunctionFactories = binaryHashFunctionFactories;
+ this.isWriteTransaction = isWriteTransaction;
+ }
+
+ @Override
+ public String toString() {
+ return "commit";
+ }
+
+ @Override
+ public IPushRuntime createPushRuntime(IHyracksTaskContext ctx) throws AlgebricksException {
+ return new CommitRuntime(ctx, jobId, datasetId, primaryKeyFields, binaryHashFunctionFactories, isWriteTransaction);
+ }
+}
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 510aa4c..05aa954 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.MetadataException;
import edu.uci.ics.asterix.metadata.MetadataManager;
@@ -43,25 +44,37 @@
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.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.ShortPointable;
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.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
+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.dataflow.PartitionedLSMInvertedIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
/**
* Contributes the runtime operator for an unnest-map representing an
* inverted-index search.
*/
public class InvertedIndexPOperator extends IndexSearchPOperator {
- public InvertedIndexPOperator(IDataSourceIndex<String, AqlSourceId> idx, boolean requiresBroadcast) {
+ private final boolean isPartitioned;
+
+ public InvertedIndexPOperator(IDataSourceIndex<String, AqlSourceId> idx, boolean requiresBroadcast,
+ boolean isPartitioned) {
super(idx, requiresBroadcast);
+ this.isPartitioned = isPartitioned;
}
@Override
public PhysicalOperatorTag getOperatorTag() {
- return PhysicalOperatorTag.INVERTED_INDEX_SEARCH;
+ if (isPartitioned) {
+ return PhysicalOperatorTag.FUZZY_INVERTED_INDEX_SEARCH;
+ } else {
+ return PhysicalOperatorTag.INVERTED_INDEX_SEARCH;
+ }
}
@Override
@@ -138,18 +151,23 @@
}
// TODO: For now we assume the type of the generated tokens is the
- // same
- // as the indexed field.
+ // same as the indexed field.
// We need a better way of expressing this because tokens may be
- // hashed,
- // or an inverted-index may index a list type, etc.
- ITypeTraits[] tokenTypeTraits = new ITypeTraits[numSecondaryKeys];
- IBinaryComparatorFactory[] tokenComparatorFactories = new IBinaryComparatorFactory[numSecondaryKeys];
+ // hashed, or an inverted-index may index a list type, etc.
+ int numTokenKeys = (!isPartitioned) ? numSecondaryKeys : numSecondaryKeys + 1;
+ ITypeTraits[] tokenTypeTraits = new ITypeTraits[numTokenKeys];
+ IBinaryComparatorFactory[] tokenComparatorFactories = new IBinaryComparatorFactory[numTokenKeys];
for (int i = 0; i < numSecondaryKeys; i++) {
tokenComparatorFactories[i] = InvertedIndexAccessMethod
.getTokenBinaryComparatorFactory(secondaryKeyType);
tokenTypeTraits[i] = InvertedIndexAccessMethod.getTokenTypeTrait(secondaryKeyType);
}
+ if (isPartitioned) {
+ // The partitioning field is hardcoded to be a short *without* an Asterix type tag.
+ tokenComparatorFactories[numSecondaryKeys] = PointableBinaryComparatorFactory
+ .of(ShortPointable.FACTORY);
+ tokenTypeTraits[numSecondaryKeys] = ShortPointable.TYPE_TRAITS;
+ }
IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(unnestMap);
List<LogicalVariable> outputVars = unnestMap.getVariables();
@@ -169,9 +187,6 @@
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadataProvider
.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataset.getDataverseName(),
datasetName, indexName);
- Pair<IFileSplitProvider, IFileSplitProvider> fileSplitProviders = metadataProvider
- .getInvertedIndexFileSplitProviders(secondarySplitsAndConstraint.first);
-
// TODO: Here we assume there is only one search key field.
int queryField = keyFields[0];
// Get tokenizer and search modifier factories.
@@ -179,12 +194,23 @@
.getSearchModifierFactory(searchModifierType, simThresh, secondaryIndex);
IBinaryTokenizerFactory queryTokenizerFactory = InvertedIndexAccessMethod.getBinaryTokenizerFactory(
searchModifierType, searchKeyType, secondaryIndex);
- InvertedIndexSearchOperatorDescriptor invIndexSearchOp = new InvertedIndexSearchOperatorDescriptor(jobSpec,
- queryField, appContext.getStorageManagerInterface(), fileSplitProviders.first,
- fileSplitProviders.second, appContext.getIndexRegistryProvider(), tokenTypeTraits,
- tokenComparatorFactories, invListsTypeTraits, invListsComparatorFactories,
- new BTreeDataflowHelperFactory(), queryTokenizerFactory, searchModifierFactory, outputRecDesc,
- retainInput, NoOpOperationCallbackProvider.INSTANCE);
+ IIndexDataflowHelperFactory dataflowHelperFactory;
+ if (!isPartitioned) {
+ dataflowHelperFactory = new LSMInvertedIndexDataflowHelperFactory(
+ AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER);
+ } else {
+ dataflowHelperFactory = new PartitionedLSMInvertedIndexDataflowHelperFactory(
+ AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER);
+ }
+ LSMInvertedIndexSearchOperatorDescriptor invIndexSearchOp = new LSMInvertedIndexSearchOperatorDescriptor(
+ jobSpec, queryField, appContext.getStorageManagerInterface(), secondarySplitsAndConstraint.first,
+ appContext.getIndexLifecycleManagerProvider(), tokenTypeTraits, tokenComparatorFactories,
+ invListsTypeTraits, invListsComparatorFactories, dataflowHelperFactory, queryTokenizerFactory,
+ searchModifierFactory, outputRecDesc, retainInput, NoOpOperationCallbackFactory.INSTANCE);
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(invIndexSearchOp,
secondarySplitsAndConstraint.second);
} catch (MetadataException e) {
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 f63cc0c..512b107 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
@@ -44,6 +44,7 @@
import edu.uci.ics.asterix.optimizer.rules.PushSimilarityFunctionsBelowJoin;
import edu.uci.ics.asterix.optimizer.rules.RemoveRedundantListifyRule;
import edu.uci.ics.asterix.optimizer.rules.RemoveUnusedOneToOneEquiJoinRule;
+import edu.uci.ics.asterix.optimizer.rules.ReplaceSinkOpWithCommitOpRule;
import edu.uci.ics.asterix.optimizer.rules.SetAsterixPhysicalOperatorsRule;
import edu.uci.ics.asterix.optimizer.rules.SetClosedRecordConstructorsRule;
import edu.uci.ics.asterix.optimizer.rules.SimilarityCheckRule;
@@ -224,6 +225,9 @@
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 ReplaceSinkOpWithCommitOpRule());
physicalRewritesAllLevels.add(new SetAlgebricksPhysicalOperatorsRule());
physicalRewritesAllLevels.add(new SetAsterixPhysicalOperatorsRule());
physicalRewritesAllLevels.add(new EnforceStructuralPropertiesRule());
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 8936425..420c713 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
@@ -267,7 +267,7 @@
// The translator will compile metadata internally. Run this compilation
// under the same transaction id as the "outer" compilation.
AqlPlusExpressionToPlanTranslator translator = new AqlPlusExpressionToPlanTranslator(
- metadataProvider.getJobTxnId(), metadataProvider, counter, null, null);
+ metadataProvider.getJobId(), metadataProvider, counter, null, 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/ReplaceSinkOpWithCommitOpRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ReplaceSinkOpWithCommitOpRule.java
new file mode 100644
index 0000000..30ed8e7
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ReplaceSinkOpWithCommitOpRule.java
@@ -0,0 +1,94 @@
+package edu.uci.ics.asterix.optimizer.rules;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.asterix.algebra.operators.CommitOperator;
+import edu.uci.ics.asterix.algebra.operators.physical.CommitPOperator;
+import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
+import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+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.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.ExtensionOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class ReplaceSinkOpWithCommitOpRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (op.getOperatorTag() != LogicalOperatorTag.SINK) {
+ return false;
+ }
+ SinkOperator sinkOperator = (SinkOperator) op;
+
+ List<Mutable<ILogicalExpression>> primaryKeyExprs = null;
+ int datasetId = 0;
+ AbstractLogicalOperator descendantOp = (AbstractLogicalOperator) sinkOperator.getInputs().get(0).getValue();
+ while (descendantOp != null) {
+ if (descendantOp.getOperatorTag() == LogicalOperatorTag.INDEX_INSERT_DELETE) {
+ IndexInsertDeleteOperator indexInsertDeleteOperator = (IndexInsertDeleteOperator) descendantOp;
+ primaryKeyExprs = indexInsertDeleteOperator.getPrimaryKeyExpressions();
+ datasetId = ((AqlDataSource) indexInsertDeleteOperator.getDataSourceIndex().getDataSource()).getDataset().getDatasetId();
+ break;
+ } else if (descendantOp.getOperatorTag() == LogicalOperatorTag.INSERT_DELETE) {
+ InsertDeleteOperator insertDeleteOperator = (InsertDeleteOperator) descendantOp;
+ primaryKeyExprs = insertDeleteOperator.getPrimaryKeyExpressions();
+ datasetId = ((AqlDataSource) insertDeleteOperator.getDataSource()).getDataset().getDatasetId();
+ break;
+ }
+ descendantOp = (AbstractLogicalOperator) descendantOp.getInputs().get(0).getValue();
+ }
+
+ if (primaryKeyExprs != null) {
+
+ //copy primaryKeyExprs
+ List<LogicalVariable> primaryKeyLogicalVars = new ArrayList<LogicalVariable>();
+ for (Mutable<ILogicalExpression> expr : primaryKeyExprs) {
+ VariableReferenceExpression varRefExpr = (VariableReferenceExpression)expr.getValue();
+ primaryKeyLogicalVars.add(new LogicalVariable(varRefExpr.getVariableReference().getId()));
+ }
+
+ //get JobId(TransactorId)
+ AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
+ JobId jobId = mp.getJobId();
+
+ //create the logical and physical operator
+ CommitOperator commitOperator = new CommitOperator(primaryKeyLogicalVars);
+ CommitPOperator commitPOperator = new CommitPOperator(jobId, datasetId, primaryKeyLogicalVars, mp.isWriteTransaction());
+ commitOperator.setPhysicalOperator(commitPOperator);
+
+ //create ExtensionOperator and put the commitOperator in it.
+ ExtensionOperator extensionOperator = new ExtensionOperator(commitOperator);
+ extensionOperator.setPhysicalOperator(commitPOperator);
+
+ //update plan link
+ extensionOperator.getInputs().add(sinkOperator.getInputs().get(0));
+ context.computeAndSetTypeEnvironmentForOperator(extensionOperator);
+ opRef.setValue(extensionOperator);
+ }
+
+ return true;
+ }
+
+}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
index 19791e3..066de80 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
@@ -173,12 +173,14 @@
op.setPhysicalOperator(new RTreeSearchPOperator(dsi, requiresBroadcast));
break;
}
- case WORD_INVIX: {
- op.setPhysicalOperator(new InvertedIndexPOperator(dsi, requiresBroadcast));
+ case WORD_INVIX:
+ case NGRAM_INVIX: {
+ op.setPhysicalOperator(new InvertedIndexPOperator(dsi, requiresBroadcast, false));
break;
}
- case NGRAM_INVIX: {
- op.setPhysicalOperator(new InvertedIndexPOperator(dsi, requiresBroadcast));
+ case FUZZY_WORD_INVIX:
+ case FUZZY_NGRAM_INVIX: {
+ op.setPhysicalOperator(new InvertedIndexPOperator(dsi, requiresBroadcast, true));
break;
}
default: {
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodUtils.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodUtils.java
index 6651ea3..d555eda 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodUtils.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodUtils.java
@@ -132,7 +132,9 @@
switch (index.getIndexType()) {
case BTREE:
case WORD_INVIX:
- case NGRAM_INVIX: {
+ case NGRAM_INVIX:
+ case FUZZY_WORD_INVIX:
+ case FUZZY_NGRAM_INVIX: {
return index.getKeyFieldNames().size();
}
case RTREE: {
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeAccessMethod.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeAccessMethod.java
index 1379bf4..1ede912 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeAccessMethod.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeAccessMethod.java
@@ -353,22 +353,26 @@
if (!isPrimaryIndex) {
primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceScan, dataset, recordType,
secondaryIndexUnnestOp, context, true, retainInput, false);
+
+ // Replace the datasource scan with the new plan rooted at
+ // primaryIndexUnnestMap.
+ indexSubTree.dataSourceScanRef.setValue(primaryIndexUnnestOp); //kisskys
} else {
List<Object> primaryIndexOutputTypes = new ArrayList<Object>();
AccessMethodUtils.appendPrimaryIndexTypes(dataset, recordType, primaryIndexOutputTypes);
primaryIndexUnnestOp = new UnnestMapOperator(dataSourceScan.getVariables(),
secondaryIndexUnnestOp.getExpressionRef(), primaryIndexOutputTypes, retainInput);
primaryIndexUnnestOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
- }
- List<Mutable<ILogicalExpression>> remainingFuncExprs = new ArrayList<Mutable<ILogicalExpression>>();
- getNewConditionExprs(conditionRef, replacedFuncExprs, remainingFuncExprs);
- // Generate new condition.
- if (!remainingFuncExprs.isEmpty()) {
- ILogicalExpression pulledCond = createSelectCondition(remainingFuncExprs);
- conditionRef.setValue(pulledCond);
- } else {
- conditionRef.setValue(null);
+ List<Mutable<ILogicalExpression>> remainingFuncExprs = new ArrayList<Mutable<ILogicalExpression>>();
+ getNewConditionExprs(conditionRef, replacedFuncExprs, remainingFuncExprs);
+ // Generate new condition.
+ if (!remainingFuncExprs.isEmpty()) {
+ ILogicalExpression pulledCond = createSelectCondition(remainingFuncExprs);
+ conditionRef.setValue(pulledCond);
+ } else {
+ conditionRef.setValue(null);
+ }
}
return primaryIndexUnnestOp;
}
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 84a86b1..d4d6b43 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
@@ -12,7 +12,6 @@
import edu.uci.ics.asterix.algebra.base.LogicalOperatorDeepCopyVisitor;
import edu.uci.ics.asterix.aql.util.FunctionUtils;
import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
-import edu.uci.ics.asterix.dataflow.data.common.ListEditDistanceSearchModifierFactory;
import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
import edu.uci.ics.asterix.formats.nontagged.AqlBinaryTokenizerFactoryProvider;
import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
@@ -60,11 +59,12 @@
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.search.ListEditDistanceSearchModifierFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
/**
* Class for helping rewrite rules to choose and apply inverted indexes.
@@ -773,7 +773,8 @@
AInt32 edThresh = (AInt32) intObj;
int mergeThreshold = 0;
// We can only optimize edit distance on strings using an ngram index.
- if (listOrStrObj.getType().getTypeTag() == ATypeTag.STRING && index.getIndexType() == IndexType.NGRAM_INVIX) {
+ if (listOrStrObj.getType().getTypeTag() == ATypeTag.STRING
+ && (index.getIndexType() == IndexType.NGRAM_INVIX || index.getIndexType() == IndexType.FUZZY_NGRAM_INVIX)) {
AString astr = (AString) listOrStrObj;
// Compute merge threshold.
mergeThreshold = (astr.getStringValue().length() + index.getGramLength() - 1)
@@ -781,7 +782,7 @@
}
// We can only optimize edit distance on lists using a word index.
if ((listOrStrObj.getType().getTypeTag() == ATypeTag.ORDEREDLIST || listOrStrObj.getType().getTypeTag() == ATypeTag.UNORDEREDLIST)
- && index.getIndexType() == IndexType.WORD_INVIX) {
+ && (index.getIndexType() == IndexType.WORD_INVIX || index.getIndexType() == IndexType.FUZZY_WORD_INVIX)) {
IACollection alist = (IACollection) listOrStrObj;
// Compute merge threshold.
mergeThreshold = alist.size() - edThresh.getIntegerValue();
@@ -807,11 +808,11 @@
AbstractFunctionCallExpression nonConstfuncExpr = (AbstractFunctionCallExpression) nonConstArg;
// We can use this index if the tokenization function matches the index type.
if (nonConstfuncExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.WORD_TOKENS
- && index.getIndexType() == IndexType.WORD_INVIX) {
+ && (index.getIndexType() == IndexType.WORD_INVIX || index.getIndexType() == IndexType.FUZZY_WORD_INVIX)) {
return true;
}
if (nonConstfuncExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.GRAM_TOKENS
- && index.getIndexType() == IndexType.NGRAM_INVIX) {
+ && (index.getIndexType() == IndexType.NGRAM_INVIX || index.getIndexType() == IndexType.FUZZY_NGRAM_INVIX)) {
return true;
}
}
@@ -824,7 +825,7 @@
}
// We can only optimize contains with ngram indexes.
if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == AsterixBuiltinFunctions.CONTAINS
- && index.getIndexType() == IndexType.NGRAM_INVIX) {
+ && (index.getIndexType() == IndexType.NGRAM_INVIX || index.getIndexType() == IndexType.FUZZY_NGRAM_INVIX)) {
// Check that the constant search string has at least gramLength characters.
AsterixConstantValue strConstVal = (AsterixConstantValue) optFuncExpr.getConstantVal(0);
IAObject strObj = strConstVal.getObject();
@@ -877,10 +878,12 @@
public static IBinaryTokenizerFactory getBinaryTokenizerFactory(SearchModifierType searchModifierType,
ATypeTag searchKeyType, Index index) throws AlgebricksException {
switch (index.getIndexType()) {
- case WORD_INVIX: {
+ case WORD_INVIX:
+ case FUZZY_WORD_INVIX: {
return AqlBinaryTokenizerFactoryProvider.INSTANCE.getWordTokenizerFactory(searchKeyType, false);
}
- case NGRAM_INVIX: {
+ case NGRAM_INVIX:
+ case FUZZY_NGRAM_INVIX: {
// Make sure not to use pre- and postfixing for conjunctive searches.
boolean prePost = (searchModifierType == SearchModifierType.CONJUNCTIVE) ? false : true;
return AqlBinaryTokenizerFactoryProvider.INSTANCE.getNGramTokenizerFactory(searchKeyType,
@@ -895,10 +898,12 @@
public static IBinaryTokenizerFactory getBinaryTokenizerFactory(ATypeTag keyType, IndexType indexType,
int gramLength) throws AlgebricksException {
switch (indexType) {
- case WORD_INVIX: {
+ case WORD_INVIX:
+ case FUZZY_WORD_INVIX: {
return AqlBinaryTokenizerFactoryProvider.INSTANCE.getWordTokenizerFactory(keyType, false);
}
- case NGRAM_INVIX: {
+ case NGRAM_INVIX:
+ case FUZZY_NGRAM_INVIX: {
return AqlBinaryTokenizerFactoryProvider.INSTANCE.getNGramTokenizerFactory(keyType, gramLength, true,
false);
}
@@ -921,11 +926,13 @@
case EDIT_DISTANCE: {
int edThresh = ((AInt32) simThresh).getIntegerValue();
switch (index.getIndexType()) {
- case NGRAM_INVIX: {
+ case NGRAM_INVIX:
+ case FUZZY_NGRAM_INVIX: {
// Edit distance on strings, filtered with overlapping grams.
return new EditDistanceSearchModifierFactory(index.getGramLength(), edThresh);
}
- case WORD_INVIX: {
+ case WORD_INVIX:
+ case FUZZY_WORD_INVIX: {
// Edit distance on two lists. The list-elements are non-overlapping.
return new ListEditDistanceSearchModifierFactory(edThresh);
}
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 fdf9c0b..7028e23 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
@@ -163,7 +163,6 @@
private final ICompiledDmlStatement stmt;
private static AtomicLong outputFileID = new AtomicLong(0);
private static final String OUTPUT_FILE_PREFIX = "OUTPUT_";
-
private static LogicalVariable METADATA_DUMMY_VAR = new LogicalVariable(-1);
public AqlExpressionToPlanTranslator(AqlMetadataProvider metadataProvider, int currentVarCounter,
@@ -186,7 +185,6 @@
new EmptyTupleSourceOperator()));
ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<Mutable<ILogicalOperator>>();
-
boolean isTransactionalWrite = false;
ILogicalOperator topOp = p.first;
ProjectOperator project = (ProjectOperator) topOp;
@@ -207,11 +205,9 @@
AqlDataSource targetDatasource = validateDatasetInfo(metadataProvider, stmt.getDataverseName(),
stmt.getDatasetName());
-
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(targetDatasource.getDataset());
for (String keyFieldName : partitionKeys) {
IFunctionInfo finfoAccess = AsterixBuiltinFunctions
@@ -236,7 +232,6 @@
switch (stmt.getKind()) {
case WRITE_FROM_QUERY_RESULT: {
-
leafOperator = new WriteResultOperator(targetDatasource, varRef, varRefsForLoading);
leafOperator.getInputs().add(new MutableObject<ILogicalOperator>(assign));
break;
@@ -271,7 +266,6 @@
}
topOp = leafOperator;
}
-
globalPlanRoots.add(new MutableObject<ILogicalOperator>(topOp));
ILogicalPlan plan = new ALogicalPlanImpl(globalPlanRoots);
return plan;
@@ -292,7 +286,6 @@
throw new AlgebricksException("Cannot write output to an external dataset.");
}
return dataSource;
-
}
private FileSplit getDefaultOutputFileLocation() throws MetadataException {
@@ -309,7 +302,6 @@
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;
@@ -408,7 +400,6 @@
AssignOperator a = new AssignOperator(v, new MutableObject<ILogicalExpression>(fldAccess));
a.getInputs().add(p.second);
return new Pair<ILogicalOperator, LogicalVariable>(a, v);
-
}
@Override
@@ -513,7 +504,6 @@
private AbstractFunctionCallExpression lookupBuiltinFunction(String functionName, int arity,
List<Mutable<ILogicalExpression>> args) {
AbstractFunctionCallExpression f = null;
-
FunctionIdentifier fi = new FunctionIdentifier(AlgebricksBuiltinFunctions.ALGEBRICKS_NS, functionName, arity);
AsterixFunctionInfo afi = AsterixBuiltinFunctions.lookupFunction(fi);
FunctionIdentifier builtinAquafi = afi == null ? null : afi.getFunctionIdentifier();
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 be6e2af..3440ce8 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
@@ -94,6 +94,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.asterix.translator.CompiledStatements.ICompiledDmlStatement;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
@@ -190,7 +191,7 @@
}
}
- private final long txnId;
+ private final JobId jobId;
private TranslationContext context;
private String outputDatasetName;
private ICompiledDmlStatement stmt;
@@ -202,9 +203,9 @@
- public AqlPlusExpressionToPlanTranslator(long txnId, AqlMetadataProvider metadataProvider,
+ public AqlPlusExpressionToPlanTranslator(JobId jobId, AqlMetadataProvider metadataProvider,
Counter currentVarCounter, String outputDatasetName, ICompiledDmlStatement stmt) {
- this.txnId = txnId;
+ this.jobId = jobId;
this.metadataProvider = metadataProvider;
this.context = new TranslationContext(currentVarCounter);
this.outputDatasetName = outputDatasetName;
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/aqlj/server/APIClientThread.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/aqlj/server/APIClientThread.java
index 9879a2d..98fe0a1 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/aqlj/server/APIClientThread.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/aqlj/server/APIClientThread.java
@@ -14,7 +14,6 @@
*/
package edu.uci.ics.asterix.api.aqlj.server;
-import java.io.File;
import java.io.IOException;
import java.io.PrintWriter;
import java.io.StringReader;
@@ -37,9 +36,8 @@
import edu.uci.ics.asterix.aql.translator.AqlTranslator;
import edu.uci.ics.asterix.aql.translator.QueryResult;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.hyracks.bootstrap.AsterixNodeState;
+import edu.uci.ics.asterix.hyracks.bootstrap.CCBootstrapImpl;
import edu.uci.ics.asterix.metadata.MetadataManager;
-import edu.uci.ics.asterix.metadata.api.IAsterixStateProxy;
import edu.uci.ics.asterix.metadata.bootstrap.AsterixProperties;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
@@ -63,9 +61,9 @@
private final AQLJStream clientStream;
private final String outputFilePath;
private final String outputNodeName;
- private final String outputNodeIP;
private final String binaryOutputClause;
+ private String outputNodeIP;
private AQLJStream nodeDataServerStream;
private int nodeDataServerPort;
private String dataverse;
@@ -75,37 +73,26 @@
this.hcc = hcc;
clientStream = new AQLJStream(clientSocket);
this.appContext = appCtx;
+ outputNodeName = AsterixProperties.INSTANCE.getMetadataNodeName();
+ outputFilePath = AsterixProperties.INSTANCE.getOutputDir() + System.currentTimeMillis() + ".adm";
- // get the name of the first node controller that we find
- // all query results will be written to this node
Map<String, Set<String>> nodeNameMap = new HashMap<String, Set<String>>();
try {
this.appContext.getCCContext().getIPAddressNodeMap(nodeNameMap);
} catch (Exception e) {
throw new IOException(" unable to obtain IP address node map", e);
}
- outputNodeIP = (String) nodeNameMap.keySet().toArray()[0];
- outputNodeName = (String) nodeNameMap.get(outputNodeIP).toArray()[0];
-
- // get the port of the node data server that is running on the first nc
- IAsterixStateProxy proxy = (IAsterixStateProxy) appCtx.getDistributedState();
- nodeDataServerPort = ((AsterixNodeState) proxy.getAsterixNodeState(outputNodeName)).getAPINodeDataServerPort();
- nodeDataServerStream = null;
-
- // write the data into the output stores directory of the nc
- // if output stores are unavailable (could they ever be?), then write to
- // tmpdir which can be overridden
- // Also, use milliseconds in path name of output file to differentiate
- // queries
- Map<String, String[]> storesMap = AsterixProperties.INSTANCE.getStores();
- String[] outputStores = storesMap.get(outputNodeName);
- if (outputStores.length > 0) {
- outputFilePath = outputStores[0] + System.currentTimeMillis() + ".adm";
- } else {
- outputFilePath = System.getProperty("java.io.tmpdir") + File.pathSeparator + System.currentTimeMillis()
- + ".adm";
+ for (String ip : nodeNameMap.keySet()) {
+ Set<String> nodeNames = nodeNameMap.get(ip);
+ if (nodeNames.contains(outputNodeName)) {
+ outputNodeIP = ip;
+ break;
+ }
}
+ nodeDataServerPort = CCBootstrapImpl.DEFAULT_API_NODEDATA_SERVER_PORT;
+ nodeDataServerStream = null;
+
// the "write output..." clause is inserted into incoming AQL statements
binaryOutputClause = "write output to " + outputNodeName + ":\"" + outputFilePath
+ "\" using \"edu.uci.ics.hyracks.algebricks.runtime.writers.SerializedDataWriterFactory\";";
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 1aad2ec..468bca6 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
@@ -29,6 +29,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.JobIdFactory;
import edu.uci.ics.asterix.translator.AqlExpressionToPlanTranslator;
import edu.uci.ics.asterix.translator.CompiledStatements.ICompiledDmlStatement;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
@@ -54,6 +55,7 @@
import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IOptimizationContextFactory;
import edu.uci.ics.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.job.IJobletEventListenerFactory;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobSpecification;
@@ -90,6 +92,8 @@
RuleCollections.buildAccessMethodRuleCollection()));
defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,
RuleCollections.buildPlanCleanupRuleCollection()));
+
+ //put TXnRuleCollection!
return defaultLogicalRewrites;
}
@@ -131,8 +135,10 @@
HTML
}
- public static Pair<Query, Integer> reWriteQuery(List<FunctionDecl> declaredFunctions, AqlMetadataProvider metadataProvider,
- Query q, SessionConfig pc, PrintWriter out, DisplayFormat pdf) throws AsterixException {
+ public static Pair<Query, Integer> reWriteQuery(List<FunctionDecl> declaredFunctions,
+ AqlMetadataProvider metadataProvider, Query q, SessionConfig pc, PrintWriter out, DisplayFormat pdf)
+ throws AsterixException {
+
if (!pc.isPrintPhysicalOpsOnly() && pc.isPrintExprParam()) {
out.println();
switch (pdf) {
@@ -195,6 +201,9 @@
}
+ edu.uci.ics.asterix.transaction.management.service.transaction.JobId asterixJobId = JobIdFactory
+ .generateJobId();
+ queryMetadataProvider.setJobId(asterixJobId);
AqlExpressionToPlanTranslator t = new AqlExpressionToPlanTranslator(queryMetadataProvider, varCounter,
outputDatasetName, statement);
@@ -316,10 +325,10 @@
builder.setTypeTraitProvider(format.getTypeTraitProvider());
builder.setNormalizedKeyComputerFactoryProvider(format.getNormalizedKeyComputerFactoryProvider());
- JobSpecification spec = compiler.createJob(AsterixAppContextInfoImpl.INSTANCE);
- // set the job event listener
- spec.setJobletEventListenerFactory(new JobEventListenerFactory(queryMetadataProvider.getJobTxnId(),
- isWriteTransaction));
+ IJobletEventListenerFactory jobEventListenerFactory = new JobEventListenerFactory(asterixJobId,
+ isWriteTransaction);
+ JobSpecification spec = compiler.createJob(AsterixAppContextInfoImpl.INSTANCE, jobEventListenerFactory);
+
if (pc.isPrintJob()) {
switch (pdf) {
case HTML: {
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
index 80293b0..a2b2163 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
@@ -82,7 +82,7 @@
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.asterix.om.types.TypeSignature;
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.DatasetIdFactory;
import edu.uci.ics.asterix.translator.AbstractAqlTranslator;
import edu.uci.ics.asterix.translator.CompiledStatements.CompiledBeginFeedStatement;
import edu.uci.ics.asterix.translator.CompiledStatements.CompiledControlFeedStatement;
@@ -374,7 +374,7 @@
}
}
MetadataManager.INSTANCE.addDataset(metadataProvider.getMetadataTxnContext(), new Dataset(dataverseName,
- datasetName, itemTypeName, datasetDetails, dsType));
+ datasetName, itemTypeName, datasetDetails, dsType, DatasetIdFactory.generateDatasetId()));
if (dd.getDatasetType() == DatasetType.INTERNAL || dd.getDatasetType() == DatasetType.FEED) {
Dataverse dataverse = MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(),
dataverseName);
@@ -693,9 +693,6 @@
sessionConfig, out, pdf);
// Query Compilation (happens under the same ongoing metadata transaction)
- if (metadataProvider.isWriteTransaction()) {
- metadataProvider.setJobTxnId(TransactionIDFactory.generateTransactionId());
- }
JobSpecification spec = APIFramework.compileQuery(declaredFunctions, metadataProvider, query,
reWrittenQuery.second, stmt == null ? null : stmt.getDatasetName(), sessionConfig, out, pdf, stmt);
@@ -834,5 +831,4 @@
}
return format;
}
-
}
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 8d804a4..b32f129 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.MetadataManager;
@@ -37,7 +36,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.CompiledStatements.CompiledDatasetDropStatement;
import edu.uci.ics.asterix.translator.CompiledStatements.CompiledLoadFromFileStatement;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
@@ -70,14 +71,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.common.impls.NoOpOperationCallbackFactory;
+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 {
@@ -102,9 +106,6 @@
LOGGER.info("DROP DATASETPATH: " + datasetPath);
- IIndexRegistryProvider<IIndex> indexRegistryProvider = AsterixIndexRegistryProvider.INSTANCE;
- IStorageManagerInterface storageManager = AsterixStorageManagerInterface.INSTANCE;
-
Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
if (dataset == null) {
throw new AlgebricksException("DROP DATASET: No metadata for dataset " + datasetName);
@@ -132,8 +133,35 @@
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> idxSplitsAndConstraint = metadataProvider
.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataset.getDataverseName(),
datasetName, index.getIndexName());
- TreeIndexDropOperatorDescriptor secondaryBtreeDrop = new TreeIndexDropOperatorDescriptor(specs[i],
- storageManager, indexRegistryProvider, idxSplitsAndConstraint.first);
+ IIndexDataflowHelperFactory dfhFactory;
+ switch (index.getIndexType()) {
+ case BTREE:
+ dfhFactory = new LSMBTreeDataflowHelperFactory(
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER);
+ break;
+ case RTREE:
+ dfhFactory = new LSMRTreeDataflowHelperFactory(
+ new IPrimitiveValueProviderFactory[] { null }, RTreePolicyType.RTREE,
+ new IBinaryComparatorFactory[] { null },
+ AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, null);
+ break;
+ case NGRAM_INVIX:
+ case WORD_INVIX:
+ dfhFactory = new LSMInvertedIndexDataflowHelperFactory(
+ AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER);
+ break;
+ default:
+ throw new AsterixException("Unknown index type provided.");
+ }
+ IndexDropOperatorDescriptor secondaryBtreeDrop = new IndexDropOperatorDescriptor(specs[i],
+ AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+ AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, idxSplitsAndConstraint.first, dfhFactory);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(specs[i], secondaryBtreeDrop,
idxSplitsAndConstraint.second);
i++;
@@ -148,8 +176,12 @@
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadataProvider
.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataset.getDataverseName(), datasetName,
datasetName);
- TreeIndexDropOperatorDescriptor primaryBtreeDrop = new TreeIndexDropOperatorDescriptor(specPrimary,
- storageManager, indexRegistryProvider, splitsAndConstraint.first);
+ IndexDropOperatorDescriptor primaryBtreeDrop = new IndexDropOperatorDescriptor(specPrimary,
+ AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+ splitsAndConstraint.first, new LSMBTreeDataflowHelperFactory(
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER));
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(specPrimary, primaryBtreeDrop,
splitsAndConstraint.second);
@@ -184,11 +216,15 @@
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.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+ splitsAndConstraint.first, typeTraits, comparatorFactories, new LSMBTreeDataflowHelperFactory(
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER), localResourceFactoryProvider,
+ NoOpOperationCallbackFactory.INSTANCE);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, indexCreateOp,
splitsAndConstraint.second);
spec.addRoot(indexCreateOp);
@@ -211,6 +247,9 @@
}
JobSpecification spec = new JobSpecification();
+ edu.uci.ics.asterix.transaction.management.service.transaction.JobId asterixJobId = JobIdFactory
+ .generateJobId();
+
ARecordType itemType = (ARecordType) MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataverseName,
dataset.getItemTypeName()).getDatatype();
ISerializerDeserializer payloadSerde = format.getSerdeProvider().getSerializerDeserializer(itemType);
@@ -259,12 +298,13 @@
}
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(),
- NoOpOperationCallbackProvider.INSTANCE);
+ AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+ splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation,
+ GlobalConfig.DEFAULT_BTREE_FILL_FACTOR, false, new LSMBTreeDataflowHelperFactory(
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER), NoOpOperationCallbackFactory.INSTANCE);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, btreeBulkLoad,
splitsAndConstraint.second);
@@ -287,6 +327,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 a8f18ec..1cbc4dc 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
@@ -1,8 +1,7 @@
package edu.uci.ics.asterix.file;
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.AqlMetadataProvider;
@@ -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 {
@@ -47,13 +44,15 @@
String indexName = indexDropStmt.getIndexName();
JobSpecification spec = new JobSpecification();
- IIndexRegistryProvider<IIndex> indexRegistryProvider = AsterixIndexRegistryProvider.INSTANCE;
- IStorageManagerInterface storageManager = AsterixStorageManagerInterface.INSTANCE;
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadataProvider
.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverseName, datasetName, indexName);
- TreeIndexDropOperatorDescriptor btreeDrop = new TreeIndexDropOperatorDescriptor(spec, storageManager,
- indexRegistryProvider, splitsAndConstraint.first);
+ IndexDropOperatorDescriptor btreeDrop = new IndexDropOperatorDescriptor(spec,
+ AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+ splitsAndConstraint.first, new LSMBTreeDataflowHelperFactory(
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER));
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..a9b9050 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,13 @@
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.common.impls.NoOpOperationCallbackFactory;
+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 +28,15 @@
@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.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
- new BTreeDataflowHelperFactory(), NoOpOperationCallbackProvider.INSTANCE);
+ new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER), localResourceFactoryProvider,
+ NoOpOperationCallbackFactory.INSTANCE);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,
secondaryPartitionConstraint);
spec.addRoot(secondaryIndexCreateOp);
@@ -63,7 +68,9 @@
// Create secondary BTree bulk load op.
TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(spec, numSecondaryKeys,
- new BTreeDataflowHelperFactory(), BTree.DEFAULT_FILL_FACTOR);
+ new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER), 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 34ba208..7b89c3f 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.common.impls.NoOpOperationCallbackFactory;
+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,
@@ -102,7 +101,9 @@
break;
}
case WORD_INVIX:
- case NGRAM_INVIX: {
+ case NGRAM_INVIX:
+ case FUZZY_WORD_INVIX:
+ case FUZZY_NGRAM_INVIX: {
indexCreator = new SecondaryInvertedIndexCreator(physOptConf);
break;
}
@@ -137,13 +138,12 @@
numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
numSecondaryKeys = createIndexStmt.getKeyFields().size();
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> primarySplitsAndConstraint = metadataProvider
- .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
- dataverseName, datasetName, datasetName);
+ .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverseName, datasetName, datasetName);
primaryFileSplitProvider = primarySplitsAndConstraint.first;
primaryPartitionConstraint = primarySplitsAndConstraint.second;
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadataProvider
- .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
- dataverseName, datasetName, secondaryIndexName);
+ .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverseName, datasetName,
+ secondaryIndexName);
secondaryFileSplitProvider = secondarySplitsAndConstraint.first;
secondaryPartitionConstraint = secondarySplitsAndConstraint.second;
// Must be called in this order.
@@ -230,10 +230,13 @@
// +Infinity
int[] highKeyFields = null;
BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
- AsterixStorageManagerInterface.INSTANCE, AsterixIndexRegistryProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
primaryFileSplitProvider, primaryRecDesc.getTypeTraits(), primaryComparatorFactories, lowKeyFields,
- highKeyFields, true, true, new BTreeDataflowHelperFactory(), false,
- NoOpOperationCallbackProvider.INSTANCE);
+ highKeyFields, true, true, new LSMBTreeDataflowHelperFactory(
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER), false,
+ NoOpOperationCallbackFactory.INSTANCE);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, primarySearchOp,
primaryPartitionConstraint);
return primarySearchOp;
@@ -286,12 +289,12 @@
fieldPermutation[i] = i;
}
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadataProvider
- .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
- dataverseName, datasetName, secondaryIndexName);
+ .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverseName, datasetName,
+ secondaryIndexName);
TreeIndexBulkLoadOperatorDescriptor treeIndexBulkLoadOp = new TreeIndexBulkLoadOperatorDescriptor(spec,
- AsterixStorageManagerInterface.INSTANCE, AsterixIndexRegistryProvider.INSTANCE,
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
secondarySplitsAndConstraint.first, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
- fieldPermutation, fillFactor, dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+ fieldPermutation, fillFactor, false, dataflowHelperFactory, NoOpOperationCallbackFactory.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 dc12814..df00769 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,8 @@
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.config.DatasetConfig.IndexType;
+import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
import edu.uci.ics.asterix.metadata.entities.Index;
@@ -25,17 +25,23 @@
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.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.ShortPointable;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.ShortSerializerDeserializer;
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.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.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
+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.dataflow.PartitionedLSMInvertedIndexDataflowHelperFactory;
+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 {
@@ -44,11 +50,11 @@
private IBinaryComparatorFactory[] tokenComparatorFactories;
private ITypeTraits[] tokenTypeTraits;
private IBinaryTokenizerFactory tokenizerFactory;
- private Pair<IFileSplitProvider, IFileSplitProvider> fileSplitProviders;
// For tokenization, sorting and loading. Represents <token, primary keys>.
private int numTokenKeyPairFields;
private IBinaryComparatorFactory[] tokenKeyPairComparatorFactories;
private RecordDescriptor tokenKeyPairRecDesc;
+ private boolean isPartitioned;
protected SecondaryInvertedIndexCreator(PhysicalOptimizationConfig physOptConf) {
super(physOptConf);
@@ -56,8 +62,8 @@
@Override
@SuppressWarnings("rawtypes")
- protected void setSecondaryRecDescAndComparators(CompiledCreateIndexStatement createIndexStmt, AqlMetadataProvider metadata)
- throws AlgebricksException, AsterixException {
+ protected void setSecondaryRecDescAndComparators(CompiledCreateIndexStatement createIndexStmt,
+ AqlMetadataProvider metadata) throws AlgebricksException, AsterixException {
// Sanity checks.
if (numPrimaryKeys > 1) {
throw new AsterixException("Cannot create inverted index on dataset with composite primary key.");
@@ -65,7 +71,14 @@
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.
+ if (createIndexStmt.getIndexType() == IndexType.FUZZY_WORD_INVIX
+ || createIndexStmt.getIndexType() == IndexType.FUZZY_NGRAM_INVIX) {
+ isPartitioned = true;
+ } else {
+ isPartitioned = false;
+ }
+ // 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];
@@ -84,25 +97,30 @@
}
secondaryRecDesc = new RecordDescriptor(secondaryRecFields, secondaryTypeTraits);
// Comparators and type traits for tokens.
- tokenComparatorFactories = new IBinaryComparatorFactory[numSecondaryKeys];
- tokenTypeTraits = new ITypeTraits[numSecondaryKeys];
+ int numTokenFields = (!isPartitioned) ? numSecondaryKeys : numSecondaryKeys + 1;
+ tokenComparatorFactories = new IBinaryComparatorFactory[numTokenFields];
+ tokenTypeTraits = new ITypeTraits[numTokenFields];
tokenComparatorFactories[0] = InvertedIndexAccessMethod.getTokenBinaryComparatorFactory(secondaryKeyType);
tokenTypeTraits[0] = InvertedIndexAccessMethod.getTokenTypeTrait(secondaryKeyType);
+ if (isPartitioned) {
+ // The partitioning field is hardcoded to be a short *without* an Asterix type tag.
+ tokenComparatorFactories[1] = PointableBinaryComparatorFactory.of(ShortPointable.FACTORY);
+ tokenTypeTraits[1] = ShortPointable.TYPE_TRAITS;
+ }
// 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];
}
- // Get file split providers for the BTree and inverted-list files.
- fileSplitProviders = metadata.getInvertedIndexFileSplitProviders(secondaryFileSplitProvider);
// For tokenization, sorting and loading.
- // One token + primary keys.
- numTokenKeyPairFields = 1 + numPrimaryKeys;
+ // One token (+ optional partitioning field) + primary keys.
+ numTokenKeyPairFields = (!isPartitioned) ? 1 + numPrimaryKeys : 2 + numPrimaryKeys;
ISerializerDeserializer[] tokenKeyPairFields = new ISerializerDeserializer[numTokenKeyPairFields];
ITypeTraits[] tokenKeyPairTypeTraits = new ITypeTraits[numTokenKeyPairFields];
tokenKeyPairComparatorFactories = new IBinaryComparatorFactory[numTokenKeyPairFields];
@@ -110,10 +128,17 @@
tokenKeyPairTypeTraits[0] = tokenTypeTraits[0];
tokenKeyPairComparatorFactories[0] = InvertedIndexAccessMethod
.getTokenBinaryComparatorFactory(secondaryKeyType);
+ int pkOff = 1;
+ if (isPartitioned) {
+ tokenKeyPairFields[1] = ShortSerializerDeserializer.INSTANCE;
+ tokenKeyPairTypeTraits[1] = tokenTypeTraits[1];
+ tokenKeyPairComparatorFactories[1] = PointableBinaryComparatorFactory.of(ShortPointable.FACTORY);
+ pkOff = 2;
+ }
for (int i = 0; i < numPrimaryKeys; i++) {
- tokenKeyPairFields[i + 1] = primaryRecDesc.getFields()[i];
- tokenKeyPairTypeTraits[i + 1] = primaryRecDesc.getTypeTraits()[i];
- tokenKeyPairComparatorFactories[i + 1] = primaryComparatorFactories[i];
+ tokenKeyPairFields[i + pkOff] = primaryRecDesc.getFields()[i];
+ tokenKeyPairTypeTraits[i + pkOff] = primaryRecDesc.getTypeTraits()[i];
+ tokenKeyPairComparatorFactories[i + pkOff] = primaryComparatorFactories[i];
}
tokenKeyPairRecDesc = new RecordDescriptor(tokenKeyPairFields, tokenKeyPairTypeTraits);
}
@@ -121,11 +146,14 @@
@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();
+ IIndexDataflowHelperFactory dataflowHelperFactory = createDataflowHelperFactory();
+ LSMInvertedIndexCreateOperatorDescriptor invIndexCreateOp = new LSMInvertedIndexCreateOperatorDescriptor(spec,
+ AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, secondaryFileSplitProvider,
+ AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, tokenTypeTraits, tokenComparatorFactories,
+ invListsTypeTraits, primaryComparatorFactories, tokenizerFactory, dataflowHelperFactory,
+ localResourceFactoryProvider, NoOpOperationCallbackFactory.INSTANCE);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, invIndexCreateOp,
secondaryPartitionConstraint);
spec.addRoot(invIndexCreateOp);
@@ -137,7 +165,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.
@@ -146,7 +174,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);
@@ -159,7 +188,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);
@@ -178,16 +207,13 @@
}
private AbstractOperatorDescriptor createTokenizerOp(JobSpecification spec) throws AlgebricksException {
- int[] fieldsToTokenize = new int[numSecondaryKeys];
- for (int i = 0; i < numSecondaryKeys; i++) {
- fieldsToTokenize[i] = i;
- }
+ int docField = 0;
int[] primaryKeyFields = new int[numPrimaryKeys];
for (int i = 0; i < numPrimaryKeys; i++) {
primaryKeyFields[i] = numSecondaryKeys + i;
}
BinaryTokenizerOperatorDescriptor tokenizerOp = new BinaryTokenizerOperatorDescriptor(spec,
- tokenKeyPairRecDesc, tokenizerFactory, fieldsToTokenize, primaryKeyFields);
+ tokenKeyPairRecDesc, tokenizerFactory, docField, primaryKeyFields, isPartitioned);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, tokenizerOp,
primaryPartitionConstraint);
return tokenizerOp;
@@ -207,18 +233,33 @@
return sortOp;
}
- private InvertedIndexBulkLoadOperatorDescriptor createInvertedIndexBulkLoadOp(JobSpecification spec) {
- int[] fieldPermutation = new int[numSecondaryKeys + numPrimaryKeys];
+ private LSMInvertedIndexBulkLoadOperatorDescriptor createInvertedIndexBulkLoadOp(JobSpecification spec) {
+ int[] fieldPermutation = new int[numTokenKeyPairFields];
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,
+ IIndexDataflowHelperFactory dataflowHelperFactory = createDataflowHelperFactory();
+ LSMInvertedIndexBulkLoadOperatorDescriptor invIndexBulkLoadOp = new LSMInvertedIndexBulkLoadOperatorDescriptor(
+ spec, fieldPermutation, false, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+ secondaryFileSplitProvider, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, tokenTypeTraits,
tokenComparatorFactories, invListsTypeTraits, primaryComparatorFactories, tokenizerFactory,
- new BTreeDataflowHelperFactory(), NoOpOperationCallbackProvider.INSTANCE);
+ dataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, invIndexBulkLoadOp,
secondaryPartitionConstraint);
return invIndexBulkLoadOp;
}
+
+ private IIndexDataflowHelperFactory createDataflowHelperFactory() {
+ if (!isPartitioned) {
+ return new LSMInvertedIndexDataflowHelperFactory(
+ AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER);
+ } else {
+ return new PartitionedLSMInvertedIndexDataflowHelperFactory(
+ AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER);
+ }
+ }
}
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 ba8f6cd..c1d6bd6 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,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.dataflow.data.nontagged.valueproviders.AqlPrimitiveValueProviderFactory;
import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
@@ -11,6 +10,7 @@
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.CompiledStatements.CompiledCreateIndexStatement;
@@ -32,14 +32,17 @@
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.common.impls.NoOpOperationCallbackFactory;
+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);
@@ -48,10 +51,17 @@
@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.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
- new RTreeDataflowHelperFactory(valueProviderFactories), NoOpOperationCallbackProvider.INSTANCE);
+ new LSMRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
+ primaryComparatorFactories, AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, AqlMetadataProvider.proposeLinearizer(
+ keyType, secondaryComparatorFactories.length)), localResourceFactoryProvider,
+ NoOpOperationCallbackFactory.INSTANCE);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,
secondaryPartitionConstraint);
spec.addRoot(secondaryIndexCreateOp);
@@ -60,8 +70,8 @@
}
@Override
- protected void setSecondaryRecDescAndComparators(CompiledCreateIndexStatement createIndexStmt, AqlMetadataProvider metadata)
- throws AlgebricksException, AsterixException {
+ protected void setSecondaryRecDescAndComparators(CompiledCreateIndexStatement createIndexStmt,
+ AqlMetadataProvider metadata) throws AlgebricksException, AsterixException {
List<String> secondaryKeyFields = createIndexStmt.getKeyFields();
int numSecondaryKeys = secondaryKeyFields.size();
if (numSecondaryKeys != 1) {
@@ -86,6 +96,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);
@@ -124,9 +135,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.LSMRTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, 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
deleted file mode 100644
index c295f55..0000000
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/TestKeywordIndexJob.java
+++ /dev/null
@@ -1,228 +0,0 @@
-package edu.uci.ics.asterix.file;
-
-import java.io.DataOutput;
-import java.io.File;
-import java.util.ArrayList;
-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.exceptions.AsterixException;
-import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AObjectSerializerDeserializer;
-import edu.uci.ics.asterix.om.base.AString;
-import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import edu.uci.ics.hyracks.api.client.HyracksConnection;
-import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.DoubleParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.FloatParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.LongParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
-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.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;
-
-public class TestKeywordIndexJob {
-
- private static final HashMap<ATypeTag, IValueParserFactory> typeToValueParserFactMap = new HashMap<ATypeTag, IValueParserFactory>();
- static {
- typeToValueParserFactMap.put(ATypeTag.INT32, IntegerParserFactory.INSTANCE);
- typeToValueParserFactMap.put(ATypeTag.FLOAT, FloatParserFactory.INSTANCE);
- typeToValueParserFactMap.put(ATypeTag.DOUBLE, DoubleParserFactory.INSTANCE);
- typeToValueParserFactMap.put(ATypeTag.INT64, LongParserFactory.INSTANCE);
- typeToValueParserFactMap.put(ATypeTag.STRING, UTF8StringParserFactory.INSTANCE);
- }
-
- public static int DEFAULT_INPUT_DATA_COLUMN = 0;
- public static LogicalVariable METADATA_DUMMY_VAR = new LogicalVariable(-1);
-
- @SuppressWarnings("unchecked")
- public JobSpecification createJobSpec() throws AsterixException, HyracksDataException {
-
- JobSpecification spec = new JobSpecification();
-
- // ---------- START GENERAL BTREE STUFF
-
- IIndexRegistryProvider<IIndex> indexRegistryProvider = AsterixIndexRegistryProvider.INSTANCE;
- IStorageManagerInterface storageManager = AsterixStorageManagerInterface.INSTANCE;
-
- // ---------- END GENERAL BTREE STUFF
-
- List<String> nodeGroup = new ArrayList<String>();
- nodeGroup.add("nc1");
- nodeGroup.add("nc2");
-
- // ---------- START KEY PROVIDER OP
-
- // TODO: should actually be empty tuple source
- // build tuple containing low and high search keys
- ArrayTupleBuilder tb = new ArrayTupleBuilder(1); // just one dummy field
- DataOutput dos = tb.getDataOutput();
-
- tb.reset();
- AObjectSerializerDeserializer.INSTANCE.serialize(new AString("Jodi Rotruck"), dos); // dummy
- // field
- tb.addFieldEndOffset();
-
- ISerializerDeserializer[] keyRecDescSers = { AObjectSerializerDeserializer.INSTANCE };
- RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
-
- ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
- keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
- String[] keyProviderOpLocationConstraint = new String[nodeGroup.size()];
- for (int p = 0; p < nodeGroup.size(); p++) {
- keyProviderOpLocationConstraint[p] = nodeGroup.get(p);
- }
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, keyProviderOpLocationConstraint);
-
- // ---------- END KEY PROVIDER OP
-
- // ---------- START SECONRARY INDEX SCAN
-
- ITypeTraits[] secondaryTypeTraits = new ITypeTraits[2];
- secondaryTypeTraits[0] = new ITypeTraits() {
-
- @Override
- public boolean isFixedLength() {
- return false;
- }
-
- @Override
- public int getFixedLength() {
- return -1;
- }
- };
-
- secondaryTypeTraits[1] = new ITypeTraits() {
-
- @Override
- public boolean isFixedLength() {
- return true;
- }
-
- @Override
- public int getFixedLength() {
- return 5;
- }
- };
-
- ISerializerDeserializer[] secondaryRecFields = new ISerializerDeserializer[2];
- secondaryRecFields[0] = AObjectSerializerDeserializer.INSTANCE;
- secondaryRecFields[1] = AObjectSerializerDeserializer.INSTANCE;
- IBinaryComparatorFactory[] secondaryComparatorFactories = new IBinaryComparatorFactory[2];
- secondaryComparatorFactories[0] = AObjectAscBinaryComparatorFactory.INSTANCE;
- secondaryComparatorFactories[1] = AObjectAscBinaryComparatorFactory.INSTANCE;
-
- int[] lowKeyFields = null;
- int[] highKeyFields = null;
- RecordDescriptor secondaryRecDesc = new RecordDescriptor(secondaryRecFields);
- // TODO: change file splits according to mount points in cluster config
- IFileSplitProvider secondarySplitProvider = new ConstantFileSplitProvider(new FileSplit[] {
- 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);
- String[] secondarySearchOpLocationConstraint = new String[nodeGroup.size()];
- for (int p = 0; p < nodeGroup.size(); p++) {
- secondarySearchOpLocationConstraint[p] = nodeGroup.get(p);
- }
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondarySearchOp,
- secondarySearchOpLocationConstraint);
-
- // ---------- END SECONDARY INDEX SCAN
-
- PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
- String[] printerLocationConstraint = new String[nodeGroup.size()];
- for (int p = 0; p < nodeGroup.size(); p++) {
- printerLocationConstraint[p] = nodeGroup.get(p);
- }
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, printerLocationConstraint);
-
- // ---------- START CONNECT THE OPERATORS
-
- spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, secondarySearchOp, 0);
-
- spec.connect(new OneToOneConnectorDescriptor(spec), secondarySearchOp, 0, printer, 0);
-
- // ---------- END CONNECT THE OPERATORS
-
- spec.addRoot(printer);
-
- return spec;
- }
-
- public static void main(String[] args) throws Exception {
- String host;
- String appName;
- String ddlFile;
-
- switch (args.length) {
- case 0: {
- host = "127.0.0.1";
- appName = "asterix";
- ddlFile = "/home/abehm/workspace/asterix/src/test/resources/demo0927/local/create-index.aql";
- System.out.println("No arguments specified, using defauls:");
- System.out.println("HYRACKS HOST: " + host);
- System.out.println("APPNAME: " + appName);
- System.out.println("DDLFILE: " + ddlFile);
- }
- break;
-
- case 3: {
- host = args[0];
- appName = args[1];
- ddlFile = args[2];
- }
- break;
-
- default: {
- System.out.println("USAGE:");
- System.out.println("ARG 1: Hyracks Host (IP or Hostname)");
- System.out.println("ARG 2: Application Name (e.g., asterix)");
- System.out.println("ARG 3: DDL File");
- host = null;
- appName = null;
- ddlFile = null;
- System.exit(0);
- }
- break;
- }
-
- int port = 1098;
- IHyracksClientConnection hcc = new HyracksConnection(host, port);
-
- TestKeywordIndexJob tij = new TestKeywordIndexJob();
- JobSpecification jobSpec = tij.createJobSpec();
-
- long start = System.currentTimeMillis();
- JobId jobId = hcc.startJob("asterix", jobSpec);
- hcc.waitForCompletion(jobId);
- long end = System.currentTimeMillis();
- System.err.println(start + " " + end + " " + (end - start));
- }
-}
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
deleted file mode 100644
index 5d2cdc5..0000000
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/TestSecondaryIndexJob.java
+++ /dev/null
@@ -1,228 +0,0 @@
-package edu.uci.ics.asterix.file;
-
-import java.io.DataOutput;
-import java.io.File;
-import java.util.ArrayList;
-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.exceptions.AsterixException;
-import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AObjectSerializerDeserializer;
-import edu.uci.ics.asterix.om.base.AString;
-import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import edu.uci.ics.hyracks.api.client.HyracksConnection;
-import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.DoubleParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.FloatParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.LongParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
-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.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;
-
-public class TestSecondaryIndexJob {
-
- private static final HashMap<ATypeTag, IValueParserFactory> typeToValueParserFactMap = new HashMap<ATypeTag, IValueParserFactory>();
- static {
- typeToValueParserFactMap.put(ATypeTag.INT32, IntegerParserFactory.INSTANCE);
- typeToValueParserFactMap.put(ATypeTag.FLOAT, FloatParserFactory.INSTANCE);
- typeToValueParserFactMap.put(ATypeTag.DOUBLE, DoubleParserFactory.INSTANCE);
- typeToValueParserFactMap.put(ATypeTag.INT64, LongParserFactory.INSTANCE);
- typeToValueParserFactMap.put(ATypeTag.STRING, UTF8StringParserFactory.INSTANCE);
- }
-
- public static int DEFAULT_INPUT_DATA_COLUMN = 0;
- public static LogicalVariable METADATA_DUMMY_VAR = new LogicalVariable(-1);
-
- @SuppressWarnings("unchecked")
- public JobSpecification createJobSpec() throws AsterixException, HyracksDataException {
-
- JobSpecification spec = new JobSpecification();
-
- // ---------- START GENERAL BTREE STUFF
-
- IIndexRegistryProvider<IIndex> indexRegistryProvider = AsterixIndexRegistryProvider.INSTANCE;
- IStorageManagerInterface storageManager = AsterixStorageManagerInterface.INSTANCE;
-
- // ---------- END GENERAL BTREE STUFF
-
- List<String> nodeGroup = new ArrayList<String>();
- nodeGroup.add("nc1");
- nodeGroup.add("nc2");
-
- // ---------- START KEY PROVIDER OP
-
- // TODO: should actually be empty tuple source
- // build tuple containing low and high search keys
- ArrayTupleBuilder tb = new ArrayTupleBuilder(1); // just one dummy field
- DataOutput dos = tb.getDataOutput();
-
- tb.reset();
- AObjectSerializerDeserializer.INSTANCE.serialize(new AString("Jodi Rotruck"), dos); // dummy
- // field
- tb.addFieldEndOffset();
-
- ISerializerDeserializer[] keyRecDescSers = { AObjectSerializerDeserializer.INSTANCE };
- RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
-
- ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
- keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
- String[] keyProviderOpLocationConstraint = new String[nodeGroup.size()];
- for (int p = 0; p < nodeGroup.size(); p++) {
- keyProviderOpLocationConstraint[p] = nodeGroup.get(p);
- }
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, keyProviderOpLocationConstraint);
-
- // ---------- END KEY PROVIDER OP
-
- // ---------- START SECONRARY INDEX SCAN
-
- ITypeTraits[] secondaryTypeTraits = new ITypeTraits[2];
- secondaryTypeTraits[0] = new ITypeTraits() {
-
- @Override
- public boolean isFixedLength() {
- return false;
- }
-
- @Override
- public int getFixedLength() {
- return -1;
- }
- };
-
- secondaryTypeTraits[1] = new ITypeTraits() {
-
- @Override
- public boolean isFixedLength() {
- return true;
- }
-
- @Override
- public int getFixedLength() {
- return 5;
- }
- };
-
- ISerializerDeserializer[] secondaryRecFields = new ISerializerDeserializer[2];
- secondaryRecFields[0] = AObjectSerializerDeserializer.INSTANCE;
- secondaryRecFields[1] = AObjectSerializerDeserializer.INSTANCE;
- IBinaryComparatorFactory[] secondaryComparatorFactories = new IBinaryComparatorFactory[2];
- secondaryComparatorFactories[0] = AObjectAscBinaryComparatorFactory.INSTANCE;
- secondaryComparatorFactories[1] = AObjectAscBinaryComparatorFactory.INSTANCE;
-
- int[] lowKeyFields = null; // -infinity
- int[] highKeyFields = null; // +infinity
- RecordDescriptor secondaryRecDesc = new RecordDescriptor(secondaryRecFields);
- // TODO: change file splits according to mount points in cluster config
- IFileSplitProvider secondarySplitProvider = new ConstantFileSplitProvider(new FileSplit[] {
- 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);
- String[] secondarySearchOpLocationConstraint = new String[nodeGroup.size()];
- for (int p = 0; p < nodeGroup.size(); p++) {
- secondarySearchOpLocationConstraint[p] = nodeGroup.get(p);
- }
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondarySearchOp,
- secondarySearchOpLocationConstraint);
-
- // ---------- END SECONDARY INDEX SCAN
-
- PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
- String[] printerLocationConstraint = new String[nodeGroup.size()];
- for (int p = 0; p < nodeGroup.size(); p++) {
- printerLocationConstraint[p] = nodeGroup.get(p);
- }
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, printerLocationConstraint);
-
- // ---------- START CONNECT THE OPERATORS
-
- spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, secondarySearchOp, 0);
-
- spec.connect(new OneToOneConnectorDescriptor(spec), secondarySearchOp, 0, printer, 0);
-
- // ---------- END CONNECT THE OPERATORS
-
- spec.addRoot(printer);
-
- return spec;
- }
-
- public static void main(String[] args) throws Exception {
- String host;
- String appName;
- String ddlFile;
-
- switch (args.length) {
- case 0: {
- host = "127.0.0.1";
- appName = "asterix";
- ddlFile = "/home/nicnic/workspace/asterix/trunk/asterix/asterix-app/src/test/resources/demo0927/local/create-index.aql";
- System.out.println("No arguments specified, using defauls:");
- System.out.println("HYRACKS HOST: " + host);
- System.out.println("APPNAME: " + appName);
- System.out.println("DDLFILE: " + ddlFile);
- }
- break;
-
- case 3: {
- host = args[0];
- appName = args[1];
- ddlFile = args[2];
- }
- break;
-
- default: {
- System.out.println("USAGE:");
- System.out.println("ARG 1: Hyracks Host (IP or Hostname)");
- System.out.println("ARG 2: Application Name (e.g., asterix)");
- System.out.println("ARG 3: DDL File");
- host = null;
- appName = null;
- ddlFile = null;
- System.exit(0);
- }
- break;
- }
-
- int port = 1098;
- IHyracksClientConnection hcc = new HyracksConnection(host, port);
-
- TestSecondaryIndexJob tij = new TestSecondaryIndexJob();
- JobSpecification jobSpec = tij.createJobSpec();
-
- long start = System.currentTimeMillis();
- JobId jobId = hcc.startJob("asterix", jobSpec);
- hcc.waitForCompletion(jobId);
- long end = System.currentTimeMillis();
- System.err.println(start + " " + end + " " + (end - start));
- }
-}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/AsterixNodeState.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/AsterixNodeState.java
deleted file mode 100644
index dbf1625..0000000
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/AsterixNodeState.java
+++ /dev/null
@@ -1,33 +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.hyracks.bootstrap;
-
-import java.io.Serializable;
-
-public class AsterixNodeState implements Serializable {
-
- private static final long serialVersionUID = 1L;
-
- private int apiNodeDataServerPort;
-
- public int getAPINodeDataServerPort() {
- return apiNodeDataServerPort;
- }
-
- public void setAPINodeDataServerPort(int port) {
- this.apiNodeDataServerPort = port;
- }
-
-}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCBootstrapImpl.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCBootstrapImpl.java
index ccba498..fcd50b5 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCBootstrapImpl.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCBootstrapImpl.java
@@ -14,11 +14,6 @@
*/
package edu.uci.ics.asterix.hyracks.bootstrap;
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
import java.util.logging.Level;
import java.util.logging.Logger;
@@ -43,7 +38,7 @@
private static final int DEFAULT_WEB_SERVER_PORT = 19001;
public static final int DEFAULT_API_SERVER_PORT = 14600;
- private static final int DEFAULT_API_NODEDATA_SERVER_PORT = 14601;
+ public static final int DEFAULT_API_NODEDATA_SERVER_PORT = 14601;
private Server webServer;
private static IAsterixStateProxy proxy;
@@ -79,7 +74,7 @@
LOGGER.info("Stopping Asterix cluster controller");
}
AsterixStateProxy.unregisterRemoteObject();
-
+
webServer.stop();
apiServer.shutdown();
}
@@ -104,25 +99,6 @@
}
private void setupAPIServer() throws Exception {
- // set the APINodeDataServer ports
- int startPort = DEFAULT_API_NODEDATA_SERVER_PORT;
- Map<String, Set<String>> nodeNameMap = new HashMap<String, Set<String>>();
- try {
- appCtx.getCCContext().getIPAddressNodeMap(nodeNameMap);
- } catch (Exception e) {
- throw new IOException("Unable to obtain IP address node map", e);
- }
-
- for (Map.Entry<String, Set<String>> entry : nodeNameMap.entrySet()) {
- Set<String> nodeNames = entry.getValue();
- Iterator<String> it = nodeNames.iterator();
- while (it.hasNext()) {
- AsterixNodeState ns = new AsterixNodeState();
- ns.setAPINodeDataServerPort(startPort++);
- proxy.setAsterixNodeState(it.next(), ns);
- }
- }
-
apiServer = new ThreadedServer(DEFAULT_API_SERVER_PORT, new APIClientThreadFactory(appCtx));
}
}
\ No newline at end of file
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/NCBootstrapImpl.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/NCBootstrapImpl.java
index b1e7481..24cf3a8 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/NCBootstrapImpl.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/NCBootstrapImpl.java
@@ -65,15 +65,14 @@
MetadataManager.INSTANCE.init();
MetadataBootstrap.startUniverse(proxy.getAsterixProperties(), ncApplicationContext);
+ // Start a sub-component for the API server. This server is only connected to by the
+ // API server that lives on the CC and never by a client wishing to execute AQL.
+ // TODO: The API sub-system will change dramatically in the future and this code will go away,
+ // but leave it for now.
+ apiNodeDataServer = new ThreadedServer(CCBootstrapImpl.DEFAULT_API_NODEDATA_SERVER_PORT,
+ new NodeDataClientThreadFactory());
+ apiNodeDataServer.start();
}
-
- // Start a sub-component for the API server. This server is only connected to by the
- // API server that lives on the CC and never by a client wishing to execute AQL.
- // TODO: The API sub-system will change dramatically in the future and this code will go away,
- // but leave it for now.
- AsterixNodeState ns = (AsterixNodeState) proxy.getAsterixNodeState(nodeId);
- apiNodeDataServer = new ThreadedServer(ns.getAPINodeDataServerPort(), new NodeDataClientThreadFactory());
- apiNodeDataServer.start();
}
public void registerRemoteMetadataNode(IAsterixStateProxy proxy) throws RemoteException {
diff --git a/asterix-app/src/main/resources/asterix-idefix.properties b/asterix-app/src/main/resources/asterix-idefix.properties
deleted file mode 100755
index 278bcb4..0000000
--- a/asterix-app/src/main/resources/asterix-idefix.properties
+++ /dev/null
@@ -1,2 +0,0 @@
-nc1.stores=/home/nicnic/Work/Asterix/tests/tpch/nc1data
-nc2.stores=/home/nicnic/Work/Asterix/tests/tpch/nc2data
diff --git a/asterix-app/src/main/resources/asterix-peach.properties b/asterix-app/src/main/resources/asterix-peach.properties
deleted file mode 100644
index 20a6eeb..0000000
--- a/asterix-app/src/main/resources/asterix-peach.properties
+++ /dev/null
@@ -1,2 +0,0 @@
-nc1.stores=/tmp/nc1/
-nc2.stores=/tmp/nc2/
diff --git a/asterix-app/src/main/resources/asterix-rainbow.properties b/asterix-app/src/main/resources/asterix-rainbow.properties
deleted file mode 100644
index d5febe4..0000000
--- a/asterix-app/src/main/resources/asterix-rainbow.properties
+++ /dev/null
@@ -1,5 +0,0 @@
-rainbow-01.stores=/data/onose/rainbow-01/
-rainbow-02.stores=/data/onose/rainbow-02/
-rainbow-03.stores=/data/onose/rainbow-03/
-rainbow-04.stores=/data/onose/rainbow-04/
-rainbow-05.stores=/data/onose/rainbow-05/
\ No newline at end of file
diff --git a/asterix-app/src/main/resources/asterix.properties b/asterix-app/src/main/resources/asterix.properties
deleted file mode 100755
index 78cd2b9..0000000
--- a/asterix-app/src/main/resources/asterix.properties
+++ /dev/null
@@ -1,10 +0,0 @@
-asterix-001.stores=/mnt/data/sda/space/onose,/mnt/data/sdb/space/onose,/mnt/data/sdc/space/onose,/mnt/data/sdd/space/onose
-asterix-002.stores=/mnt/data/sda/space/onose,/mnt/data/sdb/space/onose,/mnt/data/sdc/space/onose,/mnt/data/sdd/space/onose
-asterix-003.stores=/mnt/data/sda/space/onose,/mnt/data/sdb/space/onose,/mnt/data/sdc/space/onose,/mnt/data/sdd/space/onose
-asterix-004.stores=/mnt/data/sda/space/onose,/mnt/data/sdb/space/onose,/mnt/data/sdc/space/onose,/mnt/data/sdd/space/onose
-asterix-005.stores=/mnt/data/sda/space/onose,/mnt/data/sdb/space/onose,/mnt/data/sdc/space/onose,/mnt/data/sdd/space/onose
-asterix-006.stores=/mnt/data/sda/space/onose,/mnt/data/sdb/space/onose,/mnt/data/sdc/space/onose,/mnt/data/sdd/space/onose
-asterix-007.stores=/mnt/data/sda/space/onose,/mnt/data/sdb/space/onose,/mnt/data/sdc/space/onose,/mnt/data/sdd/space/onose
-asterix-008.stores=/mnt/data/sda/space/onose,/mnt/data/sdb/space/onose,/mnt/data/sdc/space/onose,/mnt/data/sdd/space/onose
-asterix-009.stores=/mnt/data/sda/space/onose,/mnt/data/sdb/space/onose,/mnt/data/sdc/space/onose,/mnt/data/sdd/space/onose
-asterix-010.stores=/mnt/data/sda/space/onose,/mnt/data/sdb/space/onose,/mnt/data/sdc/space/onose,/mnt/data/sdd/space/onose
diff --git a/asterix-app/src/main/resources/hyracks-initdb-deployment.properties b/asterix-app/src/main/resources/hyracks-initdb-deployment.properties
deleted file mode 100644
index e40db59..0000000
--- a/asterix-app/src/main/resources/hyracks-initdb-deployment.properties
+++ /dev/null
@@ -1,2 +0,0 @@
-cc.bootstrap.class=edu.uci.ics.initdb.hyracks.bootstrap.CCBootstrapImpl
-nc.bootstrap.class=edu.uci.ics.initdb.hyracks.bootstrap.NCBootstrapImpl
\ No newline at end of file
diff --git a/asterix-app/src/main/resources/idefix-4nc.properties b/asterix-app/src/main/resources/idefix-4nc.properties
deleted file mode 100755
index 747eb41..0000000
--- a/asterix-app/src/main/resources/idefix-4nc.properties
+++ /dev/null
@@ -1,4 +0,0 @@
-nc1.stores=/home/nicnic/Work/Asterix/tests/tpch/nc1data
-nc2.stores=/home/nicnic/Work/Asterix/tests/tpch/nc2data
-nc3.stores=/home/nicnic/Work/Asterix/tests/tpch/nc3data
-nc4.stores=/home/nicnic/Work/Asterix/tests/tpch/nc4data
diff --git a/asterix-app/src/main/resources/test.properties b/asterix-app/src/main/resources/test.properties
index 01a593b..4947dbf 100755
--- a/asterix-app/src/main/resources/test.properties
+++ b/asterix-app/src/main/resources/test.properties
@@ -1,5 +1,5 @@
MetadataNode=nc1
NewUniverse=true
-nc1.stores=/tmp/nc1data/
-nc2.stores=/tmp/nc2data/
+nc1.stores=nc1data
+nc2.stores=nc2data
OutputDir=/tmp/asterix_output/
diff --git a/asterix-app/src/main/resources/testnc1.properties b/asterix-app/src/main/resources/testnc1.properties
deleted file mode 100755
index c0ad3de..0000000
--- a/asterix-app/src/main/resources/testnc1.properties
+++ /dev/null
@@ -1 +0,0 @@
-nc1.stores=nc1data
diff --git a/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/ExecutionTest.java b/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/ExecutionTest.java
index d35001e..111f76b 100644
--- a/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/ExecutionTest.java
+++ b/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/ExecutionTest.java
@@ -93,6 +93,13 @@
List<CompilationUnit> cUnits = tcCtx.getTestCase().getCompilationUnit();
for (CompilationUnit cUnit : cUnits) {
File testFile = tcCtx.getTestFile(cUnit);
+
+ /*************** to avoid run failure cases ****************
+ if (!testFile.getAbsolutePath().contains("runtimets/queries/failure/")) {
+ continue;
+ }
+ ************************************************************/
+
File expectedResultFile = tcCtx.getExpectedResultFile(cUnit);
File actualFile = new File(PATH_ACTUAL + File.separator
+ tcCtx.getTestCase().getFilePath().replace(File.separator, "_") + "_" + cUnit.getName() + ".adm");
diff --git a/asterix-app/src/test/resources/metadata-transactions/init-state-queries.txt b/asterix-app/src/test/resources/metadata-transactions/init-state-queries.txt
index 5359017..c9ef6ee 100644
--- a/asterix-app/src/test/resources/metadata-transactions/init-state-queries.txt
+++ b/asterix-app/src/test/resources/metadata-transactions/init-state-queries.txt
@@ -1 +1 @@
-customers_orders.aql
\ No newline at end of file
+customers_orders.aql
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join-multiindex.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join-multiindex.plan
index 947404f..7411774 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join-multiindex.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join-multiindex.plan
@@ -2,21 +2,22 @@
-- RANDOM_MERGE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$29(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- BROADCAST_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$29(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_01.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_01.plan
index 7e0c315..fb53c5b 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_01.plan
@@ -3,18 +3,20 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$13(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$13(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- BROADCAST_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_02.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_02.plan
index 7e0c315..fb53c5b 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_02.plan
@@ -3,18 +3,20 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$13(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$13(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- BROADCAST_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_03.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_03.plan
index 7e0c315..fb53c5b 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_03.plan
@@ -3,18 +3,20 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$13(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$13(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- BROADCAST_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-38.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-38.plan
index 7c1c1a3..4f254dd 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-38.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-38.plan
@@ -1,14 +1,15 @@
-- SINK_WRITE |PARTITIONED|
-- RANDOM_MERGE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$16(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$16(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-45.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-45.plan
index 35c7ebb..616b2a2 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-45.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-45.plan
@@ -1,14 +1,17 @@
-- SINK_WRITE |PARTITIONED|
-- RANDOM_MERGE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-46.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-46.plan
index 35c7ebb..616b2a2 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-46.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-46.plan
@@ -1,14 +1,17 @@
-- SINK_WRITE |PARTITIONED|
-- RANDOM_MERGE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-49.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-49.plan
index 35c7ebb..616b2a2 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-49.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-49.plan
@@ -1,14 +1,17 @@
-- SINK_WRITE |PARTITIONED|
-- RANDOM_MERGE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-50.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-50.plan
index 35c7ebb..616b2a2 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-50.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-50.plan
@@ -1,14 +1,17 @@
-- SINK_WRITE |PARTITIONED|
-- RANDOM_MERGE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-51.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-51.plan
index 35c7ebb..616b2a2 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-51.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-51.plan
@@ -1,14 +1,17 @@
-- SINK_WRITE |PARTITIONED|
-- RANDOM_MERGE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-52.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-52.plan
index e6e6ddf..1445f65 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-52.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-52.plan
@@ -1,14 +1,15 @@
-- SINK_WRITE |PARTITIONED|
-- RANDOM_MERGE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$8(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$8(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-53.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-53.plan
index e6e6ddf..1445f65 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-53.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-53.plan
@@ -1,14 +1,15 @@
-- SINK_WRITE |PARTITIONED|
-- RANDOM_MERGE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$8(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$8(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-54.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-54.plan
index e6e6ddf..1445f65 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-54.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-54.plan
@@ -1,14 +1,15 @@
-- SINK_WRITE |PARTITIONED|
-- RANDOM_MERGE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$8(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$8(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-55.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-55.plan
index e6e6ddf..1445f65 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-55.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-55.plan
@@ -1,14 +1,15 @@
-- SINK_WRITE |PARTITIONED|
-- RANDOM_MERGE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$8(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$8(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-56.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-56.plan
index 16e6d6a..d57f0c4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-56.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-56.plan
@@ -1,14 +1,17 @@
-- SINK_WRITE |PARTITIONED|
-- RANDOM_MERGE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$12(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$12(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-57.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-57.plan
index 16e6d6a..d57f0c4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-57.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-57.plan
@@ -1,14 +1,17 @@
-- SINK_WRITE |PARTITIONED|
-- RANDOM_MERGE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$12(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$12(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-58.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-58.plan
index edeed53..761c8a9 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-58.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-58.plan
@@ -1,14 +1,15 @@
-- SINK_WRITE |PARTITIONED|
-- RANDOM_MERGE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$9(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$9(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/orders-composite-index-search.plan b/asterix-app/src/test/resources/optimizerts/results/orders-composite-index-search.plan
index 327253a..8340c89 100644
--- a/asterix-app/src/test/resources/optimizerts/results/orders-composite-index-search.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/orders-composite-index-search.plan
@@ -2,15 +2,18 @@
-- RANDOM_MERGE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$21(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$21(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/orders-index-search-open.plan b/asterix-app/src/test/resources/optimizerts/results/orders-index-search-open.plan
index 3a6ffad..bba6dbc 100644
--- a/asterix-app/src/test/resources/optimizerts/results/orders-index-search-open.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/orders-index-search-open.plan
@@ -2,15 +2,18 @@
-- RANDOM_MERGE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$13(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$13(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/orders-index-search.plan b/asterix-app/src/test/resources/optimizerts/results/orders-index-search.plan
index 3a6ffad..bba6dbc 100644
--- a/asterix-app/src/test/resources/optimizerts/results/orders-index-search.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/orders-index-search.plan
@@ -2,15 +2,18 @@
-- RANDOM_MERGE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$13(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$13(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/scan-delete-all.plan b/asterix-app/src/test/resources/optimizerts/results/scan-delete-all.plan
index 395fec7..e969e5d 100644
--- a/asterix-app/src/test/resources/optimizerts/results/scan-delete-all.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/scan-delete-all.plan
@@ -1,4 +1,4 @@
--- SINK |PARTITIONED|
+-- COMMIT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- INSERT_DELETE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan b/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan
index 79daefd..bf8978b 100644
--- a/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan
@@ -1,4 +1,4 @@
--- SINK |PARTITIONED|
+-- COMMIT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- INDEX_INSERT_DELETE |PARTITIONED|
@@ -22,4 +22,4 @@
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/results/scan-delete.plan b/asterix-app/src/test/resources/optimizerts/results/scan-delete.plan
index 0e7cf78..0608f69 100644
--- a/asterix-app/src/test/resources/optimizerts/results/scan-delete.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/scan-delete.plan
@@ -1,4 +1,4 @@
--- SINK |PARTITIONED|
+-- COMMIT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- INSERT_DELETE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/scan-insert-secondary-index.plan b/asterix-app/src/test/resources/optimizerts/results/scan-insert-secondary-index.plan
index 650f512..e6cf237 100644
--- a/asterix-app/src/test/resources/optimizerts/results/scan-insert-secondary-index.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/scan-insert-secondary-index.plan
@@ -1,4 +1,4 @@
--- SINK |PARTITIONED|
+-- COMMIT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- INDEX_INSERT_DELETE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/scan-insert.plan b/asterix-app/src/test/resources/optimizerts/results/scan-insert.plan
index 5cfcdf3..0b756d9 100644
--- a/asterix-app/src/test/resources/optimizerts/results/scan-insert.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/scan-insert.plan
@@ -1,4 +1,4 @@
--- SINK |PARTITIONED|
+-- COMMIT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- INSERT_DELETE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/runtimets/ignore.txt b/asterix-app/src/test/resources/runtimets/ignore.txt
new file mode 100644
index 0000000..6722cdb
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/ignore.txt
@@ -0,0 +1,44 @@
+custord/join_q_04.aql
+scan/spatial_types_02.aql
+scan/temp_types_02.aql
+fuzzyjoin/dblp-splits-3_1.aql
+fuzzyjoin/events-users-aqlplus_1.aql
+subset-collection/04.aql
+custord/freq-clerk.aql
+custord/denorm-cust-order_01.aql
+custord/denorm-cust-order_03.aql
+custord/co.aql
+comparison/numeric-comparison_01.aql
+dapd/q3.aql
+failure/q1_pricing_summary_report_failure.aql
+dml/load-from-hdfs.aql
+open-closed/open-closed-15
+open-closed/open-closed-16
+open-closed/open-closed-17
+open-closed/open-closed-18
+open-closed/open-closed-19
+open-closed/open-closed-20
+open-closed/open-closed-21
+open-closed/open-closed-22
+open-closed/open-closed-28
+open-closed/open-closed-30
+open-closed/heterog-list02
+open-closed/heterog-list03
+open-closed/c2c
+quantifiers/somesat_03.aql
+quantifiers/somesat_04.aql
+quantifiers/somesat_05.aql
+quantifiers/everysat_02.aql
+quantifiers/everysat_03.aql
+flwor
+string/startwith03.aql
+aggregate/droptype.aql
+failure/delete-rtree.aql
+failure/delete.aql
+failure/insert-rtree.aql
+failure/insert.aql
+failure/q1_pricing_summary_report_failure.aql
+failure/verify_delete-rtree.aql
+failure/verify_delete.aql
+failure/verify_insert-rtree.aql
+failure/verify_insert.aql
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/failure/delete-rtree.aql b/asterix-app/src/test/resources/runtimets/queries/failure/delete-rtree.aql
index 465d94f..6fc11e5 100644
--- a/asterix-app/src/test/resources/runtimets/queries/failure/delete-rtree.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/failure/delete-rtree.aql
@@ -18,7 +18,7 @@
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
(("path"="nc1://data/twitter/smalltweets.txt"),("format"="adm")) pre-sorted;
-delete $l from dataset MyData where $l.id>=50 die after 1500;
+delete $l from dataset MyData where $l.id>=50 die after 1000;
write output to nc1:"rttest/failure_delete-rtree.adm";
diff --git a/asterix-app/src/test/resources/runtimets/queries/failure/delete.aql b/asterix-app/src/test/resources/runtimets/queries/failure/delete.aql
index cd4a922..3da88f8 100644
--- a/asterix-app/src/test/resources/runtimets/queries/failure/delete.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/failure/delete.aql
@@ -29,7 +29,7 @@
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
-delete $l from dataset LineItem where $l.l_orderkey>=10 die after 1500;
+delete $l from dataset LineItem where $l.l_orderkey>=10 die after 1000;
write output to nc1:"rttest/failure_delete.adm";
for $c in dataset('LineItem')
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-contains.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-contains.aql
new file mode 100644
index 0000000..54ec794
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-contains.aql
@@ -0,0 +1,29 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+ id: int32,
+ dblpid: string,
+ title: string,
+ authors: string,
+ misc: string
+}
+
+create nodegroup group1 if not exists on nc1, nc2;
+
+create dataset DBLP(DBLPType)
+ partitioned by key id on group1;
+
+load dataset DBLP
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index ngram_index on DBLP(title) type fuzzy ngram(3);
+
+write output to nc1:"rttest/index-selection_fuzzy-inverted-index-ngram-contains.adm";
+
+for $o in dataset('DBLP')
+where contains($o.title, "Multimedia")
+order by $o.id
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance-panic.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance-panic.aql
new file mode 100644
index 0000000..8bfc878
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance-panic.aql
@@ -0,0 +1,29 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+ id: int32,
+ dblpid: string,
+ title: string,
+ authors: string,
+ misc: string
+}
+
+create nodegroup group1 if not exists on nc1, nc2;
+
+create dataset DBLP(DBLPType)
+ partitioned by key id on group1;
+
+load dataset DBLP
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index ngram_index on DBLP(authors) type fuzzy ngram(3);
+
+write output to nc1:"rttest/index-selection_fuzzy-inverted-index-ngram-edit-distance-panic.adm";
+
+for $o in dataset('DBLP')
+let $ed := edit-distance-check($o.authors, "Amihay Motro", 5)
+where $ed[0]
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance.aql
new file mode 100644
index 0000000..6077389
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance.aql
@@ -0,0 +1,29 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+ id: int32,
+ dblpid: string,
+ title: string,
+ authors: string,
+ misc: string
+}
+
+create nodegroup group1 if not exists on nc1, nc2;
+
+create dataset DBLP(DBLPType)
+ partitioned by key id on group1;
+
+load dataset DBLP
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index ngram_index on DBLP(authors) type fuzzy ngram(3);
+
+write output to nc1:"rttest/index-selection_fuzzy-inverted-index-ngram-edit-distance.adm";
+
+for $o in dataset('DBLP')
+let $ed := edit-distance-check($o.authors, "Amihay Motro", 1)
+where $ed[0]
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-jaccard.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-jaccard.aql
new file mode 100644
index 0000000..314c3bd
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-jaccard.aql
@@ -0,0 +1,29 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+ id: int32,
+ dblpid: string,
+ title: string,
+ authors: string,
+ misc: string
+}
+
+create nodegroup group1 if not exists on nc1, nc2;
+
+create dataset DBLP(DBLPType)
+ partitioned by key id on group1;
+
+load dataset DBLP
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index ngram_index on DBLP(title) type fuzzy ngram(3);
+
+write output to nc1:"rttest/index-selection_fuzzy-inverted-index-ngram-jaccard.adm";
+
+for $o in dataset('DBLP')
+let $jacc := similarity-jaccard-check(gram-tokens($o.title, 3, false), gram-tokens("Transactions for Cooperative Environments", 3, false), 0.5f)
+where $jacc[0]
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance-panic.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance-panic.aql
new file mode 100644
index 0000000..3a3a303
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance-panic.aql
@@ -0,0 +1,37 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+ number: int32,
+ street: string,
+ city: string
+}
+
+create type CustomerType as closed {
+ cid: int32,
+ name: string,
+ age: int32?,
+ address: AddressType?,
+ interests: [string],
+ children: [ { name: string, age: int32? } ]
+}
+
+create nodegroup group1 if not exists on nc1;
+
+create dataset Customers(CustomerType)
+ partitioned by key cid on group1;
+
+load dataset Customers
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type fuzzy keyword;
+
+write output to nc1:"rttest/index-selection_fuzzy-inverted-index-olist-edit-distance-panic.adm";
+
+for $c in dataset('Customers')
+let $ed := edit-distance-check($c.interests, ["computers", "wine", "walking"], 3)
+where $ed[0]
+order by $c.cid
+return $c
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance.aql
new file mode 100644
index 0000000..b1ce7c4
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance.aql
@@ -0,0 +1,37 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+ number: int32,
+ street: string,
+ city: string
+}
+
+create type CustomerType as closed {
+ cid: int32,
+ name: string,
+ age: int32?,
+ address: AddressType?,
+ interests: [string],
+ children: [ { name: string, age: int32? } ]
+}
+
+create nodegroup group1 if not exists on nc1;
+
+create dataset Customers(CustomerType)
+ partitioned by key cid on group1;
+
+load dataset Customers
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type fuzzy keyword;
+
+write output to nc1:"rttest/index-selection_fuzzy-inverted-index-olist-edit-distance.adm";
+
+for $c in dataset('Customers')
+let $ed := edit-distance-check($c.interests, ["computers", "wine", "walking"], 1)
+where $ed[0]
+order by $c.cid
+return $c
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-jaccard.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-jaccard.aql
new file mode 100644
index 0000000..b495ffe
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-jaccard.aql
@@ -0,0 +1,36 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+ number: int32,
+ street: string,
+ city: string
+}
+
+create type CustomerType as closed {
+ cid: int32,
+ name: string,
+ age: int32?,
+ address: AddressType?,
+ interests: [string],
+ children: [ { name: string, age: int32? } ]
+}
+
+create nodegroup group1 if not exists on nc1;
+
+create dataset Customers(CustomerType)
+ partitioned by key cid on group1;
+
+load dataset Customers
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type fuzzy keyword;
+
+write output to nc1:"rttest/index-selection_fuzzy-inverted-index-olist-jaccard.adm";
+
+for $c in dataset('Customers')
+let $jacc := similarity-jaccard-check($c.interests, ["databases", "computers", "wine"], 0.7f)
+where $jacc[0]
+return $c
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ulist-jaccard.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ulist-jaccard.aql
new file mode 100644
index 0000000..4427369
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ulist-jaccard.aql
@@ -0,0 +1,36 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+ number: int32,
+ street: string,
+ city: string
+}
+
+create type CustomerType as closed {
+ cid: int32,
+ name: string,
+ age: int32?,
+ address: AddressType?,
+ interests: {{string}},
+ children: [ { name: string, age: int32? } ]
+}
+
+create nodegroup group1 if not exists on nc1;
+
+create dataset Customers(CustomerType)
+ partitioned by key cid on group1;
+
+load dataset Customers
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type fuzzy keyword;
+
+write output to nc1:"rttest/index-selection_fuzzy-inverted-index-ulist-jaccard.adm";
+
+for $c in dataset('Customers')
+let $jacc := similarity-jaccard-check($c.interests, ["databases", "computers", "wine"], 0.7f)
+where $jacc[0]
+return $c
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-contains.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-contains.aql
new file mode 100644
index 0000000..888a381
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-contains.aql
@@ -0,0 +1,29 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+ id: int32,
+ dblpid: string,
+ title: string,
+ authors: string,
+ misc: string
+}
+
+create nodegroup group1 if not exists on nc1, nc2;
+
+create dataset DBLP(DBLPType)
+ partitioned by key id on group1;
+
+load dataset DBLP
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index keyword_index on DBLP(title) type fuzzy keyword;
+
+write output to nc1:"rttest/index-selection_fuzzy-inverted-index-word-contains.adm";
+
+for $o in dataset('DBLP')
+where contains($o.title, "Multimedia")
+order by $o.id
+return $o
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-jaccard.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-jaccard.aql
new file mode 100644
index 0000000..23c9289
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-jaccard.aql
@@ -0,0 +1,30 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+ id: int32,
+ dblpid: string,
+ title: string,
+ authors: string,
+ misc: string
+}
+
+create nodegroup group1 if not exists on nc1, nc2;
+
+create dataset DBLP(DBLPType)
+ partitioned by key id on group1;
+
+load dataset DBLP
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index keyword_index on DBLP(title) type fuzzy keyword;
+
+write output to nc1:"rttest/index-selection_fuzzy-inverted-index-word-jaccard.adm";
+
+for $o in dataset('DBLP')
+let $jacc := similarity-jaccard-check(word-tokens($o.title), word-tokens("Transactions for Cooperative Environments"), 0.5f)
+where $jacc[0]
+return $o
+
diff --git a/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-ngram-contains.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-ngram-contains.adm
new file mode 100644
index 0000000..8a99b26
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-ngram-contains.adm
@@ -0,0 +1,3 @@
+{ "id": 4, "dblpid": "books/acm/kim95/ChristodoulakisK95", "title": "Multimedia Information Systems Issues and Approaches.", "authors": "Stavros Christodoulakis Leonidas Koveos", "misc": "2002-01-03 318-337 1995 Modern Database Systems db/books/collections/kim95.html#ChristodoulakisK95" }
+{ "id": 89, "dblpid": "conf/icip/SchonfeldL98", "title": "VORTEX Video Retrieval and Tracking from Compressed Multimedia Databases.", "authors": "Dan Schonfeld Dan Lelescu", "misc": "2002-11-05 123-127 1998 ICIP (3) db/conf/icip/icip1998-3.html#SchonfeldL98" }
+{ "id": 90, "dblpid": "conf/hicss/SchonfeldL99", "title": "VORTEX Video Retrieval and Tracking from Compressed Multimedia Databases ¾ Visual Search Engine.", "authors": "Dan Schonfeld Dan Lelescu", "misc": "2002-01-03 1999 HICSS http //computer.org/proceedings/hicss/0001/00013/00013006abs.htm db/conf/hicss/hicss1999-3.html#SchonfeldL99" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-ngram-edit-distance-panic.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-ngram-edit-distance-panic.adm
new file mode 100644
index 0000000..a218d95
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-ngram-edit-distance-panic.adm
@@ -0,0 +1 @@
+{ "id": 22, "dblpid": "books/acm/kim95/Motro95", "title": "Management of Uncerainty in database Systems.", "authors": "Amihai Motro", "misc": "2002-01-03 457-476 1995 Modern Database Systems db/books/collections/kim95.html#Motro95" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-ngram-edit-distance.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-ngram-edit-distance.adm
new file mode 100644
index 0000000..a218d95
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-ngram-edit-distance.adm
@@ -0,0 +1 @@
+{ "id": 22, "dblpid": "books/acm/kim95/Motro95", "title": "Management of Uncerainty in database Systems.", "authors": "Amihai Motro", "misc": "2002-01-03 457-476 1995 Modern Database Systems db/books/collections/kim95.html#Motro95" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-ngram-jaccard.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-ngram-jaccard.adm
new file mode 100644
index 0000000..5bf6ae0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-ngram-jaccard.adm
@@ -0,0 +1 @@
+{ "id": 9, "dblpid": "books/acm/kim95/Kaiser95", "title": "Cooperative Transactions for Multiuser Environments.", "authors": "Gail E. Kaiser", "misc": "2002-01-03 409-433 1995 Modern Database Systems db/books/collections/kim95.html#Kaiser95" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-olist-edit-distance-panic.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-olist-edit-distance-panic.adm
new file mode 100644
index 0000000..9e33b16
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-olist-edit-distance-panic.adm
@@ -0,0 +1,854 @@
+{ "cid": 1, "name": "Trudie Minick", "age": 75, "address": { "number": 6740, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ "Fishing", "Squash" ], "children": [ { "name": "Arie Minick", "age": 56 }, { "name": "Alline Minick", "age": 57 }, { "name": "Petronila Minick", "age": 56 } ] }
+{ "cid": 2, "name": "Elin Debell", "age": 82, "address": { "number": 5649, "street": "Hill St.", "city": "Portland" }, "interests": [ "Bass", "Wine" ], "children": [ { "name": "Elvina Debell", "age": null }, { "name": "Renaldo Debell", "age": 51 }, { "name": "Divina Debell", "age": 57 } ] }
+{ "cid": 3, "name": "Phung Wheetley", "age": 12, "address": { "number": 5549, "street": "Hill St.", "city": "Mountain View" }, "interests": [ "Wine" ], "children": [ { "name": "Raelene Wheetley", "age": null }, { "name": "Dudley Wheetley", "age": null } ] }
+{ "cid": 4, "name": "Bernita Gungor", "age": 87, "address": { "number": 1208, "street": "Cedar St.", "city": "Mountain View" }, "interests": [ "Walking" ], "children": [ { "name": "Valencia Gungor", "age": 72 }, { "name": "Evangeline Gungor", "age": 76 }, { "name": "Odell Gungor", "age": null }, { "name": "Denny Gungor", "age": null } ] }
+{ "cid": 5, "name": "Heide Naifeh", "age": null, "address": null, "interests": [ "Music", "Databases" ], "children": [ { "name": "Deirdre Naifeh", "age": null }, { "name": "Jacquelyne Naifeh", "age": 39 } ] }
+{ "cid": 6, "name": "Cris Kager", "age": 70, "address": { "number": 8402, "street": "View St.", "city": "Los Angeles" }, "interests": [ "Walking" ], "children": [ { "name": "Carmelo Kager", "age": 34 }, { "name": "Faustina Kager", "age": null } ] }
+{ "cid": 7, "name": "Karie Kaehler", "age": 59, "address": { "number": 9875, "street": "View St.", "city": "San Jose" }, "interests": [ "Computers", "Skiing", "Basketball", "Movies" ], "children": [ { "name": "Spring Kaehler", "age": 17 } ] }
+{ "cid": 8, "name": "Audria Haylett", "age": 44, "address": { "number": 4872, "street": "Washington St.", "city": "Portland" }, "interests": [ "Cooking", "Fishing", "Video Games" ], "children": [ { "name": "Lacie Haylett", "age": 19 } ] }
+{ "cid": 9, "name": "Dreama Nuccio", "age": 55, "address": { "number": 95, "street": "Main St.", "city": "San Jose" }, "interests": [ ], "children": [ { "name": "Ricardo Nuccio", "age": 28 }, { "name": "See Nuccio", "age": 34 } ] }
+{ "cid": 10, "name": "Trent Liedy", "age": 51, "address": { "number": 1758, "street": "Oak St.", "city": "San Jose" }, "interests": [ ], "children": [ ] }
+{ "cid": 11, "name": "Meta Simek", "age": 13, "address": { "number": 4384, "street": "7th St.", "city": "San Jose" }, "interests": [ "Wine", "Walking" ], "children": [ { "name": "Oretha Simek", "age": null }, { "name": "Terence Simek", "age": null } ] }
+{ "cid": 12, "name": "Laurinda Raimann", "age": null, "address": null, "interests": [ "Basketball", "Coffee" ], "children": [ { "name": "Lulu Raimann", "age": null }, { "name": "Refugia Raimann", "age": 19 }, { "name": "Jimmie Raimann", "age": 10 }, { "name": "Cindy Raimann", "age": null } ] }
+{ "cid": 13, "name": "Nicol Kolmer", "age": null, "address": null, "interests": [ "Coffee" ], "children": [ { "name": "Erika Kolmer", "age": 40 }, { "name": "Justin Kolmer", "age": null }, { "name": "Dorathy Kolmer", "age": null }, { "name": "Anastacia Kolmer", "age": 27 } ] }
+{ "cid": 14, "name": "Chance Nicoson", "age": null, "address": null, "interests": [ "Tennis" ], "children": [ { "name": "Willette Nicoson", "age": 39 }, { "name": "Glennis Nicoson", "age": null }, { "name": "Philip Nicoson", "age": null }, { "name": "Cody Nicoson", "age": 26 } ] }
+{ "cid": 15, "name": "Berry Faubel", "age": 55, "address": { "number": 2806, "street": "Oak St.", "city": "Seattle" }, "interests": [ ], "children": [ { "name": "Tiffiny Faubel", "age": 12 }, { "name": "Hilaria Faubel", "age": 19 }, { "name": "Wesley Faubel", "age": 37 }, { "name": "Wei Faubel", "age": 28 } ] }
+{ "cid": 16, "name": "Felisa Auletta", "age": 55, "address": { "number": 7737, "street": "View St.", "city": "San Jose" }, "interests": [ "Skiing", "Coffee", "Wine" ], "children": [ { "name": "Rosalia Auletta", "age": 36 } ] }
+{ "cid": 17, "name": "Ingeborg Monkhouse", "age": null, "address": null, "interests": [ "Base Jumping", "Cigars", "Movies" ], "children": [ ] }
+{ "cid": 18, "name": "Dewayne Ardan", "age": 32, "address": { "number": 8229, "street": "Hill St.", "city": "San Jose" }, "interests": [ "Wine", "Walking", "Bass" ], "children": [ { "name": "Wen Ardan", "age": null }, { "name": "Sachiko Ardan", "age": 11 }, { "name": "Francis Ardan", "age": 20 } ] }
+{ "cid": 20, "name": "Annice Fulwider", "age": 59, "address": { "number": 4257, "street": "Park St.", "city": "Portland" }, "interests": [ ], "children": [ { "name": "Arica Fulwider", "age": 47 }, { "name": "Charlotte Fulwider", "age": 16 }, { "name": "Robbi Fulwider", "age": 29 } ] }
+{ "cid": 21, "name": "Gidget Galamay", "age": 34, "address": { "number": 2854, "street": "Washington St.", "city": "Los Angeles" }, "interests": [ ], "children": [ { "name": "Brunilda Galamay", "age": null }, { "name": "Bethel Galamay", "age": null }, { "name": "Devon Galamay", "age": 17 } ] }
+{ "cid": 22, "name": "Sarita Burrer", "age": null, "address": null, "interests": [ "Cigars", "Computers" ], "children": [ ] }
+{ "cid": 23, "name": "Micheal Konen", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Myong Konen", "age": 26 }, { "name": "Celinda Konen", "age": 33 }, { "name": "Tammy Konen", "age": 53 }, { "name": "Chester Konen", "age": null } ] }
+{ "cid": 24, "name": "Hosea Wilburn", "age": null, "address": null, "interests": [ ], "children": [ ] }
+{ "cid": 25, "name": "Goldie Vanhandel", "age": 37, "address": { "number": 6568, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ "Bass", "Fishing", "Cigars" ], "children": [ ] }
+{ "cid": 26, "name": "Jone Okuna", "age": 78, "address": { "number": 6006, "street": "7th St.", "city": "Portland" }, "interests": [ ], "children": [ { "name": "Franchesca Okuna", "age": null }, { "name": "Fred Okuna", "age": 17 }, { "name": "Marcellus Okuna", "age": null } ] }
+{ "cid": 27, "name": "Hollie Hyun", "age": null, "address": null, "interests": [ "Skiing", "Walking" ], "children": [ { "name": "Morton Hyun", "age": null }, { "name": "Farrah Hyun", "age": 40 }, { "name": "Ali Hyun", "age": null } ] }
+{ "cid": 28, "name": "Ariana Gillert", "age": 54, "address": { "number": 7331, "street": "Lake St.", "city": "Mountain View" }, "interests": [ "Databases" ], "children": [ { "name": "Inge Gillert", "age": null }, { "name": "Jeraldine Gillert", "age": 13 } ] }
+{ "cid": 29, "name": "Ruthanne Tavana", "age": null, "address": null, "interests": [ "Movies" ], "children": [ ] }
+{ "cid": 30, "name": "Deedee Centner", "age": null, "address": null, "interests": [ "Skiing", "Wine", "Databases", "Movies" ], "children": [ { "name": "Lorilee Centner", "age": 30 }, { "name": "Thad Centner", "age": null } ] }
+{ "cid": 31, "name": "Venus Toboz", "age": 44, "address": { "number": 9465, "street": "View St.", "city": "Mountain View" }, "interests": [ "Running" ], "children": [ { "name": "Ashlie Toboz", "age": null } ] }
+{ "cid": 32, "name": "Tia Berkley", "age": 30, "address": { "number": 4507, "street": "Park St.", "city": "Sunnyvale" }, "interests": [ "Base Jumping", "Music" ], "children": [ { "name": "Carmon Berkley", "age": null }, { "name": "Kristina Berkley", "age": null }, { "name": "Cristi Berkley", "age": 19 } ] }
+{ "cid": 33, "name": "Rayford Velmontes", "age": null, "address": null, "interests": [ "Fishing", "Video Games" ], "children": [ ] }
+{ "cid": 34, "name": "Sam Tannahill", "age": null, "address": null, "interests": [ "Books" ], "children": [ ] }
+{ "cid": 36, "name": "Neoma Preist", "age": 69, "address": { "number": 4830, "street": "Lake St.", "city": "San Jose" }, "interests": [ "Databases", "Computers", "Coffee" ], "children": [ { "name": "Shery Preist", "age": null }, { "name": "Kelvin Preist", "age": 43 } ] }
+{ "cid": 37, "name": "Eliana Vient", "age": 89, "address": { "number": 4882, "street": "View St.", "city": "Seattle" }, "interests": [ ], "children": [ { "name": "Dario Vient", "age": 43 } ] }
+{ "cid": 38, "name": "Lawanna Abadi", "age": 35, "address": { "number": 6942, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [ ], "children": [ { "name": "Arthur Abadi", "age": 10 } ] }
+{ "cid": 39, "name": "Brock Froncillo", "age": 72, "address": { "number": 4645, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Base Jumping", "Skiing" ], "children": [ { "name": "Cole Froncillo", "age": null }, { "name": "Ivana Froncillo", "age": null }, { "name": "Hugh Froncillo", "age": 23 } ] }
+{ "cid": 40, "name": "Fidelia Connie", "age": 81, "address": { "number": 2298, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ "Basketball", "Base Jumping", "Walking", "Skiing" ], "children": [ { "name": "Elfreda Connie", "age": 43 }, { "name": "Josephine Connie", "age": 30 }, { "name": "Lucas Connie", "age": null } ] }
+{ "cid": 41, "name": "Kevin Giottonini", "age": null, "address": null, "interests": [ "Skiing", "Bass" ], "children": [ { "name": "Victor Giottonini", "age": 37 }, { "name": "Alverta Giottonini", "age": null } ] }
+{ "cid": 42, "name": "Asley Simco", "age": 38, "address": { "number": 3322, "street": "Main St.", "city": "Mountain View" }, "interests": [ "Fishing", "Running", "Cigars" ], "children": [ { "name": "Micheal Simco", "age": null }, { "name": "Lawerence Simco", "age": null } ] }
+{ "cid": 44, "name": "Agustin Clubs", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Maxwell Clubs", "age": 31 }, { "name": "Rayna Clubs", "age": null }, { "name": "Darwin Clubs", "age": null } ] }
+{ "cid": 46, "name": "Columbus Huntington", "age": 22, "address": { "number": 3809, "street": "Washington St.", "city": "Mountain View" }, "interests": [ "Movies" ], "children": [ { "name": "Dana Huntington", "age": 10 }, { "name": "Rosa Huntington", "age": null } ] }
+{ "cid": 48, "name": "Delia Salveson", "age": 44, "address": { "number": 5596, "street": "7th St.", "city": "Portland" }, "interests": [ "Cigars", "Running", "Walking", "Running" ], "children": [ { "name": "Logan Salveson", "age": 21 }, { "name": "Temple Salveson", "age": 17 }, { "name": "Kimi Salveson", "age": null }, { "name": "Jacob Salveson", "age": 20 } ] }
+{ "cid": 49, "name": "Asa Schwing", "age": 70, "address": { "number": 2261, "street": "7th St.", "city": "Sunnyvale" }, "interests": [ "Tennis" ], "children": [ { "name": "Joy Schwing", "age": 15 } ] }
+{ "cid": 50, "name": "Lise Gorelli", "age": null, "address": null, "interests": [ "Books", "Wine", "Skiing", "Computers" ], "children": [ { "name": "Darleen Gorelli", "age": null }, { "name": "Latia Gorelli", "age": null }, { "name": "Page Gorelli", "age": null }, { "name": "Columbus Gorelli", "age": null } ] }
+{ "cid": 51, "name": "Simonne Cape", "age": null, "address": null, "interests": [ "Bass", "Bass", "Books" ], "children": [ { "name": "Leland Cape", "age": null }, { "name": "Gearldine Cape", "age": null } ] }
+{ "cid": 52, "name": "Janna Tish", "age": 12, "address": { "number": 2598, "street": "Washington St.", "city": "San Jose" }, "interests": [ ], "children": [ { "name": "Mackenzie Tish", "age": null }, { "name": "Ettie Tish", "age": null }, { "name": "Hortencia Tish", "age": null }, { "name": "Paul Tish", "age": null } ] }
+{ "cid": 53, "name": "Ricardo Greiwe", "age": 24, "address": { "number": 8983, "street": "View St.", "city": "Portland" }, "interests": [ ], "children": [ ] }
+{ "cid": 54, "name": "Haywood Vasiloff", "age": 63, "address": { "number": 8780, "street": "View St.", "city": "Sunnyvale" }, "interests": [ ], "children": [ { "name": "Celsa Vasiloff", "age": 40 }, { "name": "Shawana Vasiloff", "age": 43 }, { "name": "Joel Vasiloff", "age": 42 }, { "name": "Timmy Vasiloff", "age": 33 } ] }
+{ "cid": 55, "name": "Terrence Bryant", "age": 12, "address": { "number": 3188, "street": "Park St.", "city": "Seattle" }, "interests": [ "Wine", "Cooking" ], "children": [ { "name": "Dayna Bryant", "age": null } ] }
+{ "cid": 56, "name": "Andria Killelea", "age": null, "address": null, "interests": [ "Cigars", "Skiing" ], "children": [ ] }
+{ "cid": 57, "name": "Celestine Mac", "age": null, "address": null, "interests": [ "Wine", "Computers", "Books" ], "children": [ { "name": "Kathyrn Mac", "age": 44 } ] }
+{ "cid": 58, "name": "Rosemarie Mattei", "age": 80, "address": { "number": 1390, "street": "Park St.", "city": "Sunnyvale" }, "interests": [ ], "children": [ { "name": "Sonya Mattei", "age": 52 }, { "name": "Elenor Mattei", "age": null } ] }
+{ "cid": 59, "name": "Rea Villicana", "age": null, "address": null, "interests": [ ], "children": [ ] }
+{ "cid": 61, "name": "Linsey Mose", "age": 17, "address": { "number": 9198, "street": "Lake St.", "city": "Portland" }, "interests": [ "Puzzles" ], "children": [ { "name": "Tilda Mose", "age": null }, { "name": "Lillie Mose", "age": null }, { "name": "Robyn Mose", "age": null } ] }
+{ "cid": 62, "name": "Kiley Machnik", "age": null, "address": null, "interests": [ ], "children": [ ] }
+{ "cid": 64, "name": "Victor Susor", "age": 32, "address": { "number": 1690, "street": "Main St.", "city": "Portland" }, "interests": [ "Running", "Computers" ], "children": [ ] }
+{ "cid": 66, "name": "Lenny Latson", "age": null, "address": null, "interests": [ "Music", "Video Games" ], "children": [ ] }
+{ "cid": 67, "name": "Tobie Mattan", "age": null, "address": null, "interests": [ ], "children": [ ] }
+{ "cid": 68, "name": "Chery Basini", "age": null, "address": null, "interests": [ "Video Games" ], "children": [ ] }
+{ "cid": 69, "name": "Many Yeargain", "age": null, "address": null, "interests": [ "Coffee" ], "children": [ { "name": "Brande Yeargain", "age": null }, { "name": "Tawna Yeargain", "age": null }, { "name": "Doris Yeargain", "age": null }, { "name": "Valeria Yeargain", "age": 51 } ] }
+{ "cid": 70, "name": "Mellisa Lek", "age": 62, "address": { "number": 4281, "street": "Oak St.", "city": "Mountain View" }, "interests": [ "Bass", "Running", "Databases" ], "children": [ ] }
+{ "cid": 71, "name": "Alva Sieger", "age": null, "address": null, "interests": [ "Movies", "Walking" ], "children": [ { "name": "Renetta Sieger", "age": null }, { "name": "Shiloh Sieger", "age": 57 }, { "name": "Lavina Sieger", "age": null }, { "name": "Larraine Sieger", "age": null } ] }
+{ "cid": 73, "name": "Kelsey Flever", "age": 20, "address": { "number": 3555, "street": "Main St.", "city": "Portland" }, "interests": [ "Tennis", "Puzzles", "Video Games" ], "children": [ { "name": "Isis Flever", "age": null }, { "name": "Gonzalo Flever", "age": null } ] }
+{ "cid": 74, "name": "Lonnie Ercolani", "age": 79, "address": { "number": 2655, "street": "Lake St.", "city": "Los Angeles" }, "interests": [ "Music", "Coffee" ], "children": [ { "name": "Cassi Ercolani", "age": null } ] }
+{ "cid": 76, "name": "Opal Blewett", "age": null, "address": null, "interests": [ "Running", "Coffee", "Fishing" ], "children": [ { "name": "Violette Blewett", "age": null } ] }
+{ "cid": 77, "name": "Chantal Parriera", "age": 78, "address": { "number": 5967, "street": "Lake St.", "city": "San Jose" }, "interests": [ "Squash", "Movies", "Coffee" ], "children": [ ] }
+{ "cid": 78, "name": "Wesley Huggler", "age": 80, "address": { "number": 3078, "street": "7th St.", "city": "Los Angeles" }, "interests": [ "Base Jumping", "Movies", "Skiing" ], "children": [ { "name": "Chassidy Huggler", "age": null }, { "name": "Emogene Huggler", "age": null }, { "name": "Cheryle Huggler", "age": null } ] }
+{ "cid": 79, "name": "Alyce Schoenle", "age": 57, "address": { "number": 1345, "street": "Main St.", "city": "Portland" }, "interests": [ ], "children": [ { "name": "Stewart Schoenle", "age": 16 }, { "name": "Bruce Schoenle", "age": 44 } ] }
+{ "cid": 81, "name": "Lavonda Manford", "age": 87, "address": { "number": 2423, "street": "Main St.", "city": "San Jose" }, "interests": [ ], "children": [ ] }
+{ "cid": 82, "name": "Gloria Junkins", "age": null, "address": null, "interests": [ "Basketball" ], "children": [ ] }
+{ "cid": 83, "name": "Filiberto Couillard", "age": null, "address": null, "interests": [ "Cooking", "Books" ], "children": [ { "name": "Diane Couillard", "age": 19 }, { "name": "Asa Couillard", "age": 23 }, { "name": "Zaida Couillard", "age": 57 }, { "name": "Shavonne Couillard", "age": null } ] }
+{ "cid": 84, "name": "Huong Kachel", "age": null, "address": null, "interests": [ "Music", "Tennis", "Base Jumping" ], "children": [ { "name": "Katlyn Kachel", "age": 40 }, { "name": "Sherman Kachel", "age": null }, { "name": "Susana Kachel", "age": 32 } ] }
+{ "cid": 85, "name": "Fatimah Steltenpohl", "age": 25, "address": { "number": 6175, "street": "Park St.", "city": "Sunnyvale" }, "interests": [ ], "children": [ { "name": "Genoveva Steltenpohl", "age": 14 } ] }
+{ "cid": 86, "name": "Sofia Mongiovi", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Rosamaria Mongiovi", "age": 25 } ] }
+{ "cid": 87, "name": "Torie Horuath", "age": 21, "address": { "number": 2713, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ "Coffee", "Puzzles", "Cigars", "Walking" ], "children": [ { "name": "Joshua Horuath", "age": 10 } ] }
+{ "cid": 88, "name": "Courtney Muckleroy", "age": null, "address": null, "interests": [ "Wine", "Movies", "Skiing" ], "children": [ { "name": "Alona Muckleroy", "age": 30 }, { "name": "Flora Muckleroy", "age": 41 }, { "name": "Angel Muckleroy", "age": null }, { "name": "Daniella Muckleroy", "age": null } ] }
+{ "cid": 89, "name": "Calandra Hedden", "age": 33, "address": { "number": 1231, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ "Wine" ], "children": [ { "name": "Damien Hedden", "age": 19 } ] }
+{ "cid": 90, "name": "Dorethea Korns", "age": null, "address": null, "interests": [ "Cooking", "Computers" ], "children": [ { "name": "Catheryn Korns", "age": 22 } ] }
+{ "cid": 91, "name": "Luna Machen", "age": null, "address": null, "interests": [ "Wine" ], "children": [ { "name": "Randal Machen", "age": 59 }, { "name": "Emely Machen", "age": null } ] }
+{ "cid": 92, "name": "Kenny Laychock", "age": 15, "address": { "number": 4790, "street": "Washington St.", "city": "Portland" }, "interests": [ "Video Games", "Basketball" ], "children": [ ] }
+{ "cid": 93, "name": "Garth Raigosa", "age": null, "address": null, "interests": [ "Basketball" ], "children": [ ] }
+{ "cid": 94, "name": "Edgardo Dunnegan", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Lyndia Dunnegan", "age": null } ] }
+{ "cid": 95, "name": "Gavin Locey", "age": 86, "address": { "number": 8162, "street": "Lake St.", "city": "Portland" }, "interests": [ ], "children": [ { "name": "Terrell Locey", "age": null }, { "name": "Kazuko Locey", "age": 36 }, { "name": "Risa Locey", "age": null }, { "name": "Dorethea Locey", "age": 13 } ] }
+{ "cid": 96, "name": "Mara Aument", "age": 72, "address": { "number": 7709, "street": "Hill St.", "city": "Sunnyvale" }, "interests": [ "Cigars", "Cooking", "Movies" ], "children": [ { "name": "Leonardo Aument", "age": 22 } ] }
+{ "cid": 97, "name": "Mui Slosek", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Susanne Slosek", "age": 29 }, { "name": "Colleen Slosek", "age": null } ] }
+{ "cid": 98, "name": "Casimira Hilbrand", "age": 72, "address": { "number": 9693, "street": "Main St.", "city": "Los Angeles" }, "interests": [ ], "children": [ { "name": "Gudrun Hilbrand", "age": 18 }, { "name": "Dacia Hilbrand", "age": 26 }, { "name": "Kortney Hilbrand", "age": null }, { "name": "Luci Hilbrand", "age": null } ] }
+{ "cid": 99, "name": "Bernardina Thacher", "age": 35, "address": { "number": 1582, "street": "Main St.", "city": "Los Angeles" }, "interests": [ "Movies", "Fishing", "Fishing" ], "children": [ { "name": "Randee Thacher", "age": null }, { "name": "China Thacher", "age": null } ] }
+{ "cid": 101, "name": "Meaghan Vandel", "age": null, "address": null, "interests": [ "Music", "Base Jumping", "Books" ], "children": [ { "name": "Larissa Vandel", "age": null } ] }
+{ "cid": 102, "name": "Melany Rotan", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Christiana Rotan", "age": 21 }, { "name": "Lavina Rotan", "age": null }, { "name": "Billy Rotan", "age": null } ] }
+{ "cid": 103, "name": "Rosamond Milera", "age": null, "address": null, "interests": [ "Cigars" ], "children": [ ] }
+{ "cid": 104, "name": "Neda Dilts", "age": null, "address": null, "interests": [ "Basketball" ], "children": [ { "name": "Nona Dilts", "age": 28 }, { "name": "Wm Dilts", "age": null }, { "name": "Svetlana Dilts", "age": 46 }, { "name": "Iva Dilts", "age": 59 } ] }
+{ "cid": 105, "name": "Camilla Lohman", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Melania Lohman", "age": 50 }, { "name": "Mike Lohman", "age": 53 }, { "name": "Cassaundra Lohman", "age": 32 }, { "name": "Jay Lohman", "age": null } ] }
+{ "cid": 106, "name": "Charles Verna", "age": null, "address": null, "interests": [ "Bass", "Books" ], "children": [ { "name": "Betsy Verna", "age": 37 }, { "name": "Chae Verna", "age": 35 }, { "name": "Naoma Verna", "age": 42 } ] }
+{ "cid": 110, "name": "Karmen Milanesi", "age": 67, "address": { "number": 6223, "street": "Cedar St.", "city": "Portland" }, "interests": [ "Squash", "Squash" ], "children": [ { "name": "Emely Milanesi", "age": null }, { "name": "Adam Milanesi", "age": null }, { "name": "Gregg Milanesi", "age": null }, { "name": "Sean Milanesi", "age": 37 } ] }
+{ "cid": 111, "name": "Eddy Ortea", "age": 16, "address": { "number": 6874, "street": "Main St.", "city": "Los Angeles" }, "interests": [ ], "children": [ { "name": "Shera Ortea", "age": null } ] }
+{ "cid": 112, "name": "Dorie Lave", "age": 10, "address": { "number": 2286, "street": "Lake St.", "city": "Los Angeles" }, "interests": [ "Coffee" ], "children": [ { "name": "Grady Lave", "age": null }, { "name": "Daysi Lave", "age": null } ] }
+{ "cid": 113, "name": "Alayna Daleske", "age": 87, "address": { "number": 4739, "street": "Main St.", "city": "Sunnyvale" }, "interests": [ ], "children": [ { "name": "Hester Daleske", "age": null }, { "name": "Magnolia Daleske", "age": null }, { "name": "Bettye Daleske", "age": 32 } ] }
+{ "cid": 114, "name": "Stephine Capinpin", "age": 78, "address": { "number": 5618, "street": "Main St.", "city": "Sunnyvale" }, "interests": [ "Puzzles", "Basketball" ], "children": [ { "name": "Krystal Capinpin", "age": 31 }, { "name": "Angelic Capinpin", "age": 45 } ] }
+{ "cid": 115, "name": "Jason Oakden", "age": 89, "address": { "number": 8182, "street": "Park St.", "city": "Los Angeles" }, "interests": [ "Music", "Basketball", "Movies" ], "children": [ { "name": "Johnson Oakden", "age": null }, { "name": "Neva Oakden", "age": null }, { "name": "Juliann Oakden", "age": null }, { "name": "Elmer Oakden", "age": null } ] }
+{ "cid": 116, "name": "Conrad Zozaya", "age": 81, "address": { "number": 1667, "street": "View St.", "city": "San Jose" }, "interests": [ ], "children": [ { "name": "Jenette Zozaya", "age": 17 } ] }
+{ "cid": 118, "name": "Ellis Skillom", "age": 78, "address": { "number": 9337, "street": "View St.", "city": "Mountain View" }, "interests": [ "Running", "Cigars" ], "children": [ { "name": "Emory Skillom", "age": null } ] }
+{ "cid": 119, "name": "Chan Morreau", "age": 22, "address": { "number": 1774, "street": "Lake St.", "city": "Mountain View" }, "interests": [ "Puzzles", "Squash" ], "children": [ { "name": "Arlette Morreau", "age": null } ] }
+{ "cid": 120, "name": "Jan Gianandrea", "age": null, "address": null, "interests": [ "Databases", "Movies", "Cigars" ], "children": [ { "name": "Keesha Gianandrea", "age": null }, { "name": "Vashti Gianandrea", "age": 35 }, { "name": "Larry Gianandrea", "age": 29 } ] }
+{ "cid": 121, "name": "Shiela Gaustad", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Phebe Gaustad", "age": null }, { "name": "Mavis Gaustad", "age": null }, { "name": "Zula Gaustad", "age": 37 } ] }
+{ "cid": 122, "name": "Wei Perpall", "age": 43, "address": { "number": 916, "street": "Washington St.", "city": "Los Angeles" }, "interests": [ "Bass" ], "children": [ { "name": "Mitchel Perpall", "age": 11 }, { "name": "Aliza Perpall", "age": null }, { "name": "King Perpall", "age": null }, { "name": "Santana Perpall", "age": 22 } ] }
+{ "cid": 123, "name": "Marian Courrege", "age": 30, "address": { "number": 7321, "street": "Main St.", "city": "Sunnyvale" }, "interests": [ "Coffee" ], "children": [ ] }
+{ "cid": 124, "name": "Kelley Dressman", "age": null, "address": null, "interests": [ "Squash", "Databases", "Fishing" ], "children": [ { "name": "Evie Dressman", "age": null }, { "name": "Fredericka Dressman", "age": null }, { "name": "Leigh Dressman", "age": null }, { "name": "Luna Dressman", "age": 29 } ] }
+{ "cid": 125, "name": "Leigh Pusey", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Elbert Pusey", "age": 44 }, { "name": "Golden Pusey", "age": null }, { "name": "Maria Pusey", "age": null } ] }
+{ "cid": 126, "name": "Grayce Keir", "age": null, "address": null, "interests": [ "Wine" ], "children": [ { "name": "Antonia Keir", "age": 25 } ] }
+{ "cid": 127, "name": "Christian Anthes", "age": 32, "address": { "number": 6258, "street": "7th St.", "city": "Portland" }, "interests": [ "Running", "Bass" ], "children": [ { "name": "Sophia Anthes", "age": null } ] }
+{ "cid": 128, "name": "Edwin Harwick", "age": null, "address": null, "interests": [ "Fishing", "Squash", "Basketball" ], "children": [ { "name": "Tomeka Harwick", "age": 34 }, { "name": "Caroline Harwick", "age": 57 }, { "name": "Peter Harwick", "age": null }, { "name": "Adele Harwick", "age": null } ] }
+{ "cid": 129, "name": "Marisha Canzoneri", "age": 84, "address": { "number": 5507, "street": "View St.", "city": "Mountain View" }, "interests": [ "Music", "Databases", "Walking", "Walking" ], "children": [ ] }
+{ "cid": 130, "name": "Kandis Hissem", "age": null, "address": null, "interests": [ "Tennis" ], "children": [ { "name": "Arianna Hissem", "age": null }, { "name": "Necole Hissem", "age": 53 }, { "name": "Manie Hissem", "age": null }, { "name": "Deshawn Hissem", "age": 27 } ] }
+{ "cid": 131, "name": "Kourtney Whitesel", "age": null, "address": null, "interests": [ ], "children": [ ] }
+{ "cid": 132, "name": "Cindi Turntine", "age": 64, "address": { "number": 9432, "street": "Park St.", "city": "Portland" }, "interests": [ "Computers", "Wine" ], "children": [ { "name": "Howard Turntine", "age": null } ] }
+{ "cid": 134, "name": "Alica Frontiero", "age": null, "address": null, "interests": [ "Puzzles" ], "children": [ ] }
+{ "cid": 135, "name": "Josette Dries", "age": null, "address": null, "interests": [ "Base Jumping", "Movies" ], "children": [ { "name": "Ben Dries", "age": 36 }, { "name": "Wm Dries", "age": 29 } ] }
+{ "cid": 136, "name": "Aubrey Kasuboski", "age": null, "address": null, "interests": [ "Cigars" ], "children": [ ] }
+{ "cid": 137, "name": "Camellia Pressman", "age": 81, "address": { "number": 3947, "street": "Park St.", "city": "Seattle" }, "interests": [ "Movies", "Books", "Bass" ], "children": [ { "name": "Dwana Pressman", "age": null }, { "name": "Johnathan Pressman", "age": null }, { "name": "Kasey Pressman", "age": null }, { "name": "Mitch Pressman", "age": null } ] }
+{ "cid": 138, "name": "Ora Villafane", "age": null, "address": null, "interests": [ "Walking", "Cooking" ], "children": [ { "name": "Deeann Villafane", "age": 22 }, { "name": "Cody Villafane", "age": 47 } ] }
+{ "cid": 139, "name": "Micheline Argenal", "age": null, "address": null, "interests": [ "Bass", "Walking", "Movies" ], "children": [ { "name": "Joye Argenal", "age": 51 }, { "name": "Richard Argenal", "age": 46 }, { "name": "Sarah Argenal", "age": 21 }, { "name": "Jacinda Argenal", "age": 21 } ] }
+{ "cid": 140, "name": "Maryland Neas", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Brunilda Neas", "age": 28 } ] }
+{ "cid": 141, "name": "Adena Klockars", "age": null, "address": null, "interests": [ "Skiing", "Computers", "Bass", "Cigars" ], "children": [ ] }
+{ "cid": 142, "name": "Ervin Softleigh", "age": null, "address": null, "interests": [ "Computers", "Skiing", "Cooking", "Coffee" ], "children": [ { "name": "Russell Softleigh", "age": 50 }, { "name": "Kristy Softleigh", "age": 54 }, { "name": "Refugio Softleigh", "age": null } ] }
+{ "cid": 143, "name": "Katelynn Kanzler", "age": 80, "address": { "number": 9453, "street": "Washington St.", "city": "Seattle" }, "interests": [ ], "children": [ { "name": "Carl Kanzler", "age": null } ] }
+{ "cid": 144, "name": "Celesta Sosebee", "age": 19, "address": { "number": 2683, "street": "7th St.", "city": "Portland" }, "interests": [ "Databases", "Databases" ], "children": [ { "name": "Jesse Sosebee", "age": null }, { "name": "Oralee Sosebee", "age": null }, { "name": "Sunday Sosebee", "age": null } ] }
+{ "cid": 145, "name": "Carey Bousman", "age": 61, "address": { "number": 16, "street": "Oak St.", "city": "Mountain View" }, "interests": [ ], "children": [ { "name": "Lynda Bousman", "age": 32 }, { "name": "Evalyn Bousman", "age": 17 } ] }
+{ "cid": 146, "name": "Glennis Vanruiten", "age": 14, "address": { "number": 8272, "street": "Park St.", "city": "Los Angeles" }, "interests": [ "Squash", "Databases" ], "children": [ { "name": "Joanie Vanruiten", "age": null }, { "name": "Long Vanruiten", "age": null }, { "name": "Abdul Vanruiten", "age": null } ] }
+{ "cid": 147, "name": "Marla Pollan", "age": 24, "address": { "number": 9271, "street": "Oak St.", "city": "Portland" }, "interests": [ "Music" ], "children": [ { "name": "Song Pollan", "age": 11 }, { "name": "Lili Pollan", "age": 13 }, { "name": "Shaunte Pollan", "age": 12 }, { "name": "Sandie Pollan", "age": null } ] }
+{ "cid": 148, "name": "Coy Dulay", "age": 66, "address": { "number": 9793, "street": "Hill St.", "city": "Seattle" }, "interests": [ ], "children": [ { "name": "Emile Dulay", "age": null }, { "name": "Letitia Dulay", "age": 38 } ] }
+{ "cid": 149, "name": "Marcella Diamond", "age": 62, "address": { "number": 720, "street": "7th St.", "city": "Mountain View" }, "interests": [ ], "children": [ { "name": "Ezra Diamond", "age": null } ] }
+{ "cid": 150, "name": "Jesus Vanleeuwen", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Sueann Vanleeuwen", "age": 47 }, { "name": "Refugia Vanleeuwen", "age": null }, { "name": "Taisha Vanleeuwen", "age": null }, { "name": "Nathaniel Vanleeuwen", "age": null } ] }
+{ "cid": 151, "name": "Charlyn Soyars", "age": 21, "address": { "number": 2796, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ ], "children": [ ] }
+{ "cid": 153, "name": "Randy Hueso", "age": 11, "address": { "number": 1957, "street": "Oak St.", "city": "San Jose" }, "interests": [ "Computers", "Wine", "Databases", "Walking" ], "children": [ ] }
+{ "cid": 156, "name": "Bobbye Kauppi", "age": 79, "address": { "number": 2051, "street": "Hill St.", "city": "Sunnyvale" }, "interests": [ "Base Jumping", "Cigars", "Movies" ], "children": [ ] }
+{ "cid": 157, "name": "Mckenzie Tahir", "age": 78, "address": { "number": 6752, "street": "Hill St.", "city": "Seattle" }, "interests": [ ], "children": [ { "name": "Margarita Tahir", "age": 18 }, { "name": "Mia Tahir", "age": 47 }, { "name": "Gaylord Tahir", "age": null } ] }
+{ "cid": 158, "name": "Rosalva Harvath", "age": 84, "address": { "number": 5569, "street": "Washington St.", "city": "Mountain View" }, "interests": [ "Puzzles", "Wine", "Skiing", "Coffee" ], "children": [ { "name": "Taneka Harvath", "age": null }, { "name": "Ina Harvath", "age": 54 }, { "name": "Joanne Harvath", "age": 51 } ] }
+{ "cid": 159, "name": "Jeanmarie Franchini", "age": null, "address": null, "interests": [ "Music" ], "children": [ { "name": "Nikita Franchini", "age": null }, { "name": "Willetta Franchini", "age": null }, { "name": "Ester Franchini", "age": 12 } ] }
+{ "cid": 160, "name": "Yevette Chanez", "age": null, "address": null, "interests": [ "Bass", "Wine", "Coffee" ], "children": [ { "name": "Walter Chanez", "age": 11 }, { "name": "Pa Chanez", "age": 27 } ] }
+{ "cid": 161, "name": "Lucia Tata", "age": 85, "address": { "number": 8058, "street": "Park St.", "city": "Seattle" }, "interests": [ "Basketball", "Bass" ], "children": [ { "name": "Jenifer Tata", "age": 70 }, { "name": "Erna Tata", "age": null } ] }
+{ "cid": 162, "name": "Chang Reek", "age": 85, "address": { "number": 5943, "street": "Washington St.", "city": "Portland" }, "interests": [ "Tennis", "Movies" ], "children": [ { "name": "Camelia Reek", "age": null }, { "name": "Eleonora Reek", "age": 36 }, { "name": "Shalonda Reek", "age": 39 }, { "name": "Stefan Reek", "age": 64 } ] }
+{ "cid": 163, "name": "Marcelene Sparano", "age": 36, "address": { "number": 5722, "street": "View St.", "city": "San Jose" }, "interests": [ "Basketball", "Databases" ], "children": [ { "name": "Luz Sparano", "age": null }, { "name": "Cassandra Sparano", "age": 21 }, { "name": "Martina Sparano", "age": 21 }, { "name": "Elisabeth Sparano", "age": null } ] }
+{ "cid": 164, "name": "Lucrecia Dahlhauser", "age": null, "address": null, "interests": [ "Wine" ], "children": [ ] }
+{ "cid": 165, "name": "Melodie Starrick", "age": null, "address": null, "interests": [ "Walking" ], "children": [ { "name": "Adria Starrick", "age": null }, { "name": "Tasha Starrick", "age": 25 } ] }
+{ "cid": 166, "name": "Gregorio Plummer", "age": null, "address": null, "interests": [ "Base Jumping" ], "children": [ { "name": "Santiago Plummer", "age": null }, { "name": "Malisa Plummer", "age": 59 }, { "name": "Tracie Plummer", "age": 40 }, { "name": "Florentina Plummer", "age": 23 } ] }
+{ "cid": 169, "name": "Casandra Fierge", "age": 55, "address": { "number": 175, "street": "Cedar St.", "city": "Mountain View" }, "interests": [ "Cigars" ], "children": [ ] }
+{ "cid": 170, "name": "Dana Lese", "age": 38, "address": { "number": 575, "street": "Lake St.", "city": "Seattle" }, "interests": [ "Walking", "Coffee" ], "children": [ { "name": "Yasmine Lese", "age": 24 }, { "name": "Ezekiel Lese", "age": 20 }, { "name": "Ammie Lese", "age": 27 }, { "name": "Robert Lese", "age": 15 } ] }
+{ "cid": 171, "name": "Eddie Shebchuk", "age": 86, "address": { "number": 3304, "street": "Lake St.", "city": "Portland" }, "interests": [ "Books" ], "children": [ { "name": "Harmony Shebchuk", "age": null } ] }
+{ "cid": 172, "name": "Weldon Alquesta", "age": null, "address": null, "interests": [ "Music", "Fishing", "Music" ], "children": [ { "name": "Kip Alquesta", "age": null } ] }
+{ "cid": 173, "name": "Annamae Lucien", "age": 46, "address": { "number": 1253, "street": "Hill St.", "city": "Mountain View" }, "interests": [ "Puzzles", "Cooking", "Squash" ], "children": [ { "name": "Sanjuana Lucien", "age": 21 }, { "name": "Nathanael Lucien", "age": 27 }, { "name": "Jae Lucien", "age": null }, { "name": "Judith Lucien", "age": null } ] }
+{ "cid": 174, "name": "Taneka Baldassare", "age": 50, "address": { "number": 5787, "street": "Park St.", "city": "Portland" }, "interests": [ ], "children": [ { "name": "Junko Baldassare", "age": null }, { "name": "Denisha Baldassare", "age": null }, { "name": "Hermina Baldassare", "age": 17 }, { "name": "Lexie Baldassare", "age": null } ] }
+{ "cid": 175, "name": "Loise Obhof", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Susann Obhof", "age": null }, { "name": "Signe Obhof", "age": 38 } ] }
+{ "cid": 176, "name": "Kellie Andruszkiewic", "age": null, "address": null, "interests": [ "Fishing", "Puzzles", "Wine", "Skiing" ], "children": [ { "name": "Xiao Andruszkiewic", "age": null }, { "name": "Al Andruszkiewic", "age": 43 } ] }
+{ "cid": 177, "name": "Wilda Hanisch", "age": null, "address": null, "interests": [ "Wine", "Computers" ], "children": [ { "name": "Shannan Hanisch", "age": null }, { "name": "Marissa Hanisch", "age": 30 }, { "name": "Keely Hanisch", "age": 54 }, { "name": "Humberto Hanisch", "age": 17 } ] }
+{ "cid": 178, "name": "Athena Kaluna", "age": null, "address": null, "interests": [ "Running", "Computers", "Basketball" ], "children": [ { "name": "Rosalba Kaluna", "age": 48 }, { "name": "Max Kaluna", "age": 10 } ] }
+{ "cid": 179, "name": "Antonette Bernice", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Solange Bernice", "age": null } ] }
+{ "cid": 180, "name": "Theda Hilz", "age": 35, "address": { "number": 9918, "street": "Oak St.", "city": "Los Angeles" }, "interests": [ ], "children": [ { "name": "Ethan Hilz", "age": null }, { "name": "Bill Hilz", "age": 12 } ] }
+{ "cid": 181, "name": "Toni Sanghani", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Hollie Sanghani", "age": 29 } ] }
+{ "cid": 182, "name": "Christiana Westlie", "age": null, "address": null, "interests": [ "Skiing", "Bass" ], "children": [ { "name": "Ilda Westlie", "age": 18 } ] }
+{ "cid": 183, "name": "Ladawn Vyas", "age": 64, "address": { "number": 2663, "street": "View St.", "city": "Portland" }, "interests": [ ], "children": [ ] }
+{ "cid": 184, "name": "Mirtha Ricciardi", "age": null, "address": null, "interests": [ "Music" ], "children": [ { "name": "Elsa Ricciardi", "age": 30 }, { "name": "Vicente Ricciardi", "age": null }, { "name": "Sau Ricciardi", "age": 28 } ] }
+{ "cid": 185, "name": "Abigail Zugg", "age": 22, "address": { "number": 6676, "street": "Washington St.", "city": "Seattle" }, "interests": [ "Computers", "Basketball", "Video Games", "Basketball" ], "children": [ { "name": "Peter Zugg", "age": 10 }, { "name": "Ariane Zugg", "age": null } ] }
+{ "cid": 187, "name": "Seema Hartsch", "age": 80, "address": { "number": 6629, "street": "Lake St.", "city": "Portland" }, "interests": [ "Coffee", "Coffee", "Cigars" ], "children": [ { "name": "Suellen Hartsch", "age": null }, { "name": "Pennie Hartsch", "age": 20 }, { "name": "Aubrey Hartsch", "age": null }, { "name": "Randy Hartsch", "age": 32 } ] }
+{ "cid": 188, "name": "Brynn Bendorf", "age": 23, "address": { "number": 1168, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ "Skiing" ], "children": [ { "name": "Leesa Bendorf", "age": 11 }, { "name": "Daine Bendorf", "age": null } ] }
+{ "cid": 189, "name": "Shyla Saathoff", "age": 85, "address": { "number": 9679, "street": "Main St.", "city": "Mountain View" }, "interests": [ ], "children": [ { "name": "Johanne Saathoff", "age": 61 }, { "name": "Janett Saathoff", "age": null } ] }
+{ "cid": 190, "name": "Kristel Axelson", "age": null, "address": null, "interests": [ "Movies", "Books" ], "children": [ { "name": "Deja Axelson", "age": null } ] }
+{ "cid": 191, "name": "Lula Pangburn", "age": 42, "address": { "number": 1309, "street": "Lake St.", "city": "Seattle" }, "interests": [ "Skiing", "Cooking", "Walking", "Video Games" ], "children": [ { "name": "Love Pangburn", "age": 11 }, { "name": "Bryant Pangburn", "age": 13 }, { "name": "Kenda Pangburn", "age": 14 } ] }
+{ "cid": 193, "name": "Melisa Maccarter", "age": 50, "address": { "number": 1494, "street": "View St.", "city": "Los Angeles" }, "interests": [ "Basketball" ], "children": [ { "name": "Yetta Maccarter", "age": null }, { "name": "Geralyn Maccarter", "age": null } ] }
+{ "cid": 194, "name": "Leslee Apking", "age": 41, "address": { "number": 8107, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ "Puzzles" ], "children": [ { "name": "Irena Apking", "age": null }, { "name": "Arla Apking", "age": null } ] }
+{ "cid": 195, "name": "Annetta Demille", "age": 17, "address": { "number": 5722, "street": "Park St.", "city": "Portland" }, "interests": [ "Bass" ], "children": [ { "name": "Natacha Demille", "age": null }, { "name": "Giuseppe Demille", "age": null }, { "name": "Kami Demille", "age": null }, { "name": "Jewell Demille", "age": null } ] }
+{ "cid": 196, "name": "Darwin Seekell", "age": null, "address": null, "interests": [ "Skiing" ], "children": [ { "name": "Kathryne Seekell", "age": null }, { "name": "Marlon Seekell", "age": null }, { "name": "Shiloh Seekell", "age": 51 } ] }
+{ "cid": 197, "name": "Garth Giannitti", "age": null, "address": null, "interests": [ "Coffee", "Cigars" ], "children": [ { "name": "Patsy Giannitti", "age": null }, { "name": "Ray Giannitti", "age": 35 }, { "name": "Kamala Giannitti", "age": 35 }, { "name": "Lauran Giannitti", "age": 25 } ] }
+{ "cid": 198, "name": "Thelma Youkers", "age": null, "address": null, "interests": [ "Basketball", "Movies", "Cooking" ], "children": [ { "name": "Shamika Youkers", "age": 28 } ] }
+{ "cid": 199, "name": "Rogelio Hannan", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Blanche Hannan", "age": null }, { "name": "Elvira Hannan", "age": null }, { "name": "Cinderella Hannan", "age": null } ] }
+{ "cid": 200, "name": "Stacey Bertran", "age": 78, "address": { "number": 9050, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ ], "children": [ { "name": "Eugenia Bertran", "age": 59 }, { "name": "Lorri Bertran", "age": 29 }, { "name": "Corrie Bertran", "age": 52 } ] }
+{ "cid": 201, "name": "Tiny Hoysradt", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Simon Hoysradt", "age": 24 } ] }
+{ "cid": 202, "name": "Evangelina Poloskey", "age": 46, "address": { "number": 8285, "street": "Main St.", "city": "Los Angeles" }, "interests": [ "Wine", "Squash" ], "children": [ { "name": "Anthony Poloskey", "age": 27 }, { "name": "Olga Poloskey", "age": 10 }, { "name": "Carmon Poloskey", "age": 13 }, { "name": "Tanja Poloskey", "age": 20 } ] }
+{ "cid": 203, "name": "Elke Mazurowski", "age": 52, "address": { "number": 9276, "street": "View St.", "city": "Mountain View" }, "interests": [ ], "children": [ { "name": "Esta Mazurowski", "age": null }, { "name": "Clarence Mazurowski", "age": 14 } ] }
+{ "cid": 204, "name": "Londa Herdt", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Marnie Herdt", "age": 47 } ] }
+{ "cid": 205, "name": "Moises Plake", "age": null, "address": null, "interests": [ "Puzzles", "Computers" ], "children": [ ] }
+{ "cid": 206, "name": "Armand Hauersperger", "age": 67, "address": { "number": 7266, "street": "Park St.", "city": "Seattle" }, "interests": [ "Wine" ], "children": [ { "name": "Charlott Hauersperger", "age": 47 }, { "name": "Kayla Hauersperger", "age": null }, { "name": "Maris Hauersperger", "age": 52 } ] }
+{ "cid": 207, "name": "Phyliss Honda", "age": 22, "address": { "number": 8387, "street": "Lake St.", "city": "Seattle" }, "interests": [ "Cooking", "Music", "Books" ], "children": [ { "name": "Bee Honda", "age": null }, { "name": "Cyril Honda", "age": null }, { "name": "Vertie Honda", "age": null } ] }
+{ "cid": 210, "name": "Jillian Roadruck", "age": null, "address": null, "interests": [ "Coffee", "Tennis" ], "children": [ { "name": "Marguerite Roadruck", "age": null }, { "name": "Ilana Roadruck", "age": null }, { "name": "Chantelle Roadruck", "age": 19 }, { "name": "Nikia Roadruck", "age": 43 } ] }
+{ "cid": 211, "name": "Kristian Knepshield", "age": null, "address": null, "interests": [ ], "children": [ ] }
+{ "cid": 212, "name": "Christi Vichi", "age": null, "address": null, "interests": [ "Squash" ], "children": [ ] }
+{ "cid": 213, "name": "Micheal Evoy", "age": 68, "address": { "number": 1219, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Skiing", "Computers", "Books", "Puzzles" ], "children": [ { "name": "Socorro Evoy", "age": null }, { "name": "Gertude Evoy", "age": 36 }, { "name": "Araceli Evoy", "age": null }, { "name": "Yasmin Evoy", "age": null } ] }
+{ "cid": 214, "name": "Louvenia Zaffalon", "age": null, "address": null, "interests": [ "Skiing", "Books" ], "children": [ ] }
+{ "cid": 215, "name": "Ashton Schadegg", "age": null, "address": null, "interests": [ "Databases", "Music" ], "children": [ { "name": "Ciara Schadegg", "age": null }, { "name": "Karisa Schadegg", "age": 11 }, { "name": "Hayden Schadegg", "age": 44 } ] }
+{ "cid": 216, "name": "Odilia Lampson", "age": null, "address": null, "interests": [ "Wine", "Databases", "Basketball" ], "children": [ { "name": "Callie Lampson", "age": null } ] }
+{ "cid": 217, "name": "Scott Fulks", "age": null, "address": null, "interests": [ "Computers" ], "children": [ ] }
+{ "cid": 218, "name": "Clarinda Stagliano", "age": 76, "address": { "number": 3258, "street": "Park St.", "city": "San Jose" }, "interests": [ "Video Games", "Cigars" ], "children": [ ] }
+{ "cid": 219, "name": "Joelle Valazquez", "age": 73, "address": { "number": 9775, "street": "Park St.", "city": "San Jose" }, "interests": [ ], "children": [ { "name": "Gene Valazquez", "age": null }, { "name": "Ilona Valazquez", "age": null } ] }
+{ "cid": 220, "name": "Soila Hannemann", "age": null, "address": null, "interests": [ "Wine", "Puzzles", "Basketball" ], "children": [ { "name": "Piper Hannemann", "age": 44 } ] }
+{ "cid": 221, "name": "Delois Fiqueroa", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Cherri Fiqueroa", "age": null } ] }
+{ "cid": 222, "name": "Malcom Bloomgren", "age": 39, "address": { "number": 4674, "street": "Hill St.", "city": "Mountain View" }, "interests": [ "Databases", "Skiing" ], "children": [ { "name": "Rosia Bloomgren", "age": null }, { "name": "Bryant Bloomgren", "age": 15 }, { "name": "Donnie Bloomgren", "age": null } ] }
+{ "cid": 223, "name": "Margurite Embelton", "age": 19, "address": { "number": 554, "street": "Oak St.", "city": "Portland" }, "interests": [ "Running", "Fishing" ], "children": [ { "name": "Sherie Embelton", "age": null }, { "name": "Monica Embelton", "age": null }, { "name": "Jeanne Embelton", "age": null }, { "name": "Santiago Embelton", "age": null } ] }
+{ "cid": 224, "name": "Rene Rowey", "age": null, "address": null, "interests": [ "Base Jumping", "Base Jumping", "Walking", "Computers" ], "children": [ { "name": "Necole Rowey", "age": 26 }, { "name": "Sharyl Rowey", "age": 20 }, { "name": "Yvone Rowey", "age": 36 } ] }
+{ "cid": 225, "name": "Shantel Drapeaux", "age": null, "address": null, "interests": [ "Databases" ], "children": [ { "name": "Felicidad Drapeaux", "age": null }, { "name": "Wanetta Drapeaux", "age": 52 }, { "name": "Louise Drapeaux", "age": 28 }, { "name": "Pat Drapeaux", "age": null } ] }
+{ "cid": 226, "name": "Debrah Deppert", "age": 62, "address": { "number": 7699, "street": "7th St.", "city": "Mountain View" }, "interests": [ "Coffee" ], "children": [ { "name": "Tonie Deppert", "age": 25 }, { "name": "Neil Deppert", "age": null } ] }
+{ "cid": 227, "name": "Carlos Skyes", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Cortney Skyes", "age": 32 } ] }
+{ "cid": 228, "name": "Donnette Brumbley", "age": null, "address": null, "interests": [ "Databases", "Music" ], "children": [ { "name": "Madlyn Brumbley", "age": null }, { "name": "Apolonia Brumbley", "age": 13 }, { "name": "Stephine Brumbley", "age": null }, { "name": "Zelma Brumbley", "age": 51 } ] }
+{ "cid": 229, "name": "Raymundo Meurin", "age": null, "address": null, "interests": [ "Bass", "Basketball", "Databases" ], "children": [ { "name": "Mariela Meurin", "age": null } ] }
+{ "cid": 230, "name": "Tobias Vicars", "age": 66, "address": { "number": 638, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ "Wine", "Walking", "Books", "Walking" ], "children": [ ] }
+{ "cid": 231, "name": "Arianne Wedlow", "age": 68, "address": { "number": 9663, "street": "7th St.", "city": "Sunnyvale" }, "interests": [ ], "children": [ { "name": "Birdie Wedlow", "age": 32 }, { "name": "Pearle Wedlow", "age": 13 }, { "name": "Jordon Wedlow", "age": 43 }, { "name": "Katherin Wedlow", "age": 18 } ] }
+{ "cid": 232, "name": "Joey Potes", "age": null, "address": null, "interests": [ "Bass", "Bass", "Base Jumping" ], "children": [ { "name": "Bobby Potes", "age": null } ] }
+{ "cid": 233, "name": "Sammy Coalter", "age": null, "address": null, "interests": [ "Fishing", "Base Jumping" ], "children": [ { "name": "Twana Coalter", "age": null }, { "name": "Nenita Coalter", "age": 30 } ] }
+{ "cid": 234, "name": "Ilana Brothern", "age": 36, "address": { "number": 4850, "street": "Lake St.", "city": "Portland" }, "interests": [ "Puzzles", "Walking", "Fishing" ], "children": [ { "name": "Shayne Brothern", "age": null }, { "name": "Phillis Brothern", "age": null } ] }
+{ "cid": 235, "name": "Orpha Craycraft", "age": null, "address": null, "interests": [ "Skiing", "Squash" ], "children": [ ] }
+{ "cid": 236, "name": "Muriel Laib", "age": 25, "address": { "number": 4481, "street": "Oak St.", "city": "San Jose" }, "interests": [ "Fishing", "Tennis" ], "children": [ { "name": "Jann Laib", "age": null }, { "name": "Lila Laib", "age": 10 }, { "name": "Elyse Laib", "age": 11 } ] }
+{ "cid": 237, "name": "Sona Hehn", "age": 47, "address": { "number": 3720, "street": "Oak St.", "city": "Portland" }, "interests": [ "Computers", "Squash", "Coffee" ], "children": [ { "name": "Marquerite Hehn", "age": null }, { "name": "Suellen Hehn", "age": 29 }, { "name": "Herb Hehn", "age": 29 } ] }
+{ "cid": 238, "name": "Marcelina Redic", "age": null, "address": null, "interests": [ "Cigars", "Cigars", "Coffee" ], "children": [ { "name": "Renate Redic", "age": null }, { "name": "Kyoko Redic", "age": null }, { "name": "Dorthey Redic", "age": null } ] }
+{ "cid": 239, "name": "Celsa Fondow", "age": null, "address": null, "interests": [ "Base Jumping", "Computers", "Cooking", "Wine" ], "children": [ ] }
+{ "cid": 241, "name": "Lesha Ambrosia", "age": 49, "address": { "number": 6133, "street": "Cedar St.", "city": "Portland" }, "interests": [ "Base Jumping", "Running" ], "children": [ { "name": "Venice Ambrosia", "age": null } ] }
+{ "cid": 242, "name": "Jerold Shabot", "age": null, "address": null, "interests": [ "Fishing", "Walking", "Walking", "Puzzles" ], "children": [ { "name": "Marie Shabot", "age": 26 } ] }
+{ "cid": 243, "name": "Love Hoftiezer", "age": 88, "address": { "number": 2491, "street": "Main St.", "city": "Portland" }, "interests": [ "Cigars", "Coffee", "Books" ], "children": [ { "name": "Kellee Hoftiezer", "age": 77 } ] }
+{ "cid": 244, "name": "Rene Shenk", "age": null, "address": null, "interests": [ "Puzzles", "Puzzles", "Skiing" ], "children": [ { "name": "Victor Shenk", "age": 28 }, { "name": "Doris Shenk", "age": null }, { "name": "Max Shenk", "age": 51 } ] }
+{ "cid": 245, "name": "Lupe Abshear", "age": 55, "address": { "number": 7269, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ ], "children": [ { "name": "Song Abshear", "age": null }, { "name": "Honey Abshear", "age": 31 } ] }
+{ "cid": 246, "name": "Kenda Heikkinen", "age": 63, "address": { "number": 8924, "street": "View St.", "city": "Mountain View" }, "interests": [ "Databases" ], "children": [ ] }
+{ "cid": 247, "name": "Minda Heron", "age": 25, "address": { "number": 1629, "street": "Hill St.", "city": "Mountain View" }, "interests": [ "Tennis" ], "children": [ ] }
+{ "cid": 249, "name": "Kiana Satiago", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Stacy Satiago", "age": null } ] }
+{ "cid": 250, "name": "Angeles Saltonstall", "age": null, "address": null, "interests": [ "Tennis", "Fishing", "Movies" ], "children": [ { "name": "Suzanna Saltonstall", "age": null } ] }
+{ "cid": 251, "name": "Janeen Galston", "age": null, "address": null, "interests": [ "Basketball", "Base Jumping" ], "children": [ ] }
+{ "cid": 252, "name": "Almeda Charity", "age": 19, "address": { "number": 5553, "street": "View St.", "city": "San Jose" }, "interests": [ ], "children": [ { "name": "Rosia Charity", "age": null } ] }
+{ "cid": 254, "name": "Jeanice Longanecker", "age": 74, "address": { "number": 2613, "street": "Oak St.", "city": "San Jose" }, "interests": [ "Books", "Base Jumping" ], "children": [ ] }
+{ "cid": 255, "name": "Cherri Piegaro", "age": 64, "address": { "number": 3802, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ ], "children": [ { "name": "Elwood Piegaro", "age": null } ] }
+{ "cid": 256, "name": "Chester Rosenberg", "age": 46, "address": { "number": 8673, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Basketball" ], "children": [ { "name": "Gemma Rosenberg", "age": null }, { "name": "Marty Rosenberg", "age": null } ] }
+{ "cid": 257, "name": "Altha Jastrzebski", "age": 21, "address": { "number": 4405, "street": "Lake St.", "city": "Portland" }, "interests": [ "Puzzles" ], "children": [ ] }
+{ "cid": 258, "name": "Florentina Hense", "age": 20, "address": { "number": 8495, "street": "View St.", "city": "Portland" }, "interests": [ ], "children": [ { "name": "Noelle Hense", "age": null }, { "name": "Roxann Hense", "age": null } ] }
+{ "cid": 259, "name": "Aurelio Darrigo", "age": 45, "address": { "number": 1114, "street": "Park St.", "city": "San Jose" }, "interests": [ "Cooking", "Running" ], "children": [ { "name": "Leonard Darrigo", "age": 22 }, { "name": "Aron Darrigo", "age": null }, { "name": "Pamelia Darrigo", "age": 14 } ] }
+{ "cid": 260, "name": "Hedwig Caminero", "age": 81, "address": { "number": 4305, "street": "7th St.", "city": "Portland" }, "interests": [ "Video Games", "Databases" ], "children": [ { "name": "Hal Caminero", "age": null }, { "name": "Cierra Caminero", "age": 32 } ] }
+{ "cid": 263, "name": "Mellisa Machalek", "age": null, "address": null, "interests": [ "Bass", "Coffee", "Skiing" ], "children": [ ] }
+{ "cid": 264, "name": "Leon Yoshizawa", "age": 81, "address": { "number": 608, "street": "Washington St.", "city": "San Jose" }, "interests": [ "Running", "Books", "Running" ], "children": [ { "name": "Carmela Yoshizawa", "age": 34 } ] }
+{ "cid": 265, "name": "Donte Stempien", "age": 25, "address": { "number": 3882, "street": "Oak St.", "city": "Los Angeles" }, "interests": [ "Wine", "Books" ], "children": [ ] }
+{ "cid": 266, "name": "Carlee Friddle", "age": 74, "address": { "number": 6538, "street": "Main St.", "city": "San Jose" }, "interests": [ "Databases" ], "children": [ { "name": "Candie Friddle", "age": null }, { "name": "Zoila Friddle", "age": 59 } ] }
+{ "cid": 267, "name": "Renay Huddelston", "age": 68, "address": { "number": 1939, "street": "Washington St.", "city": "Mountain View" }, "interests": [ "Wine", "Base Jumping" ], "children": [ { "name": "Colene Huddelston", "age": null } ] }
+{ "cid": 268, "name": "Fernando Pingel", "age": null, "address": null, "interests": [ "Computers", "Tennis", "Books" ], "children": [ { "name": "Latrice Pingel", "age": null }, { "name": "Wade Pingel", "age": 13 }, { "name": "Christal Pingel", "age": null }, { "name": "Melania Pingel", "age": null } ] }
+{ "cid": 269, "name": "Dante Sharko", "age": null, "address": null, "interests": [ "Base Jumping" ], "children": [ { "name": "Ahmad Sharko", "age": 34 }, { "name": "Mona Sharko", "age": null }, { "name": "Stephaine Sharko", "age": 42 }, { "name": "Adrianna Sharko", "age": null } ] }
+{ "cid": 270, "name": "Lavon Ascenzo", "age": null, "address": null, "interests": [ "Books", "Skiing" ], "children": [ ] }
+{ "cid": 271, "name": "Carey Ronin", "age": 44, "address": { "number": 8141, "street": "Oak St.", "city": "Mountain View" }, "interests": [ "Cigars", "Video Games" ], "children": [ { "name": "Lonny Ronin", "age": null }, { "name": "Armanda Ronin", "age": null } ] }
+{ "cid": 272, "name": "Frederick Valla", "age": 15, "address": { "number": 6805, "street": "Lake St.", "city": "San Jose" }, "interests": [ "Video Games" ], "children": [ { "name": "Carroll Valla", "age": null } ] }
+{ "cid": 273, "name": "Corrinne Seaquist", "age": 24, "address": { "number": 6712, "street": "7th St.", "city": "Portland" }, "interests": [ "Puzzles", "Coffee", "Wine" ], "children": [ { "name": "Mignon Seaquist", "age": null }, { "name": "Leo Seaquist", "age": null } ] }
+{ "cid": 274, "name": "Claude Harral", "age": null, "address": null, "interests": [ "Squash", "Bass", "Cooking" ], "children": [ { "name": "Archie Harral", "age": null }, { "name": "Royal Harral", "age": null } ] }
+{ "cid": 275, "name": "Natalie Ifeanyi", "age": null, "address": null, "interests": [ ], "children": [ ] }
+{ "cid": 276, "name": "Denyse Groth", "age": 81, "address": { "number": 6825, "street": "Main St.", "city": "Sunnyvale" }, "interests": [ "Databases", "Fishing", "Movies" ], "children": [ { "name": "Marilee Groth", "age": 12 }, { "name": "Lyla Groth", "age": 46 }, { "name": "Sarah Groth", "age": null } ] }
+{ "cid": 277, "name": "Malena Smock", "age": null, "address": null, "interests": [ "Running", "Base Jumping" ], "children": [ { "name": "Inocencia Smock", "age": 50 }, { "name": "Cleveland Smock", "age": null } ] }
+{ "cid": 278, "name": "Deb Nicole", "age": 59, "address": { "number": 9003, "street": "Park St.", "city": "Seattle" }, "interests": [ "Books", "Computers", "Walking", "Cooking" ], "children": [ { "name": "Len Nicole", "age": null } ] }
+{ "cid": 279, "name": "Saundra Croan", "age": null, "address": null, "interests": [ "Movies" ], "children": [ { "name": "Jena Croan", "age": 37 }, { "name": "Sarai Croan", "age": null }, { "name": "Junita Croan", "age": null }, { "name": "Ferdinand Croan", "age": 43 } ] }
+{ "cid": 280, "name": "Marlo Maung", "age": null, "address": null, "interests": [ "Movies" ], "children": [ { "name": "Harold Maung", "age": null } ] }
+{ "cid": 282, "name": "Emelda Dawood", "age": 32, "address": { "number": 5261, "street": "View St.", "city": "Portland" }, "interests": [ ], "children": [ { "name": "Venus Dawood", "age": 12 }, { "name": "Gertrude Dawood", "age": null }, { "name": "Yen Dawood", "age": null }, { "name": "Theresa Dawood", "age": 16 } ] }
+{ "cid": 283, "name": "Pilar Fritts", "age": null, "address": null, "interests": [ "Tennis" ], "children": [ { "name": "Jeneva Fritts", "age": null }, { "name": "Gail Fritts", "age": 25 } ] }
+{ "cid": 285, "name": "Edgar Farlin", "age": 75, "address": { "number": 3833, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ "Coffee", "Databases" ], "children": [ { "name": "Stefanie Farlin", "age": 60 }, { "name": "Catina Farlin", "age": null }, { "name": "Lizzie Farlin", "age": null }, { "name": "Beau Farlin", "age": null } ] }
+{ "cid": 286, "name": "Tara Sioma", "age": 18, "address": { "number": 9425, "street": "Cedar St.", "city": "Mountain View" }, "interests": [ "Fishing" ], "children": [ { "name": "Dawna Sioma", "age": null }, { "name": "Jeanne Sioma", "age": null } ] }
+{ "cid": 288, "name": "Sharice Bachicha", "age": null, "address": null, "interests": [ ], "children": [ ] }
+{ "cid": 289, "name": "Clarence Milette", "age": 16, "address": { "number": 3778, "street": "Oak St.", "city": "Seattle" }, "interests": [ "Books", "Base Jumping", "Music" ], "children": [ ] }
+{ "cid": 290, "name": "Kimberly Gullatte", "age": 51, "address": { "number": 4130, "street": "Park St.", "city": "San Jose" }, "interests": [ "Running", "Squash", "Databases" ], "children": [ { "name": "Micheal Gullatte", "age": null }, { "name": "Estrella Gullatte", "age": 40 }, { "name": "Corrine Gullatte", "age": null }, { "name": "Ward Gullatte", "age": null } ] }
+{ "cid": 291, "name": "Svetlana Moone", "age": null, "address": null, "interests": [ "Skiing", "Computers", "Running", "Walking" ], "children": [ { "name": "Emelina Moone", "age": null }, { "name": "Candi Moone", "age": null } ] }
+{ "cid": 292, "name": "Mariana Cosselman", "age": null, "address": null, "interests": [ "Squash" ], "children": [ { "name": "Madge Cosselman", "age": 43 } ] }
+{ "cid": 293, "name": "Terresa Hofstetter", "age": 15, "address": { "number": 3338, "street": "Lake St.", "city": "Los Angeles" }, "interests": [ "Computers", "Running", "Cigars", "Fishing" ], "children": [ { "name": "Hubert Hofstetter", "age": null }, { "name": "Jolie Hofstetter", "age": null } ] }
+{ "cid": 294, "name": "Foster Salimi", "age": 79, "address": { "number": 8439, "street": "Cedar St.", "city": "Sunnyvale" }, "interests": [ ], "children": [ { "name": "Pei Salimi", "age": null } ] }
+{ "cid": 295, "name": "Guillermina Florek", "age": 61, "address": { "number": 3704, "street": "Washington St.", "city": "Mountain View" }, "interests": [ "Movies", "Books" ], "children": [ { "name": "Donnie Florek", "age": null }, { "name": "Jeannetta Florek", "age": 38 }, { "name": "Leigha Florek", "age": null }, { "name": "Zenobia Florek", "age": 10 } ] }
+{ "cid": 296, "name": "Doreen Kea", "age": 89, "address": { "number": 7034, "street": "Cedar St.", "city": "Sunnyvale" }, "interests": [ "Movies" ], "children": [ { "name": "Lyndsay Kea", "age": 68 }, { "name": "Trena Kea", "age": 18 } ] }
+{ "cid": 297, "name": "Adeline Frierson", "age": null, "address": null, "interests": [ "Coffee", "Computers", "Fishing" ], "children": [ { "name": "Marci Frierson", "age": null }, { "name": "Rolanda Frierson", "age": null }, { "name": "Del Frierson", "age": null } ] }
+{ "cid": 298, "name": "Brittny Christin", "age": null, "address": null, "interests": [ "Databases", "Video Games" ], "children": [ { "name": "Hilario Christin", "age": null }, { "name": "Clarine Christin", "age": null } ] }
+{ "cid": 299, "name": "Jacob Wainman", "age": 76, "address": { "number": 4551, "street": "Washington St.", "city": "Portland" }, "interests": [ "Base Jumping", "Wine", "Coffee" ], "children": [ { "name": "Abram Wainman", "age": 28 }, { "name": "Ramonita Wainman", "age": 18 }, { "name": "Sheryll Wainman", "age": null } ] }
+{ "cid": 300, "name": "Garret Colgrove", "age": 85, "address": { "number": 9937, "street": "Hill St.", "city": "Sunnyvale" }, "interests": [ "Base Jumping", "Puzzles", "Fishing" ], "children": [ { "name": "Janna Colgrove", "age": null }, { "name": "Jerilyn Colgrove", "age": 35 } ] }
+{ "cid": 301, "name": "Cherry Steenwyk", "age": 88, "address": { "number": 4138, "street": "Lake St.", "city": "San Jose" }, "interests": [ "Movies" ], "children": [ { "name": "Toccara Steenwyk", "age": 66 }, { "name": "Tari Steenwyk", "age": null }, { "name": "Lawanna Steenwyk", "age": null }, { "name": "Ossie Steenwyk", "age": 26 } ] }
+{ "cid": 302, "name": "Rosalie Laderer", "age": null, "address": null, "interests": [ "Tennis", "Movies", "Movies" ], "children": [ { "name": "Moriah Laderer", "age": null }, { "name": "Liana Laderer", "age": 21 }, { "name": "Genia Laderer", "age": 45 } ] }
+{ "cid": 303, "name": "Michel Bayird", "age": 37, "address": { "number": 7939, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ ], "children": [ { "name": "Shan Bayird", "age": 12 } ] }
+{ "cid": 304, "name": "Francine Reddin", "age": 39, "address": { "number": 9392, "street": "Hill St.", "city": "Seattle" }, "interests": [ "Music", "Base Jumping" ], "children": [ { "name": "Millicent Reddin", "age": null } ] }
+{ "cid": 305, "name": "Tuyet Leinbach", "age": null, "address": null, "interests": [ "Puzzles", "Walking" ], "children": [ ] }
+{ "cid": 306, "name": "Laurie Tuff", "age": null, "address": null, "interests": [ "Computers", "Base Jumping", "Bass", "Basketball" ], "children": [ { "name": "Sharie Tuff", "age": null }, { "name": "Ollie Tuff", "age": 53 }, { "name": "Gonzalo Tuff", "age": null }, { "name": "Thomas Tuff", "age": null } ] }
+{ "cid": 307, "name": "Abraham Lanphear", "age": 20, "address": { "number": 7552, "street": "Washington St.", "city": "San Jose" }, "interests": [ "Video Games" ], "children": [ { "name": "Toccara Lanphear", "age": null }, { "name": "Milly Lanphear", "age": null } ] }
+{ "cid": 308, "name": "Solomon Schwenke", "age": null, "address": null, "interests": [ "Puzzles" ], "children": [ { "name": "Gertrude Schwenke", "age": null }, { "name": "Marcell Schwenke", "age": 41 }, { "name": "Shalon Schwenke", "age": null } ] }
+{ "cid": 309, "name": "Lise Baiz", "age": 46, "address": { "number": 352, "street": "Oak St.", "city": "San Jose" }, "interests": [ "Bass", "Squash" ], "children": [ { "name": "Alisa Baiz", "age": 18 }, { "name": "Elidia Baiz", "age": 28 }, { "name": "Ray Baiz", "age": 19 } ] }
+{ "cid": 311, "name": "Ria Haflett", "age": 14, "address": { "number": 9513, "street": "Park St.", "city": "Los Angeles" }, "interests": [ "Walking" ], "children": [ { "name": "Jimmie Haflett", "age": null }, { "name": "Dario Haflett", "age": null }, { "name": "Robbyn Haflett", "age": null } ] }
+{ "cid": 312, "name": "Epifania Chorney", "age": 62, "address": { "number": 9749, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ "Wine", "Puzzles", "Tennis" ], "children": [ { "name": "Lizeth Chorney", "age": 22 } ] }
+{ "cid": 313, "name": "Lasandra Raigosa", "age": null, "address": null, "interests": [ "Walking", "Walking" ], "children": [ { "name": "Lanelle Raigosa", "age": null } ] }
+{ "cid": 314, "name": "Gwendolyn Abeb", "age": 85, "address": { "number": 3977, "street": "Hill St.", "city": "Seattle" }, "interests": [ "Basketball", "Music", "Squash", "Walking" ], "children": [ { "name": "Aurelia Abeb", "age": 14 }, { "name": "Young Abeb", "age": null }, { "name": "Shay Abeb", "age": null }, { "name": "Lavina Abeb", "age": 15 } ] }
+{ "cid": 315, "name": "Kallie Eiselein", "age": null, "address": null, "interests": [ "Computers", "Tennis" ], "children": [ ] }
+{ "cid": 316, "name": "Patrina Whitting", "age": 74, "address": { "number": 4772, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ "Music", "Video Games", "Bass" ], "children": [ { "name": "Rubye Whitting", "age": null } ] }
+{ "cid": 317, "name": "Zona Caffarel", "age": 52, "address": { "number": 9419, "street": "Cedar St.", "city": "Seattle" }, "interests": [ "Tennis", "Coffee" ], "children": [ { "name": "Cortez Caffarel", "age": null } ] }
+{ "cid": 318, "name": "Shaunna Royal", "age": 86, "address": { "number": 8681, "street": "7th St.", "city": "San Jose" }, "interests": [ ], "children": [ { "name": "Shantell Royal", "age": 37 }, { "name": "Shalon Royal", "age": 50 }, { "name": "Chung Royal", "age": 26 } ] }
+{ "cid": 319, "name": "Ashlie Rott", "age": 42, "address": { "number": 366, "street": "Cedar St.", "city": "Mountain View" }, "interests": [ "Computers", "Cooking", "Databases" ], "children": [ ] }
+{ "cid": 320, "name": "Charley Hermenegildo", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Melda Hermenegildo", "age": 51 }, { "name": "Lashon Hermenegildo", "age": null } ] }
+{ "cid": 322, "name": "Jaclyn Ettl", "age": 83, "address": { "number": 4500, "street": "Main St.", "city": "Sunnyvale" }, "interests": [ "Databases", "Skiing" ], "children": [ { "name": "Noah Ettl", "age": 30 }, { "name": "Kesha Ettl", "age": null } ] }
+{ "cid": 323, "name": "Rebeca Grisostomo", "age": 26, "address": { "number": 399, "street": "View St.", "city": "Portland" }, "interests": [ "Music" ], "children": [ { "name": "Iva Grisostomo", "age": 12 }, { "name": "Ha Grisostomo", "age": null }, { "name": "Lorna Grisostomo", "age": null } ] }
+{ "cid": 324, "name": "Wendolyn Centorino", "age": null, "address": null, "interests": [ ], "children": [ ] }
+{ "cid": 325, "name": "Ai Tarleton", "age": null, "address": null, "interests": [ "Coffee", "Music" ], "children": [ { "name": "Risa Tarleton", "age": 24 }, { "name": "Leonila Tarleton", "age": null }, { "name": "Thomasina Tarleton", "age": null } ] }
+{ "cid": 326, "name": "Tad Tellers", "age": null, "address": null, "interests": [ "Books", "Tennis", "Base Jumping" ], "children": [ { "name": "Fannie Tellers", "age": null } ] }
+{ "cid": 327, "name": "Minnie Scali", "age": null, "address": null, "interests": [ "Cooking", "Squash", "Skiing" ], "children": [ { "name": "Jalisa Scali", "age": null }, { "name": "Preston Scali", "age": null }, { "name": "Stephani Scali", "age": 47 }, { "name": "Candra Scali", "age": null } ] }
+{ "cid": 328, "name": "Mallory Sheffey", "age": 27, "address": { "number": 8532, "street": "Washington St.", "city": "Mountain View" }, "interests": [ "Cooking" ], "children": [ { "name": "Regan Sheffey", "age": 14 } ] }
+{ "cid": 330, "name": "Noma Tollefsen", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Melody Tollefsen", "age": 45 }, { "name": "Caridad Tollefsen", "age": 15 } ] }
+{ "cid": 331, "name": "Willena Provenza", "age": 43, "address": { "number": 6742, "street": "Main St.", "city": "Portland" }, "interests": [ "Basketball" ], "children": [ { "name": "Alesha Provenza", "age": 32 }, { "name": "Marty Provenza", "age": null }, { "name": "Lindy Provenza", "age": 21 }, { "name": "Junita Provenza", "age": null } ] }
+{ "cid": 332, "name": "Malcom Cafasso", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Marie Cafasso", "age": null }, { "name": "Asley Cafasso", "age": 38 } ] }
+{ "cid": 333, "name": "Conchita Olivera", "age": 37, "address": { "number": 8519, "street": "Oak St.", "city": "Mountain View" }, "interests": [ "Base Jumping" ], "children": [ { "name": "Trenton Olivera", "age": null }, { "name": "Shin Olivera", "age": 26 }, { "name": "Everett Olivera", "age": 15 }, { "name": "Shera Olivera", "age": 20 } ] }
+{ "cid": 335, "name": "Odessa Dammeyer", "age": 18, "address": { "number": 6828, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [ "Basketball", "Bass", "Cigars" ], "children": [ { "name": "Lindsey Dammeyer", "age": null } ] }
+{ "cid": 336, "name": "Jalisa Talamantez", "age": 78, "address": { "number": 9902, "street": "Lake St.", "city": "Mountain View" }, "interests": [ "Video Games", "Squash" ], "children": [ ] }
+{ "cid": 337, "name": "Kay Durney", "age": 52, "address": { "number": 4203, "street": "View St.", "city": "Seattle" }, "interests": [ "Walking" ], "children": [ { "name": "Velia Durney", "age": 38 }, { "name": "Erin Durney", "age": null } ] }
+{ "cid": 338, "name": "Dorthey Roncskevitz", "age": 38, "address": { "number": 4366, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ "Computers" ], "children": [ { "name": "Mindy Roncskevitz", "age": null } ] }
+{ "cid": 339, "name": "Sharonda Catalino", "age": 15, "address": { "number": 7616, "street": "Washington St.", "city": "Portland" }, "interests": [ ], "children": [ { "name": "Lorine Catalino", "age": null } ] }
+{ "cid": 340, "name": "Erick Faiola", "age": null, "address": null, "interests": [ "Coffee" ], "children": [ { "name": "Marquita Faiola", "age": null }, { "name": "Tasia Faiola", "age": null }, { "name": "Micheal Faiola", "age": 24 }, { "name": "Salvatore Faiola", "age": null } ] }
+{ "cid": 343, "name": "Kaylee Ozaine", "age": 78, "address": { "number": 3367, "street": "Washington St.", "city": "Seattle" }, "interests": [ ], "children": [ { "name": "Darwin Ozaine", "age": 35 }, { "name": "Anne Ozaine", "age": 13 }, { "name": "Kenneth Ozaine", "age": null }, { "name": "Pat Ozaine", "age": 53 } ] }
+{ "cid": 346, "name": "Elden Choma", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Valorie Choma", "age": null }, { "name": "Leslee Choma", "age": null } ] }
+{ "cid": 347, "name": "Patrick Feighan", "age": 34, "address": { "number": 7613, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [ "Puzzles", "Books" ], "children": [ { "name": "Madaline Feighan", "age": null } ] }
+{ "cid": 348, "name": "Matthew Pantaleo", "age": 80, "address": { "number": 9782, "street": "Washington St.", "city": "Seattle" }, "interests": [ ], "children": [ { "name": "Faviola Pantaleo", "age": null }, { "name": "Yang Pantaleo", "age": null }, { "name": "Christopher Pantaleo", "age": null }, { "name": "Jacqui Pantaleo", "age": 58 } ] }
+{ "cid": 349, "name": "Cristine Hila", "age": null, "address": null, "interests": [ "Books" ], "children": [ { "name": "Nyla Hila", "age": 51 } ] }
+{ "cid": 352, "name": "Bonny Sischo", "age": null, "address": null, "interests": [ "Bass", "Movies", "Computers" ], "children": [ { "name": "Judith Sischo", "age": 43 }, { "name": "Adeline Sischo", "age": null }, { "name": "Dayna Sischo", "age": null } ] }
+{ "cid": 353, "name": "Melody Bernas", "age": 76, "address": { "number": 6783, "street": "Main St.", "city": "San Jose" }, "interests": [ "Base Jumping" ], "children": [ { "name": "Kristel Bernas", "age": 45 }, { "name": "Clorinda Bernas", "age": 10 }, { "name": "Natosha Bernas", "age": null } ] }
+{ "cid": 354, "name": "Marian Munzell", "age": 73, "address": { "number": 4504, "street": "Oak St.", "city": "San Jose" }, "interests": [ "Fishing", "Puzzles" ], "children": [ ] }
+{ "cid": 355, "name": "Elois Leckband", "age": null, "address": null, "interests": [ "Skiing", "Wine" ], "children": [ ] }
+{ "cid": 356, "name": "Pearlene Sakumoto", "age": 22, "address": { "number": 5895, "street": "7th St.", "city": "San Jose" }, "interests": [ "Computers", "Bass", "Base Jumping", "Coffee" ], "children": [ ] }
+{ "cid": 357, "name": "Dario Lobach", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Kendall Lobach", "age": 37 } ] }
+{ "cid": 358, "name": "Fredricka Krum", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Darrick Krum", "age": null }, { "name": "Julieann Krum", "age": null }, { "name": "Sun Krum", "age": null }, { "name": "Rosamaria Krum", "age": 16 } ] }
+{ "cid": 360, "name": "Billye Grumet", "age": 82, "address": { "number": 7052, "street": "Main St.", "city": "Portland" }, "interests": [ "Coffee" ], "children": [ { "name": "Linnea Grumet", "age": null }, { "name": "Charline Grumet", "age": 67 } ] }
+{ "cid": 361, "name": "Angela Lacki", "age": 35, "address": { "number": 9710, "street": "Hill St.", "city": "Seattle" }, "interests": [ "Skiing" ], "children": [ ] }
+{ "cid": 362, "name": "Alta Bantug", "age": null, "address": null, "interests": [ "Computers" ], "children": [ ] }
+{ "cid": 363, "name": "Merlene Hoying", "age": 25, "address": { "number": 2105, "street": "Cedar St.", "city": "Portland" }, "interests": [ "Squash", "Squash", "Music" ], "children": [ { "name": "Andrew Hoying", "age": 10 } ] }
+{ "cid": 364, "name": "Joni Dazey", "age": 14, "address": { "number": 1237, "street": "Oak St.", "city": "Mountain View" }, "interests": [ ], "children": [ { "name": "Kraig Dazey", "age": null } ] }
+{ "cid": 366, "name": "Rosia Wenzinger", "age": null, "address": null, "interests": [ ], "children": [ ] }
+{ "cid": 367, "name": "Cassondra Fabiani", "age": null, "address": null, "interests": [ "Squash", "Tennis" ], "children": [ { "name": "Evia Fabiani", "age": null }, { "name": "Chaya Fabiani", "age": null }, { "name": "Sherman Fabiani", "age": null }, { "name": "Kathi Fabiani", "age": 54 } ] }
+{ "cid": 368, "name": "Tequila Scandalios", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Nilsa Scandalios", "age": null }, { "name": "Kaye Scandalios", "age": 23 }, { "name": "Angelo Scandalios", "age": 24 } ] }
+{ "cid": 369, "name": "Nickole Dory", "age": 10, "address": { "number": 4761, "street": "View St.", "city": "Portland" }, "interests": [ "Walking", "Cooking" ], "children": [ { "name": "Annmarie Dory", "age": null }, { "name": "Michele Dory", "age": null }, { "name": "Annamae Dory", "age": null }, { "name": "Flora Dory", "age": null } ] }
+{ "cid": 370, "name": "Shonta Furby", "age": 18, "address": { "number": 5792, "street": "Cedar St.", "city": "Mountain View" }, "interests": [ "Databases" ], "children": [ { "name": "Raleigh Furby", "age": null }, { "name": "Britta Furby", "age": null }, { "name": "Gay Furby", "age": null }, { "name": "Elenor Furby", "age": null } ] }
+{ "cid": 371, "name": "Agatha Tensley", "age": 13, "address": { "number": 1810, "street": "Hill St.", "city": "San Jose" }, "interests": [ "Bass", "Running", "Movies" ], "children": [ { "name": "Launa Tensley", "age": null } ] }
+{ "cid": 372, "name": "Zena Keglovic", "age": 22, "address": { "number": 7675, "street": "Park St.", "city": "Sunnyvale" }, "interests": [ "Basketball", "Wine" ], "children": [ ] }
+{ "cid": 373, "name": "Heather Seward", "age": null, "address": null, "interests": [ "Basketball" ], "children": [ { "name": "Glinda Seward", "age": 59 }, { "name": "Maribeth Seward", "age": null }, { "name": "Teofila Seward", "age": null }, { "name": "Clemencia Seward", "age": 38 } ] }
+{ "cid": 374, "name": "Clair Quinn", "age": null, "address": null, "interests": [ "Walking", "Books" ], "children": [ { "name": "Wesley Quinn", "age": 17 }, { "name": "Maren Quinn", "age": 50 }, { "name": "Ila Quinn", "age": 43 }, { "name": "Casie Quinn", "age": null } ] }
+{ "cid": 375, "name": "Chia Sagaser", "age": 15, "address": { "number": 6025, "street": "Park St.", "city": "Mountain View" }, "interests": [ "Skiing" ], "children": [ { "name": "Garnet Sagaser", "age": null }, { "name": "Mario Sagaser", "age": null }, { "name": "Sun Sagaser", "age": null } ] }
+{ "cid": 376, "name": "Jeffrey Hegarty", "age": null, "address": null, "interests": [ "Puzzles" ], "children": [ { "name": "April Hegarty", "age": null }, { "name": "Wilbur Hegarty", "age": null }, { "name": "Hanh Hegarty", "age": null } ] }
+{ "cid": 377, "name": "Zona Klint", "age": 22, "address": { "number": 6320, "street": "Hill St.", "city": "Sunnyvale" }, "interests": [ "Puzzles" ], "children": [ { "name": "Evie Klint", "age": null }, { "name": "Sharyl Klint", "age": 11 }, { "name": "Joaquina Klint", "age": 11 }, { "name": "Doloris Klint", "age": 11 } ] }
+{ "cid": 378, "name": "Melany Matias", "age": 10, "address": { "number": 8838, "street": "Main St.", "city": "Seattle" }, "interests": [ "Coffee", "Tennis", "Bass" ], "children": [ { "name": "Earnestine Matias", "age": null }, { "name": "Lore Matias", "age": null } ] }
+{ "cid": 379, "name": "Penney Huslander", "age": 58, "address": { "number": 6919, "street": "7th St.", "city": "Portland" }, "interests": [ "Cooking", "Running" ], "children": [ { "name": "Magaret Huslander", "age": null }, { "name": "Dodie Huslander", "age": 14 } ] }
+{ "cid": 380, "name": "Silva Purdue", "age": 33, "address": { "number": 1759, "street": "7th St.", "city": "Portland" }, "interests": [ "Music", "Squash" ], "children": [ { "name": "Marshall Purdue", "age": null }, { "name": "Yuki Purdue", "age": null }, { "name": "Val Purdue", "age": 12 }, { "name": "Dominica Purdue", "age": null } ] }
+{ "cid": 381, "name": "Kassandra Ereth", "age": null, "address": null, "interests": [ "Base Jumping", "Base Jumping", "Databases", "Walking" ], "children": [ { "name": "Angelina Ereth", "age": 46 }, { "name": "Tristan Ereth", "age": null }, { "name": "Johnny Ereth", "age": null } ] }
+{ "cid": 383, "name": "Marty Castine", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Nakisha Castine", "age": 40 }, { "name": "Mina Castine", "age": null }, { "name": "Katrice Castine", "age": 56 }, { "name": "Reuben Castine", "age": null } ] }
+{ "cid": 385, "name": "Jody Favaron", "age": 73, "address": { "number": 4724, "street": "7th St.", "city": "Sunnyvale" }, "interests": [ "Fishing" ], "children": [ { "name": "Elane Favaron", "age": 47 }, { "name": "Katherine Favaron", "age": 38 } ] }
+{ "cid": 386, "name": "Mao Gradowski", "age": 36, "address": { "number": 5116, "street": "Washington St.", "city": "Mountain View" }, "interests": [ "Computers", "Fishing" ], "children": [ { "name": "Jeneva Gradowski", "age": null }, { "name": "Thu Gradowski", "age": 22 }, { "name": "Daphine Gradowski", "age": null }, { "name": "Providencia Gradowski", "age": null } ] }
+{ "cid": 387, "name": "Leonard Mabie", "age": 33, "address": { "number": 6703, "street": "View St.", "city": "Mountain View" }, "interests": [ "Bass", "Running", "Walking" ], "children": [ { "name": "Jone Mabie", "age": 16 }, { "name": "Claire Mabie", "age": null }, { "name": "Larraine Mabie", "age": null }, { "name": "Corrina Mabie", "age": null } ] }
+{ "cid": 389, "name": "Loraine Morfee", "age": 72, "address": { "number": 2945, "street": "Lake St.", "city": "Seattle" }, "interests": [ "Wine", "Walking" ], "children": [ { "name": "Berry Morfee", "age": 30 } ] }
+{ "cid": 390, "name": "Shera Cung", "age": 69, "address": { "number": 5850, "street": "Hill St.", "city": "San Jose" }, "interests": [ "Fishing", "Computers", "Cigars", "Base Jumping" ], "children": [ { "name": "Lenore Cung", "age": 20 } ] }
+{ "cid": 391, "name": "Lynn Gregory", "age": 51, "address": { "number": 1249, "street": "Hill St.", "city": "San Jose" }, "interests": [ ], "children": [ { "name": "Jeannine Gregory", "age": null }, { "name": "Jaymie Gregory", "age": null }, { "name": "Lorrine Gregory", "age": 37 } ] }
+{ "cid": 392, "name": "Isiah Nussbaumer", "age": null, "address": null, "interests": [ "Squash" ], "children": [ ] }
+{ "cid": 393, "name": "Rossana Monton", "age": 34, "address": { "number": 4490, "street": "Main St.", "city": "Portland" }, "interests": [ "Skiing", "Base Jumping" ], "children": [ { "name": "Glayds Monton", "age": null }, { "name": "Lily Monton", "age": null }, { "name": "Raina Monton", "age": null }, { "name": "Hilma Monton", "age": null } ] }
+{ "cid": 394, "name": "Lizette Roux", "age": 57, "address": { "number": 458, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ "Bass", "Books" ], "children": [ { "name": "Doloris Roux", "age": null } ] }
+{ "cid": 395, "name": "Bob Layman", "age": 61, "address": { "number": 3646, "street": "Washington St.", "city": "Los Angeles" }, "interests": [ ], "children": [ ] }
+{ "cid": 396, "name": "Delfina Calcara", "age": null, "address": null, "interests": [ "Base Jumping" ], "children": [ { "name": "Sybil Calcara", "age": null } ] }
+{ "cid": 397, "name": "Blake Kealy", "age": 34, "address": { "number": 2156, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [ "Databases", "Wine", "Cigars" ], "children": [ { "name": "Lorenza Kealy", "age": null }, { "name": "Beula Kealy", "age": 15 }, { "name": "Kristofer Kealy", "age": null }, { "name": "Shayne Kealy", "age": null } ] }
+{ "cid": 398, "name": "Piedad Paranada", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Claribel Paranada", "age": 22 }, { "name": "Lincoln Paranada", "age": null }, { "name": "Cecilia Paranada", "age": null } ] }
+{ "cid": 399, "name": "Myra Millwee", "age": null, "address": null, "interests": [ "Tennis", "Running", "Tennis" ], "children": [ { "name": "Gaye Millwee", "age": null } ] }
+{ "cid": 400, "name": "Jeffery Maresco", "age": null, "address": null, "interests": [ "Coffee", "Bass" ], "children": [ ] }
+{ "cid": 401, "name": "Moises Jago", "age": 27, "address": { "number": 3773, "street": "Main St.", "city": "San Jose" }, "interests": [ "Music" ], "children": [ { "name": "Shoshana Jago", "age": null }, { "name": "Juliet Jago", "age": null }, { "name": "Berneice Jago", "age": 13 } ] }
+{ "cid": 402, "name": "Terrilyn Shinall", "age": null, "address": null, "interests": [ "Computers", "Skiing", "Music" ], "children": [ { "name": "Minh Shinall", "age": null }, { "name": "Diedre Shinall", "age": 22 } ] }
+{ "cid": 403, "name": "Kayleigh Houey", "age": null, "address": null, "interests": [ "Fishing", "Music" ], "children": [ { "name": "Ta Houey", "age": null }, { "name": "Ayana Houey", "age": null }, { "name": "Dominique Houey", "age": null }, { "name": "Denise Houey", "age": 48 } ] }
+{ "cid": 404, "name": "Harriette Abo", "age": null, "address": null, "interests": [ "Walking", "Running" ], "children": [ ] }
+{ "cid": 405, "name": "Shawnda Landborg", "age": 73, "address": { "number": 2396, "street": "Hill St.", "city": "Mountain View" }, "interests": [ ], "children": [ { "name": "Cherrie Landborg", "age": 10 } ] }
+{ "cid": 406, "name": "Addie Mandez", "age": null, "address": null, "interests": [ "Tennis", "Cigars", "Books" ], "children": [ { "name": "Rosendo Mandez", "age": 34 } ] }
+{ "cid": 407, "name": "Bebe Cotney", "age": null, "address": null, "interests": [ "Books", "Tennis" ], "children": [ { "name": "Daren Cotney", "age": null }, { "name": "Lady Cotney", "age": 48 } ] }
+{ "cid": 408, "name": "Ava Zornes", "age": null, "address": null, "interests": [ "Music" ], "children": [ ] }
+{ "cid": 410, "name": "Jennie Longhenry", "age": 82, "address": { "number": 7427, "street": "Main St.", "city": "San Jose" }, "interests": [ ], "children": [ { "name": "Charles Longhenry", "age": 61 }, { "name": "Faviola Longhenry", "age": 25 }, { "name": "Darline Longhenry", "age": null }, { "name": "Lorean Longhenry", "age": null } ] }
+{ "cid": 411, "name": "Cindi Pepin", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Fallon Pepin", "age": 39 }, { "name": "Armanda Pepin", "age": null }, { "name": "Loriann Pepin", "age": null }, { "name": "Bambi Pepin", "age": 43 } ] }
+{ "cid": 412, "name": "Devon Szalai", "age": 26, "address": { "number": 2384, "street": "Lake St.", "city": "Los Angeles" }, "interests": [ "Bass", "Books", "Books" ], "children": [ { "name": "Yolonda Szalai", "age": null }, { "name": "Denita Szalai", "age": null }, { "name": "Priscila Szalai", "age": 10 }, { "name": "Cassondra Szalai", "age": 12 } ] }
+{ "cid": 413, "name": "Maurice Landrie", "age": null, "address": null, "interests": [ "Computers", "Coffee" ], "children": [ { "name": "Gail Landrie", "age": 37 }, { "name": "Carylon Landrie", "age": null }, { "name": "Allen Landrie", "age": 16 }, { "name": "Andreas Landrie", "age": null } ] }
+{ "cid": 414, "name": "Sixta Smithheart", "age": null, "address": null, "interests": [ "Skiing", "Books", "Computers" ], "children": [ { "name": "Nicholas Smithheart", "age": null } ] }
+{ "cid": 415, "name": "Valentin Mclarney", "age": null, "address": null, "interests": [ "Squash", "Squash", "Video Games" ], "children": [ { "name": "Vanda Mclarney", "age": 17 } ] }
+{ "cid": 417, "name": "Irene Funderberg", "age": 45, "address": { "number": 8503, "street": "Hill St.", "city": "Seattle" }, "interests": [ "Music", "Skiing", "Running" ], "children": [ { "name": "Lyndia Funderberg", "age": 14 }, { "name": "Herta Funderberg", "age": null } ] }
+{ "cid": 418, "name": "Gavin Delpino", "age": null, "address": null, "interests": [ "Basketball", "Skiing", "Wine", "Fishing" ], "children": [ { "name": "Gianna Delpino", "age": null }, { "name": "Carmella Delpino", "age": 55 } ] }
+{ "cid": 419, "name": "Hector Brisbone", "age": null, "address": null, "interests": [ "Databases", "Books", "Walking", "Databases" ], "children": [ { "name": "Frederick Brisbone", "age": 17 } ] }
+{ "cid": 420, "name": "Coralie Regueira", "age": null, "address": null, "interests": [ "Books", "Tennis" ], "children": [ { "name": "Latoyia Regueira", "age": 31 }, { "name": "Obdulia Regueira", "age": 12 }, { "name": "Herlinda Regueira", "age": null } ] }
+{ "cid": 421, "name": "Rubye Dillabough", "age": 55, "address": { "number": 6980, "street": "View St.", "city": "Sunnyvale" }, "interests": [ "Squash" ], "children": [ { "name": "Hyacinth Dillabough", "age": 19 }, { "name": "Arie Dillabough", "age": null } ] }
+{ "cid": 422, "name": "Annmarie Whitcher", "age": null, "address": null, "interests": [ "Cigars" ], "children": [ { "name": "Honey Whitcher", "age": null }, { "name": "Dan Whitcher", "age": 22 } ] }
+{ "cid": 424, "name": "Camila Rightmire", "age": 25, "address": { "number": 7542, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ "Bass", "Running", "Puzzles" ], "children": [ { "name": "Donny Rightmire", "age": 14 }, { "name": "Karlene Rightmire", "age": 10 }, { "name": "Nicholas Rightmire", "age": null }, { "name": "Margareta Rightmire", "age": null } ] }
+{ "cid": 426, "name": "Agripina Philley", "age": 79, "address": { "number": 1533, "street": "Main St.", "city": "Portland" }, "interests": [ ], "children": [ { "name": "Georgianne Philley", "age": null }, { "name": "Neville Philley", "age": null }, { "name": "Brande Philley", "age": 42 }, { "name": "Tanisha Philley", "age": null } ] }
+{ "cid": 427, "name": "Janay Presutti", "age": null, "address": null, "interests": [ "Walking" ], "children": [ { "name": "Julietta Presutti", "age": null } ] }
+{ "cid": 428, "name": "Tiffany Waye", "age": null, "address": null, "interests": [ "Basketball", "Cigars" ], "children": [ { "name": "Berna Waye", "age": null }, { "name": "Kiersten Waye", "age": null }, { "name": "Romeo Waye", "age": null }, { "name": "Marvel Waye", "age": 56 } ] }
+{ "cid": 429, "name": "Eladia Scannell", "age": 20, "address": { "number": 5036, "street": "Main St.", "city": "Portland" }, "interests": [ "Skiing", "Music", "Movies" ], "children": [ ] }
+{ "cid": 430, "name": "Cari Woll", "age": 45, "address": { "number": 8226, "street": "Park St.", "city": "San Jose" }, "interests": [ "Cooking", "Walking", "Cooking" ], "children": [ { "name": "Tomasa Woll", "age": 32 }, { "name": "Annika Woll", "age": 21 } ] }
+{ "cid": 431, "name": "Estela Tolbent", "age": 27, "address": { "number": 7186, "street": "7th St.", "city": "Los Angeles" }, "interests": [ "Databases" ], "children": [ { "name": "Joie Tolbent", "age": null }, { "name": "Angila Tolbent", "age": null }, { "name": "Anastasia Tolbent", "age": 14 } ] }
+{ "cid": 432, "name": "Judi Vinet", "age": 85, "address": { "number": 7304, "street": "Oak St.", "city": "Los Angeles" }, "interests": [ "Wine" ], "children": [ { "name": "Golden Vinet", "age": 20 }, { "name": "Maragret Vinet", "age": null }, { "name": "Keshia Vinet", "age": 10 }, { "name": "Gary Vinet", "age": 73 } ] }
+{ "cid": 433, "name": "Caleb Merrbach", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Amado Merrbach", "age": 45 } ] }
+{ "cid": 434, "name": "Tamesha Soho", "age": 33, "address": { "number": 4534, "street": "Park St.", "city": "Seattle" }, "interests": [ ], "children": [ { "name": "Cody Soho", "age": null }, { "name": "Glennie Soho", "age": 22 } ] }
+{ "cid": 435, "name": "Britni Kazemi", "age": 69, "address": { "number": 7868, "street": "Main St.", "city": "San Jose" }, "interests": [ "Databases", "Music", "Wine" ], "children": [ ] }
+{ "cid": 436, "name": "Xenia Pool", "age": null, "address": null, "interests": [ "Books" ], "children": [ ] }
+{ "cid": 437, "name": "Marlene Macintyre", "age": 86, "address": { "number": 3708, "street": "Oak St.", "city": "Mountain View" }, "interests": [ "Wine", "Walking", "Music", "Coffee" ], "children": [ { "name": "Todd Macintyre", "age": null }, { "name": "Mechelle Macintyre", "age": 50 } ] }
+{ "cid": 438, "name": "Allegra Pefanis", "age": null, "address": null, "interests": [ "Computers", "Music", "Cigars" ], "children": [ ] }
+{ "cid": 439, "name": "Lillia Villnave", "age": 34, "address": { "number": 9212, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ ], "children": [ { "name": "Otis Villnave", "age": null } ] }
+{ "cid": 440, "name": "Rosie Shappen", "age": null, "address": null, "interests": [ "Cooking", "Music", "Cigars" ], "children": [ { "name": "Jung Shappen", "age": 11 } ] }
+{ "cid": 441, "name": "Jamison Reeser", "age": 84, "address": { "number": 9376, "street": "7th St.", "city": "Mountain View" }, "interests": [ "Tennis" ], "children": [ { "name": "Elena Reeser", "age": 28 } ] }
+{ "cid": 442, "name": "Val Disorda", "age": null, "address": null, "interests": [ "Bass" ], "children": [ { "name": "Simone Disorda", "age": 53 }, { "name": "Jacalyn Disorda", "age": 41 }, { "name": "Ron Disorda", "age": null }, { "name": "Clifton Disorda", "age": null } ] }
+{ "cid": 445, "name": "Walton Komo", "age": 16, "address": { "number": 8769, "street": "Main St.", "city": "Seattle" }, "interests": [ "Running", "Basketball", "Tennis" ], "children": [ ] }
+{ "cid": 446, "name": "Lilly Grannell", "age": 21, "address": { "number": 5894, "street": "Washington St.", "city": "San Jose" }, "interests": [ "Computers", "Tennis", "Puzzles", "Books" ], "children": [ { "name": "Victor Grannell", "age": null } ] }
+{ "cid": 447, "name": "Iris Schoneman", "age": 34, "address": { "number": 7648, "street": "Washington St.", "city": "Seattle" }, "interests": [ "Bass", "Wine", "Puzzles", "Cigars" ], "children": [ { "name": "Shemika Schoneman", "age": 11 }, { "name": "Maritza Schoneman", "age": 21 }, { "name": "Martha Schoneman", "age": 20 } ] }
+{ "cid": 448, "name": "Gracie Pekas", "age": 59, "address": { "number": 4732, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Base Jumping", "Wine", "Cigars" ], "children": [ { "name": "Jeanett Pekas", "age": 35 }, { "name": "Jennifer Pekas", "age": null }, { "name": "Carrol Pekas", "age": null } ] }
+{ "cid": 449, "name": "Jacinda Markle", "age": null, "address": null, "interests": [ "Basketball", "Basketball", "Computers" ], "children": [ { "name": "Tam Markle", "age": 45 } ] }
+{ "cid": 450, "name": "Althea Mohammed", "age": null, "address": null, "interests": [ "Fishing", "Databases" ], "children": [ { "name": "Jasper Mohammed", "age": null } ] }
+{ "cid": 451, "name": "Lelia Sondelski", "age": 60, "address": { "number": 4044, "street": "Park St.", "city": "Portland" }, "interests": [ "Books", "Squash", "Walking" ], "children": [ ] }
+{ "cid": 452, "name": "Casie Marasigan", "age": null, "address": null, "interests": [ "Walking", "Computers" ], "children": [ { "name": "Connie Marasigan", "age": null }, { "name": "Kimberlie Marasigan", "age": null } ] }
+{ "cid": 453, "name": "Sherlyn Deadmond", "age": null, "address": null, "interests": [ "Tennis", "Puzzles", "Base Jumping" ], "children": [ { "name": "Torrie Deadmond", "age": 46 }, { "name": "Cleotilde Deadmond", "age": 55 }, { "name": "Garry Deadmond", "age": 34 }, { "name": "Valrie Deadmond", "age": null } ] }
+{ "cid": 454, "name": "Irving Lhuillier", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Emile Lhuillier", "age": null }, { "name": "Albert Lhuillier", "age": null }, { "name": "Ingeborg Lhuillier", "age": 23 }, { "name": "Shila Lhuillier", "age": 55 } ] }
+{ "cid": 455, "name": "Manual Altizer", "age": 70, "address": { "number": 6293, "street": "7th St.", "city": "Portland" }, "interests": [ "Running", "Fishing", "Coffee" ], "children": [ { "name": "Katherine Altizer", "age": null } ] }
+{ "cid": 456, "name": "Kim Cervera", "age": 89, "address": { "number": 3967, "street": "Lake St.", "city": "Portland" }, "interests": [ "Fishing" ], "children": [ { "name": "Winona Cervera", "age": 37 }, { "name": "Shanice Cervera", "age": null }, { "name": "Michaele Cervera", "age": null } ] }
+{ "cid": 457, "name": "Jenice Boger", "age": null, "address": null, "interests": [ "Skiing", "Databases", "Running" ], "children": [ ] }
+{ "cid": 458, "name": "Ivan Sien", "age": 17, "address": { "number": 9981, "street": "Lake St.", "city": "Portland" }, "interests": [ "Cooking", "Coffee" ], "children": [ { "name": "Laurence Sien", "age": null }, { "name": "Nelle Sien", "age": null }, { "name": "Thalia Sien", "age": null } ] }
+{ "cid": 459, "name": "Mable Ellwein", "age": 60, "address": { "number": 1138, "street": "Lake St.", "city": "Portland" }, "interests": [ ], "children": [ { "name": "Stan Ellwein", "age": 19 }, { "name": "Ashlea Ellwein", "age": 13 }, { "name": "Tiesha Ellwein", "age": 28 } ] }
+{ "cid": 460, "name": "Jeraldine Choules", "age": null, "address": null, "interests": [ "Fishing" ], "children": [ { "name": "Berneice Choules", "age": 16 }, { "name": "Jaime Choules", "age": 21 }, { "name": "Li Choules", "age": 20 }, { "name": "Leah Choules", "age": null } ] }
+{ "cid": 461, "name": "Dessie Schnibbe", "age": null, "address": null, "interests": [ ], "children": [ ] }
+{ "cid": 462, "name": "Margaret Galvis", "age": null, "address": null, "interests": [ "Base Jumping", "Movies", "Movies" ], "children": [ { "name": "Isaac Galvis", "age": 48 }, { "name": "Mei Galvis", "age": null }, { "name": "Asha Galvis", "age": null }, { "name": "Zachery Galvis", "age": null } ] }
+{ "cid": 463, "name": "Mika Rininger", "age": null, "address": null, "interests": [ "Databases", "Cooking" ], "children": [ { "name": "Inez Rininger", "age": 58 }, { "name": "Betty Rininger", "age": null }, { "name": "Laurie Rininger", "age": 48 }, { "name": "Billie Rininger", "age": null } ] }
+{ "cid": 464, "name": "Petra Kinsel", "age": null, "address": null, "interests": [ "Wine" ], "children": [ { "name": "Janise Kinsel", "age": null }, { "name": "Donnie Kinsel", "age": 26 }, { "name": "Joana Kinsel", "age": 12 } ] }
+{ "cid": 465, "name": "Rey Arango", "age": 68, "address": { "number": 1788, "street": "View St.", "city": "Los Angeles" }, "interests": [ "Tennis" ], "children": [ ] }
+{ "cid": 466, "name": "Paulene Bagen", "age": 87, "address": { "number": 4093, "street": "View St.", "city": "Mountain View" }, "interests": [ "Music" ], "children": [ { "name": "Antione Bagen", "age": null }, { "name": "Samatha Bagen", "age": null } ] }
+{ "cid": 467, "name": "Magali Ingerson", "age": null, "address": null, "interests": [ "Books", "Base Jumping" ], "children": [ { "name": "Monty Ingerson", "age": 11 }, { "name": "Noelia Ingerson", "age": 47 }, { "name": "Tennie Ingerson", "age": null }, { "name": "Merrill Ingerson", "age": null } ] }
+{ "cid": 468, "name": "Raeann Conry", "age": 68, "address": { "number": 4312, "street": "Cedar St.", "city": "Seattle" }, "interests": [ "Squash" ], "children": [ { "name": "Ellena Conry", "age": 36 }, { "name": "Lynwood Conry", "age": 13 }, { "name": "Coreen Conry", "age": 23 } ] }
+{ "cid": 470, "name": "Yesenia Doyon", "age": 78, "address": { "number": 3641, "street": "7th St.", "city": "Seattle" }, "interests": [ "Databases", "Puzzles" ], "children": [ { "name": "Halley Doyon", "age": null }, { "name": "Teisha Doyon", "age": 33 }, { "name": "Warren Doyon", "age": null } ] }
+{ "cid": 471, "name": "Nicol Majersky", "age": null, "address": null, "interests": [ "Video Games", "Books" ], "children": [ { "name": "Alise Majersky", "age": null }, { "name": "Kathline Majersky", "age": 53 }, { "name": "Charlie Majersky", "age": 45 }, { "name": "Helaine Majersky", "age": null } ] }
+{ "cid": 472, "name": "Kelley Mischler", "age": 38, "address": { "number": 7988, "street": "Lake St.", "city": "Los Angeles" }, "interests": [ "Movies", "Cooking", "Skiing" ], "children": [ { "name": "Keila Mischler", "age": 19 }, { "name": "Evie Mischler", "age": 15 } ] }
+{ "cid": 475, "name": "Brinda Gouker", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Gayle Gouker", "age": 52 } ] }
+{ "cid": 478, "name": "Sophia Whitt", "age": 26, "address": { "number": 2787, "street": "Park St.", "city": "Mountain View" }, "interests": [ "Fishing", "Databases" ], "children": [ { "name": "Irving Whitt", "age": 13 }, { "name": "Jeannette Whitt", "age": null } ] }
+{ "cid": 479, "name": "Danilo Varney", "age": 17, "address": { "number": 9330, "street": "Hill St.", "city": "Portland" }, "interests": [ "Wine" ], "children": [ { "name": "Shelby Varney", "age": null }, { "name": "Fidela Varney", "age": null }, { "name": "Maynard Varney", "age": null }, { "name": "Lindsay Varney", "age": null } ] }
+{ "cid": 480, "name": "Nigel Pitmon", "age": null, "address": null, "interests": [ "Puzzles", "Books" ], "children": [ { "name": "Janene Pitmon", "age": null }, { "name": "Louie Pitmon", "age": 19 }, { "name": "Genny Pitmon", "age": 24 }, { "name": "Robby Pitmon", "age": 55 } ] }
+{ "cid": 481, "name": "Leana Revera", "age": null, "address": null, "interests": [ "Running", "Skiing" ], "children": [ { "name": "Marquita Revera", "age": null } ] }
+{ "cid": 482, "name": "Samantha Stonis", "age": null, "address": null, "interests": [ "Databases" ], "children": [ ] }
+{ "cid": 483, "name": "Elsa Vigen", "age": null, "address": null, "interests": [ "Wine", "Databases" ], "children": [ { "name": "Larae Vigen", "age": null }, { "name": "Elwood Vigen", "age": null } ] }
+{ "cid": 484, "name": "Bennie Dragaj", "age": null, "address": null, "interests": [ "Fishing", "Databases", "Wine" ], "children": [ { "name": "Viva Dragaj", "age": 13 } ] }
+{ "cid": 485, "name": "Gene Rogoff", "age": null, "address": null, "interests": [ "Fishing" ], "children": [ { "name": "Ebonie Rogoff", "age": null } ] }
+{ "cid": 486, "name": "Willa Patman", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Ross Patman", "age": 42 }, { "name": "Erin Patman", "age": null }, { "name": "Vannessa Patman", "age": 11 }, { "name": "Hilaria Patman", "age": 28 } ] }
+{ "cid": 487, "name": "Zenia Virgilio", "age": 46, "address": { "number": 584, "street": "Main St.", "city": "Mountain View" }, "interests": [ "Walking", "Squash", "Wine" ], "children": [ { "name": "Quintin Virgilio", "age": null }, { "name": "Edith Virgilio", "age": null }, { "name": "Nicolle Virgilio", "age": 33 } ] }
+{ "cid": 489, "name": "Brigid Delosier", "age": 31, "address": { "number": 6082, "street": "Oak St.", "city": "Portland" }, "interests": [ "Tennis", "Cigars", "Music" ], "children": [ { "name": "Allegra Delosier", "age": null }, { "name": "Yong Delosier", "age": 10 }, { "name": "Steffanie Delosier", "age": 13 } ] }
+{ "cid": 492, "name": "Gene Alcazar", "age": 59, "address": { "number": 9650, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Computers" ], "children": [ { "name": "Olympia Alcazar", "age": null }, { "name": "Mark Alcazar", "age": 37 }, { "name": "Danilo Alcazar", "age": null } ] }
+{ "cid": 493, "name": "Lindsey Trout", "age": 86, "address": { "number": 7619, "street": "Cedar St.", "city": "Portland" }, "interests": [ "Base Jumping", "Skiing" ], "children": [ { "name": "Madlyn Trout", "age": 58 }, { "name": "Amie Trout", "age": 72 } ] }
+{ "cid": 494, "name": "Delma Deever", "age": 84, "address": { "number": 5044, "street": "7th St.", "city": "Seattle" }, "interests": [ "Computers", "Basketball", "Squash" ], "children": [ ] }
+{ "cid": 496, "name": "Lonna Starkweather", "age": 80, "address": { "number": 1162, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ "Coffee", "Bass", "Running" ], "children": [ { "name": "Matilda Starkweather", "age": null } ] }
+{ "cid": 497, "name": "Chantay Balak", "age": null, "address": null, "interests": [ "Bass", "Fishing" ], "children": [ { "name": "John Balak", "age": null }, { "name": "Thu Balak", "age": 38 } ] }
+{ "cid": 498, "name": "Arleen Sultzer", "age": null, "address": null, "interests": [ "Coffee", "Movies", "Skiing" ], "children": [ { "name": "Norine Sultzer", "age": 29 } ] }
+{ "cid": 499, "name": "Carlita Tarlton", "age": 43, "address": { "number": 9148, "street": "Main St.", "city": "Sunnyvale" }, "interests": [ "Computers", "Base Jumping", "Video Games" ], "children": [ ] }
+{ "cid": 500, "name": "Tierra Bjorklund", "age": null, "address": null, "interests": [ "Puzzles", "Skiing" ], "children": [ { "name": "Avelina Bjorklund", "age": 54 }, { "name": "Mallory Bjorklund", "age": null } ] }
+{ "cid": 501, "name": "Alyce Coant", "age": null, "address": null, "interests": [ "Music", "Base Jumping" ], "children": [ { "name": "Elyse Coant", "age": 50 } ] }
+{ "cid": 502, "name": "Lawana Mulik", "age": 82, "address": { "number": 3071, "street": "Park St.", "city": "Portland" }, "interests": [ "Cigars", "Cigars" ], "children": [ { "name": "Carrie Mulik", "age": null }, { "name": "Sharlene Mulik", "age": 33 }, { "name": "Leone Mulik", "age": 46 } ] }
+{ "cid": 503, "name": "Phyliss Cassani", "age": null, "address": null, "interests": [ "Squash", "Tennis" ], "children": [ { "name": "Rolando Cassani", "age": 44 }, { "name": "Rikki Cassani", "age": 18 }, { "name": "Monty Cassani", "age": 40 } ] }
+{ "cid": 504, "name": "Marla Kolenda", "age": 57, "address": { "number": 464, "street": "View St.", "city": "San Jose" }, "interests": [ "Coffee" ], "children": [ { "name": "Iliana Kolenda", "age": 34 }, { "name": "Ammie Kolenda", "age": 20 }, { "name": "Candi Kolenda", "age": 23 }, { "name": "Lyla Kolenda", "age": 23 } ] }
+{ "cid": 505, "name": "Mike Runk", "age": null, "address": null, "interests": [ "Databases", "Computers", "Running", "Video Games" ], "children": [ { "name": "Lashawn Runk", "age": 21 } ] }
+{ "cid": 506, "name": "Jonna Kolbusz", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Debrah Kolbusz", "age": null }, { "name": "Hugh Kolbusz", "age": null } ] }
+{ "cid": 507, "name": "Yuk Flanegan", "age": null, "address": null, "interests": [ "Puzzles", "Puzzles", "Squash" ], "children": [ { "name": "Alexander Flanegan", "age": null } ] }
+{ "cid": 508, "name": "Tiffany Kimmey", "age": 64, "address": { "number": 8625, "street": "7th St.", "city": "Mountain View" }, "interests": [ "Bass", "Walking" ], "children": [ ] }
+{ "cid": 509, "name": "Alvaro Johnke", "age": null, "address": null, "interests": [ "Computers" ], "children": [ { "name": "Allison Johnke", "age": null }, { "name": "Ellan Johnke", "age": null } ] }
+{ "cid": 510, "name": "Candace Morello", "age": null, "address": null, "interests": [ "Wine", "Base Jumping", "Running" ], "children": [ { "name": "Sandy Morello", "age": 57 }, { "name": "Delois Morello", "age": 15 } ] }
+{ "cid": 512, "name": "Paul Cobian", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Will Cobian", "age": 30 }, { "name": "Conrad Cobian", "age": 35 }, { "name": "Justin Cobian", "age": 11 } ] }
+{ "cid": 513, "name": "Marianna Gortman", "age": 49, "address": { "number": 927, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Databases", "Databases" ], "children": [ ] }
+{ "cid": 514, "name": "Raleigh Belling", "age": 56, "address": { "number": 7408, "street": "View St.", "city": "Mountain View" }, "interests": [ "Running" ], "children": [ ] }
+{ "cid": 515, "name": "Connie Banis", "age": null, "address": null, "interests": [ "Coffee" ], "children": [ { "name": "Brittni Banis", "age": null }, { "name": "Deloras Banis", "age": 25 } ] }
+{ "cid": 516, "name": "Taunya Berkbigler", "age": 82, "address": { "number": 5441, "street": "View St.", "city": "Seattle" }, "interests": [ "Databases", "Tennis" ], "children": [ { "name": "Cherry Berkbigler", "age": 27 }, { "name": "Perry Berkbigler", "age": null } ] }
+{ "cid": 517, "name": "Alfonso Bruderer", "age": null, "address": null, "interests": [ "Bass" ], "children": [ ] }
+{ "cid": 518, "name": "Cora Ingargiola", "age": null, "address": null, "interests": [ "Skiing", "Squash", "Movies" ], "children": [ { "name": "Katlyn Ingargiola", "age": null }, { "name": "Mike Ingargiola", "age": null }, { "name": "Lawrence Ingargiola", "age": null }, { "name": "Isabelle Ingargiola", "age": null } ] }
+{ "cid": 519, "name": "Julianna Goodsell", "age": 59, "address": { "number": 5594, "street": "Lake St.", "city": "Seattle" }, "interests": [ "Video Games", "Fishing" ], "children": [ ] }
+{ "cid": 520, "name": "Janay Bernbeck", "age": null, "address": null, "interests": [ "Databases", "Databases" ], "children": [ { "name": "Aurea Bernbeck", "age": null }, { "name": "Tiara Bernbeck", "age": null }, { "name": "Alfredia Bernbeck", "age": 26 } ] }
+{ "cid": 521, "name": "Frankie Hofmann", "age": null, "address": null, "interests": [ "Databases", "Movies" ], "children": [ { "name": "Shirlee Hofmann", "age": 32 }, { "name": "Jacque Hofmann", "age": 23 }, { "name": "Jazmin Hofmann", "age": null }, { "name": "Serena Hofmann", "age": 56 } ] }
+{ "cid": 522, "name": "Daryl Kissack", "age": 86, "address": { "number": 7825, "street": "Cedar St.", "city": "Mountain View" }, "interests": [ "Squash", "Base Jumping", "Tennis" ], "children": [ { "name": "Darrel Kissack", "age": 21 } ] }
+{ "cid": 523, "name": "Johanne Huls", "age": null, "address": null, "interests": [ "Books", "Bass" ], "children": [ { "name": "Melynda Huls", "age": null }, { "name": "Vicky Huls", "age": 16 }, { "name": "Charlott Huls", "age": null } ] }
+{ "cid": 524, "name": "Rickie Manche", "age": null, "address": null, "interests": [ ], "children": [ ] }
+{ "cid": 525, "name": "Miquel Hodnefield", "age": 12, "address": { "number": 4784, "street": "7th St.", "city": "Sunnyvale" }, "interests": [ ], "children": [ { "name": "Darnell Hodnefield", "age": null }, { "name": "Particia Hodnefield", "age": null } ] }
+{ "cid": 528, "name": "Tamela Witherbee", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Penney Witherbee", "age": null } ] }
+{ "cid": 529, "name": "Cinderella Lewis", "age": null, "address": null, "interests": [ "Base Jumping" ], "children": [ { "name": "Flor Lewis", "age": null }, { "name": "Alonzo Lewis", "age": 23 } ] }
+{ "cid": 530, "name": "Olevia Sturk", "age": 72, "address": { "number": 1939, "street": "Cedar St.", "city": "Sunnyvale" }, "interests": [ "Computers" ], "children": [ { "name": "Cindy Sturk", "age": 18 }, { "name": "Alishia Sturk", "age": null }, { "name": "Sonja Sturk", "age": 51 } ] }
+{ "cid": 531, "name": "Camelia Yoes", "age": null, "address": null, "interests": [ ], "children": [ ] }
+{ "cid": 532, "name": "Tania Fraklin", "age": 38, "address": { "number": 2857, "street": "Washington St.", "city": "Seattle" }, "interests": [ "Squash", "Databases" ], "children": [ ] }
+{ "cid": 533, "name": "Trinity Urquidez", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Corrine Urquidez", "age": 29 }, { "name": "Markita Urquidez", "age": 19 }, { "name": "Danette Urquidez", "age": null } ] }
+{ "cid": 534, "name": "Bridgett Ebel", "age": null, "address": null, "interests": [ "Cigars" ], "children": [ ] }
+{ "cid": 535, "name": "Juana Hirliman", "age": 87, "address": { "number": 6763, "street": "Hill St.", "city": "Sunnyvale" }, "interests": [ "Movies" ], "children": [ { "name": "Ursula Hirliman", "age": 40 }, { "name": "Doretha Hirliman", "age": 30 }, { "name": "Leisha Hirliman", "age": 49 } ] }
+{ "cid": 536, "name": "Wilber Rehrer", "age": null, "address": null, "interests": [ "Movies" ], "children": [ { "name": "Zulema Rehrer", "age": null }, { "name": "Lavonda Rehrer", "age": null }, { "name": "Stacey Rehrer", "age": 59 } ] }
+{ "cid": 537, "name": "Mara Hugar", "age": null, "address": null, "interests": [ "Fishing", "Skiing", "Skiing" ], "children": [ { "name": "Krista Hugar", "age": null } ] }
+{ "cid": 538, "name": "Mack Vollick", "age": null, "address": null, "interests": [ "Base Jumping", "Fishing", "Walking", "Computers" ], "children": [ { "name": "Gil Vollick", "age": 11 }, { "name": "Marica Vollick", "age": null } ] }
+{ "cid": 539, "name": "Nicky Graceffo", "age": null, "address": null, "interests": [ "Video Games" ], "children": [ ] }
+{ "cid": 540, "name": "Bryanna Herling", "age": 67, "address": { "number": 7682, "street": "View St.", "city": "Sunnyvale" }, "interests": [ ], "children": [ { "name": "Cyrstal Herling", "age": 50 }, { "name": "Vallie Herling", "age": 54 }, { "name": "Doris Herling", "age": null } ] }
+{ "cid": 541, "name": "Sammy Adamitis", "age": 71, "address": { "number": 5593, "street": "Washington St.", "city": "Seattle" }, "interests": [ "Books", "Tennis", "Cooking" ], "children": [ ] }
+{ "cid": 542, "name": "Eveline Smedley", "age": 50, "address": { "number": 5513, "street": "Oak St.", "city": "San Jose" }, "interests": [ "Skiing", "Walking" ], "children": [ { "name": "Lynsey Smedley", "age": 26 } ] }
+{ "cid": 543, "name": "Pearl Nollette", "age": null, "address": null, "interests": [ "Base Jumping", "Running" ], "children": [ ] }
+{ "cid": 544, "name": "Silas Demay", "age": 69, "address": { "number": 447, "street": "Main St.", "city": "Portland" }, "interests": [ "Tennis", "Bass" ], "children": [ { "name": "Latonya Demay", "age": null }, { "name": "Lissette Demay", "age": 37 }, { "name": "Lynell Demay", "age": 42 }, { "name": "Mikel Demay", "age": 17 } ] }
+{ "cid": 545, "name": "Dolores Ferer", "age": null, "address": null, "interests": [ "Coffee", "Bass", "Tennis" ], "children": [ { "name": "Bridgette Ferer", "age": null } ] }
+{ "cid": 547, "name": "Daryl Dambra", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Jacquline Dambra", "age": null }, { "name": "Seymour Dambra", "age": null } ] }
+{ "cid": 548, "name": "Elvia Duchesney", "age": null, "address": null, "interests": [ "Basketball" ], "children": [ { "name": "Arcelia Duchesney", "age": 22 } ] }
+{ "cid": 549, "name": "Kathrin Cruff", "age": 63, "address": { "number": 9002, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ "Tennis", "Books" ], "children": [ { "name": "Candi Cruff", "age": 49 }, { "name": "Barry Cruff", "age": 17 }, { "name": "Shane Cruff", "age": 18 }, { "name": "Brendon Cruff", "age": null } ] }
+{ "cid": 550, "name": "Aleisha Brehon", "age": 61, "address": { "number": 7835, "street": "Hill St.", "city": "Mountain View" }, "interests": [ "Squash" ], "children": [ { "name": "Vito Brehon", "age": null }, { "name": "Matthew Brehon", "age": 32 } ] }
+{ "cid": 552, "name": "Marlena Humann", "age": null, "address": null, "interests": [ ], "children": [ ] }
+{ "cid": 553, "name": "Mina Ciminera", "age": null, "address": null, "interests": [ "Base Jumping", "Databases" ], "children": [ { "name": "Cornelius Ciminera", "age": null }, { "name": "Rozanne Ciminera", "age": null }, { "name": "Byron Ciminera", "age": null } ] }
+{ "cid": 554, "name": "Darci Yafai", "age": 60, "address": { "number": 4694, "street": "Park St.", "city": "Mountain View" }, "interests": [ ], "children": [ { "name": "Lecia Yafai", "age": 47 } ] }
+{ "cid": 555, "name": "Agustina Bretthauer", "age": null, "address": null, "interests": [ "Cigars" ], "children": [ { "name": "Arthur Bretthauer", "age": 33 }, { "name": "Titus Bretthauer", "age": 33 }, { "name": "Margret Bretthauer", "age": null } ] }
+{ "cid": 557, "name": "Kaitlyn Hilleman", "age": 61, "address": { "number": 1076, "street": "Hill St.", "city": "Sunnyvale" }, "interests": [ ], "children": [ { "name": "Corrie Hilleman", "age": 31 }, { "name": "Jovan Hilleman", "age": null }, { "name": "Carmine Hilleman", "age": null } ] }
+{ "cid": 559, "name": "Carolyne Shiroma", "age": null, "address": null, "interests": [ "Movies", "Running" ], "children": [ { "name": "Ying Shiroma", "age": 57 } ] }
+{ "cid": 560, "name": "Karin Dicesare", "age": null, "address": null, "interests": [ "Wine", "Puzzles" ], "children": [ ] }
+{ "cid": 561, "name": "Renetta Cudworth", "age": null, "address": null, "interests": [ "Skiing", "Basketball" ], "children": [ ] }
+{ "cid": 563, "name": "Deirdre Landero", "age": null, "address": null, "interests": [ "Books", "Fishing", "Video Games" ], "children": [ { "name": "Norman Landero", "age": 59 }, { "name": "Jennine Landero", "age": 45 }, { "name": "Rutha Landero", "age": 19 }, { "name": "Jackie Landero", "age": 29 } ] }
+{ "cid": 564, "name": "Inger Dargin", "age": 56, "address": { "number": 8704, "street": "View St.", "city": "Mountain View" }, "interests": [ "Wine", "Running", "Computers" ], "children": [ ] }
+{ "cid": 565, "name": "Shantell Rima", "age": 82, "address": { "number": 205, "street": "Cedar St.", "city": "Sunnyvale" }, "interests": [ ], "children": [ { "name": "Boyce Rima", "age": 67 }, { "name": "Woodrow Rima", "age": 18 }, { "name": "Helene Rima", "age": null }, { "name": "David Rima", "age": null } ] }
+{ "cid": 566, "name": "Asley Grow", "age": null, "address": null, "interests": [ "Coffee", "Books", "Tennis" ], "children": [ { "name": "Dale Grow", "age": null } ] }
+{ "cid": 567, "name": "Peggie Madhavan", "age": null, "address": null, "interests": [ "Computers", "Bass" ], "children": [ ] }
+{ "cid": 569, "name": "Beata Diles", "age": 88, "address": { "number": 2198, "street": "Park St.", "city": "Mountain View" }, "interests": [ ], "children": [ { "name": "Myrtice Diles", "age": 46 }, { "name": "Stella Diles", "age": null }, { "name": "Rowena Diles", "age": 26 } ] }
+{ "cid": 570, "name": "Lee Basora", "age": null, "address": null, "interests": [ "Squash", "Cigars" ], "children": [ ] }
+{ "cid": 571, "name": "Lenita Tentler", "age": null, "address": null, "interests": [ "Running", "Fishing" ], "children": [ { "name": "Damian Tentler", "age": 16 }, { "name": "Camellia Tentler", "age": null }, { "name": "Vern Tentler", "age": 15 } ] }
+{ "cid": 572, "name": "Darcy Polycarpe", "age": 35, "address": { "number": 8051, "street": "View St.", "city": "Mountain View" }, "interests": [ "Computers", "Coffee", "Walking", "Walking" ], "children": [ { "name": "Kenneth Polycarpe", "age": null } ] }
+{ "cid": 573, "name": "Tyree Ketcher", "age": null, "address": null, "interests": [ "Computers", "Walking" ], "children": [ { "name": "Aleisha Ketcher", "age": null }, { "name": "Vonda Ketcher", "age": null }, { "name": "Cyndy Ketcher", "age": 13 }, { "name": "Chassidy Ketcher", "age": 30 } ] }
+{ "cid": 574, "name": "Camellia Toxey", "age": 52, "address": { "number": 5437, "street": "Hill St.", "city": "Portland" }, "interests": [ ], "children": [ { "name": "Deandrea Toxey", "age": null }, { "name": "Danille Toxey", "age": null } ] }
+{ "cid": 577, "name": "Alejandro Oblinger", "age": null, "address": null, "interests": [ "Movies", "Movies" ], "children": [ { "name": "Tenesha Oblinger", "age": 56 }, { "name": "Loni Oblinger", "age": 12 }, { "name": "Sherryl Oblinger", "age": null } ] }
+{ "cid": 578, "name": "Dolly Delphia", "age": null, "address": null, "interests": [ "Wine" ], "children": [ { "name": "Sharron Delphia", "age": null }, { "name": "Shemeka Delphia", "age": null }, { "name": "Rachael Delphia", "age": null } ] }
+{ "cid": 579, "name": "Sabra Yuenger", "age": 45, "address": { "number": 2681, "street": "Cedar St.", "city": "Sunnyvale" }, "interests": [ "Puzzles" ], "children": [ { "name": "Eddie Yuenger", "age": null } ] }
+{ "cid": 581, "name": "Leigha Finkenbinder", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Lorine Finkenbinder", "age": 29 }, { "name": "Stephanie Finkenbinder", "age": 28 } ] }
+{ "cid": 582, "name": "Suzie Ocallahan", "age": null, "address": null, "interests": [ "Basketball" ], "children": [ { "name": "Tamra Ocallahan", "age": null } ] }
+{ "cid": 583, "name": "Bev Yerena", "age": null, "address": null, "interests": [ "Puzzles", "Wine" ], "children": [ { "name": "Larhonda Yerena", "age": 45 }, { "name": "Josefina Yerena", "age": null }, { "name": "Sydney Yerena", "age": 42 } ] }
+{ "cid": 584, "name": "Bailey Janes", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Marylou Janes", "age": null }, { "name": "Andra Janes", "age": null } ] }
+{ "cid": 585, "name": "Young Drube", "age": 21, "address": { "number": 6960, "street": "View St.", "city": "Seattle" }, "interests": [ "Basketball", "Fishing", "Walking" ], "children": [ { "name": "Irwin Drube", "age": null }, { "name": "Gustavo Drube", "age": null } ] }
+{ "cid": 586, "name": "Jeannine Donnerberg", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Mike Donnerberg", "age": null } ] }
+{ "cid": 587, "name": "Santos Monterio", "age": 36, "address": { "number": 4454, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ "Databases", "Music", "Cooking" ], "children": [ { "name": "Lashonda Monterio", "age": null } ] }
+{ "cid": 588, "name": "Debora Laughinghouse", "age": 87, "address": { "number": 5099, "street": "View St.", "city": "San Jose" }, "interests": [ "Tennis", "Walking", "Databases" ], "children": [ { "name": "Frederica Laughinghouse", "age": 59 }, { "name": "Johnie Laughinghouse", "age": 12 }, { "name": "Numbers Laughinghouse", "age": 73 } ] }
+{ "cid": 589, "name": "Rebeca Blackwell", "age": 66, "address": { "number": 5708, "street": "View St.", "city": "Portland" }, "interests": [ ], "children": [ ] }
+{ "cid": 590, "name": "Joye Burton", "age": null, "address": null, "interests": [ "Bass", "Base Jumping" ], "children": [ { "name": "Noemi Burton", "age": 19 }, { "name": "Hulda Burton", "age": null }, { "name": "Cleotilde Burton", "age": null }, { "name": "Dara Burton", "age": null } ] }
+{ "cid": 591, "name": "Matthew Tenhaeff", "age": null, "address": null, "interests": [ "Databases", "Video Games" ], "children": [ { "name": "Jan Tenhaeff", "age": 25 }, { "name": "Nana Tenhaeff", "age": null }, { "name": "Laticia Tenhaeff", "age": null }, { "name": "Ara Tenhaeff", "age": 44 } ] }
+{ "cid": 592, "name": "Rachelle Spare", "age": 13, "address": { "number": 8088, "street": "Oak St.", "city": "Portland" }, "interests": [ "Squash", "Puzzles" ], "children": [ { "name": "Theo Spare", "age": null }, { "name": "Shizue Spare", "age": null } ] }
+{ "cid": 593, "name": "Danial Pittillo", "age": 87, "address": { "number": 815, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ "Tennis", "Base Jumping" ], "children": [ { "name": "Neva Pittillo", "age": 28 }, { "name": "Brooks Pittillo", "age": null }, { "name": "Randell Pittillo", "age": 52 }, { "name": "Allyson Pittillo", "age": 51 } ] }
+{ "cid": 594, "name": "Zenia Corban", "age": null, "address": null, "interests": [ "Puzzles", "Computers", "Video Games", "Cigars" ], "children": [ { "name": "Arielle Corban", "age": null }, { "name": "Arthur Corban", "age": 15 }, { "name": "Taneka Corban", "age": 51 }, { "name": "Claire Corban", "age": null } ] }
+{ "cid": 595, "name": "Samuel Brawdy", "age": 28, "address": { "number": 453, "street": "Main St.", "city": "Los Angeles" }, "interests": [ "Books", "Basketball" ], "children": [ { "name": "Marlen Brawdy", "age": 14 }, { "name": "Lorine Brawdy", "age": 13 }, { "name": "Brad Brawdy", "age": null } ] }
+{ "cid": 596, "name": "Juliane Maddy", "age": null, "address": null, "interests": [ "Coffee", "Computers", "Walking", "Basketball" ], "children": [ { "name": "Joannie Maddy", "age": null }, { "name": "Penny Maddy", "age": 35 }, { "name": "Joette Maddy", "age": 35 }, { "name": "Karla Maddy", "age": 54 } ] }
+{ "cid": 597, "name": "Clarine Eutsey", "age": 39, "address": { "number": 9112, "street": "7th St.", "city": "Portland" }, "interests": [ "Video Games", "Cigars", "Walking" ], "children": [ ] }
+{ "cid": 598, "name": "Venus Peat", "age": null, "address": null, "interests": [ "Coffee", "Walking", "Cigars" ], "children": [ { "name": "Antonetta Peat", "age": null }, { "name": "Shane Peat", "age": null } ] }
+{ "cid": 599, "name": "Alva Molaison", "age": 87, "address": { "number": 5974, "street": "Washington St.", "city": "Seattle" }, "interests": [ "Wine", "Squash" ], "children": [ { "name": "Milo Molaison", "age": 39 } ] }
+{ "cid": 600, "name": "Cordell Sherburn", "age": null, "address": null, "interests": [ "Squash", "Skiing", "Skiing" ], "children": [ { "name": "Shenna Sherburn", "age": 22 }, { "name": "Minna Sherburn", "age": 10 }, { "name": "Tari Sherburn", "age": null } ] }
+{ "cid": 601, "name": "Zackary Willier", "age": null, "address": null, "interests": [ "Cooking", "Databases", "Databases" ], "children": [ ] }
+{ "cid": 602, "name": "Clyde Salada", "age": 59, "address": { "number": 8316, "street": "7th St.", "city": "Sunnyvale" }, "interests": [ "Movies", "Skiing", "Cooking" ], "children": [ ] }
+{ "cid": 603, "name": "Barry Corkum", "age": null, "address": null, "interests": [ "Running", "Running" ], "children": [ { "name": "Charlesetta Corkum", "age": null }, { "name": "Helaine Corkum", "age": null }, { "name": "Erinn Corkum", "age": 28 }, { "name": "Alesia Corkum", "age": 36 } ] }
+{ "cid": 605, "name": "Sue Henriksen", "age": 78, "address": { "number": 7208, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [ ], "children": [ { "name": "Lauretta Henriksen", "age": null }, { "name": "Leigh Henriksen", "age": 11 } ] }
+{ "cid": 606, "name": "Virgilio Liebelt", "age": 11, "address": { "number": 8348, "street": "Cedar St.", "city": "Seattle" }, "interests": [ ], "children": [ { "name": "Stanford Liebelt", "age": null }, { "name": "Delaine Liebelt", "age": null }, { "name": "Kevin Liebelt", "age": null }, { "name": "Michaele Liebelt", "age": null } ] }
+{ "cid": 607, "name": "Bert Garigliano", "age": 71, "address": { "number": 3881, "street": "Washington St.", "city": "San Jose" }, "interests": [ "Walking", "Wine" ], "children": [ { "name": "Junior Garigliano", "age": 42 }, { "name": "Willa Garigliano", "age": 21 }, { "name": "Carlo Garigliano", "age": null } ] }
+{ "cid": 608, "name": "Bruce Stanley", "age": 39, "address": { "number": 4532, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ "Tennis" ], "children": [ ] }
+{ "cid": 609, "name": "Mindi Dieudonne", "age": null, "address": null, "interests": [ "Puzzles" ], "children": [ ] }
+{ "cid": 610, "name": "Elinor Notoma", "age": 66, "address": { "number": 6763, "street": "Lake St.", "city": "Mountain View" }, "interests": [ "Coffee" ], "children": [ { "name": "Dennis Notoma", "age": null }, { "name": "Carol Notoma", "age": 21 } ] }
+{ "cid": 611, "name": "Evelyne Bassette", "age": null, "address": null, "interests": [ "Coffee" ], "children": [ { "name": "Angla Bassette", "age": 13 } ] }
+{ "cid": 612, "name": "Keneth Ganie", "age": 57, "address": { "number": 7712, "street": "Washington St.", "city": "Portland" }, "interests": [ "Cigars", "Base Jumping" ], "children": [ { "name": "Connie Ganie", "age": null }, { "name": "Kamala Ganie", "age": 25 }, { "name": "Beulah Ganie", "age": 15 } ] }
+{ "cid": 613, "name": "Shanelle Leader", "age": null, "address": null, "interests": [ "Databases", "Base Jumping", "Wine", "Fishing" ], "children": [ { "name": "Florencia Leader", "age": null }, { "name": "Herbert Leader", "age": 11 }, { "name": "Jeanna Leader", "age": null } ] }
+{ "cid": 614, "name": "Wallace Chaidy", "age": null, "address": null, "interests": [ "Bass", "Movies", "Music" ], "children": [ { "name": "Refugio Chaidy", "age": null }, { "name": "Hae Chaidy", "age": 55 }, { "name": "Julian Chaidy", "age": null }, { "name": "Tabatha Chaidy", "age": null } ] }
+{ "cid": 615, "name": "Kimber Warnberg", "age": 77, "address": { "number": 1404, "street": "View St.", "city": "San Jose" }, "interests": [ ], "children": [ { "name": "Kristal Warnberg", "age": null } ] }
+{ "cid": 616, "name": "Shanda Dussault", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Darrick Dussault", "age": null } ] }
+{ "cid": 617, "name": "Jacques Gaskill", "age": null, "address": null, "interests": [ "Cigars", "Coffee", "Computers", "Wine" ], "children": [ { "name": "Angelyn Gaskill", "age": null }, { "name": "Jeanett Gaskill", "age": 40 }, { "name": "Emelda Gaskill", "age": 34 } ] }
+{ "cid": 618, "name": "Janella Hurtt", "age": null, "address": null, "interests": [ "Skiing", "Coffee", "Skiing" ], "children": [ { "name": "Lupe Hurtt", "age": 17 }, { "name": "Jae Hurtt", "age": 14 }, { "name": "Evan Hurtt", "age": 45 } ] }
+{ "cid": 619, "name": "Luanne Elmquist", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Burton Elmquist", "age": 11 }, { "name": "Melvin Elmquist", "age": null } ] }
+{ "cid": 620, "name": "Arielle Mackellar", "age": null, "address": null, "interests": [ "Cooking", "Bass" ], "children": [ { "name": "Evelin Mackellar", "age": 17 }, { "name": "Theresa Mackellar", "age": 53 }, { "name": "Ronnie Mackellar", "age": null }, { "name": "Elwanda Mackellar", "age": 54 } ] }
+{ "cid": 621, "name": "Theresa Satterthwaite", "age": 16, "address": { "number": 3249, "street": "Main St.", "city": "Mountain View" }, "interests": [ "Wine", "Skiing", "Wine", "Fishing" ], "children": [ { "name": "Rickie Satterthwaite", "age": null }, { "name": "Rina Satterthwaite", "age": null } ] }
+{ "cid": 622, "name": "Telma Rives", "age": null, "address": null, "interests": [ "Basketball" ], "children": [ { "name": "Maribeth Rives", "age": 42 }, { "name": "Youlanda Rives", "age": 13 }, { "name": "Trang Rives", "age": null }, { "name": "Hyun Rives", "age": null } ] }
+{ "cid": 624, "name": "Bong Lyall", "age": null, "address": null, "interests": [ "Databases", "Music", "Video Games" ], "children": [ ] }
+{ "cid": 625, "name": "Gale Marrazzo", "age": 25, "address": { "number": 2307, "street": "View St.", "city": "San Jose" }, "interests": [ "Fishing", "Base Jumping", "Walking", "Cooking" ], "children": [ { "name": "Coleman Marrazzo", "age": null }, { "name": "Frances Marrazzo", "age": null }, { "name": "Camellia Marrazzo", "age": 11 } ] }
+{ "cid": 626, "name": "Sydney Josten", "age": 44, "address": { "number": 4815, "street": "Hill St.", "city": "Sunnyvale" }, "interests": [ "Cigars" ], "children": [ { "name": "Basil Josten", "age": 14 }, { "name": "Yasuko Josten", "age": null } ] }
+{ "cid": 627, "name": "Fernande Ede", "age": 75, "address": { "number": 9316, "street": "Cedar St.", "city": "Mountain View" }, "interests": [ ], "children": [ { "name": "Rebeca Ede", "age": null }, { "name": "Raymond Ede", "age": 57 } ] }
+{ "cid": 628, "name": "Tomoko Alcantara", "age": 56, "address": { "number": 3556, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ "Running", "Tennis" ], "children": [ { "name": "Babara Alcantara", "age": 31 }, { "name": "Ilana Alcantara", "age": null }, { "name": "Maren Alcantara", "age": 45 } ] }
+{ "cid": 629, "name": "Mayola Clabo", "age": null, "address": null, "interests": [ "Basketball", "Skiing", "Running" ], "children": [ { "name": "Rigoberto Clabo", "age": 58 } ] }
+{ "cid": 630, "name": "Darla Domenick", "age": 14, "address": { "number": 3315, "street": "Park St.", "city": "San Jose" }, "interests": [ "Databases" ], "children": [ { "name": "Verda Domenick", "age": null } ] }
+{ "cid": 631, "name": "Brook Jenks", "age": null, "address": null, "interests": [ "Wine" ], "children": [ { "name": "Eldon Jenks", "age": null }, { "name": "Luann Jenks", "age": 53 }, { "name": "Aurora Jenks", "age": 37 } ] }
+{ "cid": 632, "name": "Keeley Goga", "age": null, "address": null, "interests": [ "Books", "Base Jumping" ], "children": [ { "name": "Walter Goga", "age": 39 }, { "name": "Chaya Goga", "age": null }, { "name": "Melodie Goga", "age": null }, { "name": "Isidro Goga", "age": 32 } ] }
+{ "cid": 633, "name": "Shalon Grauberger", "age": 34, "address": { "number": 765, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ "Music", "Base Jumping", "Tennis" ], "children": [ { "name": "Kris Grauberger", "age": 14 }, { "name": "Stuart Grauberger", "age": 12 }, { "name": "Billy Grauberger", "age": null } ] }
+{ "cid": 634, "name": "Katherina Parzych", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Modesta Parzych", "age": null }, { "name": "Darin Parzych", "age": 20 } ] }
+{ "cid": 635, "name": "Angelena Braegelmann", "age": 36, "address": { "number": 4158, "street": "Park St.", "city": "San Jose" }, "interests": [ "Wine", "Skiing" ], "children": [ { "name": "Daisey Braegelmann", "age": 18 }, { "name": "Gaston Braegelmann", "age": 19 }, { "name": "Louella Braegelmann", "age": null }, { "name": "Leonie Braegelmann", "age": null } ] }
+{ "cid": 636, "name": "Babara Shore", "age": 83, "address": { "number": 9452, "street": "Oak St.", "city": "Los Angeles" }, "interests": [ "Databases", "Movies", "Tennis" ], "children": [ { "name": "Candy Shore", "age": 58 }, { "name": "Nanci Shore", "age": null }, { "name": "Asia Shore", "age": null } ] }
+{ "cid": 639, "name": "Zena Seehusen", "age": 24, "address": { "number": 6303, "street": "Hill St.", "city": "Mountain View" }, "interests": [ "Cooking", "Movies", "Music" ], "children": [ { "name": "Hester Seehusen", "age": null }, { "name": "Coreen Seehusen", "age": 12 } ] }
+{ "cid": 640, "name": "Willy Bielak", "age": null, "address": null, "interests": [ "Squash" ], "children": [ ] }
+{ "cid": 642, "name": "Odell Nova", "age": 25, "address": { "number": 896, "street": "Park St.", "city": "San Jose" }, "interests": [ "Video Games", "Squash", "Music" ], "children": [ { "name": "Leopoldo Nova", "age": null }, { "name": "Rickey Nova", "age": null }, { "name": "Mike Nova", "age": 14 }, { "name": "Tamie Nova", "age": 14 } ] }
+{ "cid": 643, "name": "Juliet Skreen", "age": null, "address": null, "interests": [ "Walking" ], "children": [ ] }
+{ "cid": 644, "name": "Julio Gilly", "age": null, "address": null, "interests": [ "Puzzles" ], "children": [ { "name": "Eleonore Gilly", "age": null } ] }
+{ "cid": 645, "name": "Shawnda Dollinger", "age": 36, "address": { "number": 5980, "street": "Park St.", "city": "Los Angeles" }, "interests": [ ], "children": [ { "name": "Vicente Dollinger", "age": null }, { "name": "Kerrie Dollinger", "age": 10 }, { "name": "Sima Dollinger", "age": 14 } ] }
+{ "cid": 646, "name": "Pablo Catterton", "age": null, "address": null, "interests": [ "Fishing", "Computers" ], "children": [ ] }
+{ "cid": 647, "name": "Jodi Dearson", "age": null, "address": null, "interests": [ "Fishing", "Movies" ], "children": [ ] }
+{ "cid": 649, "name": "Anisha Sender", "age": null, "address": null, "interests": [ "Tennis", "Databases", "Bass" ], "children": [ { "name": "Viva Sender", "age": 40 }, { "name": "Terica Sender", "age": null } ] }
+{ "cid": 650, "name": "Darrin Orengo", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Linwood Orengo", "age": 39 } ] }
+{ "cid": 651, "name": "Delana Henk", "age": 69, "address": { "number": 5497, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ "Coffee", "Video Games", "Databases" ], "children": [ { "name": "Loan Henk", "age": null }, { "name": "Teresa Henk", "age": 20 }, { "name": "Randell Henk", "age": null }, { "name": "Micah Henk", "age": null } ] }
+{ "cid": 652, "name": "Armida Moeuy", "age": 34, "address": { "number": 8306, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ "Running" ], "children": [ { "name": "Sunshine Moeuy", "age": null }, { "name": "Leta Moeuy", "age": 19 } ] }
+{ "cid": 653, "name": "Robbie Rhump", "age": null, "address": null, "interests": [ "Squash", "Computers" ], "children": [ { "name": "Alishia Rhump", "age": 14 }, { "name": "Lyndsay Rhump", "age": 27 } ] }
+{ "cid": 654, "name": "Louis Laubersheimer", "age": 76, "address": { "number": 8010, "street": "7th St.", "city": "San Jose" }, "interests": [ "Base Jumping", "Bass", "Cooking" ], "children": [ { "name": "Jewel Laubersheimer", "age": 22 }, { "name": "Toccara Laubersheimer", "age": 45 }, { "name": "Eve Laubersheimer", "age": null } ] }
+{ "cid": 655, "name": "Shaun Brandenburg", "age": null, "address": null, "interests": [ "Skiing", "Computers", "Base Jumping" ], "children": [ { "name": "Ned Brandenburg", "age": null }, { "name": "Takako Brandenburg", "age": 41 }, { "name": "Astrid Brandenburg", "age": null }, { "name": "Patience Brandenburg", "age": null } ] }
+{ "cid": 656, "name": "Rufus Peaden", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Nathanael Peaden", "age": 57 }, { "name": "Jamaal Peaden", "age": null } ] }
+{ "cid": 657, "name": "Rory Teachman", "age": null, "address": null, "interests": [ ], "children": [ ] }
+{ "cid": 658, "name": "Truman Leitner", "age": null, "address": null, "interests": [ "Computers", "Bass", "Walking" ], "children": [ ] }
+{ "cid": 659, "name": "Daniel Groskreutz", "age": null, "address": null, "interests": [ "Databases" ], "children": [ { "name": "Mariam Groskreutz", "age": 21 }, { "name": "Carlton Groskreutz", "age": null } ] }
+{ "cid": 660, "name": "Israel Aday", "age": null, "address": null, "interests": [ "Wine", "Bass", "Cigars" ], "children": [ { "name": "Mi Aday", "age": null } ] }
+{ "cid": 661, "name": "Lorita Kraut", "age": 43, "address": { "number": 5017, "street": "Park St.", "city": "Los Angeles" }, "interests": [ "Tennis", "Movies", "Bass" ], "children": [ { "name": "Mirian Kraut", "age": null } ] }
+{ "cid": 662, "name": "Domonique Corbi", "age": 13, "address": { "number": 7286, "street": "Hill St.", "city": "Seattle" }, "interests": [ "Tennis", "Cooking", "Computers" ], "children": [ { "name": "Katrice Corbi", "age": null }, { "name": "Idalia Corbi", "age": null }, { "name": "Hayley Corbi", "age": null } ] }
+{ "cid": 663, "name": "Riley Noteboom", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Marvis Noteboom", "age": 57 } ] }
+{ "cid": 665, "name": "Garnet Desai", "age": null, "address": null, "interests": [ "Databases" ], "children": [ { "name": "Aliza Desai", "age": null } ] }
+{ "cid": 666, "name": "Pamila Burzlaff", "age": 68, "address": { "number": 6543, "street": "View St.", "city": "Portland" }, "interests": [ "Squash", "Cigars", "Movies" ], "children": [ ] }
+{ "cid": 667, "name": "Shaniqua Deist", "age": null, "address": null, "interests": [ "Puzzles", "Books", "Cigars" ], "children": [ ] }
+{ "cid": 668, "name": "Dorene Spigelman", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Chiquita Spigelman", "age": 29 }, { "name": "Anisha Spigelman", "age": 34 }, { "name": "Micah Spigelman", "age": 28 } ] }
+{ "cid": 669, "name": "Royal Abke", "age": 60, "address": { "number": 1675, "street": "Main St.", "city": "Los Angeles" }, "interests": [ ], "children": [ { "name": "Leandra Abke", "age": 25 }, { "name": "Shawanna Abke", "age": null } ] }
+{ "cid": 670, "name": "Angelo Kellar", "age": 22, "address": { "number": 3178, "street": "View St.", "city": "Seattle" }, "interests": [ "Wine", "Music", "Fishing" ], "children": [ { "name": "Zula Kellar", "age": null }, { "name": "Brittaney Kellar", "age": 10 }, { "name": "Fredia Kellar", "age": null } ] }
+{ "cid": 671, "name": "Harley Emami", "age": null, "address": null, "interests": [ "Basketball" ], "children": [ { "name": "Valentine Emami", "age": null }, { "name": "Pearlene Emami", "age": null } ] }
+{ "cid": 672, "name": "Pamelia Repka", "age": 30, "address": { "number": 8837, "street": "Oak St.", "city": "San Jose" }, "interests": [ "Coffee", "Base Jumping" ], "children": [ { "name": "Klara Repka", "age": 19 }, { "name": "Bennett Repka", "age": null }, { "name": "Randy Repka", "age": 13 }, { "name": "Ervin Repka", "age": null } ] }
+{ "cid": 673, "name": "Willard Matuszek", "age": null, "address": null, "interests": [ "Running" ], "children": [ { "name": "Kyong Matuszek", "age": null }, { "name": "Delena Matuszek", "age": null }, { "name": "Toney Matuszek", "age": null }, { "name": "Shayne Matuszek", "age": 19 } ] }
+{ "cid": 675, "name": "Camellia Brickett", "age": null, "address": null, "interests": [ "Running" ], "children": [ { "name": "Leona Brickett", "age": null }, { "name": "Mario Brickett", "age": null }, { "name": "Nadine Brickett", "age": 35 }, { "name": "Marlon Brickett", "age": 31 } ] }
+{ "cid": 676, "name": "Ima Juart", "age": 64, "address": { "number": 2498, "street": "Cedar St.", "city": "Portland" }, "interests": [ "Walking" ], "children": [ { "name": "Cortez Juart", "age": 17 }, { "name": "Guillermo Juart", "age": null }, { "name": "Shelley Juart", "age": 20 }, { "name": "Daryl Juart", "age": null } ] }
+{ "cid": 677, "name": "Brigid Sarabia", "age": 89, "address": { "number": 918, "street": "Park St.", "city": "Los Angeles" }, "interests": [ ], "children": [ { "name": "Elisa Sarabia", "age": null }, { "name": "Pura Sarabia", "age": 56 } ] }
+{ "cid": 678, "name": "Lekisha Barnell", "age": null, "address": null, "interests": [ "Movies", "Skiing", "Running" ], "children": [ { "name": "August Barnell", "age": null }, { "name": "Tiffany Barnell", "age": 55 }, { "name": "Meghan Barnell", "age": null } ] }
+{ "cid": 680, "name": "Domenica Qunnarath", "age": null, "address": null, "interests": [ ], "children": [ ] }
+{ "cid": 681, "name": "Iliana Nagele", "age": null, "address": null, "interests": [ "Movies", "Running" ], "children": [ { "name": "Sunny Nagele", "age": 55 }, { "name": "Waltraud Nagele", "age": 39 }, { "name": "Darron Nagele", "age": null } ] }
+{ "cid": 682, "name": "Krystle Weingartner", "age": 87, "address": { "number": 5293, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ "Squash" ], "children": [ { "name": "Bryanna Weingartner", "age": 19 }, { "name": "Rubie Weingartner", "age": 32 }, { "name": "Raye Weingartner", "age": null } ] }
+{ "cid": 683, "name": "Dodie Crall", "age": 37, "address": { "number": 1337, "street": "7th St.", "city": "Mountain View" }, "interests": [ "Wine" ], "children": [ { "name": "Cassy Crall", "age": null }, { "name": "Thu Crall", "age": 19 } ] }
+{ "cid": 684, "name": "Elmo Ballenger", "age": 69, "address": { "number": 2657, "street": "Park St.", "city": "Seattle" }, "interests": [ "Wine" ], "children": [ { "name": "Sheena Ballenger", "age": 53 }, { "name": "Abby Ballenger", "age": null }, { "name": "Markus Ballenger", "age": null } ] }
+{ "cid": 685, "name": "Lois Mcglothian", "age": null, "address": null, "interests": [ "Movies", "Skiing" ], "children": [ { "name": "Karon Mcglothian", "age": 35 } ] }
+{ "cid": 686, "name": "Trudi Arnette", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Adrian Arnette", "age": 43 }, { "name": "Hulda Arnette", "age": 34 }, { "name": "Shamika Arnette", "age": null } ] }
+{ "cid": 687, "name": "Adriene Glowinski", "age": null, "address": null, "interests": [ ], "children": [ ] }
+{ "cid": 688, "name": "Maryellen Leriche", "age": null, "address": null, "interests": [ "Music", "Walking", "Skiing" ], "children": [ { "name": "Dorinda Leriche", "age": 27 } ] }
+{ "cid": 689, "name": "Camila Cho", "age": 70, "address": { "number": 7731, "street": "Cedar St.", "city": "Mountain View" }, "interests": [ "Video Games", "Cigars" ], "children": [ { "name": "Myrtie Cho", "age": 57 }, { "name": "Merideth Cho", "age": 45 }, { "name": "Meta Cho", "age": 20 } ] }
+{ "cid": 691, "name": "Sharee Charrier", "age": 17, "address": { "number": 6693, "street": "Main St.", "city": "Mountain View" }, "interests": [ "Puzzles", "Cooking", "Bass" ], "children": [ { "name": "Odessa Charrier", "age": null } ] }
+{ "cid": 692, "name": "Nida Picknell", "age": 24, "address": { "number": 9053, "street": "Park St.", "city": "Mountain View" }, "interests": [ "Skiing", "Music", "Wine", "Base Jumping" ], "children": [ { "name": "Caroyln Picknell", "age": null }, { "name": "Micheline Picknell", "age": 10 } ] }
+{ "cid": 693, "name": "Ela Crisan", "age": null, "address": null, "interests": [ "Movies" ], "children": [ ] }
+{ "cid": 694, "name": "Ariel Soltani", "age": null, "address": null, "interests": [ "Databases", "Music", "Puzzles" ], "children": [ { "name": "Aldo Soltani", "age": null }, { "name": "Anglea Soltani", "age": null } ] }
+{ "cid": 695, "name": "Wyatt Eveleth", "age": 28, "address": { "number": 5421, "street": "View St.", "city": "San Jose" }, "interests": [ ], "children": [ { "name": "Orval Eveleth", "age": null }, { "name": "Beth Eveleth", "age": 11 }, { "name": "Yuki Eveleth", "age": null }, { "name": "Alyse Eveleth", "age": 14 } ] }
+{ "cid": 696, "name": "Nadia Dunklee", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Mendy Dunklee", "age": 17 }, { "name": "Edgar Dunklee", "age": null }, { "name": "Pasquale Dunklee", "age": null }, { "name": "Colin Dunklee", "age": null } ] }
+{ "cid": 697, "name": "Claud Coffel", "age": 72, "address": { "number": 8483, "street": "Cedar St.", "city": "Mountain View" }, "interests": [ ], "children": [ { "name": "Katheleen Coffel", "age": 38 }, { "name": "Tashina Coffel", "age": null } ] }
+{ "cid": 698, "name": "Tawanna Zanin", "age": 60, "address": { "number": 7979, "street": "View St.", "city": "Seattle" }, "interests": [ ], "children": [ { "name": "Denny Zanin", "age": 31 }, { "name": "Danial Zanin", "age": 43 }, { "name": "Kenyetta Zanin", "age": null }, { "name": "Aleisha Zanin", "age": null } ] }
+{ "cid": 699, "name": "Lyda Golomb", "age": 46, "address": { "number": 5049, "street": "Main St.", "city": "Seattle" }, "interests": [ "Fishing", "Basketball" ], "children": [ { "name": "Shonta Golomb", "age": null }, { "name": "Lynwood Golomb", "age": 26 }, { "name": "Leonila Golomb", "age": 30 }, { "name": "Alejandrina Golomb", "age": null } ] }
+{ "cid": 700, "name": "Suk Blondin", "age": null, "address": null, "interests": [ "Wine" ], "children": [ { "name": "Brenton Blondin", "age": null }, { "name": "Charlotte Blondin", "age": null }, { "name": "Eldon Blondin", "age": 10 }, { "name": "Leanne Blondin", "age": null } ] }
+{ "cid": 702, "name": "Lane Krog", "age": 50, "address": { "number": 1646, "street": "Lake St.", "city": "Mountain View" }, "interests": [ "Running" ], "children": [ { "name": "Carri Krog", "age": null }, { "name": "Sage Krog", "age": null }, { "name": "Bronwyn Krog", "age": null } ] }
+{ "cid": 703, "name": "Susanne Pettey", "age": null, "address": null, "interests": [ "Squash", "Basketball", "Skiing" ], "children": [ { "name": "Nancey Pettey", "age": 35 }, { "name": "Lawana Pettey", "age": null }, { "name": "Percy Pettey", "age": 25 } ] }
+{ "cid": 704, "name": "Melodee Clemons", "age": null, "address": null, "interests": [ "Base Jumping", "Tennis", "Video Games" ], "children": [ { "name": "Doreatha Clemons", "age": 22 } ] }
+{ "cid": 705, "name": "Sofia Bonniwell", "age": 81, "address": { "number": 767, "street": "Cedar St.", "city": "Portland" }, "interests": [ "Basketball" ], "children": [ { "name": "Douglass Bonniwell", "age": 58 }, { "name": "Jackeline Bonniwell", "age": 16 } ] }
+{ "cid": 706, "name": "Miquel Caesar", "age": 16, "address": { "number": 2176, "street": "Park St.", "city": "Mountain View" }, "interests": [ ], "children": [ { "name": "Shaniqua Caesar", "age": null }, { "name": "Ellis Caesar", "age": null }, { "name": "Bruna Caesar", "age": null }, { "name": "Kayleen Caesar", "age": null } ] }
+{ "cid": 708, "name": "Elease Holtmann", "age": 75, "address": { "number": 5295, "street": "Washington St.", "city": "Los Angeles" }, "interests": [ ], "children": [ { "name": "Leonardo Holtmann", "age": null }, { "name": "Katharine Holtmann", "age": null }, { "name": "Chung Holtmann", "age": 20 }, { "name": "Teodoro Holtmann", "age": 19 } ] }
+{ "cid": 709, "name": "Jazmine Twiddy", "age": null, "address": null, "interests": [ "Puzzles", "Computers", "Wine" ], "children": [ { "name": "Veronika Twiddy", "age": 21 } ] }
+{ "cid": 710, "name": "Arlen Horka", "age": null, "address": null, "interests": [ "Movies", "Coffee", "Walking" ], "children": [ { "name": "Valencia Horka", "age": null }, { "name": "Wesley Horka", "age": null } ] }
+{ "cid": 711, "name": "Agnes Andreas", "age": null, "address": null, "interests": [ "Books" ], "children": [ { "name": "Fairy Andreas", "age": null }, { "name": "Wilhemina Andreas", "age": null }, { "name": "Parthenia Andreas", "age": 53 }, { "name": "Maye Andreas", "age": null } ] }
+{ "cid": 712, "name": "Jack Lamoreux", "age": 32, "address": { "number": 4486, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [ ], "children": [ { "name": "Rubin Lamoreux", "age": 15 }, { "name": "Jonelle Lamoreux", "age": 10 }, { "name": "Shonna Lamoreux", "age": null }, { "name": "India Lamoreux", "age": 17 } ] }
+{ "cid": 713, "name": "Galina Retterbush", "age": null, "address": null, "interests": [ "Bass", "Squash" ], "children": [ { "name": "Janene Retterbush", "age": null }, { "name": "Toby Retterbush", "age": 15 }, { "name": "Renato Retterbush", "age": null }, { "name": "Annice Retterbush", "age": 22 } ] }
+{ "cid": 715, "name": "Zoraida Scribner", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Ninfa Scribner", "age": 31 } ] }
+{ "cid": 716, "name": "Deirdre Bruderer", "age": null, "address": null, "interests": [ "Computers", "Wine" ], "children": [ { "name": "Coralee Bruderer", "age": null }, { "name": "Mina Bruderer", "age": null }, { "name": "Lindsey Bruderer", "age": 35 }, { "name": "Yi Bruderer", "age": null } ] }
+{ "cid": 717, "name": "Paulette Moccasin", "age": 87, "address": { "number": 1426, "street": "View St.", "city": "Portland" }, "interests": [ "Fishing" ], "children": [ { "name": "Savannah Moccasin", "age": null }, { "name": "Mariela Moccasin", "age": 34 }, { "name": "Isadora Moccasin", "age": null }, { "name": "Vivien Moccasin", "age": 31 } ] }
+{ "cid": 718, "name": "Tandy Trick", "age": 18, "address": { "number": 1215, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Fishing", "Fishing" ], "children": [ { "name": "Edyth Trick", "age": null }, { "name": "Jimmy Trick", "age": null }, { "name": "Jacquline Trick", "age": null }, { "name": "Tyler Trick", "age": null } ] }
+{ "cid": 719, "name": "Antoinette Boursiquot", "age": 47, "address": { "number": 3652, "street": "Cedar St.", "city": "Portland" }, "interests": [ ], "children": [ { "name": "Dennis Boursiquot", "age": null }, { "name": "Katelyn Boursiquot", "age": null }, { "name": "Gabrielle Boursiquot", "age": null }, { "name": "Deidre Boursiquot", "age": null } ] }
+{ "cid": 721, "name": "Jesica Tinder", "age": 28, "address": { "number": 5526, "street": "7th St.", "city": "Mountain View" }, "interests": [ ], "children": [ ] }
+{ "cid": 723, "name": "Teressa Krol", "age": 22, "address": { "number": 8036, "street": "Park St.", "city": "Seattle" }, "interests": [ "Music" ], "children": [ { "name": "Tuan Krol", "age": null }, { "name": "Judi Krol", "age": null }, { "name": "Maddie Krol", "age": null } ] }
+{ "cid": 724, "name": "Merle Bakula", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Margart Bakula", "age": 49 }, { "name": "Mathew Bakula", "age": 36 } ] }
+{ "cid": 725, "name": "Sallie Calderon", "age": null, "address": null, "interests": [ ], "children": [ ] }
+{ "cid": 726, "name": "Brinda Raudebaugh", "age": 83, "address": { "number": 7179, "street": "View St.", "city": "Mountain View" }, "interests": [ ], "children": [ ] }
+{ "cid": 727, "name": "Valene Resecker", "age": null, "address": null, "interests": [ "Music", "Wine", "Books", "Walking" ], "children": [ ] }
+{ "cid": 728, "name": "Bruno Freeburger", "age": 84, "address": { "number": 2482, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [ "Computers" ], "children": [ { "name": "Shizuko Freeburger", "age": null } ] }
+{ "cid": 730, "name": "Marti Vandoren", "age": null, "address": null, "interests": [ "Skiing", "Bass" ], "children": [ { "name": "Carroll Vandoren", "age": null }, { "name": "Lorretta Vandoren", "age": 30 }, { "name": "Chloe Vandoren", "age": 42 }, { "name": "Ilona Vandoren", "age": null } ] }
+{ "cid": 731, "name": "Yajaira Orto", "age": null, "address": null, "interests": [ "Music", "Databases" ], "children": [ { "name": "Eliz Orto", "age": 17 }, { "name": "Gisela Orto", "age": null } ] }
+{ "cid": 732, "name": "Dania Fabio", "age": null, "address": null, "interests": [ "Skiing" ], "children": [ { "name": "Virgie Fabio", "age": null }, { "name": "Nereida Fabio", "age": 37 } ] }
+{ "cid": 733, "name": "Edie Stager", "age": 26, "address": { "number": 2691, "street": "Park St.", "city": "Seattle" }, "interests": [ ], "children": [ { "name": "Ethyl Stager", "age": 10 } ] }
+{ "cid": 734, "name": "Lera Korn", "age": null, "address": null, "interests": [ "Tennis", "Puzzles", "Cigars" ], "children": [ { "name": "Criselda Korn", "age": 37 } ] }
+{ "cid": 736, "name": "Desmond Branam", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Manuel Branam", "age": 51 } ] }
+{ "cid": 737, "name": "Jeffrey Chesson", "age": 13, "address": { "number": 6833, "street": "Lake St.", "city": "Portland" }, "interests": [ "Tennis", "Computers" ], "children": [ { "name": "Clayton Chesson", "age": null }, { "name": "Yi Chesson", "age": null } ] }
+{ "cid": 738, "name": "Josphine Rohrer", "age": 75, "address": { "number": 862, "street": "Main St.", "city": "Los Angeles" }, "interests": [ "Databases" ], "children": [ { "name": "Marvin Rohrer", "age": 22 }, { "name": "Wyatt Rohrer", "age": null }, { "name": "Deloras Rohrer", "age": null } ] }
+{ "cid": 739, "name": "Libbie Thigpin", "age": null, "address": null, "interests": [ "Databases" ], "children": [ ] }
+{ "cid": 740, "name": "Thomasine Collado", "age": null, "address": null, "interests": [ "Music" ], "children": [ { "name": "Tabetha Collado", "age": null }, { "name": "Alline Collado", "age": null }, { "name": "Delisa Collado", "age": null }, { "name": "Jack Collado", "age": 56 } ] }
+{ "cid": 741, "name": "Lesia Risatti", "age": 48, "address": { "number": 7378, "street": "Cedar St.", "city": "Portland" }, "interests": [ "Fishing", "Wine", "Databases" ], "children": [ { "name": "Tangela Risatti", "age": null }, { "name": "Leonel Risatti", "age": 33 }, { "name": "Cythia Risatti", "age": 36 } ] }
+{ "cid": 742, "name": "Andy Schifo", "age": 36, "address": { "number": 4422, "street": "View St.", "city": "Los Angeles" }, "interests": [ "Basketball" ], "children": [ ] }
+{ "cid": 743, "name": "Nona Debroux", "age": null, "address": null, "interests": [ "Bass" ], "children": [ ] }
+{ "cid": 744, "name": "Crysta Christen", "age": 57, "address": { "number": 439, "street": "Hill St.", "city": "Portland" }, "interests": [ "Basketball", "Squash", "Base Jumping" ], "children": [ ] }
+{ "cid": 745, "name": "Tabatha Hagwell", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Gaynell Hagwell", "age": null } ] }
+{ "cid": 746, "name": "Rosalinda Pola", "age": null, "address": null, "interests": [ "Cooking", "Computers", "Walking", "Cigars" ], "children": [ { "name": "Maribel Pola", "age": 19 }, { "name": "Chaya Pola", "age": null }, { "name": "Shauna Pola", "age": null }, { "name": "Elenora Pola", "age": 22 } ] }
+{ "cid": 747, "name": "Gil Dunnaway", "age": 65, "address": { "number": 3022, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ "Running", "Squash" ], "children": [ { "name": "Laurice Dunnaway", "age": null } ] }
+{ "cid": 748, "name": "Petra Ganes", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Perry Ganes", "age": null }, { "name": "Krista Ganes", "age": 54 }, { "name": "Kayce Ganes", "age": 52 }, { "name": "Eleni Ganes", "age": null } ] }
+{ "cid": 749, "name": "Pearle Mauney", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Delpha Mauney", "age": null }, { "name": "Micki Mauney", "age": 28 }, { "name": "Wayne Mauney", "age": null } ] }
+{ "cid": 750, "name": "Rosaura Gaul", "age": null, "address": null, "interests": [ "Music", "Books", "Tennis" ], "children": [ { "name": "Letisha Gaul", "age": 41 } ] }
+{ "cid": 751, "name": "Lydia Iannelli", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Teri Iannelli", "age": 36 } ] }
+{ "cid": 752, "name": "Maria Lebovic", "age": null, "address": null, "interests": [ "Bass" ], "children": [ { "name": "Thi Lebovic", "age": null }, { "name": "Rosamaria Lebovic", "age": 23 }, { "name": "Brinda Lebovic", "age": 39 } ] }
+{ "cid": 753, "name": "Maris Bannett", "age": null, "address": null, "interests": [ "Fishing", "Cigars", "Running" ], "children": [ { "name": "Libbie Bannett", "age": 11 }, { "name": "Francina Bannett", "age": 21 }, { "name": "Tuyet Bannett", "age": null }, { "name": "Zona Bannett", "age": 32 } ] }
+{ "cid": 754, "name": "Luetta Joern", "age": 25, "address": { "number": 5554, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ ], "children": [ { "name": "Hildegarde Joern", "age": null }, { "name": "Lorenza Joern", "age": 13 } ] }
+{ "cid": 755, "name": "Bette Trentz", "age": 57, "address": { "number": 2794, "street": "Park St.", "city": "Portland" }, "interests": [ ], "children": [ { "name": "Christa Trentz", "age": 14 }, { "name": "Jestine Trentz", "age": 22 }, { "name": "Shantel Trentz", "age": 37 }, { "name": "Jacklyn Trentz", "age": null } ] }
+{ "cid": 756, "name": "Marisol Noyes", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Delora Noyes", "age": null }, { "name": "Jonelle Noyes", "age": 44 } ] }
+{ "cid": 758, "name": "Akiko Hoenstine", "age": 56, "address": { "number": 8888, "street": "Lake St.", "city": "Portland" }, "interests": [ "Movies", "Walking" ], "children": [ { "name": "Maren Hoenstine", "age": null }, { "name": "Tyler Hoenstine", "age": null }, { "name": "Jesse Hoenstine", "age": 40 } ] }
+{ "cid": 759, "name": "Alaina Dadds", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Athena Dadds", "age": 36 }, { "name": "Denis Dadds", "age": null }, { "name": "Nathanial Dadds", "age": 42 }, { "name": "Molly Dadds", "age": null } ] }
+{ "cid": 761, "name": "Adele Henrikson", "age": null, "address": null, "interests": [ "Cooking", "Bass" ], "children": [ { "name": "Paulina Henrikson", "age": null }, { "name": "David Henrikson", "age": null }, { "name": "Jose Henrikson", "age": null }, { "name": "Meg Henrikson", "age": null } ] }
+{ "cid": 763, "name": "Candis Deya", "age": null, "address": null, "interests": [ "Computers" ], "children": [ { "name": "Lise Deya", "age": null }, { "name": "Jeni Deya", "age": 52 }, { "name": "Domonique Deya", "age": 24 }, { "name": "Rubie Deya", "age": null } ] }
+{ "cid": 766, "name": "Tosha Loffredo", "age": 64, "address": { "number": 5580, "street": "View St.", "city": "Mountain View" }, "interests": [ "Walking" ], "children": [ { "name": "Hellen Loffredo", "age": 32 } ] }
+{ "cid": 767, "name": "Wendi Hoecker", "age": null, "address": null, "interests": [ ], "children": [ ] }
+{ "cid": 768, "name": "Adelina Troendle", "age": null, "address": null, "interests": [ "Computers" ], "children": [ { "name": "Lenna Troendle", "age": 51 }, { "name": "Ines Troendle", "age": 48 }, { "name": "Ora Troendle", "age": null } ] }
+{ "cid": 769, "name": "Isaias Tenny", "age": 71, "address": { "number": 270, "street": "Park St.", "city": "Portland" }, "interests": [ "Wine", "Fishing", "Base Jumping" ], "children": [ { "name": "Theo Tenny", "age": null }, { "name": "Shena Tenny", "age": null }, { "name": "Coralee Tenny", "age": null }, { "name": "Orval Tenny", "age": 39 } ] }
+{ "cid": 770, "name": "Merrill Tilson", "age": null, "address": null, "interests": [ "Computers", "Skiing" ], "children": [ { "name": "Elna Tilson", "age": null } ] }
+{ "cid": 771, "name": "Marisela Tredo", "age": null, "address": null, "interests": [ "Tennis", "Coffee" ], "children": [ { "name": "Ardell Tredo", "age": 21 }, { "name": "Evelynn Tredo", "age": 16 } ] }
+{ "cid": 773, "name": "Leatrice Zysett", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Bee Zysett", "age": 30 }, { "name": "Russ Zysett", "age": 11 }, { "name": "Jeff Zysett", "age": 39 }, { "name": "Herman Zysett", "age": 27 } ] }
+{ "cid": 774, "name": "Nadene Rigel", "age": null, "address": null, "interests": [ "Cigars", "Cigars" ], "children": [ { "name": "Rebbeca Rigel", "age": 33 } ] }
+{ "cid": 776, "name": "Dagmar Sarkis", "age": null, "address": null, "interests": [ "Basketball", "Running", "Wine" ], "children": [ { "name": "Tari Sarkis", "age": null }, { "name": "Rana Sarkis", "age": 56 }, { "name": "Merissa Sarkis", "age": null }, { "name": "Lori Sarkis", "age": 26 } ] }
+{ "cid": 777, "name": "Coralee Vaugh", "age": 51, "address": { "number": 4130, "street": "Hill St.", "city": "San Jose" }, "interests": [ ], "children": [ { "name": "Dean Vaugh", "age": 31 }, { "name": "Stanton Vaugh", "age": 39 }, { "name": "Marti Vaugh", "age": 33 }, { "name": "Eden Vaugh", "age": 27 } ] }
+{ "cid": 778, "name": "Shellie Sario", "age": null, "address": null, "interests": [ "Puzzles" ], "children": [ ] }
+{ "cid": 779, "name": "Vinita Bockskopf", "age": null, "address": null, "interests": [ "Tennis", "Video Games" ], "children": [ ] }
+{ "cid": 780, "name": "Penny Poortinga", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Estella Poortinga", "age": null } ] }
+{ "cid": 781, "name": "Christy Darcangelo", "age": 42, "address": { "number": 2178, "street": "Washington St.", "city": "Portland" }, "interests": [ "Computers", "Fishing" ], "children": [ { "name": "Luis Darcangelo", "age": 21 }, { "name": "Omega Darcangelo", "age": 26 }, { "name": "Remedios Darcangelo", "age": 28 }, { "name": "Domenic Darcangelo", "age": 21 } ] }
+{ "cid": 782, "name": "Shameka Haifa", "age": 16, "address": { "number": 9555, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Cigars", "Computers", "Coffee", "Skiing" ], "children": [ { "name": "Dannette Haifa", "age": null } ] }
+{ "cid": 783, "name": "Johnnie Kesby", "age": 56, "address": { "number": 9798, "street": "View St.", "city": "Seattle" }, "interests": [ "Puzzles", "Tennis" ], "children": [ ] }
+{ "cid": 784, "name": "Omar Hasen", "age": null, "address": null, "interests": [ "Movies" ], "children": [ { "name": "Hugh Hasen", "age": null } ] }
+{ "cid": 785, "name": "Gabriel Breidel", "age": 32, "address": { "number": 9288, "street": "Park St.", "city": "San Jose" }, "interests": [ "Cigars", "Bass" ], "children": [ { "name": "Bernie Breidel", "age": null } ] }
+{ "cid": 786, "name": "Johnsie Maheux", "age": null, "address": null, "interests": [ "Cigars" ], "children": [ { "name": "Danuta Maheux", "age": null } ] }
+{ "cid": 787, "name": "Sara Yerly", "age": 12, "address": { "number": 872, "street": "7th St.", "city": "Seattle" }, "interests": [ "Fishing" ], "children": [ { "name": "Nettie Yerly", "age": null }, { "name": "Regine Yerly", "age": null }, { "name": "Hyo Yerly", "age": null } ] }
+{ "cid": 789, "name": "Carli Notto", "age": null, "address": null, "interests": [ "Cigars" ], "children": [ ] }
+{ "cid": 790, "name": "Dustin Brumble", "age": null, "address": null, "interests": [ "Computers", "Databases", "Tennis" ], "children": [ { "name": "Oda Brumble", "age": null }, { "name": "Jennefer Brumble", "age": 26 }, { "name": "Ricardo Brumble", "age": 37 }, { "name": "Graciela Brumble", "age": 10 } ] }
+{ "cid": 791, "name": "Jame Apresa", "age": 66, "address": { "number": 8417, "street": "Main St.", "city": "San Jose" }, "interests": [ "Running", "Puzzles", "Base Jumping" ], "children": [ { "name": "Awilda Apresa", "age": null }, { "name": "Nelle Apresa", "age": 40 }, { "name": "Terrell Apresa", "age": null }, { "name": "Malia Apresa", "age": 43 } ] }
+{ "cid": 793, "name": "Shondra Gollman", "age": null, "address": null, "interests": [ "Skiing" ], "children": [ { "name": "Paul Gollman", "age": 30 }, { "name": "Katherina Gollman", "age": 53 } ] }
+{ "cid": 794, "name": "Annabel Leins", "age": 75, "address": { "number": 9761, "street": "Park St.", "city": "Los Angeles" }, "interests": [ "Bass", "Computers", "Bass", "Cigars" ], "children": [ { "name": "Oswaldo Leins", "age": 21 } ] }
+{ "cid": 795, "name": "Sharilyn Branstad", "age": null, "address": null, "interests": [ "Databases", "Music" ], "children": [ { "name": "Ashlee Branstad", "age": 24 }, { "name": "Bobbye Branstad", "age": 26 }, { "name": "Natalya Branstad", "age": null }, { "name": "Edith Branstad", "age": null } ] }
+{ "cid": 796, "name": "Daniele Brisk", "age": null, "address": null, "interests": [ "Walking", "Bass" ], "children": [ ] }
+{ "cid": 797, "name": "Frederica Kale", "age": 77, "address": { "number": 6861, "street": "Oak St.", "city": "Los Angeles" }, "interests": [ "Puzzles", "Bass" ], "children": [ { "name": "Shanice Kale", "age": null }, { "name": "Soraya Kale", "age": 64 }, { "name": "Laurena Kale", "age": 57 } ] }
+{ "cid": 799, "name": "Ronny Piefer", "age": 45, "address": { "number": 7724, "street": "7th St.", "city": "Mountain View" }, "interests": [ "Fishing" ], "children": [ { "name": "Chantal Piefer", "age": 24 }, { "name": "Tiffany Piefer", "age": null }, { "name": "Farrah Piefer", "age": 21 }, { "name": "Dee Piefer", "age": null } ] }
+{ "cid": 800, "name": "Karon Johnsen", "age": null, "address": null, "interests": [ "Movies" ], "children": [ { "name": "Roselee Johnsen", "age": 25 } ] }
+{ "cid": 802, "name": "Sang Hollman", "age": null, "address": null, "interests": [ "Skiing" ], "children": [ { "name": "Carman Hollman", "age": null }, { "name": "Kirstie Hollman", "age": 40 }, { "name": "Jacquetta Hollman", "age": null } ] }
+{ "cid": 803, "name": "Yolonda Korf", "age": null, "address": null, "interests": [ "Bass", "Skiing", "Music" ], "children": [ { "name": "Ivette Korf", "age": null }, { "name": "Lashon Korf", "age": null } ] }
+{ "cid": 804, "name": "Joaquina Burlin", "age": 77, "address": { "number": 5479, "street": "7th St.", "city": "Sunnyvale" }, "interests": [ "Running", "Wine", "Running" ], "children": [ ] }
+{ "cid": 805, "name": "Gaylord Ginder", "age": null, "address": null, "interests": [ "Databases", "Coffee" ], "children": [ { "name": "Lucina Ginder", "age": null }, { "name": "Harriett Ginder", "age": null } ] }
+{ "cid": 806, "name": "Corliss Sharratt", "age": null, "address": null, "interests": [ "Basketball", "Cigars", "Cooking" ], "children": [ { "name": "Albertine Sharratt", "age": null }, { "name": "Nobuko Sharratt", "age": 29 }, { "name": "Neil Sharratt", "age": null } ] }
+{ "cid": 807, "name": "Maryanne Kuzminski", "age": 21, "address": { "number": 1601, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ "Running" ], "children": [ { "name": "India Kuzminski", "age": null }, { "name": "Adell Kuzminski", "age": null } ] }
+{ "cid": 808, "name": "Brande Decius", "age": null, "address": null, "interests": [ "Basketball", "Fishing", "Puzzles" ], "children": [ { "name": "Li Decius", "age": 56 }, { "name": "Eusebio Decius", "age": 50 }, { "name": "Clementina Decius", "age": 29 } ] }
+{ "cid": 809, "name": "Dagny Mangiaracina", "age": 44, "address": { "number": 5993, "street": "Lake St.", "city": "San Jose" }, "interests": [ ], "children": [ { "name": "Bari Mangiaracina", "age": 31 }, { "name": "Tiara Mangiaracina", "age": 12 }, { "name": "Milly Mangiaracina", "age": null }, { "name": "Chelsie Mangiaracina", "age": null } ] }
+{ "cid": 810, "name": "Myron Dumlao", "age": null, "address": null, "interests": [ "Wine", "Coffee" ], "children": [ { "name": "Josie Dumlao", "age": 36 } ] }
+{ "cid": 811, "name": "Marti Whitmyre", "age": null, "address": null, "interests": [ "Music", "Walking" ], "children": [ ] }
+{ "cid": 812, "name": "Bee Godette", "age": 26, "address": { "number": 1757, "street": "Washington St.", "city": "Portland" }, "interests": [ "Video Games", "Base Jumping", "Tennis" ], "children": [ { "name": "Madaline Godette", "age": 10 }, { "name": "Shasta Godette", "age": 15 }, { "name": "Parthenia Godette", "age": 11 }, { "name": "Priscila Godette", "age": 13 } ] }
+{ "cid": 813, "name": "Leann Domagala", "age": 47, "address": { "number": 4472, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [ "Computers" ], "children": [ { "name": "Alvera Domagala", "age": 36 }, { "name": "Rosalva Domagala", "age": 27 }, { "name": "Eugenia Domagala", "age": null }, { "name": "My Domagala", "age": 32 } ] }
+{ "cid": 814, "name": "Harriette Kasmarek", "age": 68, "address": { "number": 7191, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ "Music", "Skiing" ], "children": [ { "name": "Melani Kasmarek", "age": 24 }, { "name": "Jesica Kasmarek", "age": 22 } ] }
+{ "cid": 815, "name": "Leigha Bires", "age": 11, "address": { "number": 7263, "street": "Oak St.", "city": "Portland" }, "interests": [ "Running" ], "children": [ { "name": "Val Bires", "age": null } ] }
+{ "cid": 816, "name": "Cheyenne Eddie", "age": null, "address": null, "interests": [ "Walking", "Cooking" ], "children": [ { "name": "Kathe Eddie", "age": null }, { "name": "Charles Eddie", "age": null } ] }
+{ "cid": 818, "name": "Nellie Whetzell", "age": null, "address": null, "interests": [ "Walking" ], "children": [ ] }
+{ "cid": 819, "name": "Twanna Finnley", "age": null, "address": null, "interests": [ "Squash", "Cigars" ], "children": [ { "name": "Reba Finnley", "age": null }, { "name": "Moises Finnley", "age": null } ] }
+{ "cid": 820, "name": "Lacy Caudill", "age": 22, "address": { "number": 8679, "street": "Main St.", "city": "Mountain View" }, "interests": [ "Wine" ], "children": [ { "name": "Sybil Caudill", "age": null } ] }
+{ "cid": 821, "name": "Carole Edlund", "age": 76, "address": { "number": 4008, "street": "Park St.", "city": "Los Angeles" }, "interests": [ "Computers", "Cooking", "Running", "Basketball" ], "children": [ { "name": "Garfield Edlund", "age": 54 }, { "name": "Brooks Edlund", "age": null }, { "name": "Gertrudis Edlund", "age": null }, { "name": "Tabitha Edlund", "age": 58 } ] }
+{ "cid": 824, "name": "Vonda Czaplewski", "age": 72, "address": { "number": 4597, "street": "7th St.", "city": "Portland" }, "interests": [ "Skiing" ], "children": [ { "name": "Gaynelle Czaplewski", "age": null }, { "name": "India Czaplewski", "age": null } ] }
+{ "cid": 825, "name": "Kirstie Rinebold", "age": 57, "address": { "number": 9463, "street": "Oak St.", "city": "Portland" }, "interests": [ "Cooking", "Cigars", "Books" ], "children": [ { "name": "Vonda Rinebold", "age": null }, { "name": "Man Rinebold", "age": 21 } ] }
+{ "cid": 826, "name": "Ressie Feenstra", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Sasha Feenstra", "age": null } ] }
+{ "cid": 827, "name": "Clementina Papin", "age": null, "address": null, "interests": [ "Music", "Basketball", "Cigars" ], "children": [ { "name": "Catina Papin", "age": null }, { "name": "Demetrius Papin", "age": 59 }, { "name": "Marylou Papin", "age": 12 }, { "name": "Apryl Papin", "age": 16 } ] }
+{ "cid": 828, "name": "Marcelle Steinhour", "age": null, "address": null, "interests": [ "Running", "Basketball", "Walking" ], "children": [ { "name": "Jimmie Steinhour", "age": 13 }, { "name": "Kirstie Steinhour", "age": 19 } ] }
+{ "cid": 831, "name": "Raina Rys", "age": 62, "address": { "number": 7048, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ "Walking" ], "children": [ { "name": "Ezra Rys", "age": null }, { "name": "Carl Rys", "age": null }, { "name": "Loraine Rys", "age": null } ] }
+{ "cid": 832, "name": "Alina Hosley", "age": null, "address": null, "interests": [ "Databases", "Databases", "Music" ], "children": [ { "name": "Sebrina Hosley", "age": null }, { "name": "Dyan Hosley", "age": null } ] }
+{ "cid": 833, "name": "Lakisha Petkoff", "age": null, "address": null, "interests": [ "Coffee" ], "children": [ { "name": "Brittanie Petkoff", "age": null }, { "name": "Ashli Petkoff", "age": null } ] }
+{ "cid": 834, "name": "Luvenia Grandstaff", "age": null, "address": null, "interests": [ "Squash" ], "children": [ { "name": "Joleen Grandstaff", "age": 28 }, { "name": "Elvera Grandstaff", "age": null }, { "name": "Leonia Grandstaff", "age": 35 }, { "name": "Jaclyn Grandstaff", "age": 28 } ] }
+{ "cid": 835, "name": "Raphael Marzili", "age": null, "address": null, "interests": [ "Music" ], "children": [ { "name": "Angelic Marzili", "age": 38 } ] }
+{ "cid": 836, "name": "Elden Shumski", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Weldon Shumski", "age": null }, { "name": "Anneliese Shumski", "age": null } ] }
+{ "cid": 837, "name": "Denice Wolken", "age": 28, "address": { "number": 5010, "street": "7th St.", "city": "Mountain View" }, "interests": [ ], "children": [ { "name": "Kattie Wolken", "age": null } ] }
+{ "cid": 838, "name": "Karan Aharon", "age": 88, "address": { "number": 8033, "street": "Washington St.", "city": "Portland" }, "interests": [ "Computers", "Movies", "Walking" ], "children": [ { "name": "Matha Aharon", "age": 16 } ] }
+{ "cid": 841, "name": "Omar Enwall", "age": null, "address": null, "interests": [ "Skiing", "Skiing", "Books" ], "children": [ { "name": "Kirby Enwall", "age": 31 }, { "name": "Cythia Enwall", "age": 24 }, { "name": "August Enwall", "age": null } ] }
+{ "cid": 843, "name": "Lenny Acerno", "age": 64, "address": { "number": 7656, "street": "Main St.", "city": "Seattle" }, "interests": [ "Base Jumping", "Squash" ], "children": [ ] }
+{ "cid": 844, "name": "Madelene Ten", "age": null, "address": null, "interests": [ "Squash" ], "children": [ { "name": "Johanne Ten", "age": 39 }, { "name": "Lurline Ten", "age": null }, { "name": "Cathy Ten", "age": 49 } ] }
+{ "cid": 845, "name": "Burt Earp", "age": 21, "address": { "number": 7626, "street": "Lake St.", "city": "Seattle" }, "interests": [ "Computers" ], "children": [ { "name": "Denny Earp", "age": null }, { "name": "Blaine Earp", "age": null }, { "name": "Wilson Earp", "age": 10 }, { "name": "Joan Earp", "age": null } ] }
+{ "cid": 846, "name": "Kieth Norlund", "age": 15, "address": { "number": 4039, "street": "Park St.", "city": "Mountain View" }, "interests": [ "Wine", "Walking", "Puzzles" ], "children": [ { "name": "Shawn Norlund", "age": null } ] }
+{ "cid": 847, "name": "Ashton Korba", "age": 25, "address": { "number": 6450, "street": "Park St.", "city": "Sunnyvale" }, "interests": [ "Cigars", "Computers", "Walking", "Video Games" ], "children": [ ] }
+{ "cid": 848, "name": "Myrta Kopf", "age": null, "address": null, "interests": [ "Wine", "Basketball", "Base Jumping" ], "children": [ ] }
+{ "cid": 850, "name": "Garnet Younce", "age": null, "address": null, "interests": [ "Databases", "Video Games", "Books" ], "children": [ { "name": "Syble Younce", "age": 16 } ] }
+{ "cid": 851, "name": "Darrel Machia", "age": 31, "address": { "number": 3290, "street": "View St.", "city": "Seattle" }, "interests": [ ], "children": [ { "name": "Coy Machia", "age": 13 }, { "name": "Janean Machia", "age": 13 }, { "name": "Sandi Machia", "age": 18 } ] }
+{ "cid": 852, "name": "Terrell Ramsay", "age": null, "address": null, "interests": [ ], "children": [ ] }
+{ "cid": 853, "name": "Denisse Peralto", "age": 25, "address": { "number": 3931, "street": "7th St.", "city": "Portland" }, "interests": [ "Tennis", "Walking", "Basketball" ], "children": [ { "name": "Asha Peralto", "age": 14 }, { "name": "Clark Peralto", "age": null }, { "name": "Jessika Peralto", "age": null }, { "name": "Nadene Peralto", "age": null } ] }
+{ "cid": 854, "name": "Angie Oyster", "age": 32, "address": { "number": 8860, "street": "Main St.", "city": "San Jose" }, "interests": [ "Coffee", "Movies", "Fishing" ], "children": [ { "name": "Hugh Oyster", "age": 10 } ] }
+{ "cid": 855, "name": "Rosette Reen", "age": 57, "address": { "number": 2767, "street": "Lake St.", "city": "Mountain View" }, "interests": [ "Basketball" ], "children": [ ] }
+{ "cid": 857, "name": "Kasie Fujioka", "age": null, "address": null, "interests": [ "Skiing", "Cigars" ], "children": [ { "name": "Leontine Fujioka", "age": null }, { "name": "Nga Fujioka", "age": 21 }, { "name": "Nathanael Fujioka", "age": 27 } ] }
+{ "cid": 858, "name": "Maricruz Dittberner", "age": null, "address": null, "interests": [ "Tennis", "Wine", "Cigars", "Video Games" ], "children": [ ] }
+{ "cid": 859, "name": "Mozelle Catillo", "age": 61, "address": { "number": 253, "street": "View St.", "city": "Los Angeles" }, "interests": [ "Databases", "Cooking", "Wine" ], "children": [ ] }
+{ "cid": 860, "name": "Isabelle Sept", "age": 88, "address": { "number": 4382, "street": "Washington St.", "city": "Portland" }, "interests": [ "Puzzles", "Books" ], "children": [ ] }
+{ "cid": 861, "name": "Hugh Mcbrien", "age": null, "address": null, "interests": [ "Skiing", "Cigars", "Cooking" ], "children": [ { "name": "Otha Mcbrien", "age": 38 } ] }
+{ "cid": 862, "name": "Constance Bries", "age": 77, "address": { "number": 2585, "street": "Oak St.", "city": "Los Angeles" }, "interests": [ ], "children": [ { "name": "Lizzie Bries", "age": 42 }, { "name": "Shenika Bries", "age": null }, { "name": "Phillip Bries", "age": null } ] }
+{ "cid": 864, "name": "Katharyn Zanotti", "age": 62, "address": { "number": 8336, "street": "7th St.", "city": "Sunnyvale" }, "interests": [ "Puzzles" ], "children": [ { "name": "Magan Zanotti", "age": null }, { "name": "Jacinto Zanotti", "age": null } ] }
+{ "cid": 865, "name": "Moon Marino", "age": 43, "address": { "number": 5710, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ "Skiing" ], "children": [ { "name": "Markita Marino", "age": 10 } ] }
+{ "cid": 866, "name": "Bonita Kauphusman", "age": null, "address": null, "interests": [ ], "children": [ ] }
+{ "cid": 869, "name": "Lino Wooderson", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Nola Wooderson", "age": null }, { "name": "Leticia Wooderson", "age": 36 }, { "name": "Bernardine Wooderson", "age": null } ] }
+{ "cid": 870, "name": "Natosha Lufsey", "age": null, "address": null, "interests": [ "Cigars", "Walking" ], "children": [ { "name": "Tiffany Lufsey", "age": null } ] }
+{ "cid": 871, "name": "Lona Dacus", "age": null, "address": null, "interests": [ "Base Jumping" ], "children": [ { "name": "Pablo Dacus", "age": null }, { "name": "Darlene Dacus", "age": 45 }, { "name": "Darius Dacus", "age": 31 }, { "name": "Cordia Dacus", "age": null } ] }
+{ "cid": 872, "name": "Michele Herschel", "age": 39, "address": { "number": 4287, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [ ], "children": [ ] }
+{ "cid": 875, "name": "Ramon Crepps", "age": null, "address": null, "interests": [ "Coffee", "Movies", "Skiing" ], "children": [ { "name": "Elisha Crepps", "age": null } ] }
+{ "cid": 876, "name": "Chelsie Motten", "age": null, "address": null, "interests": [ "Music", "Squash", "Music", "Walking" ], "children": [ { "name": "Nida Motten", "age": null }, { "name": "Taneka Motten", "age": 10 }, { "name": "Maynard Motten", "age": 57 } ] }
+{ "cid": 877, "name": "Nicki Lipkind", "age": null, "address": null, "interests": [ "Books", "Movies" ], "children": [ { "name": "Yahaira Lipkind", "age": 12 } ] }
+{ "cid": 878, "name": "Migdalia Bisker", "age": 50, "address": { "number": 6699, "street": "Oak St.", "city": "Los Angeles" }, "interests": [ "Computers", "Basketball" ], "children": [ { "name": "Moira Bisker", "age": null }, { "name": "Tanisha Bisker", "age": null } ] }
+{ "cid": 879, "name": "Vinnie Antoniewicz", "age": 45, "address": { "number": 1633, "street": "Hill St.", "city": "Seattle" }, "interests": [ "Cooking", "Puzzles" ], "children": [ ] }
+{ "cid": 880, "name": "Sara Abo", "age": null, "address": null, "interests": [ "Squash" ], "children": [ ] }
+{ "cid": 881, "name": "Leora Chesnutt", "age": 49, "address": { "number": 6487, "street": "Hill St.", "city": "Seattle" }, "interests": [ "Movies" ], "children": [ { "name": "Myrtle Chesnutt", "age": null }, { "name": "Serina Chesnutt", "age": 11 }, { "name": "Jana Chesnutt", "age": 10 } ] }
+{ "cid": 883, "name": "Odilia Bugtong", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Mark Bugtong", "age": 15 }, { "name": "Paula Bugtong", "age": null }, { "name": "Jenee Bugtong", "age": 17 }, { "name": "Lilian Bugtong", "age": 44 } ] }
+{ "cid": 884, "name": "Laila Marta", "age": null, "address": null, "interests": [ "Fishing", "Movies" ], "children": [ { "name": "Carlota Marta", "age": 19 } ] }
+{ "cid": 885, "name": "Les Legere", "age": 87, "address": { "number": 3998, "street": "Cedar St.", "city": "Portland" }, "interests": [ "Bass", "Tennis", "Fishing" ], "children": [ { "name": "Concetta Legere", "age": 45 }, { "name": "Tamica Legere", "age": null }, { "name": "Aurora Legere", "age": null } ] }
+{ "cid": 887, "name": "Jermaine Folz", "age": 35, "address": { "number": 8487, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ "Computers", "Puzzles", "Cooking" ], "children": [ { "name": "Sharice Folz", "age": null } ] }
+{ "cid": 888, "name": "Natalie Nocella", "age": 66, "address": { "number": 2856, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ ], "children": [ { "name": "Noel Nocella", "age": 26 }, { "name": "Damon Nocella", "age": 29 }, { "name": "Joesph Nocella", "age": 33 }, { "name": "Nidia Nocella", "age": null } ] }
+{ "cid": 889, "name": "Elvis Schoff", "age": 83, "address": { "number": 6724, "street": "Hill St.", "city": "Mountain View" }, "interests": [ ], "children": [ { "name": "Spring Schoff", "age": 43 }, { "name": "Davis Schoff", "age": 55 }, { "name": "Ryann Schoff", "age": 58 }, { "name": "Clarinda Schoff", "age": 11 } ] }
+{ "cid": 890, "name": "Janise Maccarthy", "age": 66, "address": { "number": 7337, "street": "Main St.", "city": "San Jose" }, "interests": [ "Wine", "Computers" ], "children": [ ] }
+{ "cid": 891, "name": "Jesusita Bhatia", "age": 57, "address": { "number": 1476, "street": "Lake St.", "city": "Mountain View" }, "interests": [ "Walking" ], "children": [ ] }
+{ "cid": 892, "name": "Madge Hendson", "age": 79, "address": { "number": 8832, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Databases", "Fishing", "Skiing" ], "children": [ { "name": "Elia Hendson", "age": 48 }, { "name": "Lashawn Hendson", "age": 27 } ] }
+{ "cid": 893, "name": "Norberto Banchero", "age": null, "address": null, "interests": [ ], "children": [ ] }
+{ "cid": 894, "name": "Reginald Julien", "age": 16, "address": { "number": 1107, "street": "Lake St.", "city": "Mountain View" }, "interests": [ "Databases", "Wine" ], "children": [ { "name": "Arthur Julien", "age": null }, { "name": "Evia Julien", "age": null } ] }
+{ "cid": 897, "name": "Gerald Roehrman", "age": null, "address": null, "interests": [ "Bass", "Wine" ], "children": [ { "name": "Virgie Roehrman", "age": 28 }, { "name": "Akiko Roehrman", "age": 59 }, { "name": "Robbie Roehrman", "age": 10 }, { "name": "Flavia Roehrman", "age": null } ] }
+{ "cid": 898, "name": "Thao Seufert", "age": 78, "address": { "number": 3529, "street": "Hill St.", "city": "Seattle" }, "interests": [ "Bass", "Squash", "Coffee" ], "children": [ { "name": "Classie Seufert", "age": null } ] }
+{ "cid": 899, "name": "Ada Kamealoha", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Juliann Kamealoha", "age": null }, { "name": "Ilana Kamealoha", "age": 25 }, { "name": "Herminia Kamealoha", "age": 55 }, { "name": "Carli Kamealoha", "age": null } ] }
+{ "cid": 901, "name": "Riva Ziko", "age": null, "address": null, "interests": [ "Running", "Tennis", "Video Games" ], "children": [ { "name": "Leandra Ziko", "age": 49 }, { "name": "Torrie Ziko", "age": null } ] }
+{ "cid": 903, "name": "Elise Morenz", "age": 17, "address": { "number": 8968, "street": "View St.", "city": "Mountain View" }, "interests": [ ], "children": [ ] }
+{ "cid": 904, "name": "Holley Tofil", "age": 51, "address": { "number": 8946, "street": "Oak St.", "city": "Mountain View" }, "interests": [ "Music", "Squash" ], "children": [ { "name": "Kristal Tofil", "age": null } ] }
+{ "cid": 905, "name": "Pandora Azzarella", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Lane Azzarella", "age": null }, { "name": "Joi Azzarella", "age": 19 } ] }
+{ "cid": 907, "name": "Princess Sudol", "age": 73, "address": { "number": 9770, "street": "Oak St.", "city": "San Jose" }, "interests": [ "Computers", "Base Jumping" ], "children": [ { "name": "Bronwyn Sudol", "age": 22 }, { "name": "Judith Sudol", "age": null } ] }
+{ "cid": 908, "name": "Ferdinand Auila", "age": 82, "address": { "number": 1071, "street": "Lake St.", "city": "Portland" }, "interests": [ "Base Jumping", "Running", "Wine" ], "children": [ { "name": "Ai Auila", "age": 69 }, { "name": "Laurel Auila", "age": null } ] }
+{ "cid": 909, "name": "Mariko Sharar", "age": null, "address": null, "interests": [ "Squash", "Movies", "Computers" ], "children": [ ] }
+{ "cid": 910, "name": "Everette Moe", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Berna Moe", "age": 56 }, { "name": "Harold Moe", "age": 28 }, { "name": "See Moe", "age": 20 } ] }
+{ "cid": 911, "name": "Eileen Bartolomeo", "age": 20, "address": { "number": 8915, "street": "Main St.", "city": "Portland" }, "interests": [ ], "children": [ ] }
+{ "cid": 912, "name": "Alessandra Kaskey", "age": 52, "address": { "number": 6906, "street": "View St.", "city": "Los Angeles" }, "interests": [ "Skiing", "Walking", "Basketball" ], "children": [ { "name": "Mack Kaskey", "age": null } ] }
+{ "cid": 913, "name": "Evelynn Fague", "age": 42, "address": { "number": 5729, "street": "7th St.", "city": "Seattle" }, "interests": [ "Books", "Databases", "Cooking" ], "children": [ ] }
+{ "cid": 914, "name": "Hunter Flournoy", "age": null, "address": null, "interests": [ "Cooking", "Squash" ], "children": [ { "name": "Christopher Flournoy", "age": 59 }, { "name": "Earnestine Flournoy", "age": null } ] }
+{ "cid": 916, "name": "Kris Mcmarlin", "age": null, "address": null, "interests": [ "Movies", "Music", "Puzzles" ], "children": [ ] }
+{ "cid": 917, "name": "Jerri Blachowski", "age": null, "address": null, "interests": [ "Skiing" ], "children": [ { "name": "Chet Blachowski", "age": 43 }, { "name": "Mallory Blachowski", "age": null }, { "name": "Akilah Blachowski", "age": null } ] }
+{ "cid": 919, "name": "Fairy Wansley", "age": 45, "address": { "number": 9020, "street": "Park St.", "city": "Los Angeles" }, "interests": [ "Wine", "Walking", "Databases", "Video Games" ], "children": [ { "name": "Marvella Wansley", "age": null }, { "name": "Hisako Wansley", "age": null }, { "name": "Shaunta Wansley", "age": null }, { "name": "Gemma Wansley", "age": 21 } ] }
+{ "cid": 920, "name": "Mirtha Dellbringge", "age": null, "address": null, "interests": [ "Walking", "Basketball", "Basketball" ], "children": [ { "name": "Morgan Dellbringge", "age": 51 }, { "name": "Alease Dellbringge", "age": 35 } ] }
+{ "cid": 921, "name": "Mario Nolden", "age": 17, "address": { "number": 3977, "street": "Cedar St.", "city": "Sunnyvale" }, "interests": [ ], "children": [ { "name": "Gertrude Nolden", "age": null }, { "name": "Ray Nolden", "age": null }, { "name": "Inocencia Nolden", "age": null } ] }
+{ "cid": 922, "name": "Shanice Lingle", "age": 26, "address": { "number": 4753, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [ ], "children": [ { "name": "Sandie Lingle", "age": 12 }, { "name": "Nia Lingle", "age": 13 }, { "name": "Marilyn Lingle", "age": 15 } ] }
+{ "cid": 923, "name": "Bobbi Ursino", "age": null, "address": null, "interests": [ "Movies", "Books", "Walking" ], "children": [ { "name": "Shon Ursino", "age": null }, { "name": "Lorean Ursino", "age": null } ] }
+{ "cid": 924, "name": "Kathleen Lash", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Clementina Lash", "age": 58 }, { "name": "Zula Lash", "age": null }, { "name": "Mellissa Lash", "age": 54 } ] }
+{ "cid": 925, "name": "Quintin Kizzie", "age": null, "address": null, "interests": [ "Computers", "Tennis", "Bass", "Movies" ], "children": [ { "name": "Julius Kizzie", "age": 11 }, { "name": "Melissia Kizzie", "age": null }, { "name": "Olga Kizzie", "age": 42 } ] }
+{ "cid": 927, "name": "Lillia Hartlein", "age": 55, "address": { "number": 5856, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ "Base Jumping", "Coffee", "Cigars" ], "children": [ { "name": "Nicky Hartlein", "age": null }, { "name": "Cassaundra Hartlein", "age": 10 }, { "name": "Micheline Hartlein", "age": 26 }, { "name": "Anton Hartlein", "age": 32 } ] }
+{ "cid": 928, "name": "Maddie Diclaudio", "age": 33, "address": { "number": 4674, "street": "Washington St.", "city": "San Jose" }, "interests": [ "Base Jumping", "Databases", "Bass" ], "children": [ { "name": "Dominique Diclaudio", "age": 12 } ] }
+{ "cid": 929, "name": "Jean Guitierrez", "age": 75, "address": { "number": 9736, "street": "Lake St.", "city": "Mountain View" }, "interests": [ "Wine", "Wine", "Fishing" ], "children": [ ] }
+{ "cid": 930, "name": "Kathie Gier", "age": 37, "address": { "number": 5075, "street": "Main St.", "city": "Portland" }, "interests": [ ], "children": [ { "name": "Onie Gier", "age": 16 } ] }
+{ "cid": 931, "name": "Octavia Koiner", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Ardath Koiner", "age": 32 }, { "name": "Milly Koiner", "age": null }, { "name": "Arlinda Koiner", "age": null }, { "name": "Debby Koiner", "age": null } ] }
+{ "cid": 932, "name": "Kraig Bomia", "age": null, "address": null, "interests": [ "Music" ], "children": [ ] }
+{ "cid": 933, "name": "Eartha Hershberger", "age": 81, "address": { "number": 7013, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [ "Puzzles" ], "children": [ { "name": "Waneta Hershberger", "age": null }, { "name": "Katherine Hershberger", "age": 67 }, { "name": "Johnnie Hershberger", "age": 25 }, { "name": "Jovan Hershberger", "age": 30 } ] }
+{ "cid": 934, "name": "Dessie Lockmiller", "age": 70, "address": { "number": 4313, "street": "Lake St.", "city": "San Jose" }, "interests": [ "Coffee", "Puzzles" ], "children": [ ] }
+{ "cid": 935, "name": "Sharita Aspegren", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Russell Aspegren", "age": 35 }, { "name": "Bernardina Aspegren", "age": null }, { "name": "Isobel Aspegren", "age": 11 }, { "name": "Reva Aspegren", "age": null } ] }
+{ "cid": 937, "name": "Annika Pauline", "age": 78, "address": { "number": 8563, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ ], "children": [ { "name": "Mikki Pauline", "age": 34 } ] }
+{ "cid": 938, "name": "Parthenia Dromgoole", "age": 36, "address": { "number": 527, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ "Fishing" ], "children": [ ] }
+{ "cid": 940, "name": "Kitty Nalepka", "age": null, "address": null, "interests": [ "Movies", "Wine", "Basketball" ], "children": [ { "name": "Kendra Nalepka", "age": null } ] }
+{ "cid": 941, "name": "Jamey Jakobson", "age": null, "address": null, "interests": [ "Books", "Cooking", "Video Games" ], "children": [ { "name": "Elmer Jakobson", "age": 14 }, { "name": "Minh Jakobson", "age": 30 } ] }
+{ "cid": 942, "name": "Emerson Keblish", "age": null, "address": null, "interests": [ "Tennis" ], "children": [ { "name": "Leonora Keblish", "age": null } ] }
+{ "cid": 943, "name": "Kathryne Blacock", "age": 82, "address": { "number": 3510, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ "Running", "Bass", "Music" ], "children": [ ] }
+{ "cid": 944, "name": "Johana Hisman", "age": null, "address": null, "interests": [ "Wine" ], "children": [ { "name": "Kirstin Hisman", "age": 43 }, { "name": "Darwin Hisman", "age": 29 } ] }
+{ "cid": 945, "name": "Hildegard Dedinas", "age": 70, "address": { "number": 3273, "street": "View St.", "city": "Sunnyvale" }, "interests": [ ], "children": [ { "name": "Renato Dedinas", "age": 35 } ] }
+{ "cid": 946, "name": "Taylor Parrigan", "age": null, "address": null, "interests": [ "Music" ], "children": [ { "name": "Salome Parrigan", "age": 50 }, { "name": "Gary Parrigan", "age": 25 }, { "name": "Harold Parrigan", "age": null } ] }
+{ "cid": 948, "name": "Thad Scialpi", "age": 22, "address": { "number": 8731, "street": "Washington St.", "city": "Portland" }, "interests": [ "Base Jumping", "Tennis", "Wine" ], "children": [ { "name": "Harlan Scialpi", "age": 10 }, { "name": "Lucile Scialpi", "age": 11 }, { "name": "Audria Scialpi", "age": null } ] }
+{ "cid": 949, "name": "Elissa Rogue", "age": null, "address": null, "interests": [ "Fishing", "Music" ], "children": [ { "name": "Noriko Rogue", "age": 41 }, { "name": "Lavona Rogue", "age": 39 } ] }
+{ "cid": 950, "name": "Young Bayn", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Evangeline Bayn", "age": 38 }, { "name": "Darcy Bayn", "age": 45 }, { "name": "Rosita Bayn", "age": null }, { "name": "Austin Bayn", "age": 46 } ] }
+{ "cid": 951, "name": "Janine Martorano", "age": 65, "address": { "number": 6420, "street": "7th St.", "city": "Los Angeles" }, "interests": [ "Books", "Music" ], "children": [ { "name": "Idella Martorano", "age": null } ] }
+{ "cid": 955, "name": "Liliana Stenkamp", "age": null, "address": null, "interests": [ "Music" ], "children": [ ] }
+{ "cid": 956, "name": "Laquanda Bynoe", "age": 79, "address": { "number": 6122, "street": "Main St.", "city": "Portland" }, "interests": [ ], "children": [ { "name": "Joel Bynoe", "age": null }, { "name": "Brian Bynoe", "age": 61 }, { "name": "Shana Bynoe", "age": null } ] }
+{ "cid": 957, "name": "Lucius Schurr", "age": 75, "address": { "number": 3918, "street": "Main St.", "city": "Mountain View" }, "interests": [ ], "children": [ { "name": "Willetta Schurr", "age": 22 }, { "name": "Andre Schurr", "age": null }, { "name": "Merrilee Schurr", "age": 32 } ] }
+{ "cid": 958, "name": "Ricardo Pezzica", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Delois Pezzica", "age": 11 } ] }
+{ "cid": 960, "name": "Lenore Limardi", "age": null, "address": null, "interests": [ "Music" ], "children": [ { "name": "Kris Limardi", "age": 12 } ] }
+{ "cid": 961, "name": "Mirian Herpolsheimer", "age": null, "address": null, "interests": [ "Music", "Fishing", "Computers" ], "children": [ { "name": "Larissa Herpolsheimer", "age": 41 }, { "name": "Markus Herpolsheimer", "age": null }, { "name": "Natacha Herpolsheimer", "age": null } ] }
+{ "cid": 962, "name": "Taryn Coley", "age": null, "address": null, "interests": [ "Running", "Basketball", "Cooking" ], "children": [ ] }
+{ "cid": 963, "name": "Mila Ditmars", "age": 29, "address": { "number": 5850, "street": "View St.", "city": "Sunnyvale" }, "interests": [ "Music" ], "children": [ ] }
+{ "cid": 964, "name": "Stephany Soders", "age": null, "address": null, "interests": [ "Tennis", "Wine", "Computers" ], "children": [ ] }
+{ "cid": 965, "name": "Mellie Risen", "age": null, "address": null, "interests": [ "Tennis" ], "children": [ { "name": "Coreen Risen", "age": 36 }, { "name": "Faith Risen", "age": 34 }, { "name": "Crystle Risen", "age": 54 } ] }
+{ "cid": 966, "name": "Brigitte Quimby", "age": 13, "address": { "number": 203, "street": "Main St.", "city": "Mountain View" }, "interests": [ "Skiing", "Tennis" ], "children": [ { "name": "Ilona Quimby", "age": null }, { "name": "Shaunte Quimby", "age": null }, { "name": "Lorie Quimby", "age": null } ] }
+{ "cid": 968, "name": "Alix Levier", "age": 44, "address": { "number": 7241, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ "Databases", "Fishing", "Wine" ], "children": [ { "name": "Florentina Levier", "age": null }, { "name": "Hyon Levier", "age": null }, { "name": "Dannielle Levier", "age": null } ] }
+{ "cid": 970, "name": "Pia Sudderth", "age": null, "address": null, "interests": [ "Databases" ], "children": [ { "name": "Ernestina Sudderth", "age": 15 }, { "name": "Larue Sudderth", "age": 46 }, { "name": "Toshia Sudderth", "age": 27 } ] }
+{ "cid": 974, "name": "Alexis Malcomson", "age": null, "address": null, "interests": [ "Movies", "Books" ], "children": [ { "name": "Kerri Malcomson", "age": null } ] }
+{ "cid": 975, "name": "Gary Whitemore", "age": null, "address": null, "interests": [ ], "children": [ ] }
+{ "cid": 976, "name": "Madalyn Nidiffer", "age": 35, "address": { "number": 7635, "street": "Main St.", "city": "San Jose" }, "interests": [ "Coffee", "Wine", "Music" ], "children": [ { "name": "Tricia Nidiffer", "age": 10 }, { "name": "Kevin Nidiffer", "age": 24 }, { "name": "Elyse Nidiffer", "age": null } ] }
+{ "cid": 978, "name": "Rudy Watsky", "age": 32, "address": { "number": 2754, "street": "Oak St.", "city": "Seattle" }, "interests": [ "Cooking" ], "children": [ ] }
+{ "cid": 979, "name": "Yoko Bailony", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Vivienne Bailony", "age": null }, { "name": "Lori Bailony", "age": 47 } ] }
+{ "cid": 980, "name": "Harley Lappe", "age": 56, "address": { "number": 647, "street": "Hill St.", "city": "Mountain View" }, "interests": [ "Books", "Cigars", "Basketball" ], "children": [ { "name": "Maxwell Lappe", "age": null }, { "name": "Gemma Lappe", "age": 32 }, { "name": "Ester Lappe", "age": 40 }, { "name": "Myles Lappe", "age": 36 } ] }
+{ "cid": 981, "name": "Lilliam Lopus", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Tracey Lopus", "age": null } ] }
+{ "cid": 982, "name": "Jude Brandsrud", "age": 41, "address": { "number": 7133, "street": "Washington St.", "city": "Seattle" }, "interests": [ "Bass", "Skiing" ], "children": [ { "name": "Scottie Brandsrud", "age": null }, { "name": "Gennie Brandsrud", "age": 10 }, { "name": "Agnes Brandsrud", "age": null }, { "name": "Clarinda Brandsrud", "age": 17 } ] }
+{ "cid": 984, "name": "Janett Kitchens", "age": 66, "address": { "number": 7558, "street": "View St.", "city": "Mountain View" }, "interests": [ "Coffee", "Movies", "Squash" ], "children": [ { "name": "Grayce Kitchens", "age": 14 }, { "name": "Dwayne Kitchens", "age": null }, { "name": "Wilber Kitchens", "age": 51 }, { "name": "Nancey Kitchens", "age": null } ] }
+{ "cid": 985, "name": "Arnette Farlow", "age": 23, "address": { "number": 7843, "street": "Main St.", "city": "Portland" }, "interests": [ "Running", "Databases" ], "children": [ { "name": "Lora Farlow", "age": 12 }, { "name": "Arlen Farlow", "age": 11 }, { "name": "Rodney Farlow", "age": null }, { "name": "Tori Farlow", "age": 11 } ] }
+{ "cid": 986, "name": "Tennille Wikle", "age": 78, "address": { "number": 3428, "street": "View St.", "city": "Portland" }, "interests": [ "Movies", "Databases", "Wine" ], "children": [ { "name": "Lourie Wikle", "age": null }, { "name": "Laure Wikle", "age": null } ] }
+{ "cid": 987, "name": "Sharolyn Demchak", "age": 36, "address": { "number": 4672, "street": "Lake St.", "city": "San Jose" }, "interests": [ ], "children": [ ] }
+{ "cid": 988, "name": "Dagmar Plasky", "age": 89, "address": { "number": 1219, "street": "Park St.", "city": "Portland" }, "interests": [ ], "children": [ { "name": "Dann Plasky", "age": 59 }, { "name": "Raye Plasky", "age": null }, { "name": "Sammie Plasky", "age": 36 }, { "name": "Kasi Plasky", "age": 24 } ] }
+{ "cid": 991, "name": "Leonel Toepperwein", "age": 62, "address": { "number": 8356, "street": "Washington St.", "city": "Seattle" }, "interests": [ "Coffee", "Books" ], "children": [ { "name": "Sean Toepperwein", "age": null }, { "name": "Charline Toepperwein", "age": 49 }, { "name": "Hattie Toepperwein", "age": 22 }, { "name": "Melida Toepperwein", "age": null } ] }
+{ "cid": 992, "name": "Staci Alexandropoul", "age": null, "address": null, "interests": [ "Databases", "Movies", "Tennis" ], "children": [ { "name": "Casimira Alexandropoul", "age": null }, { "name": "Kena Alexandropoul", "age": 54 }, { "name": "Ellie Alexandropoul", "age": null }, { "name": "Ambrose Alexandropoul", "age": null } ] }
+{ "cid": 993, "name": "Shawn Irie", "age": null, "address": null, "interests": [ "Fishing", "Cigars" ], "children": [ { "name": "Tonette Irie", "age": null } ] }
+{ "cid": 994, "name": "Isa Gravelle", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Lashonda Gravelle", "age": null }, { "name": "Carry Gravelle", "age": 58 } ] }
+{ "cid": 995, "name": "Kiersten Basila", "age": null, "address": null, "interests": [ ], "children": [ { "name": "Norman Basila", "age": 17 }, { "name": "Reginia Basila", "age": null }, { "name": "Gilberto Basila", "age": null }, { "name": "Elvira Basila", "age": 49 } ] }
+{ "cid": 996, "name": "Elouise Wider", "age": null, "address": null, "interests": [ "Coffee", "Computers", "Base Jumping" ], "children": [ ] }
+{ "cid": 997, "name": "Yesenia Gao", "age": 38, "address": { "number": 5990, "street": "View St.", "city": "Portland" }, "interests": [ "Computers", "Computers", "Puzzles", "Puzzles" ], "children": [ { "name": "Jared Gao", "age": 11 }, { "name": "Sang Gao", "age": null }, { "name": "Jeanne Gao", "age": 13 }, { "name": "Lavona Gao", "age": 23 } ] }
+{ "cid": 998, "name": "Barry Schmaus", "age": 65, "address": { "number": 4894, "street": "View St.", "city": "Sunnyvale" }, "interests": [ ], "children": [ { "name": "Ma Schmaus", "age": 40 }, { "name": "Lashawn Schmaus", "age": 13 }, { "name": "Georgianne Schmaus", "age": 38 } ] }
+{ "cid": 999, "name": "Bo Chaim", "age": 59, "address": { "number": 8050, "street": "View St.", "city": "Seattle" }, "interests": [ ], "children": [ { "name": "Zandra Chaim", "age": 42 }, { "name": "Theda Chaim", "age": 14 }, { "name": "Sharika Chaim", "age": 22 } ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-olist-edit-distance.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-olist-edit-distance.adm
new file mode 100644
index 0000000..6d89122
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-olist-edit-distance.adm
@@ -0,0 +1,8 @@
+{ "cid": 11, "name": "Meta Simek", "age": 13, "address": { "number": 4384, "street": "7th St.", "city": "San Jose" }, "interests": [ "Wine", "Walking" ], "children": [ { "name": "Oretha Simek", "age": null }, { "name": "Terence Simek", "age": null } ] }
+{ "cid": 132, "name": "Cindi Turntine", "age": 64, "address": { "number": 9432, "street": "Park St.", "city": "Portland" }, "interests": [ "Computers", "Wine" ], "children": [ { "name": "Howard Turntine", "age": null } ] }
+{ "cid": 153, "name": "Randy Hueso", "age": 11, "address": { "number": 1957, "street": "Oak St.", "city": "San Jose" }, "interests": [ "Computers", "Wine", "Databases", "Walking" ], "children": [ ] }
+{ "cid": 389, "name": "Loraine Morfee", "age": 72, "address": { "number": 2945, "street": "Lake St.", "city": "Seattle" }, "interests": [ "Wine", "Walking" ], "children": [ { "name": "Berry Morfee", "age": 30 } ] }
+{ "cid": 573, "name": "Tyree Ketcher", "age": null, "address": null, "interests": [ "Computers", "Walking" ], "children": [ { "name": "Aleisha Ketcher", "age": null }, { "name": "Vonda Ketcher", "age": null }, { "name": "Cyndy Ketcher", "age": 13 }, { "name": "Chassidy Ketcher", "age": 30 } ] }
+{ "cid": 658, "name": "Truman Leitner", "age": null, "address": null, "interests": [ "Computers", "Bass", "Walking" ], "children": [ ] }
+{ "cid": 716, "name": "Deirdre Bruderer", "age": null, "address": null, "interests": [ "Computers", "Wine" ], "children": [ { "name": "Coralee Bruderer", "age": null }, { "name": "Mina Bruderer", "age": null }, { "name": "Lindsey Bruderer", "age": 35 }, { "name": "Yi Bruderer", "age": null } ] }
+{ "cid": 838, "name": "Karan Aharon", "age": 88, "address": { "number": 8033, "street": "Washington St.", "city": "Portland" }, "interests": [ "Computers", "Movies", "Walking" ], "children": [ { "name": "Matha Aharon", "age": 16 } ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-olist-jaccard.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-olist-jaccard.adm
new file mode 100644
index 0000000..71bb9d7
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-olist-jaccard.adm
@@ -0,0 +1 @@
+{ "cid": 153, "name": "Randy Hueso", "age": 11, "address": { "number": 1957, "street": "Oak St.", "city": "San Jose" }, "interests": [ "Computers", "Wine", "Databases", "Walking" ], "children": [ ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-ulist-jaccard.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-ulist-jaccard.adm
new file mode 100644
index 0000000..fd1b75e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-ulist-jaccard.adm
@@ -0,0 +1 @@
+{ "cid": 153, "name": "Randy Hueso", "age": 11, "address": { "number": 1957, "street": "Oak St.", "city": "San Jose" }, "interests": {{ "Computers", "Wine", "Databases", "Walking" }}, "children": [ ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-word-contains.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-word-contains.adm
new file mode 100644
index 0000000..8a99b26
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-word-contains.adm
@@ -0,0 +1,3 @@
+{ "id": 4, "dblpid": "books/acm/kim95/ChristodoulakisK95", "title": "Multimedia Information Systems Issues and Approaches.", "authors": "Stavros Christodoulakis Leonidas Koveos", "misc": "2002-01-03 318-337 1995 Modern Database Systems db/books/collections/kim95.html#ChristodoulakisK95" }
+{ "id": 89, "dblpid": "conf/icip/SchonfeldL98", "title": "VORTEX Video Retrieval and Tracking from Compressed Multimedia Databases.", "authors": "Dan Schonfeld Dan Lelescu", "misc": "2002-11-05 123-127 1998 ICIP (3) db/conf/icip/icip1998-3.html#SchonfeldL98" }
+{ "id": 90, "dblpid": "conf/hicss/SchonfeldL99", "title": "VORTEX Video Retrieval and Tracking from Compressed Multimedia Databases ¾ Visual Search Engine.", "authors": "Dan Schonfeld Dan Lelescu", "misc": "2002-01-03 1999 HICSS http //computer.org/proceedings/hicss/0001/00013/00013006abs.htm db/conf/hicss/hicss1999-3.html#SchonfeldL99" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-word-jaccard.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-word-jaccard.adm
new file mode 100644
index 0000000..5bf6ae0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-word-jaccard.adm
@@ -0,0 +1 @@
+{ "id": 9, "dblpid": "books/acm/kim95/Kaiser95", "title": "Cooperative Transactions for Multiuser Environments.", "authors": "Gail E. Kaiser", "misc": "2002-01-03 409-433 1995 Modern Database Systems db/books/collections/kim95.html#Kaiser95" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterix-app/src/test/resources/runtimets/testsuite.xml
index 1e2f235..d1a1757 100644
--- a/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -1651,6 +1651,56 @@
</compilation-unit>
</test-case>
<test-case FilePath="index-selection">
+ <compilation-unit name="fuzzy-inverted-index-ngram-contains">
+ <output-file compare="Text">fuzzy-inverted-index-ngram-contains.adm</output-file>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="fuzzy-inverted-index-ngram-edit-distance-panic">
+ <output-file compare="Text">fuzzy-inverted-index-ngram-edit-distance-panic.adm</output-file>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="fuzzy-inverted-index-ngram-edit-distance">
+ <output-file compare="Text">fuzzy-inverted-index-ngram-edit-distance.adm</output-file>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="fuzzy-inverted-index-ngram-jaccard">
+ <output-file compare="Text">fuzzy-inverted-index-ngram-jaccard.adm</output-file>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="fuzzy-inverted-index-olist-edit-distance-panic">
+ <output-file compare="Text">fuzzy-inverted-index-olist-edit-distance-panic.adm</output-file>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="fuzzy-inverted-index-olist-edit-distance">
+ <output-file compare="Text">fuzzy-inverted-index-olist-edit-distance.adm</output-file>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="fuzzy-inverted-index-olist-jaccard">
+ <output-file compare="Text">fuzzy-inverted-index-olist-jaccard.adm</output-file>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="fuzzy-inverted-index-ulist-jaccard">
+ <output-file compare="Text">fuzzy-inverted-index-ulist-jaccard.adm</output-file>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="fuzzy-inverted-index-word-contains">
+ <output-file compare="Text">fuzzy-inverted-index-word-contains.adm</output-file>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="fuzzy-inverted-index-word-jaccard">
+ <output-file compare="Text">fuzzy-inverted-index-word-jaccard.adm</output-file>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
<compilation-unit name="inverted-index-ngram-contains">
<output-file compare="Text">inverted-index-ngram-contains.adm</output-file>
</compilation-unit>
diff --git a/asterix-aql/src/main/javacc/AQL.jj b/asterix-aql/src/main/javacc/AQL.jj
index eaa4260..59a7753 100644
--- a/asterix-aql/src/main/javacc/AQL.jj
+++ b/asterix-aql/src/main/javacc/AQL.jj
@@ -398,9 +398,10 @@
("," <IDENTIFIER> { cis.addFieldExpr(token.image); })*
<RIGHTPAREN>
("type"
- ("btree" { cis.setIndexType(IndexType.BTREE); }
- | "keyword" { cis.setIndexType(IndexType.WORD_INVIX); }
+ ("btree" { cis.setIndexType(IndexType.BTREE); }
| "rtree" { cis.setIndexType(IndexType.RTREE); }
+ | "keyword" { cis.setIndexType(IndexType.WORD_INVIX); }
+ | "fuzzy keyword" { cis.setIndexType(IndexType.FUZZY_WORD_INVIX); }
| "ngram"
<LEFTPAREN>
(<INTEGER_LITERAL>
@@ -409,7 +410,16 @@
cis.setGramLength(Integer.valueOf(token.image));
}
)
- <RIGHTPAREN>
+ <RIGHTPAREN>
+ | "fuzzy ngram"
+ <LEFTPAREN>
+ (<INTEGER_LITERAL>
+ {
+ cis.setIndexType(IndexType.FUZZY_NGRAM_INVIX);
+ cis.setGramLength(Integer.valueOf(token.image));
+ }
+ )
+ <RIGHTPAREN>
)
";"
| ";"
diff --git a/asterix-common/pom.xml b/asterix-common/pom.xml
index 048c037..f7f3767 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.2-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..cef79bf 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.NOINDEX_PROVIDER;
}
public static void setNodeControllerInfo(Map<String, Set<String>> nodeControllerInfo) {
@@ -37,4 +30,9 @@
return nodeControllerMap;
}
+ @Override
+ public IIndexLifecycleManagerProvider getIndexLifecycleManagerProvider() {
+ return AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER;
+ }
+
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/DatasetConfig.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/DatasetConfig.java
index e860a0a..cbe96f7 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/DatasetConfig.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/DatasetConfig.java
@@ -41,7 +41,9 @@
BTREE,
RTREE,
WORD_INVIX,
- NGRAM_INVIX
+ NGRAM_INVIX,
+ FUZZY_WORD_INVIX,
+ FUZZY_NGRAM_INVIX
}
}
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..997dce6 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
@@ -5,11 +5,17 @@
import edu.uci.ics.asterix.common.config.GlobalConfig;
import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
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.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ConstantMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ImmediateScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.RefCountingOperationTrackerFactory;
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,15 +24,28 @@
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 static final int DEFAULT_MAX_OPEN_FILES = 500;
private final INCApplicationContext ncApplicationContext;
-
- private IndexRegistry<IIndex> indexRegistry;
+
+ private IIndexLifecycleManager indexLifecycleManager;
private IFileMapManager fileMapManager;
private IBufferCache bufferCache;
- private TransactionProvider provider;
+ private TransactionSubsystem txnSubsystem;
+
+ private ILSMMergePolicy mergePolicy;
+ private ILSMOperationTrackerFactory opTrackerFactory;
+ private ILSMIOOperationScheduler lsmIOScheduler;
+ private ILocalResourceRepository localResourceRepository;
+ private ResourceIdFactory resourceIdFactory;
public AsterixAppRuntimeContext(INCApplicationContext ncApplicationContext) {
this.ncApplicationContext = ncApplicationContext;
@@ -36,20 +55,22 @@
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);
+ bufferCache = new BufferCache(ioMgr, allocator, prs, fileMapManager, pageSize, numPages, DEFAULT_MAX_OPEN_FILES);
+ indexLifecycleManager = new IndexLifecycleManager(DEFAULT_LIFECYCLEMANAGER_MEMORY_BUDGET);
+ txnSubsystem = new TransactionSubsystem(ncApplicationContext.getNodeId());
- // Initialize the index registry
- indexRegistry = new IndexRegistry<IIndex>();
+ lsmIOScheduler = ImmediateScheduler.INSTANCE;
+ mergePolicy = new ConstantMergePolicy(lsmIOScheduler, 3);
+ opTrackerFactory = RefCountingOperationTrackerFactory.INSTANCE;
- // Initialize the transaction sub-system
- provider = new TransactionProvider(ncApplicationContext.getNodeId());
+ ILocalResourceRepositoryFactory persistentLocalResourceRepositoryFactory = new PersistentLocalResourceRepositoryFactory(
+ ioMgr);
+ localResourceRepository = persistentLocalResourceRepositoryFactory.createRepository();
+ resourceIdFactory = (new ResourceIdFactoryProvider(localResourceRepository)).createResourceIdFactory();
}
private int getBufferCachePageSize() {
@@ -107,12 +128,31 @@
return fileMapManager;
}
- public IndexRegistry<IIndex> getIndexRegistry() {
- return indexRegistry;
+ public TransactionSubsystem getTransactionSubsystem() {
+ return txnSubsystem;
}
- public TransactionProvider getTransactionProvider() {
- return provider;
+ public IIndexLifecycleManager getIndexLifecycleManager() {
+ return indexLifecycleManager;
}
+ public ILSMMergePolicy getLSMMergePolicy() {
+ return mergePolicy;
+ }
+
+ public ILSMOperationTrackerFactory getLSMOperationTrackerFactory() {
+ return opTrackerFactory;
+ }
+
+ 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..2021269
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixRuntimeComponentsProvider.java
@@ -0,0 +1,86 @@
+package edu.uci.ics.asterix.common.context;
+
+import edu.uci.ics.asterix.transaction.management.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
+import edu.uci.ics.asterix.transaction.management.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
+import edu.uci.ics.asterix.transaction.management.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
+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.ILSMIOOperationCallbackFactory;
+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.ILSMIndex;
+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.ILSMOperationTrackerFactory;
+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 class AsterixRuntimeComponentsProvider implements IIndexLifecycleManagerProvider, IStorageManagerInterface,
+ ILSMIOOperationSchedulerProvider, ILSMMergePolicyProvider, ILSMOperationTrackerFactory {
+ private final ILSMIOOperationCallbackFactory ioOpCallbackFactory;
+
+ public static final AsterixRuntimeComponentsProvider LSMBTREE_PROVIDER = new AsterixRuntimeComponentsProvider(
+ LSMBTreeIOOperationCallbackFactory.INSTANCE);
+ public static final AsterixRuntimeComponentsProvider LSMRTREE_PROVIDER = new AsterixRuntimeComponentsProvider(
+ LSMRTreeIOOperationCallbackFactory.INSTANCE);
+ public static final AsterixRuntimeComponentsProvider LSMINVERTEDINDEX_PROVIDER = new AsterixRuntimeComponentsProvider(
+ LSMInvertedIndexIOOperationCallbackFactory.INSTANCE);
+ public static final AsterixRuntimeComponentsProvider NOINDEX_PROVIDER = new AsterixRuntimeComponentsProvider(null);
+
+ private AsterixRuntimeComponentsProvider(ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
+ this.ioOpCallbackFactory = ioOpCallbackFactory;
+ }
+
+ @Override
+ public ILSMOperationTracker createOperationTracker(ILSMIndex index) {
+ return new IndexOperationTracker(index, ioOpCallbackFactory);
+ }
+
+ @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/context/TransactionSubsystemProvider.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/TransactionSubsystemProvider.java
new file mode 100644
index 0000000..85e5375
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/TransactionSubsystemProvider.java
@@ -0,0 +1,34 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.common.context;
+
+import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionSubsystemProvider;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+/**
+ * The purpose of this provider is to work around a cyclic dependency between asterix-common and asterix-transactions.
+ * The operation callbacks would depend on the AsterixAppRuntimeContext to get the transaction subsystem,
+ * while at the same time the AsterixAppRuntimeContext depends on asterix-transactions for the TransactionSubsystem.
+ */
+public class TransactionSubsystemProvider implements ITransactionSubsystemProvider {
+ @Override
+ public TransactionSubsystem getTransactionSubsystem(IHyracksTaskContext ctx) {
+ AsterixAppRuntimeContext runtimeCtx = (AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext()
+ .getApplicationObject();
+ return runtimeCtx.getTransactionSubsystem();
+ }
+}
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 9f1b9b5..918e804 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
@@ -23,8 +23,8 @@
import edu.uci.ics.asterix.metadata.api.IMetadataManager;
import edu.uci.ics.asterix.metadata.api.IMetadataNode;
import edu.uci.ics.asterix.metadata.bootstrap.MetadataConstants;
-import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
import edu.uci.ics.asterix.metadata.entities.Datatype;
import edu.uci.ics.asterix.metadata.entities.Dataverse;
import edu.uci.ics.asterix.metadata.entities.Function;
@@ -32,7 +32,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.
@@ -108,36 +109,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);
}
@@ -147,7 +148,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);
}
@@ -175,7 +176,7 @@
return dataverse;
}
try {
- dataverse = metadataNode.getDataverse(ctx.getTxnId(), dataverseName);
+ dataverse = metadataNode.getDataverse(ctx.getJobId(), dataverseName);
} catch (RemoteException e) {
throw new MetadataException(e);
}
@@ -194,7 +195,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);
}
@@ -206,7 +207,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);
}
@@ -217,7 +218,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);
}
@@ -253,7 +254,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);
}
@@ -270,7 +271,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);
}
@@ -280,7 +281,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);
}
@@ -291,7 +292,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);
}
@@ -327,7 +328,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);
}
@@ -342,7 +343,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);
}
@@ -351,7 +352,7 @@
@Override
public void addAdapter(MetadataTransactionContext mdTxnCtx, DatasourceAdapter adapter) throws MetadataException {
try {
- metadataNode.addAdapter(mdTxnCtx.getTxnId(), adapter);
+ metadataNode.addAdapter(mdTxnCtx.getJobId(), adapter);
} catch (RemoteException e) {
throw new MetadataException(e);
}
@@ -363,7 +364,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);
}
@@ -373,7 +374,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);
}
@@ -382,7 +383,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);
}
@@ -391,7 +392,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);
}
@@ -401,7 +402,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);
}
@@ -429,7 +430,7 @@
return nodeGroup;
}
try {
- nodeGroup = metadataNode.getNodeGroup(ctx.getTxnId(), nodeGroupName);
+ nodeGroup = metadataNode.getNodeGroup(ctx.getJobId(), nodeGroupName);
} catch (RemoteException e) {
throw new MetadataException(e);
}
@@ -444,7 +445,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);
}
@@ -455,7 +456,7 @@
public void dropFunction(MetadataTransactionContext ctx, FunctionSignature functionSignature)
throws MetadataException {
try {
- metadataNode.dropFunction(ctx.getTxnId(), functionSignature);
+ metadataNode.dropFunction(ctx.getJobId(), functionSignature);
} catch (RemoteException e) {
throw new MetadataException(e);
}
@@ -489,7 +490,7 @@
return function;
}
try {
- function = metadataNode.getFunction(ctx.getTxnId(), functionSignature);
+ function = metadataNode.getFunction(ctx.getJobId(), functionSignature);
} catch (RemoteException e) {
throw new MetadataException(e);
}
@@ -503,13 +504,22 @@
}
@Override
+ public void initializeDatasetIdFactory(MetadataTransactionContext ctx) throws MetadataException {
+ try {
+ metadataNode.initializeDatasetIdFactory(ctx.getJobId());
+ } catch (RemoteException e) {
+ throw new MetadataException(e);
+ }
+ }
+
+ @Override
public List<Function> getDataverseFunctions(MetadataTransactionContext ctx, String dataverseName)
throws MetadataException {
List<Function> dataverseFunctions;
try {
// Assuming that the transaction can read its own writes on the
// metadata node.
- dataverseFunctions = metadataNode.getDataverseFunctions(ctx.getTxnId(), dataverseName);
+ dataverseFunctions = metadataNode.getDataverseFunctions(ctx.getJobId(), dataverseName);
} catch (RemoteException e) {
throw new MetadataException(e);
}
@@ -521,7 +531,7 @@
@Override
public void dropAdapter(MetadataTransactionContext ctx, String dataverseName, String name) throws MetadataException {
try {
- metadataNode.dropAdapter(ctx.getTxnId(), dataverseName, name);
+ metadataNode.dropAdapter(ctx.getJobId(), dataverseName, name);
} catch (RemoteException e) {
throw new MetadataException(e);
}
@@ -532,11 +542,10 @@
throws MetadataException {
DatasourceAdapter adapter = null;
try {
- adapter = metadataNode.getAdapter(ctx.getTxnId(), dataverseName, name);
+ adapter = metadataNode.getAdapter(ctx.getJobId(), dataverseName, name);
} catch (RemoteException e) {
throw new MetadataException(e);
}
return adapter;
}
-
}
\ 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 120f1a5..a9406e2 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
@@ -30,8 +30,8 @@
import edu.uci.ics.asterix.metadata.api.IValueExtractor;
import edu.uci.ics.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
import edu.uci.ics.asterix.metadata.bootstrap.MetadataSecondaryIndexes;
-import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
import edu.uci.ics.asterix.metadata.entities.Datatype;
import edu.uci.ics.asterix.metadata.entities.Dataverse;
import edu.uci.ics.asterix.metadata.entities.Function;
@@ -39,8 +39,8 @@
import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
import edu.uci.ics.asterix.metadata.entities.Node;
import edu.uci.ics.asterix.metadata.entities.NodeGroup;
-import edu.uci.ics.asterix.metadata.entitytupletranslators.DatasourceAdapterTupleTranslator;
import edu.uci.ics.asterix.metadata.entitytupletranslators.DatasetTupleTranslator;
+import edu.uci.ics.asterix.metadata.entitytupletranslators.DatasourceAdapterTupleTranslator;
import edu.uci.ics.asterix.metadata.entitytupletranslators.DatatypeTupleTranslator;
import edu.uci.ics.asterix.metadata.entitytupletranslators.DataverseTupleTranslator;
import edu.uci.ics.asterix.metadata.entitytupletranslators.FunctionTupleTranslator;
@@ -56,9 +56,12 @@
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;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
@@ -66,18 +69,15 @@
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.MultiComparator;
public class MetadataNode implements IMetadataNode {
@@ -85,9 +85,10 @@
// 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 TransactionProvider transactionProvider;
+ private IIndexLifecycleManager indexLifecycleManager;
+ private TransactionSubsystem transactionProvider;
public static final MetadataNode INSTANCE = new MetadataNode();
@@ -96,26 +97,26 @@
}
public void initialize(AsterixAppRuntimeContext runtimeContext) {
- this.transactionProvider = runtimeContext.getTransactionProvider();
- this.indexRegistry = runtimeContext.getIndexRegistry();
+ this.transactionProvider = runtimeContext.getTransactionSubsystem();
+ 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);
- transactionProvider.getTransactionManager().commitTransaction(txnCtx);
+ public void commitTransaction(JobId jobId) throws RemoteException, ACIDException {
+ TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(jobId);
+ transactionProvider.getTransactionManager().commitTransaction(txnCtx, new DatasetId(-1), -1);
}
@Override
- public void abortTransaction(long txnId) throws RemoteException, ACIDException {
+ public void abortTransaction(JobId jobId) throws RemoteException, ACIDException {
try {
- TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(txnId);
- transactionProvider.getTransactionManager().abortTransaction(txnCtx);
+ TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(jobId);
+ transactionProvider.getTransactionManager().abortTransaction(txnCtx, new DatasetId(-1), -1);
} catch (ACIDException e) {
e.printStackTrace();
throw e;
@@ -123,23 +124,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);
@@ -149,26 +150,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);
@@ -178,11 +179,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) {
@@ -191,11 +192,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) {
@@ -204,11 +205,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);
@@ -218,11 +219,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) {
@@ -231,12 +232,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 function with this name " + function.getName() + " and arity "
@@ -246,65 +247,69 @@
}
}
- 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);
+ //TODO: extract the key from the tuple and get the PKHashValue from the key.
+ //index.getIndexLogger().generateLogRecord(transactionProvider, txnCtx, index.getDatasetId().getId(), null,
+ // resourceID, IndexOperation.INSERT, tuple, null, null);
+ 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());
}
}
// As a side effect, acquires an S lock on the 'Function' dataset
// on behalf of txnId.
- List<Function> dataverseFunctions = getDataverseFunctions(txnId, dataverseName);
+ List<Function> dataverseFunctions = getDataverseFunctions(jobId, dataverseName);
if (dataverseFunctions != null && dataverseFunctions.size() > 0) {
// Drop all functions in this dataverse.
for (Function function : dataverseFunctions) {
- dropFunction(txnId, new FunctionSignature(dataverseName, function.getName(), function.getArity()));
+ dropFunction(jobId, new FunctionSignature(dataverseName, function.getName(), function.getArity()));
}
}
// As a side effect, acquires an S lock on the 'Adapter' dataset
// on behalf of txnId.
- List<DatasourceAdapter> dataverseAdapters = getDataverseAdapters(txnId, dataverseName);
+ List<DatasourceAdapter> dataverseAdapters = getDataverseAdapters(jobId, dataverseName);
if (dataverseAdapters != null && dataverseAdapters.size() > 0) {
// Drop all functions in this dataverse.
for (DatasourceAdapter adapter : dataverseAdapters) {
- dropAdapter(txnId, dataverseName, adapter.getAdapterIdentifier().getAdapterName());
+ dropAdapter(jobId, dataverseName, adapter.getAdapterIdentifier().getAdapterName());
}
}
@@ -312,8 +317,8 @@
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) {
@@ -324,11 +329,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);
}
@@ -340,30 +345,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.,
@@ -376,14 +381,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) {
@@ -395,10 +400,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);
}
@@ -414,8 +419,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) {
@@ -426,13 +431,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);
}
@@ -457,15 +462,15 @@
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);
+ ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey);
// This call uses the secondary index on datatype. Get nested types before deleting entry from secondary index.
- List<String> nestedTypes = getNestedDatatypeNames(txnId, dataverseName, datatypeName);
- deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
- deleteFromDatatypeSecondaryIndex(txnId, dataverseName, datatypeName);
+ List<String> nestedTypes = getNestedDatatypeNames(jobId, dataverseName, datatypeName);
+ deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
+ deleteFromDatatypeSecondaryIndex(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.,
@@ -477,14 +482,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) {
@@ -496,17 +501,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.
@@ -519,31 +524,35 @@
}
}
- 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);
+ //TODO: extract the key from the tuple and get the PKHashValue from the key.
+ //check how to get the oldValue.
+ //index.getIndexLogger().generateLogRecord(transactionProvider, txnCtx, index.getDatasetId().getId(), null,
+ // resourceID, IndexOperation.DELETE, tuple, operation, null);
+ 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;
}
@@ -555,28 +564,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);
@@ -584,14 +593,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;
}
@@ -601,13 +610,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, datatypeName);
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) {
@@ -615,13 +624,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) {
@@ -629,13 +638,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) {
@@ -643,13 +652,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);
@@ -657,14 +666,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;
}
@@ -675,14 +684,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);
@@ -690,14 +699,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;
}
@@ -708,13 +717,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;
}
@@ -725,7 +734,7 @@
}
@Override
- public Function getFunction(long txnId, FunctionSignature functionSignature) throws MetadataException,
+ public Function getFunction(JobId jobId, FunctionSignature functionSignature) throws MetadataException,
RemoteException {
try {
ITupleReference searchKey = createTuple(functionSignature.getNamespace(), functionSignature.getName(), ""
@@ -733,7 +742,7 @@
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;
}
@@ -745,15 +754,27 @@
}
@Override
- public void dropFunction(long txnId, FunctionSignature functionSignature) throws MetadataException, RemoteException {
+ public void dropFunction(JobId jobId, FunctionSignature functionSignature) throws MetadataException,
+ RemoteException {
+
+ Function function;
+ try {
+ function = getFunction(jobId, functionSignature);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ if (function == null) {
+ throw new MetadataException("Cannot drop function '" + functionSignature.toString()
+ + "' because it doesn't exist.");
+ }
try {
// Delete entry from the 'function' dataset.
ITupleReference searchKey = createTuple(functionSignature.getNamespace(), functionSignature.getName());
// 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.
@@ -765,11 +786,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.
@@ -779,17 +800,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 = (ITreeIndexCursor) indexAccessor.createSearchCursor();
IBinaryComparator[] searchCmps = new IBinaryComparator[searchKey.getFieldCount()];
for (int i = 0; i < searchKey.getFieldCount(); i++) {
searchCmps[i] = comparatorFactories[i].createBinaryComparator();
@@ -801,7 +822,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);
}
@@ -809,6 +830,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
@@ -829,14 +888,14 @@
}
@Override
- public List<Function> getDataverseFunctions(long txnId, String dataverseName) throws MetadataException,
+ public List<Function> getDataverseFunctions(JobId jobId, String dataverseName) throws MetadataException,
RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName);
FunctionTupleTranslator tupleReaderWriter = new FunctionTupleTranslator(false);
IValueExtractor<Function> valueExtractor = new MetadataEntityValueExtractor<Function>(tupleReaderWriter);
List<Function> results = new ArrayList<Function>();
- searchIndex(txnId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey, valueExtractor, results);
+ searchIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey, valueExtractor, results);
return results;
} catch (Exception e) {
throw new MetadataException(e);
@@ -844,12 +903,12 @@
}
@Override
- public void addAdapter(long txnId, DatasourceAdapter adapter) throws MetadataException, RemoteException {
+ public void addAdapter(JobId jobId, DatasourceAdapter adapter) throws MetadataException, RemoteException {
try {
// Insert into the 'Adapter' dataset.
DatasourceAdapterTupleTranslator tupleReaderWriter = new DatasourceAdapterTupleTranslator(true);
ITupleReference adapterTuple = tupleReaderWriter.getTupleFromMetadataEntity(adapter);
- insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, adapterTuple);
+ insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, adapterTuple);
} catch (BTreeDuplicateKeyException e) {
throw new MetadataException("A adapter with this name " + adapter.getAdapterIdentifier().getAdapterName()
@@ -861,11 +920,11 @@
}
@Override
- public void dropAdapter(long txnId, String dataverseName, String adapterName) throws MetadataException,
+ public void dropAdapter(JobId jobId, String dataverseName, String adapterName) throws MetadataException,
RemoteException {
DatasourceAdapter adapter;
try {
- adapter = getAdapter(txnId, dataverseName, adapterName);
+ adapter = getAdapter(jobId, dataverseName, adapterName);
} catch (Exception e) {
throw new MetadataException(e);
}
@@ -877,8 +936,9 @@
ITupleReference searchKey = createTuple(dataverseName, adapterName);
// Searches the index for the tuple to be deleted. Acquires an S
// lock on the 'Adapter' dataset.
- ITupleReference datasetTuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey);
- deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, datasetTuple);
+ ITupleReference datasetTuple = getTupleToBeDeleted(jobId,
+ MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey);
+ deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, datasetTuple);
// TODO: Change this to be a BTree specific exception, e.g.,
// BTreeKeyDoesNotExistException.
@@ -891,14 +951,15 @@
}
@Override
- public DatasourceAdapter getAdapter(long txnId, String dataverseName, String adapterName) throws MetadataException,
- RemoteException {
+ public DatasourceAdapter getAdapter(JobId jobId, String dataverseName, String adapterName)
+ throws MetadataException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName, adapterName);
DatasourceAdapterTupleTranslator tupleReaderWriter = new DatasourceAdapterTupleTranslator(false);
List<DatasourceAdapter> results = new ArrayList<DatasourceAdapter>();
- IValueExtractor<DatasourceAdapter> valueExtractor = new MetadataEntityValueExtractor<DatasourceAdapter>(tupleReaderWriter);
- searchIndex(txnId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey, valueExtractor, results);
+ IValueExtractor<DatasourceAdapter> valueExtractor = new MetadataEntityValueExtractor<DatasourceAdapter>(
+ tupleReaderWriter);
+ searchIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey, valueExtractor, results);
if (results.isEmpty()) {
return null;
}
@@ -909,18 +970,18 @@
}
@Override
- public List<DatasourceAdapter> getDataverseAdapters(long txnId, String dataverseName) throws MetadataException,
+ public List<DatasourceAdapter> getDataverseAdapters(JobId jobId, String dataverseName) throws MetadataException,
RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName);
DatasourceAdapterTupleTranslator tupleReaderWriter = new DatasourceAdapterTupleTranslator(false);
- IValueExtractor<DatasourceAdapter> valueExtractor = new MetadataEntityValueExtractor<DatasourceAdapter>(tupleReaderWriter);
+ IValueExtractor<DatasourceAdapter> valueExtractor = new MetadataEntityValueExtractor<DatasourceAdapter>(
+ tupleReaderWriter);
List<DatasourceAdapter> results = new ArrayList<DatasourceAdapter>();
- searchIndex(txnId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey, valueExtractor, results);
+ searchIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey, valueExtractor, results);
return results;
} catch (Exception e) {
throw new MetadataException(e);
}
}
-
}
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 6c3bb5e..15bd229 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
@@ -19,13 +19,13 @@
import edu.uci.ics.asterix.common.functions.FunctionSignature;
import edu.uci.ics.asterix.external.dataset.adapter.AdapterIdentifier;
-import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
import edu.uci.ics.asterix.metadata.entities.Datatype;
import edu.uci.ics.asterix.metadata.entities.Dataverse;
import edu.uci.ics.asterix.metadata.entities.Function;
import edu.uci.ics.asterix.metadata.entities.NodeGroup;
-import edu.uci.ics.asterix.om.functions.AsterixFunction;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
/**
* Used to implement serializable transactions against the MetadataCache.
@@ -56,16 +56,16 @@
// The APIs in this class make sure that these two caches are kept in sync.
protected MetadataCache droppedCache = new MetadataCache();
- protected ArrayList<MetadataLogicalOperation> opLog = new ArrayList<MetadataLogicalOperation>();
- private final long txnId;
+ protected ArrayList<MetadataLogicalOperation> opLog = new ArrayList<MetadataLogicalOperation>();
+ 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) {
droppedCache.dropDataverse(dataverse);
@@ -97,18 +97,18 @@
logAndApply(new MetadataLogicalOperation(adapter, true));
}
+ public void dropDataset(String dataverseName, String datasetName) {
+ Dataset dataset = new Dataset(dataverseName, datasetName, null, null,
+ null, -1);
+ droppedCache.addDatasetIfNotExists(dataset);
+ logAndApply(new MetadataLogicalOperation(dataset, false));
+ }
public void dropDataverse(String dataverseName) {
Dataverse dataverse = new Dataverse(dataverseName, null);
droppedCache.addDataverseIfNotExists(dataverse);
logAndApply(new MetadataLogicalOperation(dataverse, false));
}
- public void dropDataset(String dataverseName, String datasetName) {
- Dataset dataset = new Dataset(dataverseName, datasetName, null, null, null);
- droppedCache.addDatasetIfNotExists(dataset);
- logAndApply(new MetadataLogicalOperation(dataset, false));
- }
-
public void dropDataDatatype(String dataverseName, String datatypeName) {
Datatype datatype = new Datatype(dataverseName, datatypeName, null, false);
droppedCache.addDatatypeIfNotExists(datatype);
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IAsterixStateProxy.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IAsterixStateProxy.java
index 2b915d9..5f772c7 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IAsterixStateProxy.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IAsterixStateProxy.java
@@ -32,8 +32,4 @@
public IMetadataNode getMetadataNode() throws RemoteException;
public AsterixProperties getAsterixProperties() throws RemoteException;
-
- public Object getAsterixNodeState(String nodeName) throws RemoteException;
-
- public void setAsterixNodeState(String nodeName, Object ns) throws RemoteException;
}
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..8554ddd 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
@@ -19,12 +19,14 @@
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.logging.IndexLogger;
+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.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
/**
* Descriptor interface for a primary or secondary index on metadata datasets.
@@ -58,13 +60,21 @@
public ARecordType getPayloadRecordType();
+ public void setFile(FileReference file);
+
+ public FileReference getFile();
+
public void setFileId(int fileId);
- public void initTreeLogger(ITreeIndex treeIndex) throws ACIDException;
+ public void initIndexLogger(IIndex index) throws ACIDException;
public int getFileId();
- public byte[] getResourceId();
+ public void setResourceID(long resourceID);
- public TreeLogger getTreeLogger();
+ public long getResourceID();
+
+ public IndexLogger getIndexLogger();
+
+ 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 01e9649..fe96a6f 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
@@ -30,6 +30,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,
@@ -94,7 +95,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.
@@ -430,4 +431,6 @@
public List<Function> getDataverseFunctions(MetadataTransactionContext ctx, String dataverseName)
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 76c5746..fc9fe18 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
@@ -31,6 +31,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
@@ -49,7 +50,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.
@@ -57,7 +58,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.
@@ -65,7 +66,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
@@ -74,7 +75,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.
@@ -82,13 +83,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.
@@ -96,13 +97,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.
@@ -111,13 +112,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.
@@ -125,7 +126,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;
/**
@@ -133,20 +134,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.
@@ -154,13 +155,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.
@@ -171,14 +172,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.
@@ -189,14 +190,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.
@@ -206,7 +207,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;
/**
@@ -214,7 +215,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.
@@ -222,13 +223,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.
@@ -240,14 +241,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.
@@ -258,14 +259,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.
@@ -273,13 +274,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.
@@ -290,14 +291,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.
@@ -308,14 +309,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.
@@ -323,13 +324,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.
@@ -337,13 +338,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.
@@ -352,13 +353,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.
@@ -366,10 +367,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 functionSignature
* An instance of functionSignature representing the function
@@ -377,14 +379,14 @@
* @throws MetadataException
* @throws RemoteException
*/
- public Function getFunction(long txnId, FunctionSignature functionSignature) throws MetadataException,
+ public Function getFunction(JobId jobId, FunctionSignature functionSignature) throws MetadataException,
RemoteException;
/**
* 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 functionSignature
* An instance of functionSignature representing the function
@@ -393,10 +395,10 @@
* group to be deleted.
* @throws RemoteException
*/
- public void dropFunction(long txnId, FunctionSignature functionSignature) throws MetadataException, RemoteException;
+ public void dropFunction(JobId jobId, FunctionSignature functionSignature) throws MetadataException, RemoteException;
/**
- * @param txnId
+ * @param jobId
* A globally unique id for an active metadata transaction.
* @param function
* Function to be inserted
@@ -405,7 +407,7 @@
* unknown function
* @throws RemoteException
*/
- public void addFunction(long txnId, Function function) throws MetadataException, RemoteException;
+ public void addFunction(JobId jobId, Function function) throws MetadataException, RemoteException;
/**
* @param ctx
@@ -414,7 +416,7 @@
* @throws MetadataException
* @throws RemoteException
*/
- public List<Function> getDataverseFunctions(long txnId, String dataverseName) throws MetadataException,
+ public List<Function> getDataverseFunctions(JobId jobId, String dataverseName) throws MetadataException,
RemoteException;
/**
@@ -424,17 +426,17 @@
* @throws MetadataException
* @throws RemoteException
*/
- public List<DatasourceAdapter> getDataverseAdapters(long txnId, String dataverseName) throws MetadataException,
+ public List<DatasourceAdapter> getDataverseAdapters(JobId jobId, String dataverseName) throws MetadataException,
RemoteException;
- public DatasourceAdapter getAdapter(long txnId, String dataverseName, String adapterName) throws MetadataException,
+ public DatasourceAdapter getAdapter(JobId jobId, String dataverseName, String adapterName) throws MetadataException,
RemoteException;
/**
* Deletes a adapter , 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 adapter that is to be deleted.
@@ -443,11 +445,11 @@
* if the adapter does not exists.
* @throws RemoteException
*/
- public void dropAdapter(long txnId, String dataverseName, String adapterName) throws MetadataException,
+ public void dropAdapter(JobId jobId, String dataverseName, String adapterName) throws MetadataException,
RemoteException;
/**
- * @param txnId
+ * @param jobId
* A globally unique id for an active metadata transaction.
* @param adapter
* Adapter to be inserted
@@ -455,6 +457,7 @@
* for example, if the adapter already exists.
* @throws RemoteException
*/
- public void addAdapter(long txnId, DatasourceAdapter adapter) throws MetadataException, RemoteException;
+ public void addAdapter(JobId jobId, DatasourceAdapter adapter) 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/AsterixProperties.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/AsterixProperties.java
index d0d04ee..71a7de3 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/AsterixProperties.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/AsterixProperties.java
@@ -15,6 +15,7 @@
package edu.uci.ics.asterix.metadata.bootstrap;
+import java.io.File;
import java.io.FileInputStream;
import java.io.FileNotFoundException;
import java.io.IOException;
@@ -96,6 +97,19 @@
String ncName = pn.substring(0, pn.indexOf('.'));
val = p.getProperty(pn);
String[] folderNames = val.split("\\s*,\\s*");
+ int i = 0;
+ for (String store : folderNames) {
+ boolean needsStartSep = !store.startsWith(File.separator);
+ boolean needsEndSep = !store.endsWith(File.separator);
+ if (needsStartSep && needsEndSep) {
+ folderNames[i] = File.separator + store + File.separator;
+ } else if (needsStartSep) {
+ folderNames[i] = File.separator + store;
+ } else if (needsEndSep) {
+ folderNames[i] = store + File.separator;
+ }
+ i++;
+ }
stores.put(ncName, folderNames);
nodeNames = new HashSet<String>();
nodeNames.addAll(stores.keySet());
@@ -127,7 +141,7 @@
return nodeNames;
}
- public String getOutputDir() {
+ public String getOutputDir() {
return outputDir;
}
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/AsterixStateProxy.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/AsterixStateProxy.java
index e2c02fa..3946fa6 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/AsterixStateProxy.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/AsterixStateProxy.java
@@ -17,7 +17,6 @@
import java.rmi.RemoteException;
import java.rmi.server.UnicastRemoteObject;
-import java.util.HashMap;
import java.util.logging.Logger;
import edu.uci.ics.asterix.metadata.api.IAsterixStateProxy;
@@ -33,7 +32,6 @@
private IMetadataNode metadataNode;
private AsterixProperties asterixProperties;
private static final IAsterixStateProxy cc = new AsterixStateProxy();
- private final HashMap<String, Object> nodeStateMap = new HashMap<String, Object>();
public static IAsterixStateProxy registerRemoteObject() throws RemoteException {
IAsterixStateProxy stub = (IAsterixStateProxy) UnicastRemoteObject.exportObject(cc, 0);
@@ -65,14 +63,4 @@
public AsterixProperties getAsterixProperties() throws RemoteException {
return this.asterixProperties;
}
-
- @Override
- synchronized public Object getAsterixNodeState(String nodeName) throws RemoteException {
- return nodeStateMap.get(nodeName);
- }
-
- @Override
- synchronized public void setAsterixNodeState(String nodeName, Object ns) throws RemoteException {
- nodeStateMap.put(nodeName, ns);
- }
}
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 24fe244..838de70 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
@@ -26,16 +26,15 @@
import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
import edu.uci.ics.asterix.common.context.AsterixAppRuntimeContext;
-import edu.uci.ics.asterix.common.functions.FunctionConstants;
import edu.uci.ics.asterix.external.adapter.factory.IAdapterFactory;
import edu.uci.ics.asterix.external.dataset.adapter.AdapterIdentifier;
import edu.uci.ics.asterix.metadata.IDatasetDetails;
import edu.uci.ics.asterix.metadata.MetadataManager;
import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
import edu.uci.ics.asterix.metadata.api.IMetadataIndex;
-import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
import edu.uci.ics.asterix.metadata.entities.AsterixBuiltinTypeMap;
import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
import edu.uci.ics.asterix.metadata.entities.Datatype;
import edu.uci.ics.asterix.metadata.entities.Dataverse;
import edu.uci.ics.asterix.metadata.entities.Index;
@@ -49,29 +48,33 @@
import edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat;
import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
import edu.uci.ics.asterix.transaction.management.resource.TransactionalResourceRepository;
-import edu.uci.ics.asterix.transaction.management.service.logging.DataUtil;
-import edu.uci.ics.asterix.transaction.management.service.logging.TreeResourceManager;
+import edu.uci.ics.asterix.transaction.management.service.logging.IndexResourceManager;
+import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager.ResourceType;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
import edu.uci.ics.hyracks.api.application.INCApplicationContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
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.storage.am.common.api.ITreeIndex;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+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.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
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
@@ -83,17 +86,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;
@@ -110,8 +119,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.
@@ -123,10 +131,14 @@
initLocalIndexArrays();
boolean isNewUniverse = true;
- TransactionalResourceRepository resourceRepository = runtimeContext.getTransactionProvider()
+ TransactionalResourceRepository resourceRepository = runtimeContext.getTransactionSubsystem()
.getTransactionalResourceRepository();
- resourceRepository.registerTransactionalResourceManager(TreeResourceManager.ID, new TreeResourceManager(
- runtimeContext.getTransactionProvider()));
+ resourceRepository.registerTransactionalResourceManager(ResourceType.LSM_BTREE, new IndexResourceManager(
+ ResourceType.LSM_BTREE, runtimeContext.getTransactionSubsystem()));
+ resourceRepository.registerTransactionalResourceManager(ResourceType.LSM_RTREE, new IndexResourceManager(
+ ResourceType.LSM_RTREE, runtimeContext.getTransactionSubsystem()));
+ resourceRepository.registerTransactionalResourceManager(ResourceType.LSM_INVERTED_INDEX,
+ new IndexResourceManager(ResourceType.LSM_INVERTED_INDEX, runtimeContext.getTransactionSubsystem()));
metadataNodeName = asterixProperties.getMetadataNodeName();
isNewUniverse = asterixProperties.isNewUniverse();
@@ -139,31 +151,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);
@@ -176,15 +182,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);
@@ -193,46 +200,27 @@
}
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);
+ }
+ 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 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);
- }
-
- private static void registerTransactionalResource(IMetadataIndex index,
+ private static void registerTransactionalResource(IMetadataIndex metadataIndex,
TransactionalResourceRepository resourceRepository) throws ACIDException {
- int fileId = index.getFileId();
- ITreeIndex treeIndex = (ITreeIndex) btreeRegistry.get(fileId);
- byte[] resourceId = DataUtil.intToByteArray(fileId);
- resourceRepository.registerTransactionalResource(resourceId, treeIndex);
- index.initTreeLogger(treeIndex);
+ long resourceId = metadataIndex.getResourceID();
+ IIndex index = indexLifecycleManager.getIndex(resourceId);
+ resourceRepository.registerTransactionalResource(resourceId, index);
+ metadataIndex.initIndexLogger(index);
}
public static void insertInitialDataverses(MetadataTransactionContext mdTxnCtx) throws Exception {
@@ -248,7 +236,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()));
}
}
@@ -328,37 +316,41 @@
private static DatasourceAdapter getAdapter(String adapterFactoryClassName) throws Exception {
String adapterName = ((IAdapterFactory) (Class.forName(adapterFactoryClassName).newInstance())).getName();
- return new DatasourceAdapter(new AdapterIdentifier(MetadataConstants.METADATA_DATAVERSE_NAME, adapterName), adapterFactoryClassName,
- DatasourceAdapter.AdapterType.INTERNAL);
+ return new DatasourceAdapter(new AdapterIdentifier(MetadataConstants.METADATA_DATAVERSE_NAME, adapterName),
+ adapterFactoryClassName, DatasourceAdapter.AdapterType.INTERNAL);
}
- 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.getLSMMergePolicy(),
+ runtimeContext.getLSMOperationTrackerFactory(), 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 5fcac2e..c069d0d 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
@@ -28,14 +28,16 @@
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.IAType;
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.asterix.transaction.management.service.logging.IndexLogger;
+import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
+import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager.ResourceType;
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.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
/**
* Descriptor for a primary or secondary index on metadata datasets.
@@ -61,15 +63,19 @@
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;
// Resource id of this index for use in transactions.
- protected byte[] indexResourceId;
+ protected long resourceId;
// Logger for tree indexes.
- private TreeLogger treeLogger;
+ private IndexLogger indexLogger;
+ // 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.");
@@ -115,14 +121,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
@@ -197,12 +204,11 @@
@Override
public void setFileId(int fileId) {
this.fileId = fileId;
- this.indexResourceId = DataUtil.intToByteArray(fileId);
}
@Override
- public void initTreeLogger(ITreeIndex treeIndex) throws ACIDException {
- this.treeLogger = new TreeLogger(indexResourceId, treeIndex);
+ public void initIndexLogger(IIndex index) throws ACIDException {
+ this.indexLogger = new IndexLogger(resourceId, ResourceType.LSM_BTREE, index);
}
@Override
@@ -215,12 +221,32 @@
return payloadType;
}
- @Override
- public byte[] getResourceId() {
- return indexResourceId;
+ public IndexLogger getIndexLogger() {
+ return indexLogger;
}
- 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;
+ }
+
+ @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 b1f6796..4061fed 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,18 @@
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 DATASOURCE_ADAPTER_DATASET_ID = 8;
+ public static final int FIRST_AVAILABLE_USER_DATASET_ID = 100;
+
public static IMetadataIndex DATASOURCE_ADAPTER_DATASET;
/**
@@ -49,33 +61,32 @@
}
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", "Name", "Arity" },
- MetadataRecordTypes.FUNCTION_RECORDTYPE);
+ MetadataRecordTypes.FUNCTION_RECORDTYPE, FUNCTION_DATASET_ID);
- DATASOURCE_ADAPTER_DATASET = new MetadataIndex("DatasourceAdapter", null, 3,
- new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING }, new String[] { "DataverseName", "Name" },
- MetadataRecordTypes.DATASOURCE_ADAPTER_RECORDTYPE);
-
+ DATASOURCE_ADAPTER_DATASET = new MetadataIndex("DatasourceAdapter", null, 3, new IAType[] {
+ BuiltinType.ASTRING, BuiltinType.ASTRING }, new String[] { "DataverseName", "Name" },
+ MetadataRecordTypes.DATASOURCE_ADAPTER_RECORDTYPE, DATASOURCE_ADAPTER_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 ec46e7b..50ea20a 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
@@ -137,7 +137,8 @@
private static final ARecordType createFeedDetailsRecordType() {
AOrderedListType orderedListType = new AOrderedListType(BuiltinType.ASTRING, null);
- AOrderedListType orderedListOfPropertiesType = new AOrderedListType(DATASOURCE_ADAPTER_PROPERTIES_RECORDTYPE, null);
+ AOrderedListType orderedListOfPropertiesType = new AOrderedListType(DATASOURCE_ADAPTER_PROPERTIES_RECORDTYPE,
+ null);
String[] fieldNames = { "FileStructure", "PartitioningStrategy", "PartitioningKey", "PrimaryKey", "GroupName",
"DatasourceAdapter", "Properties", "Function", "Status" };
@@ -148,8 +149,6 @@
return new ARecordType(null, fieldNames, fieldTypes, true);
}
- // 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;
@@ -158,10 +157,11 @@
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" };
+ "ExternalDetails", "FeedDetails", "Timestamp", "DatasetId" };
List<IAType> internalRecordUnionList = new ArrayList<IAType>();
internalRecordUnionList.add(BuiltinType.ANULL);
@@ -179,7 +179,7 @@
AUnionType feedRecordUnion = new AUnionType(feedRecordUnionList, null);
IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING,
- internalRecordUnion, externalRecordUnion, feedRecordUnion, BuiltinType.ASTRING };
+ internalRecordUnion, 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
new file mode 100644
index 0000000..35a9b83
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java
@@ -0,0 +1,272 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.metadata.declared;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.annotations.TypeDataGen;
+import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.formats.base.IDataFormat;
+import edu.uci.ics.asterix.metadata.MetadataException;
+import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
+import edu.uci.ics.asterix.metadata.api.IMetadataManager;
+import edu.uci.ics.asterix.metadata.bootstrap.AsterixProperties;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.Datatype;
+import edu.uci.ics.asterix.metadata.entities.Dataverse;
+import edu.uci.ics.asterix.metadata.entities.Index;
+import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
+import edu.uci.ics.asterix.metadata.entities.NodeGroup;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.data.IAWriterFactory;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+
+public class AqlCompiledMetadataDeclarations {
+ private static Logger LOGGER = Logger.getLogger(AqlCompiledMetadataDeclarations.class.getName());
+
+ // We are assuming that there is a one AqlCompiledMetadataDeclarations per
+ // transaction.
+ private final MetadataTransactionContext mdTxnCtx;
+ private String dataverseName = null;
+ private FileSplit outputFile;
+ private Map<String, String[]> stores;
+ private IDataFormat format;
+ private Map<String, String> config;
+
+ private final Map<String, IAType> types;
+ private final Map<String, TypeDataGen> typeDataGenMap;
+ private final IAWriterFactory writerFactory;
+
+ private IMetadataManager metadataManager = MetadataManager.INSTANCE;
+ private boolean isConnected = false;
+
+ public AqlCompiledMetadataDeclarations(MetadataTransactionContext mdTxnCtx, String dataverseName,
+ FileSplit outputFile, Map<String, String> config, Map<String, String[]> stores, Map<String, IAType> types,
+ Map<String, TypeDataGen> typeDataGenMap, IAWriterFactory writerFactory, boolean online) {
+ this.mdTxnCtx = mdTxnCtx;
+ this.dataverseName = dataverseName;
+ this.outputFile = outputFile;
+ this.config = config;
+ if (stores == null && online) {
+ this.stores = AsterixProperties.INSTANCE.getStores();
+ } else {
+ this.stores = stores;
+ }
+ this.types = types;
+ this.typeDataGenMap = typeDataGenMap;
+ this.writerFactory = writerFactory;
+ }
+
+ public void connectToDataverse(String dvName) throws AlgebricksException, AsterixException {
+ if (isConnected) {
+ throw new AlgebricksException("You are already connected to " + dataverseName + " dataverse");
+ }
+ Dataverse dv;
+ try {
+ dv = metadataManager.getDataverse(mdTxnCtx, dvName);
+ } catch (Exception e) {
+ throw new AsterixException(e);
+ }
+ if (dv == null) {
+ throw new AlgebricksException("There is no dataverse with this name " + dvName + " to connect to.");
+ }
+ dataverseName = dvName;
+ isConnected = true;
+ try {
+ format = (IDataFormat) Class.forName(dv.getDataFormat()).newInstance();
+ } catch (Exception e) {
+ throw new AsterixException(e);
+ }
+ }
+
+ public void disconnectFromDataverse() throws AlgebricksException {
+ if (!isConnected) {
+ throw new AlgebricksException("You are not connected to any dataverse");
+ }
+ dataverseName = null;
+ format = null;
+ isConnected = false;
+ }
+
+ public boolean isConnectedToDataverse() {
+ return isConnected;
+ }
+
+ public String getDataverseName() {
+ return dataverseName;
+ }
+
+ public FileSplit getOutputFile() {
+ return outputFile;
+ }
+
+ public IDataFormat getFormat() throws AlgebricksException {
+ if (!isConnected) {
+ throw new AlgebricksException("You need first to connect to a dataverse.");
+ }
+ return format;
+ }
+
+ public String getPropertyValue(String propertyName) {
+ return config.get(propertyName);
+ }
+
+ public IAType findType(String typeName) {
+ Datatype type;
+ try {
+ type = metadataManager.getDatatype(mdTxnCtx, dataverseName, typeName);
+ } catch (Exception e) {
+ throw new IllegalStateException();
+ }
+ if (type == null) {
+ throw new IllegalStateException();
+ }
+ return type.getDatatype();
+ }
+
+ public List<String> findNodeGroupNodeNames(String nodeGroupName) throws AlgebricksException {
+ NodeGroup ng;
+ try {
+ ng = metadataManager.getNodegroup(mdTxnCtx, nodeGroupName);
+ } catch (MetadataException e) {
+ throw new AlgebricksException(e);
+ }
+ if (ng == null) {
+ throw new AlgebricksException("No node group with this name " + nodeGroupName);
+ }
+ return ng.getNodeNames();
+ }
+
+ public Map<String, String[]> getAllStores() {
+ return stores;
+ }
+
+ public Dataset findDataset(String datasetName) throws AlgebricksException {
+ try {
+ return metadataManager.getDataset(mdTxnCtx, dataverseName, datasetName);
+ } catch (MetadataException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
+ public List<Index> getDatasetIndexes(String dataverseName, String datasetName) throws AlgebricksException {
+ 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);
+ } catch (MetadataException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
+ public Index getIndex(String dataverseName, String datasetName, String indexName) throws AlgebricksException {
+ try {
+ return metadataManager.getIndex(mdTxnCtx, dataverseName, datasetName, indexName);
+ } catch (MetadataException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
+ public void setOutputFile(FileSplit outputFile) {
+ this.outputFile = outputFile;
+ }
+
+ public Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
+ String datasetName, String targetIdxName) throws AlgebricksException {
+ FileSplit[] splits = splitsForInternalOrFeedDataset(datasetName, targetIdxName);
+ IFileSplitProvider splitProvider = new ConstantFileSplitProvider(splits);
+ String[] loc = new String[splits.length];
+ for (int p = 0; p < splits.length; p++) {
+ loc[p] = splits[p].getNodeName();
+ }
+ AlgebricksPartitionConstraint pc = new AlgebricksAbsolutePartitionConstraint(loc);
+ return new Pair<IFileSplitProvider, AlgebricksPartitionConstraint>(splitProvider, pc);
+ }
+
+ private FileSplit[] splitsForInternalOrFeedDataset(String datasetName, String targetIdxName)
+ throws AlgebricksException {
+
+ File relPathFile = new File(getRelativePath(datasetName + "_idx_" + targetIdxName));
+ Dataset dataset = findDataset(datasetName);
+ if (dataset.getDatasetType() != DatasetType.INTERNAL & dataset.getDatasetType() != DatasetType.FEED) {
+ throw new AlgebricksException("Not an internal or feed dataset");
+ }
+ InternalDatasetDetails datasetDetails = (InternalDatasetDetails) dataset.getDatasetDetails();
+ List<String> nodeGroup = findNodeGroupNodeNames(datasetDetails.getNodeGroupName());
+ if (nodeGroup == null) {
+ throw new AlgebricksException("Couldn't find node group " + datasetDetails.getNodeGroupName());
+ }
+
+ List<FileSplit> splitArray = new ArrayList<FileSplit>();
+ for (String nd : nodeGroup) {
+ String[] nodeStores = stores.get(nd);
+ if (nodeStores == null) {
+ LOGGER.warning("Node " + nd + " has no stores.");
+ throw new AlgebricksException("Node " + nd + " has no stores.");
+ } else {
+ for (int j = 0; j < nodeStores.length; j++) {
+ File f = new File(nodeStores[j] + File.separator + relPathFile);
+ splitArray.add(new FileSplit(nd, new FileReference(f)));
+ }
+ }
+ }
+ FileSplit[] splits = new FileSplit[splitArray.size()];
+ int i = 0;
+ for (FileSplit fs : splitArray) {
+ splits[i++] = fs;
+ }
+ return splits;
+ }
+
+ public String getRelativePath(String fileName) {
+ return dataverseName + File.separator + fileName;
+ }
+
+ public Map<String, TypeDataGen> getTypeDataGenMap() {
+ return typeDataGenMap;
+ }
+
+ public Map<String, IAType> getTypeDeclarations() {
+ return types;
+ }
+
+ public IAWriterFactory getWriterFactory() {
+ return writerFactory;
+ }
+
+ public MetadataTransactionContext getMetadataTransactionContext() {
+ return mdTxnCtx;
+ }
+}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
index fad1731..bdd9dc5 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
@@ -24,6 +24,8 @@
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.context.TransactionSubsystemProvider;
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;
@@ -64,7 +66,13 @@
import edu.uci.ics.asterix.runtime.base.AsterixTupleFilterFactory;
import edu.uci.ics.asterix.runtime.formats.FormatUtils;
import edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat;
-import edu.uci.ics.asterix.runtime.transaction.TreeIndexInsertUpdateDeleteOperatorDescriptor;
+import edu.uci.ics.asterix.runtime.job.listener.JobEventListenerFactory;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexModificationOperationCallbackFactory;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexSearchOperationCallbackFactory;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexModificationOperationCallbackFactory;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexSearchOperationCallbackFactory;
+import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager.ResourceType;
+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;
@@ -89,6 +97,8 @@
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.IBinaryHashFunctionFactory;
+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;
@@ -99,31 +109,34 @@
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.ISearchOperationCallbackFactory;
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.impls.NoOpOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
-import edu.uci.ics.hyracks.storage.am.rtree.dataflow.RTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.dataflow.LSMTreeIndexInsertUpdateDeleteOperatorDescriptor;
+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 static Logger LOGGER = Logger.getLogger(AqlMetadataProvider.class.getName());
-
private final MetadataTransactionContext mdTxnCtx;
private boolean isWriteTransaction;
private Map<String, String[]> stores;
private Map<String, String> config;
private IAWriterFactory writerFactory;
private FileSplit outputFile;
- private long jobTxnId;
-
private final Dataverse defaultDataverse;
+ private JobId jobId;
private static final Map<String, String> adapterFactoryMapping = initializeAdapterFactoryMapping();
@@ -148,9 +161,9 @@
this.defaultDataverse = defaultDataverse;
this.stores = AsterixProperties.INSTANCE.getStores();
}
-
- public void setJobTxnId(long txnId){
- this.jobTxnId = txnId;
+
+ public void setJobId(JobId jobId) {
+ this.jobId = jobId;
}
public Dataverse getDefaultDataverse() {
@@ -430,11 +443,37 @@
} catch (Exception e) {
throw new AlgebricksException(e);
}
+
+ ISearchOperationCallbackFactory searchCallbackFactory = null;
+ if (isSecondary) {
+ searchCallbackFactory = new SecondaryIndexSearchOperationCallbackFactory();
+ } else {
+ JobId jobId = ((JobEventListenerFactory) jobSpec.getJobletEventListenerFactory()).getJobId();
+ int datasetId = dataset.getDatasetId();
+ int[] primaryKeyFields = new int[numPrimaryKeys];
+ List<LogicalVariable> primaryKeyVars = new ArrayList<LogicalVariable>();
+ int varIndexOffset = outputVars.size() - numPrimaryKeys - 1;
+ for (int i = 0; i < numPrimaryKeys; i++) {
+ primaryKeyFields[i] = i;
+ primaryKeyVars.add(new LogicalVariable(outputVars.get(varIndexOffset + i).getId()));
+ }
+ IBinaryHashFunctionFactory[] entityIdFieldHashFunctionFactories = JobGenHelper
+ .variablesToBinaryHashFunctionFactories(primaryKeyVars, typeEnv, context);
+
+ TransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
+ searchCallbackFactory = new PrimaryIndexSearchOperationCallbackFactory(jobId, datasetId,
+ primaryKeyFields, entityIdFieldHashFunctionFactories, txnSubsystemProvider,
+ ResourceType.LSM_BTREE);
+ }
+
BTreeSearchOperatorDescriptor btreeSearchOp = new BTreeSearchOperatorDescriptor(jobSpec, outputRecDesc,
- appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(), spPc.first,
+ appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(), spPc.first,
typeTraits, comparatorFactories, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive,
- new BTreeDataflowHelperFactory(), retainInput, NoOpOperationCallbackProvider.INSTANCE);
+ new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER), retainInput, searchCallbackFactory);
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeSearchOp, spPc.second);
+
} catch (MetadataException me) {
throw new AlgebricksException(me);
}
@@ -486,11 +525,22 @@
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc = splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
dataset.getDataverseName(), dataset.getDatasetName(), indexName);
+
+ IBinaryComparatorFactory[] primaryComparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
+ dataset, recType, context.getBinaryComparatorFactoryProvider());
+ IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(keyType.getTypeTag());
+ ISearchOperationCallbackFactory searchCallbackFactory = new SecondaryIndexSearchOperationCallbackFactory();
RTreeSearchOperatorDescriptor rtreeSearchOp = new RTreeSearchOperatorDescriptor(jobSpec, outputRecDesc,
- appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(), spPc.first,
- typeTraits, comparatorFactories, keyFields, new RTreeDataflowHelperFactory(valueProviderFactories),
- retainInput, NoOpOperationCallbackProvider.INSTANCE);
+ appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(), spPc.first,
+ typeTraits, comparatorFactories, keyFields, new LSMRTreeDataflowHelperFactory(
+ valueProviderFactories, RTreePolicyType.RTREE, primaryComparatorFactories,
+ AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, proposeLinearizer(
+ nestedKeyType.getTypeTag(), comparatorFactories.length)), retainInput,
+ searchCallbackFactory);
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(rtreeSearchOp, spPc.second);
+
} catch (MetadataException me) {
throw new AlgebricksException(me);
}
@@ -598,7 +648,6 @@
String itemTypeName = dataset.getItemTypeName();
ARecordType itemType = (ARecordType) MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
dataset.getDataverseName(), itemTypeName).getDatatype();
-
ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(dataset, itemType);
IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
itemType, context.getBinaryComparatorFactoryProvider());
@@ -606,23 +655,28 @@
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
dataSource.getId().getDataverseName(), datasetName, indexName);
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
+
+ //TODO
+ //figure out the right behavior of the bulkload and then give the right callback
+ //(ex. what's the expected behavior when there is an error during bulkload?)
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(),
- NoOpOperationCallbackProvider.INSTANCE);
+ GlobalConfig.DEFAULT_BTREE_FILL_FACTOR, false, new LSMBTreeDataflowHelperFactory(
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER), NoOpOperationCallbackFactory.INSTANCE);
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad,
splitsAndConstraint.second);
-
} catch (MetadataException me) {
throw new AlgebricksException(me);
}
}
- public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertOrDeleteRuntime(IndexOp indexOp,
- IDataSource<AqlSourceId> dataSource, IOperatorSchema propagatedSchema, List<LogicalVariable> keys,
- LogicalVariable payload, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec)
- throws AlgebricksException {
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertOrDeleteRuntime(IndexOperation indexOp,
+ IDataSource<AqlSourceId> dataSource, IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv,
+ List<LogicalVariable> keys, LogicalVariable payload, RecordDescriptor recordDesc, JobGenContext context,
+ JobSpecification spec) throws AlgebricksException {
String datasetName = dataSource.getId().getDatasetName();
int numKeys = keys.size();
// Move key fields to front.
@@ -656,12 +710,32 @@
itemType, context.getBinaryComparatorFactoryProvider());
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
dataSource.getId().getDataverseName(), datasetName, indexName);
- TreeIndexInsertUpdateDeleteOperatorDescriptor btreeBulkLoad = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
- spec, recordDesc, appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(),
- splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation, indexOp,
- new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackProvider.INSTANCE, jobTxnId);
+
+ //prepare callback
+ JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
+ int datasetId = dataset.getDatasetId();
+ int[] primaryKeyFields = new int[numKeys];
+ for (i = 0; i < numKeys; i++) {
+ primaryKeyFields[i] = i;
+ }
+ IBinaryHashFunctionFactory[] entityIdFieldHashFunctionFactories = JobGenHelper
+ .variablesToBinaryHashFunctionFactories(keys, typeEnv, context);
+ TransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
+ PrimaryIndexModificationOperationCallbackFactory modificationCallbackFactory = new PrimaryIndexModificationOperationCallbackFactory(
+ jobId, datasetId, primaryKeyFields, entityIdFieldHashFunctionFactories, txnSubsystemProvider,
+ indexOp, ResourceType.LSM_BTREE);
+
+ LSMTreeIndexInsertUpdateDeleteOperatorDescriptor btreeBulkLoad = new LSMTreeIndexInsertUpdateDeleteOperatorDescriptor(
+ spec, recordDesc, appContext.getStorageManagerInterface(),
+ appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
+ comparatorFactories, fieldPermutation, indexOp, new LSMBTreeDataflowHelperFactory(
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER), null, modificationCallbackFactory);
+
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad,
splitsAndConstraint.second);
+
} catch (MetadataException me) {
throw new AlgebricksException(me);
}
@@ -669,27 +743,27 @@
@Override
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertRuntime(
- IDataSource<AqlSourceId> dataSource, IOperatorSchema propagatedSchema, List<LogicalVariable> keys,
- LogicalVariable payload, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec)
- throws AlgebricksException {
- return getInsertOrDeleteRuntime(IndexOp.INSERT, dataSource, propagatedSchema, keys, payload, recordDesc,
- context, spec);
+ IDataSource<AqlSourceId> dataSource, IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv,
+ List<LogicalVariable> keys, LogicalVariable payload, RecordDescriptor recordDesc, JobGenContext context,
+ JobSpecification spec) throws AlgebricksException {
+ return getInsertOrDeleteRuntime(IndexOperation.INSERT, dataSource, propagatedSchema, typeEnv, keys, payload,
+ recordDesc, context, spec);
}
@Override
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getDeleteRuntime(
- IDataSource<AqlSourceId> dataSource, IOperatorSchema propagatedSchema, List<LogicalVariable> keys,
- LogicalVariable payload, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec)
- throws AlgebricksException {
- return getInsertOrDeleteRuntime(IndexOp.DELETE, dataSource, propagatedSchema, keys, payload, recordDesc,
- context, spec);
+ IDataSource<AqlSourceId> dataSource, IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv,
+ List<LogicalVariable> keys, LogicalVariable payload, RecordDescriptor recordDesc, JobGenContext context,
+ JobSpecification spec) throws AlgebricksException {
+ return getInsertOrDeleteRuntime(IndexOperation.DELETE, dataSource, propagatedSchema, typeEnv, keys, payload,
+ recordDesc, context, spec);
}
- public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexInsertOrDeleteRuntime(IndexOp indexOp,
- IDataSourceIndex<String, AqlSourceId> dataSourceIndex, IOperatorSchema propagatedSchema,
- IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys,
- List<LogicalVariable> secondaryKeys, ILogicalExpression filterExpr, RecordDescriptor recordDesc,
- JobGenContext context, JobSpecification spec) throws AlgebricksException {
+ 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, JobGenContext context, JobSpecification spec) throws AlgebricksException {
String indexName = dataSourceIndex.getId();
String dataverseName = dataSourceIndex.getDataSource().getId().getDataverseName();
String datasetName = dataSourceIndex.getDataSource().getId().getDatasetName();
@@ -708,12 +782,12 @@
AsterixTupleFilterFactory filterFactory = createTupleFilterFactory(inputSchemas, typeEnv, filterExpr, context);
switch (secondaryIndex.getIndexType()) {
case BTREE: {
- return getBTreeDmlRuntime(dataverseName, datasetName, indexName, propagatedSchema, primaryKeys,
- secondaryKeys, filterFactory, recordDesc, context, spec, indexOp);
+ return getBTreeDmlRuntime(dataverseName, datasetName, indexName, propagatedSchema, typeEnv,
+ primaryKeys, secondaryKeys, filterFactory, recordDesc, context, spec, indexOp);
}
case RTREE: {
- return getRTreeDmlRuntime(dataverseName, datasetName, indexName, propagatedSchema, primaryKeys,
- secondaryKeys, filterFactory, recordDesc, context, spec, indexOp);
+ return getRTreeDmlRuntime(dataverseName, datasetName, indexName, propagatedSchema, typeEnv,
+ primaryKeys, secondaryKeys, filterFactory, recordDesc, context, spec, indexOp);
}
default: {
throw new AlgebricksException("Insert and delete not implemented for index type: "
@@ -728,8 +802,8 @@
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,
- primaryKeys, secondaryKeys, filterExpr, recordDesc, context, spec);
+ return getIndexInsertOrDeleteRuntime(IndexOperation.INSERT, dataSourceIndex, propagatedSchema, inputSchemas,
+ typeEnv, primaryKeys, secondaryKeys, filterExpr, recordDesc, context, spec);
}
@Override
@@ -738,8 +812,8 @@
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,
- primaryKeys, secondaryKeys, filterExpr, recordDesc, context, spec);
+ return getIndexInsertOrDeleteRuntime(IndexOperation.DELETE, dataSourceIndex, propagatedSchema, inputSchemas,
+ typeEnv, primaryKeys, secondaryKeys, filterExpr, recordDesc, context, spec);
}
private AsterixTupleFilterFactory createTupleFilterFactory(IOperatorSchema[] inputSchemas,
@@ -756,9 +830,11 @@
}
private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getBTreeDmlRuntime(String dataverseName,
- String datasetName, String indexName, IOperatorSchema propagatedSchema, List<LogicalVariable> primaryKeys,
- List<LogicalVariable> secondaryKeys, AsterixTupleFilterFactory filterFactory, RecordDescriptor recordDesc,
- JobGenContext context, JobSpecification spec, IndexOp indexOp) throws AlgebricksException {
+ String datasetName, String indexName, IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv,
+ List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
+ AsterixTupleFilterFactory filterFactory, RecordDescriptor recordDesc, JobGenContext context,
+ JobSpecification spec, IndexOperation indexOp) throws AlgebricksException {
+
int numKeys = primaryKeys.size() + secondaryKeys.size();
// generate field permutations
int[] fieldPermutation = new int[numKeys];
@@ -817,12 +893,33 @@
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
dataverseName, datasetName, indexName);
- TreeIndexInsertUpdateDeleteOperatorDescriptor btreeInsert = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
- spec, recordDesc, appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(),
- splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation, indexOp,
- new BTreeDataflowHelperFactory(), filterFactory, NoOpOperationCallbackProvider.INSTANCE,
- jobTxnId);
- return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeInsert,
+
+ //prepare callback
+ JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
+ int datasetId = dataset.getDatasetId();
+ int[] primaryKeyFields = new int[primaryKeys.size()];
+ i = 0;
+ for (LogicalVariable varKey : primaryKeys) {
+ int idx = propagatedSchema.findVariable(varKey);
+ primaryKeyFields[i] = idx;
+ i++;
+ }
+ IBinaryHashFunctionFactory[] entityIdFieldHashFunctionFactories = JobGenHelper
+ .variablesToBinaryHashFunctionFactories(primaryKeys, typeEnv, context);
+ TransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
+ SecondaryIndexModificationOperationCallbackFactory modificationCallbackFactory = new SecondaryIndexModificationOperationCallbackFactory(
+ jobId, datasetId, primaryKeyFields, entityIdFieldHashFunctionFactories, txnSubsystemProvider,
+ indexOp, ResourceType.LSM_BTREE);
+
+ LSMTreeIndexInsertUpdateDeleteOperatorDescriptor btreeBulkLoad = new LSMTreeIndexInsertUpdateDeleteOperatorDescriptor(
+ spec, recordDesc, appContext.getStorageManagerInterface(),
+ appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
+ comparatorFactories, fieldPermutation, indexOp, new LSMBTreeDataflowHelperFactory(
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER), filterFactory,
+ modificationCallbackFactory);
+ return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad,
splitsAndConstraint.second);
} catch (MetadataException e) {
throw new AlgebricksException(e);
@@ -830,9 +927,10 @@
}
private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getRTreeDmlRuntime(String dataverseName,
- String datasetName, String indexName, IOperatorSchema propagatedSchema, List<LogicalVariable> primaryKeys,
- List<LogicalVariable> secondaryKeys, AsterixTupleFilterFactory filterFactory, RecordDescriptor recordDesc,
- JobGenContext context, JobSpecification spec, IndexOp indexOp) throws AlgebricksException {
+ String datasetName, String indexName, IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv,
+ List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
+ AsterixTupleFilterFactory filterFactory, RecordDescriptor recordDesc, JobGenContext context,
+ JobSpecification spec, IndexOperation indexOp) throws AlgebricksException {
try {
Dataset dataset = MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverseName, datasetName);
String itemTypeName = dataset.getItemTypeName();
@@ -882,28 +980,67 @@
++i;
}
+ IBinaryComparatorFactory[] primaryComparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
+ dataset, recType, context.getBinaryComparatorFactoryProvider());
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
dataverseName, datasetName, indexName);
- TreeIndexInsertUpdateDeleteOperatorDescriptor rtreeUpdate = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
- spec, recordDesc, appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(),
- splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation, indexOp,
- new RTreeDataflowHelperFactory(valueProviderFactories), filterFactory,
- NoOpOperationCallbackProvider.INSTANCE, jobTxnId);
+
+ //prepare callback
+ JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
+ int datasetId = dataset.getDatasetId();
+ int[] primaryKeyFields = new int[numKeys];
+ i = 0;
+ for (LogicalVariable varKey : primaryKeys) {
+ int idx = propagatedSchema.findVariable(varKey);
+ primaryKeyFields[i] = idx;
+ i++;
+ }
+ IBinaryHashFunctionFactory[] entityIdFieldHashFunctionFactories = JobGenHelper
+ .variablesToBinaryHashFunctionFactories(primaryKeys, typeEnv, context);
+ TransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
+ SecondaryIndexModificationOperationCallbackFactory modificationCallbackFactory = new SecondaryIndexModificationOperationCallbackFactory(
+ jobId, datasetId, primaryKeyFields, entityIdFieldHashFunctionFactories, txnSubsystemProvider,
+ indexOp, ResourceType.LSM_RTREE);
+
+ LSMTreeIndexInsertUpdateDeleteOperatorDescriptor rtreeUpdate = new LSMTreeIndexInsertUpdateDeleteOperatorDescriptor(
+ spec, recordDesc, appContext.getStorageManagerInterface(),
+ appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
+ comparatorFactories, fieldPermutation, indexOp, new LSMRTreeDataflowHelperFactory(
+ valueProviderFactories, RTreePolicyType.RTREE, primaryComparatorFactories,
+ AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, proposeLinearizer(
+ nestedKeyType.getTypeTag(), comparatorFactories.length)), filterFactory,
+ modificationCallbackFactory);
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(rtreeUpdate, splitsAndConstraint.second);
} catch (MetadataException me) {
throw new AlgebricksException(me);
}
}
- public long getJobTxnId() {
- return jobTxnId;
+ 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 976bd87..2adf4fc 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 324429e..0cbd392 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
@@ -41,10 +41,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;
@@ -69,9 +73,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
@@ -86,6 +95,7 @@
}
private Dataset createDatasetFromARecord(ARecord datasetRecord) {
+
String dataverseName = ((AString) datasetRecord
.getValueByPos(MetadataRecordTypes.DATASET_ARECORD_DATAVERSENAME_FIELD_INDEX)).getStringValue();
String datasetName = ((AString) datasetRecord
@@ -94,6 +104,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: {
@@ -185,7 +197,7 @@
}
datasetDetails = new ExternalDatasetDetails(adapter, properties);
}
- return new Dataset(dataverseName, datasetName, typeName, datasetDetails, datasetType);
+ return new Dataset(dataverseName, datasetName, typeName, datasetDetails, datasetType, datasetId);
}
@Override
@@ -236,7 +248,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 d37fbc6..a2f5f49 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
@@ -48,6 +48,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.NotImplementedException;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -79,11 +80,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;
}
@@ -180,7 +181,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);
}
@@ -416,11 +417,11 @@
try {
if (typeName == null) {
typeName = suggestedTypeName;
- metadataNode.addDatatype(txnId, new Datatype(topLevelType.getDataverseName(), typeName, nestedType,
+ metadataNode.addDatatype(jobId, new Datatype(topLevelType.getDataverseName(), typeName, nestedType,
true));
}
- mn.insertIntoDatatypeSecondaryIndex(txnId, topLevelType.getDataverseName(), typeName,
+ mn.insertIntoDatatypeSecondaryIndex(jobId, topLevelType.getDataverseName(), typeName,
topLevelType.getDatatypeName());
} catch (BTreeDuplicateKeyException e) {
@@ -428,7 +429,6 @@
// a previous nested type.
}
return typeName;
-
}
private boolean isDerivedType(ATypeTag tag) {
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/builders/OrderedListBuilder.java b/asterix-om/src/main/java/edu/uci/ics/asterix/builders/OrderedListBuilder.java
index 1fdd7df..caad260 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/builders/OrderedListBuilder.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/builders/OrderedListBuilder.java
@@ -3,7 +3,7 @@
import java.io.IOException;
import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IToken;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
public class OrderedListBuilder extends AbstractListBuilder {
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 df3b43e..d1ee8c2 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.IOException;
import edu.uci.ics.hyracks.data.std.util.GrowableArray;
-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
deleted file mode 100644
index 2a7fdb5..0000000
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/ListEditDistanceSearchModifier.java
+++ /dev/null
@@ -1,36 +0,0 @@
-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;
-
-// TODO: Should go into hyracks.
-public class ListEditDistanceSearchModifier implements IInvertedIndexSearchModifier {
-
- private int edThresh;
-
- public ListEditDistanceSearchModifier(int edThresh) {
- 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;
- }
-
- public void setEdThresh(int edThresh) {
- this.edThresh = edThresh;
- }
-}
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
deleted file mode 100644
index b117a36..0000000
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/ListEditDistanceSearchModifierFactory.java
+++ /dev/null
@@ -1,21 +0,0 @@
-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;
-
-// TODO: Should go into hyracks.
-public class ListEditDistanceSearchModifierFactory implements IInvertedIndexSearchModifierFactory {
-
- 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 9dfe7df..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
@@ -5,11 +5,11 @@
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..eae4e62 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.2-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 261039a..d336c05 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
@@ -14,8 +14,8 @@
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.NGramUTF8StringBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+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 c316fb5..a98b754 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
@@ -12,7 +12,7 @@
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.lsm.invertedindex.tokenizers.IBinaryTokenizer;
public class WordTokensEvaluator implements ICopyEvaluator {
private 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 c4e8387..0bdb7ff 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
@@ -11,9 +11,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 90a4293..be982c0 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
@@ -11,10 +11,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 20a2977..1993453 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
@@ -11,9 +11,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 e86f86a6..43fda6f 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
@@ -11,9 +11,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 d6da522..554044a 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
@@ -11,10 +11,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 791ee6b..80cb6db 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
@@ -11,10 +11,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..45e19c6 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
@@ -2,7 +2,9 @@
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.DatasetId;
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,13 +15,17 @@
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;
}
+
+ public JobId getJobId() {
+ return jobId;
+ }
@Override
public IJobletEventListener createListener(final IHyracksJobletContext jobletContext) {
@@ -29,11 +35,11 @@
public void jobletFinish(JobStatus jobStatus) {
try {
ITransactionManager txnManager = ((AsterixAppRuntimeContext) jobletContext.getApplicationContext()
- .getApplicationObject()).getTransactionProvider().getTransactionManager();
- TransactionContext txnContext = txnManager.getTransactionContext(txnId);
+ .getApplicationObject()).getTransactionSubsystem().getTransactionManager();
+ TransactionContext txnContext = txnManager.getTransactionContext(jobId);
txnContext.setTransactionType(transactionalWrite ? TransactionType.READ_WRITE
: TransactionType.READ);
- txnManager.completedTransaction(txnContext, !(jobStatus == JobStatus.FAILURE));
+ txnManager.completedTransaction(txnContext, new DatasetId(-1), -1, !(jobStatus == JobStatus.FAILURE));
} catch (ACIDException e) {
throw new Error(e);
}
@@ -41,7 +47,12 @@
@Override
public void jobletStart() {
-
+ try {
+ ((AsterixAppRuntimeContext) jobletContext.getApplicationContext().getApplicationObject())
+ .getTransactionSubsystem().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 dc83694..45294df 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
@@ -48,6 +48,7 @@
import edu.uci.ics.asterix.om.types.TypeSignature;
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;
@@ -931,7 +932,7 @@
List<Statement> statements = 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, statements);
dmlt.translate();
typeMap = dmlt.getTypeMap();
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/translator/ADGenDmlTranslator.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/translator/ADGenDmlTranslator.java
index 7b91fcb..526580e 100644
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/translator/ADGenDmlTranslator.java
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/translator/ADGenDmlTranslator.java
@@ -19,58 +19,54 @@
public class ADGenDmlTranslator extends AbstractAqlTranslator {
- private final MetadataTransactionContext mdTxnCtx;
- private final List<Statement> aqlStatements;
- private Map<TypeSignature, IAType> types;
- private Map<TypeSignature, TypeDataGen> typeDataGenMap;
+ private final MetadataTransactionContext mdTxnCtx;
+ private final List<Statement> aqlStatements;
+ private Map<TypeSignature, IAType> types;
+ private Map<TypeSignature, TypeDataGen> typeDataGenMap;
- public ADGenDmlTranslator(MetadataTransactionContext mdTxnCtx,
- List<Statement> aqlStatements) {
- this.mdTxnCtx = mdTxnCtx;
- this.aqlStatements = aqlStatements;
- }
+ public ADGenDmlTranslator(MetadataTransactionContext mdTxnCtx, List<Statement> aqlStatements) {
+ this.mdTxnCtx = mdTxnCtx;
+ this.aqlStatements = aqlStatements;
+ }
- public void translate() throws AsterixException, MetadataException,
- AlgebricksException {
- String defaultDataverse = getDefaultDataverse();
- types = new HashMap<TypeSignature, IAType>();
- typeDataGenMap = new HashMap<TypeSignature, TypeDataGen>();
+ public void translate() throws AsterixException, MetadataException, AlgebricksException {
+ String defaultDataverse = getDefaultDataverse();
+ types = new HashMap<TypeSignature, IAType>();
+ typeDataGenMap = new HashMap<TypeSignature, TypeDataGen>();
- for (Statement stmt : aqlStatements) {
- if (stmt.getKind().equals(Statement.Kind.TYPE_DECL)) {
- TypeDecl td = (TypeDecl) stmt;
- String typeDataverse = td.getDataverseName() == null ? defaultDataverse
- : td.getDataverseName().getValue();
+ for (Statement stmt : aqlStatements) {
+ if (stmt.getKind().equals(Statement.Kind.TYPE_DECL)) {
+ TypeDecl td = (TypeDecl) stmt;
+ String typeDataverse = td.getDataverseName() == null ? defaultDataverse : td.getDataverseName()
+ .getValue();
- Map<TypeSignature, IAType> typeInStmt = TypeTranslator
- .computeTypes(mdTxnCtx, td, typeDataverse, types);
- types.putAll(typeInStmt);
+ Map<TypeSignature, IAType> typeInStmt = TypeTranslator.computeTypes(mdTxnCtx, td, typeDataverse, types);
+ types.putAll(typeInStmt);
- TypeSignature signature = new TypeSignature(typeDataverse, td
- .getIdent().getValue());
- TypeDataGen tdg = td.getDatagenAnnotation();
- if (tdg != null) {
- typeDataGenMap.put(signature, tdg);
- }
- }
- }
- }
+ TypeSignature signature = new TypeSignature(typeDataverse, td.getIdent().getValue());
+ TypeDataGen tdg = td.getDatagenAnnotation();
+ if (tdg != null) {
+ typeDataGenMap.put(signature, tdg);
+ }
+ }
+ }
+ }
- private String getDefaultDataverse() {
- for (Statement stmt : aqlStatements) {
- if (stmt.getKind().equals(Statement.Kind.DATAVERSE_DECL)) {
- return ((DataverseDecl) stmt).getDataverseName().getValue();
- }
- }
- return null;
- }
+ private String getDefaultDataverse() {
+ for (Statement stmt : aqlStatements) {
+ if (stmt.getKind().equals(Statement.Kind.DATAVERSE_DECL)) {
+ return ((DataverseDecl) stmt).getDataverseName().getValue();
+ }
+ }
+ return null;
+ }
- public Map<TypeSignature, IAType> getTypeMap() {
- return types;
- }
+ public Map<TypeSignature, IAType> getTypeMap() {
+ return types;
+ }
- public Map<TypeSignature, TypeDataGen> getTypeDataGenMap() {
- return typeDataGenMap;
- }
+ public Map<TypeSignature, TypeDataGen> getTypeDataGenMap() {
+ return typeDataGenMap;
+ }
}
diff --git a/asterix-transactions/pom.xml b/asterix-transactions/pom.xml
index c440c2a..2637220 100644
--- a/asterix-transactions/pom.xml
+++ b/asterix-transactions/pom.xml
@@ -32,5 +32,21 @@
<type>jar</type>
<scope>compile</scope>
</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>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-storage-am-lsm-invertedindex</artifactId>
+ <version>0.2.2-SNAPSHOT</version>
+ </dependency>
</dependencies>
+
</project>
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/AbstractLSMIOOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/AbstractLSMIOOperationCallback.java
new file mode 100644
index 0000000..6f7c220
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/AbstractLSMIOOperationCallback.java
@@ -0,0 +1,85 @@
+/*
+ * 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.ioopcallbacks;
+
+import java.util.List;
+
+import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
+import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
+
+public abstract class AbstractLSMIOOperationCallback implements ILSMIOOperationCallback {
+
+ // Object on which blocked LSMIndex operations are waiting.
+ protected final IndexOperationTracker opTracker;
+
+ public AbstractLSMIOOperationCallback(IndexOperationTracker opTracker) {
+ this.opTracker = opTracker;
+ }
+
+ @Override
+ public void beforeOperation(ILSMIOOperation operation) {
+ // Do nothing.
+ }
+
+ @Override
+ public synchronized void afterFinalize(ILSMIOOperation operation, ILSMComponent newComponent) {
+ // Wake up all blocked index operations that were waiting for this io operation to complete.
+ opTracker.notifyAll();
+ }
+
+ protected abstract long getComponentLSN(List<ILSMComponent> oldComponents) throws HyracksDataException;
+
+ protected void putLSNIntoMetadata(ITreeIndex treeIndex, List<ILSMComponent> oldComponents) throws HyracksDataException {
+ long componentLSN = getComponentLSN(oldComponents);
+ int fileId = treeIndex.getFileId();
+ IBufferCache bufferCache = treeIndex.getBufferCache();
+ ITreeIndexMetaDataFrame metadataFrame = treeIndex.getFreePageManager().getMetaDataFrameFactory().createFrame();
+ int metadataPageId = treeIndex.getFreePageManager().getFirstMetadataPage();
+ ICachedPage metadataPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, metadataPageId), false);
+ metadataPage.acquireWriteLatch();
+ try {
+ metadataFrame.setPage(metadataPage);
+ metadataFrame.setLSN(componentLSN);
+ } finally {
+ metadataPage.releaseWriteLatch();
+ bufferCache.unpin(metadataPage);
+ }
+ }
+
+ protected long getTreeIndexLSN(ITreeIndex treeIndex) throws HyracksDataException {
+ int fileId = treeIndex.getFileId();
+ IBufferCache bufferCache = treeIndex.getBufferCache();
+ ITreeIndexMetaDataFrame metadataFrame = treeIndex.getFreePageManager().getMetaDataFrameFactory().createFrame();
+ int metadataPageId = treeIndex.getFreePageManager().getFirstMetadataPage();
+ ICachedPage metadataPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, metadataPageId), false);
+ metadataPage.acquireReadLatch();
+ try {
+ metadataFrame.setPage(metadataPage);
+ return metadataFrame.getLSN();
+ } finally {
+ metadataPage.releaseReadLatch();
+ bufferCache.unpin(metadataPage);
+ }
+ }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMBTreeIOOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMBTreeIOOperationCallback.java
new file mode 100644
index 0000000..434e6ef
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMBTreeIOOperationCallback.java
@@ -0,0 +1,54 @@
+/*
+ * 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.ioopcallbacks;
+
+import java.util.List;
+
+import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTreeComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+
+public class LSMBTreeIOOperationCallback extends AbstractLSMIOOperationCallback {
+
+ public LSMBTreeIOOperationCallback(IndexOperationTracker opTracker) {
+ super(opTracker);
+ }
+
+ @Override
+ public void afterOperation(ILSMIOOperation operation, List<ILSMComponent> oldComponents, ILSMComponent newComponent)
+ throws HyracksDataException {
+ LSMBTreeComponent btreeComponent = (LSMBTreeComponent) newComponent;
+ putLSNIntoMetadata(btreeComponent.getBTree(), oldComponents);
+ }
+
+ @Override
+ protected long getComponentLSN(List<ILSMComponent> oldComponents) throws HyracksDataException {
+ if (oldComponents == null) {
+ // Implies a flush IO operation.
+ return opTracker.getLastLSN();
+ }
+ // Get max LSN from the oldComponents. Implies a merge IO operation.
+ long maxLSN = -1;
+ for (Object o : oldComponents) {
+ BTree btree = (BTree) o;
+ maxLSN = Math.max(getTreeIndexLSN(btree), maxLSN);
+ }
+ return maxLSN;
+ }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java
new file mode 100644
index 0000000..a51da07
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java
@@ -0,0 +1,35 @@
+/*
+ * 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.ioopcallbacks;
+
+import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+
+public class LSMBTreeIOOperationCallbackFactory implements ILSMIOOperationCallbackFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static LSMBTreeIOOperationCallbackFactory INSTANCE = new LSMBTreeIOOperationCallbackFactory();
+
+ private LSMBTreeIOOperationCallbackFactory() {
+ }
+
+ @Override
+ public ILSMIOOperationCallback createIOOperationCallback(Object syncObj) {
+ return new LSMBTreeIOOperationCallback((IndexOperationTracker) syncObj);
+ }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
new file mode 100644
index 0000000..b1450c1
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
@@ -0,0 +1,53 @@
+/*
+ * 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.ioopcallbacks;
+
+import java.util.List;
+
+import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexComponent;
+
+public class LSMInvertedIndexIOOperationCallback extends AbstractLSMIOOperationCallback {
+
+ public LSMInvertedIndexIOOperationCallback(IndexOperationTracker opTracker) {
+ super(opTracker);
+ }
+
+ @Override
+ public void afterOperation(ILSMIOOperation operation, List<ILSMComponent> oldComponents, ILSMComponent newComponent)
+ throws HyracksDataException {
+ LSMInvertedIndexComponent invIndexComponent = (LSMInvertedIndexComponent) newComponent;
+ putLSNIntoMetadata(invIndexComponent.getDeletedKeysBTree(), oldComponents);
+ }
+
+ @Override
+ protected long getComponentLSN(List<ILSMComponent> oldComponents) throws HyracksDataException {
+ if (oldComponents == null) {
+ // Implies a flush IO operation.
+ return opTracker.getLastLSN();
+ }
+ // Get max LSN from the oldComponents. Implies a merge IO operation.
+ long maxLSN = -1;
+ for (Object o : oldComponents) {
+ LSMInvertedIndexComponent invIndexComponent = (LSMInvertedIndexComponent) o;
+ maxLSN = Math.max(getTreeIndexLSN(invIndexComponent.getDeletedKeysBTree()), maxLSN);
+ }
+ return maxLSN;
+ }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java
new file mode 100644
index 0000000..790c60c
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java
@@ -0,0 +1,35 @@
+/*
+ * 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.ioopcallbacks;
+
+import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+
+public class LSMInvertedIndexIOOperationCallbackFactory implements ILSMIOOperationCallbackFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static LSMInvertedIndexIOOperationCallbackFactory INSTANCE = new LSMInvertedIndexIOOperationCallbackFactory();
+
+ private LSMInvertedIndexIOOperationCallbackFactory() {
+ }
+
+ @Override
+ public ILSMIOOperationCallback createIOOperationCallback(Object syncObj) {
+ return new LSMInvertedIndexIOOperationCallback((IndexOperationTracker) syncObj);
+ }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMRTreeIOOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMRTreeIOOperationCallback.java
new file mode 100644
index 0000000..ab3c9a6
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMRTreeIOOperationCallback.java
@@ -0,0 +1,53 @@
+/*
+ * 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.ioopcallbacks;
+
+import java.util.List;
+
+import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.LSMRTreeComponent;
+
+public class LSMRTreeIOOperationCallback extends AbstractLSMIOOperationCallback {
+
+ public LSMRTreeIOOperationCallback(IndexOperationTracker opTracker) {
+ super(opTracker);
+ }
+
+ @Override
+ public void afterOperation(ILSMIOOperation operation, List<ILSMComponent> oldComponents, ILSMComponent newComponent) throws HyracksDataException {
+ LSMRTreeComponent rtreeComponent = (LSMRTreeComponent) newComponent;
+ putLSNIntoMetadata(rtreeComponent.getRTree(), oldComponents);
+ putLSNIntoMetadata(rtreeComponent.getBTree(), oldComponents);
+ }
+
+ @Override
+ protected long getComponentLSN(List<ILSMComponent> oldComponents) throws HyracksDataException {
+ if (oldComponents == null) {
+ // Implies a flush IO operation.
+ return opTracker.getLastLSN();
+ }
+ // Get max LSN from the oldComponents. Implies a merge IO operation.
+ long maxLSN = -1;
+ for (Object o : oldComponents) {
+ LSMRTreeComponent rtreeComponent = (LSMRTreeComponent) o;
+ maxLSN = Math.max(getTreeIndexLSN(rtreeComponent.getRTree()), maxLSN);
+ }
+ return maxLSN;
+ }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java
new file mode 100644
index 0000000..4b47a95
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java
@@ -0,0 +1,35 @@
+/*
+ * 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.ioopcallbacks;
+
+import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+
+public class LSMRTreeIOOperationCallbackFactory implements ILSMIOOperationCallbackFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static LSMRTreeIOOperationCallbackFactory INSTANCE = new LSMRTreeIOOperationCallbackFactory();
+
+ private LSMRTreeIOOperationCallbackFactory() {
+ }
+
+ @Override
+ public ILSMIOOperationCallback createIOOperationCallback(Object syncObj) {
+ return new LSMRTreeIOOperationCallback((IndexOperationTracker) syncObj);
+ }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/AbstractOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/AbstractOperationCallback.java
new file mode 100644
index 0000000..1f018b4
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/AbstractOperationCallback.java
@@ -0,0 +1,71 @@
+/*
+ * 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.opcallbacks;
+
+import edu.uci.ics.asterix.transaction.management.service.locking.ILockManager;
+import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
+import edu.uci.ics.asterix.transaction.management.service.transaction.FieldsHashValueGenerator;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public abstract class AbstractOperationCallback {
+ protected final DatasetId datasetId;
+ protected final int[] primaryKeyFields;
+ protected final IBinaryHashFunction[] primaryKeyHashFunctions;
+ protected final ILockManager lockManager;
+ protected final TransactionContext txnCtx;
+ protected int transactorLocalNumActiveOperations = 0;
+
+ public AbstractOperationCallback(int datasetId, int[] primaryKeyFields,
+ IBinaryHashFunctionFactory[] primaryKeyHashFunctionFactories, TransactionContext txnCtx,
+ ILockManager lockManager) {
+ this.datasetId = new DatasetId(datasetId);
+ this.primaryKeyFields = primaryKeyFields;
+ if (primaryKeyHashFunctionFactories != null) {
+ this.primaryKeyHashFunctions = new IBinaryHashFunction[primaryKeyHashFunctionFactories.length];
+ for (int i = 0; i < primaryKeyHashFunctionFactories.length; ++i) {
+ this.primaryKeyHashFunctions[i] = primaryKeyHashFunctionFactories[i].createBinaryHashFunction();
+ }
+ } else {
+ this.primaryKeyHashFunctions = null;
+ }
+ this.txnCtx = txnCtx;
+ this.lockManager = lockManager;
+ }
+
+ public int computePrimaryKeyHashValue(ITupleReference tuple, int[] primaryKeyFields,
+ IBinaryHashFunction[] primaryKeyHashFunctions) {
+ return FieldsHashValueGenerator.computeFieldsHashValue(tuple, primaryKeyFields, primaryKeyHashFunctions);
+ }
+
+ public TransactionContext getTransactionContext() {
+ return txnCtx;
+ }
+
+ public int getLocalNumActiveOperations() {
+ return transactorLocalNumActiveOperations;
+ }
+
+ public void incrementLocalNumActiveOperations() {
+ transactorLocalNumActiveOperations++;
+ }
+
+ public void decrementLocalNumActiveOperations() {
+ transactorLocalNumActiveOperations--;
+ }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/AbstractOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/AbstractOperationCallbackFactory.java
new file mode 100644
index 0000000..84c747b
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/AbstractOperationCallbackFactory.java
@@ -0,0 +1,42 @@
+/*
+ * 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.opcallbacks;
+
+import java.io.Serializable;
+
+import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionSubsystemProvider;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+
+public abstract class AbstractOperationCallbackFactory implements Serializable{
+ protected final JobId jobId;
+ protected final int datasetId;
+ protected final int[] primaryKeyFields;
+ protected final IBinaryHashFunctionFactory[] primaryKeyHashFunctionFactories;
+ protected final ITransactionSubsystemProvider txnSubsystemProvider;
+ protected final byte resourceType;
+
+ public AbstractOperationCallbackFactory(JobId jobId, int datasetId, int[] primaryKeyFields,
+ IBinaryHashFunctionFactory[] primaryKeyHashFunctionFactories,
+ ITransactionSubsystemProvider txnSubsystemProvider, byte resourceType) {
+ this.jobId = jobId;
+ this.datasetId = datasetId;
+ this.primaryKeyFields = primaryKeyFields;
+ this.primaryKeyHashFunctionFactories = primaryKeyHashFunctionFactories;
+ this.txnSubsystemProvider = txnSubsystemProvider;
+ this.resourceType = resourceType;
+ }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/IndexOperationTracker.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/IndexOperationTracker.java
new file mode 100644
index 0000000..04b28be
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/IndexOperationTracker.java
@@ -0,0 +1,125 @@
+/*
+ * 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.opcallbacks;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+
+public class IndexOperationTracker implements ILSMOperationTracker {
+
+ // Number of active operations on a ILSMIndex instance.
+ private int numActiveOperations = 0;
+ private long lastLsn;
+ private final ILSMIndex index;
+ private final ILSMIOOperationCallback ioOpCallback;
+ private ILSMIndexAccessor accessor;
+
+ public IndexOperationTracker(ILSMIndex index, ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
+ this.index = index;
+ //TODO
+ //This code is added to avoid NullPointException when the index's comparatorFactory is null.
+ //The null comparator factory is set in the constructor of the IndexDropOperatorDescriptor.
+ if (ioOpCallbackFactory != null) {
+ ioOpCallback = ioOpCallbackFactory.createIOOperationCallback(this);
+ } else {
+ ioOpCallback = null;
+ }
+ }
+
+ @Override
+ public synchronized boolean beforeOperation(ISearchOperationCallback searchCallback,
+ IModificationOperationCallback modificationCallback, boolean tryOperation) throws HyracksDataException {
+ // Wait for pending flushes to complete.
+ // If flushFlag is set, then the flush is queued to occur by the last completing operation.
+ // This operation should wait for that flush to occur before proceeding.
+ if (index.getFlushStatus(index)) {
+ if (tryOperation) {
+ return false;
+ }
+ try {
+ this.wait();
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ numActiveOperations++;
+
+ // Increment transactor-local active operations count.
+ AbstractOperationCallback opCallback = getOperationCallback(searchCallback, modificationCallback);
+ if (opCallback != null) {
+ opCallback.incrementLocalNumActiveOperations();
+ }
+ return true;
+ }
+
+ @Override
+ public void afterOperation(ISearchOperationCallback searchCallback,
+ IModificationOperationCallback modificationCallback) throws HyracksDataException {
+ // Searches are immediately considered complete, because they should not prevent the execution of flushes.
+ if (searchCallback != null) {
+ completeOperation(searchCallback, modificationCallback);
+ }
+ }
+
+ @Override
+ public synchronized void completeOperation(ISearchOperationCallback searchCallback,
+ IModificationOperationCallback modificationCallback) throws HyracksDataException {
+ numActiveOperations--;
+
+ // Decrement transactor-local active operations count.
+ AbstractOperationCallback opCallback = getOperationCallback(searchCallback, modificationCallback);
+ if (opCallback != null) {
+ opCallback.decrementLocalNumActiveOperations();
+ }
+ // If we need a flush, and this is the last completing operation, then schedule the flush.
+ // Once the flush has completed notify all waiting operations.
+ if (index.getFlushStatus(index) && numActiveOperations == 0) {
+ if (accessor == null) {
+ accessor = (ILSMIndexAccessor) index.createAccessor(NoOpOperationCallback.INSTANCE,
+ NoOpOperationCallback.INSTANCE);
+ }
+ accessor.scheduleFlush(ioOpCallback);
+ }
+ }
+
+ private AbstractOperationCallback getOperationCallback(ISearchOperationCallback searchCallback,
+ IModificationOperationCallback modificationCallback) {
+
+ if (searchCallback == NoOpOperationCallback.INSTANCE || modificationCallback == NoOpOperationCallback.INSTANCE) {
+ return null;
+ }
+ if (searchCallback != null) {
+ return (AbstractOperationCallback) searchCallback;
+ } else {
+ return (AbstractOperationCallback) modificationCallback;
+ }
+ }
+
+ public long getLastLSN() {
+ return lastLsn;
+ }
+
+ public void setLastLSN(long lastLsn) {
+ this.lastLsn = lastLsn;
+ }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/IndexOperationTrackerFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/IndexOperationTrackerFactory.java
new file mode 100644
index 0000000..032a4f9
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/IndexOperationTrackerFactory.java
@@ -0,0 +1,38 @@
+/*
+ * 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.opcallbacks;
+
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+
+public class IndexOperationTrackerFactory implements ILSMOperationTrackerFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private final ILSMIOOperationCallbackFactory ioOpCallbackFactory;
+
+ public IndexOperationTrackerFactory(ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
+ this.ioOpCallbackFactory = ioOpCallbackFactory;
+ }
+
+ @Override
+ public ILSMOperationTracker createOperationTracker(ILSMIndex index) {
+ return new IndexOperationTracker(index, ioOpCallbackFactory);
+ }
+
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java
new file mode 100644
index 0000000..19e4396
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java
@@ -0,0 +1,82 @@
+/*
+ * 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.opcallbacks;
+
+import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.locking.ILockManager;
+import edu.uci.ics.asterix.transaction.management.service.logging.IndexLogger;
+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.TransactionSubsystem;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.tuples.LSMBTreeTupleReference;
+
+/**
+ * Assumes LSM-BTrees as primary indexes.
+ * Performs locking on primary keys, and also logs before/after images.
+ */
+public class PrimaryIndexModificationOperationCallback extends AbstractOperationCallback implements
+ IModificationOperationCallback {
+
+ protected final long resourceId;
+ protected final byte resourceType;
+ protected final IndexOperation indexOp;
+ protected final TransactionSubsystem txnSubsystem;
+
+ public PrimaryIndexModificationOperationCallback(int datasetId, int[] primaryKeyFields,
+ IBinaryHashFunctionFactory[] primaryKeyHashFunctionFactories, TransactionContext txnCtx,
+ ILockManager lockManager, TransactionSubsystem txnSubsystem, long resourceId, byte resourceType, IndexOperation indexOp) {
+ super(datasetId, primaryKeyFields, primaryKeyHashFunctionFactories, txnCtx, lockManager);
+ this.resourceId = resourceId;
+ this.resourceType = resourceType;
+ this.indexOp = indexOp;
+ this.txnSubsystem = txnSubsystem;
+ }
+
+ @Override
+ public void before(ITupleReference tuple) throws HyracksDataException {
+ int pkHash = computePrimaryKeyHashValue(tuple, primaryKeyFields, primaryKeyHashFunctions);
+ try {
+ lockManager.lock(datasetId, pkHash, LockMode.X, txnCtx);
+ } catch (ACIDException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ @Override
+ public void found(ITupleReference before, ITupleReference after) throws HyracksDataException {
+ IndexLogger logger = txnSubsystem.getTreeLoggerRepository().getIndexLogger(resourceId, resourceType);
+ int pkHash = computePrimaryKeyHashValue(after, primaryKeyFields, primaryKeyHashFunctions);
+ LSMBTreeTupleReference lsmBTreeTuple = (LSMBTreeTupleReference) before;
+ IndexOperation oldOp = IndexOperation.INSERT;
+ if (before == null) {
+ oldOp = IndexOperation.NOOP;
+ }
+ if (lsmBTreeTuple != null && lsmBTreeTuple.isAntimatter()) {
+ oldOp = IndexOperation.DELETE;
+ }
+ try {
+ logger.generateLogRecord(txnSubsystem, txnCtx, datasetId.getId(), pkHash, resourceId, indexOp, after,
+ oldOp, before);
+ } catch (ACIDException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
new file mode 100644
index 0000000..af1cb9d
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
@@ -0,0 +1,72 @@
+/*
+ * 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.opcallbacks;
+
+import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.resource.TransactionalResourceRepository;
+import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionSubsystemProvider;
+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.TransactionSubsystem;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+
+/**
+ * Assumes LSM-BTrees as primary indexes.
+ */
+public class PrimaryIndexModificationOperationCallbackFactory extends AbstractOperationCallbackFactory implements
+ IModificationOperationCallbackFactory {
+
+ private static final long serialVersionUID = 1L;
+ private final IndexOperation indexOp;
+
+ public PrimaryIndexModificationOperationCallbackFactory(JobId jobId, int datasetId, int[] primaryKeyFields,
+ IBinaryHashFunctionFactory[] primaryKeyHashFunctionFactories,
+ ITransactionSubsystemProvider txnSubsystemProvider, IndexOperation indexOp, byte resourceType) {
+ super(jobId, datasetId, primaryKeyFields, primaryKeyHashFunctionFactories, txnSubsystemProvider, resourceType);
+ this.indexOp = indexOp;
+ }
+
+ @Override
+ public IModificationOperationCallback createModificationOperationCallback(long resourceId, Object resource,
+ IHyracksTaskContext ctx) throws HyracksDataException {
+ TransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
+ TransactionalResourceRepository txnResourceRepository = txnSubsystem.getTransactionalResourceRepository();
+ ILSMIndex index = (ILSMIndex) txnResourceRepository.getTransactionalResource(resourceId);
+
+ //register the resource if it is not registered
+ if (index == null) {
+ txnSubsystem.getTransactionalResourceRepository().registerTransactionalResource(resourceId, resource);
+ index = (ILSMIndex) resource;
+ }
+
+ try {
+ TransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
+ IModificationOperationCallback modCallback = new PrimaryIndexModificationOperationCallback(datasetId,
+ primaryKeyFields, primaryKeyHashFunctionFactories, txnCtx, txnSubsystem.getLockManager(),
+ txnSubsystem, resourceId, resourceType, indexOp);
+ txnCtx.registerIndexAndCallback(index, (AbstractOperationCallback) modCallback);
+ return modCallback;
+ } catch (ACIDException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallback.java
new file mode 100644
index 0000000..85e8980
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallback.java
@@ -0,0 +1,67 @@
+/*
+ * 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.opcallbacks;
+
+import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.locking.ILockManager;
+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.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+
+/**
+ * Assumes LSM-BTrees as primary indexes. Implements try/locking and unlocking on primary keys.
+ */
+public class PrimaryIndexSearchOperationCallback extends AbstractOperationCallback implements ISearchOperationCallback {
+
+ public PrimaryIndexSearchOperationCallback(int datasetId, int[] entityIdFields,
+ IBinaryHashFunctionFactory[] entityIdFieldHashFunctionFactories, ILockManager lockManager,
+ TransactionContext txnCtx) {
+ super(datasetId, entityIdFields, entityIdFieldHashFunctionFactories, txnCtx, lockManager);
+ }
+
+ @Override
+ public boolean proceed(ITupleReference tuple) throws HyracksDataException {
+ int pkHash = computePrimaryKeyHashValue(tuple, primaryKeyFields, primaryKeyHashFunctions);
+ try {
+ return lockManager.tryLock(datasetId, pkHash, LockMode.S, txnCtx);
+ } catch (ACIDException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ @Override
+ public void reconcile(ITupleReference tuple) throws HyracksDataException {
+ int pkHash = computePrimaryKeyHashValue(tuple, primaryKeyFields, primaryKeyHashFunctions);
+ try {
+ lockManager.lock(datasetId, pkHash, LockMode.S, txnCtx);
+ } catch (ACIDException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ @Override
+ public void cancel(ITupleReference tuple) throws HyracksDataException {
+ int pkHash = computePrimaryKeyHashValue(tuple, primaryKeyFields, primaryKeyHashFunctions);
+ try {
+ lockManager.unlock(datasetId, pkHash, txnCtx);
+ } catch (ACIDException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java
new file mode 100644
index 0000000..3e6dfcb
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java
@@ -0,0 +1,53 @@
+/*
+ * 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.opcallbacks;
+
+import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionSubsystemProvider;
+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.TransactionSubsystem;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
+
+public class PrimaryIndexSearchOperationCallbackFactory extends AbstractOperationCallbackFactory implements
+ ISearchOperationCallbackFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public PrimaryIndexSearchOperationCallbackFactory(JobId jobId, int datasetId, int[] entityIdFields,
+ IBinaryHashFunctionFactory[] entityIdFieldHashFunctionFactories,
+ ITransactionSubsystemProvider txnSubsystemProvider, byte resourceType) {
+ super(jobId, datasetId, entityIdFields, entityIdFieldHashFunctionFactories, txnSubsystemProvider, resourceType);
+ }
+
+ @Override
+ public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx)
+ throws HyracksDataException {
+ TransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
+ try {
+ TransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
+ return new PrimaryIndexSearchOperationCallback(datasetId, primaryKeyFields,
+ primaryKeyHashFunctionFactories, txnSubsystem.getLockManager(), txnCtx);
+ } catch (ACIDException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallback.java
new file mode 100644
index 0000000..fa28c0c
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallback.java
@@ -0,0 +1,70 @@
+/*
+ * 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.opcallbacks;
+
+import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.locking.ILockManager;
+import edu.uci.ics.asterix.transaction.management.service.logging.IndexLogger;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+
+/**
+ * Secondary-index modifications do not require any locking.
+ * We assume that the modification of the corresponding primary index has already taken an appropriate lock.
+ * This callback performs logging of the before and/or after images for secondary indexes.
+ */
+public class SecondaryIndexModificationOperationCallback extends AbstractOperationCallback implements
+ IModificationOperationCallback {
+
+ protected final long resourceId;
+ protected final byte resourceType;
+ protected final IndexOperation indexOp;
+ protected final IndexOperation oldOp;
+ protected final TransactionSubsystem txnSubsystem;
+
+ public SecondaryIndexModificationOperationCallback(int datasetId, int[] primaryKeyFields,
+ IBinaryHashFunctionFactory[] primaryKeyHashFunctionFactories, TransactionContext txnCtx,
+ ILockManager lockManager, TransactionSubsystem txnSubsystem, long resourceId, byte resourceType, IndexOperation indexOp) {
+ super(datasetId, primaryKeyFields, primaryKeyHashFunctionFactories, txnCtx, lockManager);
+ this.resourceId = resourceId;
+ this.resourceType = resourceType;
+ this.indexOp = indexOp;
+ oldOp = (indexOp == IndexOperation.DELETE) ? IndexOperation.INSERT : IndexOperation.DELETE;
+ this.txnSubsystem = txnSubsystem;
+ }
+
+ @Override
+ public void before(ITupleReference tuple) throws HyracksDataException {
+ // Do nothing.
+ }
+
+ @Override
+ public void found(ITupleReference before, ITupleReference after) throws HyracksDataException {
+ IndexLogger logger = txnSubsystem.getTreeLoggerRepository().getIndexLogger(resourceId, resourceType);
+ int pkHash = computePrimaryKeyHashValue(after, primaryKeyFields, primaryKeyHashFunctions);
+ try {
+ logger.generateLogRecord(txnSubsystem, txnCtx, datasetId.getId(), pkHash, resourceId, indexOp, after,
+ oldOp, before);
+ } catch (ACIDException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
new file mode 100644
index 0000000..3fede95
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
@@ -0,0 +1,67 @@
+/*
+ * 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.opcallbacks;
+
+import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.resource.TransactionalResourceRepository;
+import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionSubsystemProvider;
+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.TransactionSubsystem;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+
+public class SecondaryIndexModificationOperationCallbackFactory extends AbstractOperationCallbackFactory implements
+ IModificationOperationCallbackFactory {
+
+ private static final long serialVersionUID = 1L;
+ private final IndexOperation indexOp;
+
+ public SecondaryIndexModificationOperationCallbackFactory(JobId jobId, int datasetId, int[] primaryKeyFields,
+ IBinaryHashFunctionFactory[] primaryKeyHashFunctionFactories,
+ ITransactionSubsystemProvider txnSubsystemProvider, IndexOperation indexOp, byte resourceType) {
+ super(jobId, datasetId, primaryKeyFields, primaryKeyHashFunctionFactories, txnSubsystemProvider, resourceType);
+ this.indexOp = indexOp;
+ }
+
+ @Override
+ public IModificationOperationCallback createModificationOperationCallback(long resourceId, Object resource, IHyracksTaskContext ctx)
+ throws HyracksDataException {
+ TransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
+ TransactionalResourceRepository txnResourceRepository = txnSubsystem.getTransactionalResourceRepository();
+ ILSMIndex index = (ILSMIndex) txnResourceRepository.getTransactionalResource(resourceId);
+
+ //register the resource if it is not registered
+ if (index == null) {
+ txnSubsystem.getTransactionalResourceRepository().registerTransactionalResource(resourceId, resource);
+ index = (ILSMIndex) resource;
+ }
+
+ try {
+ TransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
+ return new SecondaryIndexModificationOperationCallback(datasetId, primaryKeyFields,
+ primaryKeyHashFunctionFactories, txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId,
+ resourceType, indexOp);
+ } catch (ACIDException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexSearchOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexSearchOperationCallback.java
new file mode 100644
index 0000000..16aefee
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexSearchOperationCallback.java
@@ -0,0 +1,47 @@
+/*
+ * 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.opcallbacks;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+
+/**
+ * Secondary index searches perform no locking at all.
+ */
+public class SecondaryIndexSearchOperationCallback extends AbstractOperationCallback implements
+ ISearchOperationCallback {
+
+ public SecondaryIndexSearchOperationCallback() {
+ super(-1, null, null, null, null);
+ }
+
+ @Override
+ public boolean proceed(ITupleReference tuple) throws HyracksDataException {
+ return true;
+ }
+
+ @Override
+ public void reconcile(ITupleReference tuple) throws HyracksDataException {
+ // Do nothing.
+ }
+
+ @Override
+ public void cancel(ITupleReference tuple) throws HyracksDataException {
+ // Do nothing.
+ }
+
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexSearchOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexSearchOperationCallbackFactory.java
new file mode 100644
index 0000000..7172e06
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexSearchOperationCallbackFactory.java
@@ -0,0 +1,32 @@
+/*
+ * 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.opcallbacks;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
+
+public class SecondaryIndexSearchOperationCallbackFactory implements ISearchOperationCallbackFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx)
+ throws HyracksDataException {
+ return new SecondaryIndexSearchOperationCallback();
+ }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/TransactionalResourceRepository.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/TransactionalResourceRepository.java
index 25309e1..4532390 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/TransactionalResourceRepository.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/TransactionalResourceRepository.java
@@ -20,6 +20,7 @@
import java.util.Map;
import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager;
+import edu.uci.ics.asterix.transaction.management.service.transaction.MutableResourceId;
/**
* Represents a repository containing Resource Managers and Resources in the
@@ -32,20 +33,20 @@
*/
public class TransactionalResourceRepository {
- private Map<ByteBuffer, Object> resourceRepository = new HashMap<ByteBuffer, Object>(); // repository
+ private Map<MutableResourceId, Object> resourceRepository = new HashMap<MutableResourceId, Object>(); // repository
private Map<Byte, IResourceManager> resourceMgrRepository = new HashMap<Byte, IResourceManager>(); // repository
+
+ private MutableResourceId mutableResourceId = new MutableResourceId(0);
- public void registerTransactionalResource(byte[] resourceBytes, Object resource) {
- // convert to ByteBuffer so that a byte[] can be used as a key in a hash map.
- ByteBuffer resourceId = ByteBuffer.wrap(resourceBytes); // need to
-
+ public void registerTransactionalResource(long resourceId, Object resource) {
synchronized (resourceRepository) {
+ mutableResourceId.setId(resourceId);
+// MutableResourceId newMutableResourceId = new MutableResourceId(resourceId);
+// resourceRepository.put(newMutableResourceId, resource);
if (resourceRepository.get(resourceId) == null) {
- resourceRepository.put(resourceId, resource);
-
- // wake up threads waiting for the resource
- resourceRepository.notifyAll();
+ MutableResourceId newMutableResourceId = new MutableResourceId(resourceId);
+ resourceRepository.put(newMutableResourceId, resource);
}
}
}
@@ -54,26 +55,14 @@
synchronized (resourceMgrRepository) {
if (resourceMgrRepository.get(id) == null) {
resourceMgrRepository.put(id, resourceMgr);
-
- // wake up threads waiting for the resource manager
- resourceMgrRepository.notifyAll();
}
}
}
- public Object getTransactionalResource(byte[] resourceIdBytes) {
- ByteBuffer buffer = ByteBuffer.wrap(resourceIdBytes);
+ public Object getTransactionalResource(long resourceId) {
synchronized (resourceRepository) {
- while (resourceRepository.get(buffer) == null) {
- try {
- resourceRepository.wait();
- } catch (InterruptedException ie) {
- ie.printStackTrace();
- break; // the thread might be interrupted due to other
- // failures occurring elsewhere, break from the loop
- }
- }
- return resourceRepository.get(buffer);
+ mutableResourceId.setId(resourceId);
+ return resourceRepository.get(mutableResourceId);
}
}
@@ -81,7 +70,5 @@
synchronized (resourceMgrRepository) {
return resourceMgrRepository.get(id);
}
-
}
-
}
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..1341cc1 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,72 @@
* 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;
+ public void unlock(DatasetId datasetId, int entityHashValue, TransactionContext txnContext) 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
+ * @param datasetId
+ * @param entityHashValue
+ * @param txnContext
+ * @throws ACIDException TODO
*/
- public boolean convertLock(TransactionContext context, byte[] resourceID, int mode) throws ACIDException;
-
+ public void unlock(DatasetId datasetId, int entityHashValue, TransactionContext txnContext, boolean commitFlag) 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..6df3928 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,1706 @@
+/*
+ * 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.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.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.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.TransactionProvider;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
/**
- * @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 TransactionSubsystem txnSubsystem;
- 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);
+ //TODO
+ //This code should be taken care properly when there is a way to avoid doubling memory space for txnIds.
+ private LogicalLogLocator logicalLogLocator;
+
+ public LockManager(TransactionSubsystem txnSubsystem) throws ACIDException {
+ this.txnSubsystem = txnSubsystem;
+ 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();
+
+ this.logicalLogLocator = LogUtil.getDummyLogicalLogLocator(txnSubsystem.getLogManager());
+
+ 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);
-
- if (txrInfo == null) {
- throw new ACIDException("No lock is granted to the transaction, to convert");
- }
-
- 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");
- }
-
- curMode = tInfo.getMode();
- if (mode == curMode) { // Redundant
- return true; // We do not increment the counter, because it is a
- // conversion
- }
-
- if (!lMtx.isConversion(curMode, mode)) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Transaction " + txId + " already has grabbed a stronger mode (" + curMode + ") than "
- + mode);
- }
-
- return true;
- }
-
- lInfo = lmTables.getLockInfo(resourceID);
- if (lInfo == null) {
- throw new ACIDException("No lock on the resource, to convert");
- }
- }
-
- try {
- return convertLockForNewTransaction(context, resourceID, curMode, mode);
- } catch (InterruptedException e) {
- e.printStackTrace();
- }
-
- throw new ACIDException("Problem in Lock Converting for Transaction " + txId
- + " (We unexpectedly returned from convert lock for new transaction)");
+ public void unlock(DatasetId datasetId, int entityHashValue, TransactionContext txnContext) throws ACIDException {
+ internalUnlock(datasetId, entityHashValue, txnContext, false);
}
- 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 ");
- }
- return true;
- }
- // --------------------------------------------
+ @Override
+ public void unlock(DatasetId datasetId, int entityHashValue, TransactionContext txnContext, boolean commitFlag)
+ throws ACIDException {
+ internalUnlock(datasetId, entityHashValue, txnContext, commitFlag);
+ }
- int updatedMask = lInfo.getUpdatedMask(txId, curMode, LockInfo.UNKNOWN_IX);
- if (lMtx.conflicts(updatedMask, reqMode)) { // if Conflicting, Txr
- // needs to wait
+ private void internalUnlock(DatasetId datasetId, int entityHashValue, TransactionContext txnContext,
+ boolean commitFlag) throws ACIDException {
+ JobId jobId = txnContext.getJobId();
+ int eLockInfo = -1;
+ DatasetLockInfo dLockInfo = null;
+ JobInfo jobInfo;
+ int entityInfo = -1;
- // ---- 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;
- }
- // ---------------------------
-
- else {
- waitObj = woManager.allocate();
- if (waitObj == null) {
- throw new ACIDException("Invalid (null) object allocated as the WaitEntry for Txr " + txId);
- }
- lInfo.addToConvert(txId, reqMode, waitObj);
- txrInfo.setWaitOnRid(resourceId);
- context.setStartWaitTime(System.currentTimeMillis());
- }
- }
-
- 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 ");
- }
- return true;
+ if (IS_DEBUG_MODE) {
+ if (entityHashValue == -1) {
+ throw new UnsupportedOperationException(
+ "Unsupported unlock request: dataset-granule unlock is not supported");
}
}
- if (shouldAbort) {
- requestTxrAbort(context);
- return false;
+ latchLockTable();
+ validateJob(txnContext);
+
+ if (IS_DEBUG_MODE) {
+ trackLockRequest("Requested", RequestType.UNLOCK, datasetId, entityHashValue, (byte) 0, 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()) {
+ //find the resource to be unlocked
+ dLockInfo = datasetResourceHT.get(datasetId);
+ jobInfo = jobHT.get(jobId);
+ if (dLockInfo == null || jobInfo == null) {
+ unlatchLockTable();
+ throw new IllegalStateException("Invalid unlock request: Corresponding lock info doesn't exist.");
+ }
- waitObj.wait();
- }
+ eLockInfo = dLockInfo.getEntityResourceHT().get(entityHashValue);
- 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);
+ if (eLockInfo == -1) {
+ unlatchLockTable();
+ throw new IllegalStateException("Invalid unlock request: Corresponding lock info doesn't exist.");
+ }
+
+ //find the corresponding entityInfo
+ entityInfo = entityLockInfoManager.findEntityInfoFromHolderList(eLockInfo, jobId.getId(), entityHashValue);
+ if (entityInfo == -1) {
+ unlatchLockTable();
+ 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;
+
+ //TODO
+ //This code should be taken care properly when there is a way to avoid doubling memory space for txnIds.
+ //This commit log is written here in order to avoid increasing the memory space for managing transactionIds
+ if (commitFlag) {
+ if (txnContext.getTransactionType().equals(TransactionContext.TransactionType.READ_WRITE)) {
+ try {
+ txnSubsystem.getLogManager().log(LogType.COMMIT, txnContext, datasetId.getId(),
+ entityHashValue, -1, (byte) 0, 0, null, null, logicalLogLocator);
+ } catch (ACIDException e) {
+ try {
+ requestAbort(txnContext);
+ } finally {
+ unlatchLockTable();
+ }
+ }
}
+
+ txnContext.setLastLSNToIndexes(logicalLogLocator.getLsn());
+ }
+
+ //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;
+ }
+ waiterObjId = waiterObj.getNextWaiterObjId();
+ }
+ 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);
+ }
+ }
+
+ //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);
+ }
+
+ //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();
+ }
+
+ @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!!!");
+ }
+ }
+
+ //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);
+
+ //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);
+ }
+ }
+
+ 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+"]");
+ // }
+ }
+ }
+
+ //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);
+ }
+ }
+ }
+
+ 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);
+ }
+ }
+
+ unlatchLockTable();
+
+ return isSuccess;
+ }
+
+ 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(Thread.currentThread().getId() + ":");
+ 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);
+ }
+ }
}
- return true;
+ if (isInterruptedExceptionOccurred) {
+ throw new ACIDException("InterruptedException is caught");
+ }
+
+ //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)");
}
- @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);
+ /**
+ * 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() + ")");
}
- int wlMode = lInfo.getLockMode(wlModeIx);
- String wLModeRep = (wlMode == 0 ? "S" : "X");
- status += " for " + wLModeRep + " lock";
+ waiterObj.notifyAll();
}
-
- 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";
+ 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 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
- */
-
- 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) {
- 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 int getId() {
- return id;
- }
-
- public void setNext(int n) {
- next = n;
- }
-
- public int getNext() {
- return next;
- }
-
- 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..a6ed47c
--- /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.TransactionSubsystem;
+
+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;
+ TransactionSubsystem 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 TransactionSubsystem("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;
+ TransactionSubsystem txnProvider;
+ ILockManager lockMgr;
+ WorkerReadyQueue workerReadyQueue;
+ LockRequest lockRequest;
+ boolean needWait;
+ boolean isAwaken;
+ boolean isDone;
+
+ public LockRequestWorker(TransactionSubsystem 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..a8860bd
--- /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.TransactionSubsystem;
+
+/**
+ * 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;
+ TransactionSubsystem txnProvider = new TransactionSubsystem("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(TransactionSubsystem 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(TransactionSubsystem 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(TransactionSubsystem 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(TransactionSubsystem 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..91ce7ed 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,24 @@
return bytes;
}
+ public static long byteArrayToLong(byte[] bytes, int offset) {
+ return ((bytes[offset] & 0xff) << 56) + ((bytes[offset + 1] & 0xff) << 48) + ((bytes[offset + 2] & 0xff) << 40)
+ + ((bytes[offset + 3] & 0xff) << 32) + ((bytes[offset + 4] & 0xff) << 24)
+ + ((bytes[offset + 5] & 0xff) << 16) + ((bytes[offset + 6] & 0xff) << 8)
+ + ((bytes[offset + 7] & 0xff) << 0);
+ }
+
+ 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/ILogCursor.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogCursor.java
index 437c92b..991de1b 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogCursor.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogCursor.java
@@ -23,7 +23,7 @@
*/
public interface ILogCursor {
- public boolean next(LogicalLogLocator next) throws IOException, ACIDException;
+ public boolean next(LogicalLogLocator currentLogLocator) throws IOException, ACIDException;
public ILogFilter getLogFilter();
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogManager.java
index 1158029..c629d03 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogManager.java
@@ -15,51 +15,30 @@
package edu.uci.ics.asterix.transaction.management.service.logging;
import java.io.IOException;
-import java.util.Map;
import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.logging.IndexLogger.ReusableLogContentObject;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
public interface ILogManager {
/**
- * An API to write a log record.
- *
- * @param logicalLogLocator
- * A reusable object passed in by the caller. When the call
- * returns, this object has the physical location of the log
- * record that was written.
- * @param context
- * the transaction context associated with the transaction that
- * is writing the log record
- * @param resourceMgrId
- * the unique identifier of the resource manager that would be
- * handling (interpreting) the log record if there is a need to
- * process and apply the log record during a redo/undo task.
- * @param pageId
- * the unique identifier of the page where the operation
- * corresponding to the log record is applied
* @param logType
- * the type of log record (@see LogType)
- * @param logActionType
- * the action that needs to be taken when processing the log
- * record (@see LogActionType)
- * @param length
- * the length of the content inside the log record. This does not
- * include the header or the checksum size.
+ * @param context
+ * @param datasetId
+ * @param PKHashValue
+ * @param resourceId
+ * @param resourceMgrId
+ * @param logContentSize
+ * @param reusableLogContentObject
* @param logger
- * an implementation of the @see ILogger interface that is
- * invoked by the ILogManager instance to get the actual content
- * for the log record.
- * @param loggerArguments
- * Represent any additional arguments that needs to be passed
- * back in the call the to ILogger interface APIs.
+ * @param logicalLogLocator
* @throws ACIDException
*/
- public void log(LogicalLogLocator logicalLogLocator, TransactionContext context, byte resourceMgrId, long pageId,
- byte logType, byte logActionType, int length, ILogger logger, Map<Object, Object> loggerArguments)
- throws ACIDException;
+ void log(byte logType, TransactionContext context, int datasetId, int PKHashValue, long resourceId,
+ byte resourceMgrId, int logContentSize, ReusableLogContentObject reusableLogContentObject, ILogger logger,
+ LogicalLogLocator logicalLogLocator) throws ACIDException;
/**
* @param physicalLogLocator
@@ -85,13 +64,12 @@
public ILogCursor readLog(ILogFilter logFilter) throws ACIDException;
/**
+ * @param logicalLogLocator TODO
* @param PhysicalLogLocator
* specifies the location of the log record to be read
- * @return LogicalLogLocator represents the in-memory location of the log
- * record that has been fetched
* @throws ACIDException
*/
- public LogicalLogLocator readLog(PhysicalLogLocator physicalLogLocator) throws ACIDException;
+ public void readLog(long lsnValue, LogicalLogLocator logicalLogLocator) throws ACIDException;
/**
* Flushes the log records up to the lsn represented by the
@@ -123,8 +101,8 @@
* Returns the Transaction Provider associated with this ILogManager
* instance
*
- * @return TransactionProvider
+ * @return TransactionSubsystem
*/
- public TransactionProvider getTransactionProvider();
+ public TransactionSubsystem getTransactionSubsystem();
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogRecordHelper.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogRecordHelper.java
index ab4bfea..80f74cb 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogRecordHelper.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogRecordHelper.java
@@ -25,38 +25,41 @@
public interface ILogRecordHelper {
- public byte getLogType(LogicalLogLocator logicalLogLocator);
+ byte getLogType(LogicalLogLocator logicalLogLocator);
- public int getLogLength(LogicalLogLocator logicalLogLocator);
+ int getJobId(LogicalLogLocator logicalLogLocator);
- public long getLogTimestamp(LogicalLogLocator logicalLogLocator);
+ int getDatasetId(LogicalLogLocator logicalLogLocator);
- public long getLogChecksum(LogicalLogLocator logicalLogLocator);
+ int getPKHashValue(LogicalLogLocator logicalLogLocator);
- public long getLogTransactionId(LogicalLogLocator logicalLogLocator);
+ PhysicalLogLocator getPrevLSN(LogicalLogLocator logicalLogLocator);
- public byte getResourceMgrId(LogicalLogLocator logicalLogLocator);
+ boolean getPrevLSN(PhysicalLogLocator physicalLogLocator, LogicalLogLocator logicalLogLocator);
+
+ long getResourceId(LogicalLogLocator logicalLogLocator);
+
+ byte getResourceMgrId(LogicalLogLocator logicalLogLocater);
- public long getPageId(LogicalLogLocator logicalLogLocator);
+ int getLogContentSize(LogicalLogLocator logicalLogLocater);
- public int getLogContentBeginPos(LogicalLogLocator logicalLogLocator);
+ long getLogChecksum(LogicalLogLocator logicalLogLocator);
- public int getLogContentEndPos(LogicalLogLocator logicalLogLocator);
+ int getLogContentBeginPos(LogicalLogLocator logicalLogLocator);
- public String getLogRecordForDisplay(LogicalLogLocator logicalLogLocator);
+ int getLogContentEndPos(LogicalLogLocator logicalLogLocator);
- public byte getLogActionType(LogicalLogLocator logicalLogLocator);
+ String getLogRecordForDisplay(LogicalLogLocator logicalLogLocator);
- public PhysicalLogLocator getPreviousLsnByTransaction(LogicalLogLocator logicalLogLocator);
+ void writeLogHeader(LogicalLogLocator logicalLogLocator, byte logType, TransactionContext context, int datasetId,
+ int PKHashValue, long prevLogicalLogLocator, long resourceId, byte resourceMgrId, int logRecordSize);
- public boolean getPreviousLsnByTransaction(PhysicalLogLocator physicalLogLocator,
- LogicalLogLocator logicalLogLocator);
+ boolean validateLogRecord(LogicalLogLocator logicalLogLocator);
- public void writeLogHeader(TransactionContext context, LogicalLogLocator logicalLogLocator, byte resourceMgrId,
- long pageId, byte logType, byte logActionType, int logContentSize, long prevLsnValue);
+ int getLogRecordSize(byte logType, int logBodySize);
- public void writeLogTail(LogicalLogLocator logicalLogLocator, ILogManager logManager);
+ int getLogHeaderSize(byte logType);
- public boolean validateLogRecord(LogManagerProperties logManagerProperties, LogicalLogLocator logicalLogLocator);
+ int getLogChecksumSize();
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogger.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogger.java
index 92984cb..e26a3cc 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogger.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogger.java
@@ -14,9 +14,9 @@
*/
package edu.uci.ics.asterix.transaction.management.service.logging;
-import java.util.Map;
import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.logging.IndexLogger.ReusableLogContentObject;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
/**
@@ -25,11 +25,11 @@
*/
public interface ILogger {
- public void preLog(TransactionContext context, Map<Object, Object> loggerArguments) throws ACIDException;
+ public void preLog(TransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException;
- public void log(TransactionContext context, final LogicalLogLocator logicalLogLocator, int logRecordSize,
- Map<Object, Object> loggerArguments) throws ACIDException;
+ public void log(TransactionContext context, final LogicalLogLocator logicalLogLocator, int logContentSize,
+ ReusableLogContentObject reusableLogContentObject) throws ACIDException;
- public void postLog(TransactionContext context, Map<Object, Object> loggerArguments) throws ACIDException;
+ public void postLog(TransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException;
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLogger.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLogger.java
new file mode 100644
index 0000000..1eeadff
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLogger.java
@@ -0,0 +1,273 @@
+/*
+ * 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.transaction.management.service.logging;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+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.service.transaction.IResourceManager.ResourceType;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.common.tuples.SimpleTupleWriter;
+
+public class IndexLogger implements ILogger, ICloseable {
+
+ private final Map<Object, Object> jobId2ReusableLogContentObjectRepositoryMap = new ConcurrentHashMap<Object, Object>();
+
+ public static final String TREE_INDEX = "TREE_INDEX";
+ public static final String TUPLE_REFERENCE = "TUPLE_REFERENCE";
+ public static final String TUPLE_WRITER = "TUPLE_WRITER";
+ public static final String INDEX_OPERATION = "INDEX_OPERATION";
+ public static final String RESOURCE_ID = "RESOURCE_ID";
+
+ private final long resourceId;
+ private final byte resourceType;
+ private final SimpleTupleWriter tupleWriter;
+
+ public class BTreeOperationCodes {
+ public static final byte INSERT = 0;
+ public static final byte DELETE = 1;
+ }
+
+ public IndexLogger(long resourceId, byte resourceType, IIndex index) {
+ this.resourceId = resourceId;
+ this.resourceType = resourceType;
+ this.tupleWriter = new SimpleTupleWriter();
+ }
+
+ public synchronized void close(TransactionContext context) {
+ ReusableLogContentObjectRepository txnThreadStateRepository = (ReusableLogContentObjectRepository) jobId2ReusableLogContentObjectRepositoryMap
+ .get(context.getJobId());
+ txnThreadStateRepository.remove(Thread.currentThread().getId());
+ jobId2ReusableLogContentObjectRepositoryMap.remove(context.getJobId());
+ }
+
+ public void generateLogRecord(TransactionSubsystem txnSubsystem, TransactionContext context, int datasetId,
+ int PKHashValue, long resourceId, IndexOperation newOperation, ITupleReference newValue,
+ IndexOperation oldOperation, ITupleReference oldValue) throws ACIDException {
+
+ if (this.resourceId != resourceId) {
+ throw new ACIDException("IndexLogger mistach");
+ }
+
+ context.addCloseableResource(this); // the close method would be called
+ // on this TreeLogger instance at
+ // the time of transaction
+ // commit/abort.
+ if (newOperation != IndexOperation.INSERT && newOperation != IndexOperation.DELETE) {
+ throw new ACIDException("Loging for Operation " + newOperation + " not supported");
+ }
+
+ ReusableLogContentObject reusableLogContentObject = null;
+ ReusableLogContentObjectRepository reusableLogContentObjectRepository = null;
+ reusableLogContentObjectRepository = (ReusableLogContentObjectRepository) jobId2ReusableLogContentObjectRepositoryMap
+ .get(context.getJobId());
+ if (reusableLogContentObjectRepository == null) {
+ synchronized (context) { // threads belonging to different
+ // transaction do not need to
+ // synchronize amongst them.
+ if (reusableLogContentObjectRepository == null) {
+ reusableLogContentObjectRepository = new ReusableLogContentObjectRepository();
+ jobId2ReusableLogContentObjectRepositoryMap.put(context.getJobId(),
+ reusableLogContentObjectRepository);
+ }
+ }
+ }
+
+ reusableLogContentObject = reusableLogContentObjectRepository.getObject(Thread.currentThread().getId());
+ if (reusableLogContentObject == null) {
+ LogicalLogLocator logicalLogLocator = LogUtil.getDummyLogicalLogLocator(txnSubsystem.getLogManager());
+ reusableLogContentObject = new ReusableLogContentObject(logicalLogLocator, newOperation, newValue,
+ oldOperation, oldValue);
+ reusableLogContentObjectRepository.putObject(Thread.currentThread().getId(), reusableLogContentObject);
+ } else {
+ reusableLogContentObject.setNewOperation(newOperation);
+ reusableLogContentObject.setNewValue(newValue);
+ reusableLogContentObject.setOldOperation(oldOperation);
+ reusableLogContentObject.setOldValue(oldValue);
+ }
+
+ int logContentSize = 4/*TupleFieldCount*/+ 1/*NewOperation*/+ 4/*newValueLength*/;
+ if (newValue != null) {
+ logContentSize += tupleWriter.bytesRequired(newValue);
+ }
+
+ logContentSize += 1/*OldOperation*/+ 4/*oldValueLength*/;
+ if (oldValue != null) {
+ logContentSize += tupleWriter.bytesRequired(oldValue);
+ }
+
+ txnSubsystem.getLogManager().log(LogType.UPDATE, context, datasetId, PKHashValue, resourceId, resourceType,
+ logContentSize, reusableLogContentObject, this, reusableLogContentObject.getLogicalLogLocator());
+ }
+
+ @Override
+ public void log(TransactionContext context, LogicalLogLocator logicalLogLocator, int logContentSize,
+ ReusableLogContentObject reusableLogContentObject) throws ACIDException {
+ int offset = 0;
+ int tupleSize = 0;
+
+ //tuple field count
+ (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + offset, reusableLogContentObject
+ .getNewValue().getFieldCount());
+ offset += 4;
+
+ //new operation
+ (logicalLogLocator.getBuffer()).put(logicalLogLocator.getMemoryOffset() + offset,
+ (byte) reusableLogContentObject.getNewOperation().ordinal());
+ offset += 1;
+
+ //new tuple size
+ if (reusableLogContentObject.getNewValue() != null) {
+ tupleSize = tupleWriter.bytesRequired(reusableLogContentObject.getNewValue());
+ }
+ (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + offset, tupleSize);
+ offset += 4;
+
+ //new tuple
+ if (tupleSize != 0) {
+ tupleWriter.writeTuple(reusableLogContentObject.getNewValue(), logicalLogLocator.getBuffer().getArray(),
+ logicalLogLocator.getMemoryOffset() + offset);
+ offset += tupleSize;
+ }
+
+ if (resourceType == ResourceType.LSM_BTREE) {
+ //old operation
+ (logicalLogLocator.getBuffer()).put(logicalLogLocator.getMemoryOffset() + offset,
+ (byte) reusableLogContentObject.getOldOperation().ordinal());
+ offset += 1;
+
+ if (reusableLogContentObject.getOldOperation() != IndexOperation.NOOP) {
+ //old tuple size
+ if (reusableLogContentObject.getOldValue() != null) {
+ tupleSize = tupleWriter.bytesRequired(reusableLogContentObject.getOldValue());
+ } else {
+ tupleSize = 0;
+ }
+ (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + offset, tupleSize);
+ offset += 4;
+
+ if (tupleSize != 0) {
+ //old tuple
+ tupleWriter.writeTuple(reusableLogContentObject.getNewValue(), logicalLogLocator.getBuffer()
+ .getArray(), logicalLogLocator.getMemoryOffset() + offset);
+ }
+ }
+ }
+ }
+
+ @Override
+ public void postLog(TransactionContext context, ReusableLogContentObject reusableLogContentObject)
+ throws ACIDException {
+ }
+
+ @Override
+ public void preLog(TransactionContext context, ReusableLogContentObject reusableLogContentObject)
+ throws ACIDException {
+ }
+
+ /**
+ * Represents a utility class for generating log records corresponding to
+ * operations on a ITreeIndex implementation. A TreeLogger instance is thread
+ * safe and can be shared across multiple threads that may belong to same or
+ * different transactions.
+ */
+ public class ReusableLogContentObjectRepository {
+
+ private final Map<Long, ReusableLogContentObject> id2Object = new HashMap<Long, ReusableLogContentObject>();
+
+ public synchronized ReusableLogContentObject getObject(long threadId) {
+ return id2Object.get(threadId);
+ }
+
+ public synchronized void putObject(long threadId, ReusableLogContentObject reusableLogContentObject) {
+ this.id2Object.put(threadId, reusableLogContentObject);
+ }
+
+ public synchronized void remove(long threadId) {
+ id2Object.remove(threadId);
+ }
+ }
+
+ /**
+ * Represents the state of a transaction thread. The state contains information
+ * that includes the tuple being operated, the operation and the location of the
+ * log record corresponding to the operation.
+ */
+ public class ReusableLogContentObject {
+
+ private LogicalLogLocator logicalLogLocator;
+ private IndexOperation newOperation;
+ private ITupleReference newValue;
+ private IndexOperation oldOperation;
+ private ITupleReference oldValue;
+
+ public ReusableLogContentObject(LogicalLogLocator logicalLogLocator, IndexOperation newOperation,
+ ITupleReference newValue, IndexOperation oldOperation, ITupleReference oldValue) {
+ this.logicalLogLocator = logicalLogLocator;
+ this.newOperation = newOperation;
+ this.newValue = newValue;
+ this.oldOperation = oldOperation;
+ this.oldValue = oldValue;
+ }
+
+ public synchronized LogicalLogLocator getLogicalLogLocator() {
+ return logicalLogLocator;
+ }
+
+ public synchronized void setLogicalLogLocator(LogicalLogLocator logicalLogLocator) {
+ this.logicalLogLocator = logicalLogLocator;
+ }
+
+ public synchronized void setNewOperation(IndexOperation newOperation) {
+ this.newOperation = newOperation;
+ }
+
+ public synchronized IndexOperation getNewOperation() {
+ return newOperation;
+ }
+
+ public synchronized void setNewValue(ITupleReference newValue) {
+ this.newValue = newValue;
+ }
+
+ public synchronized ITupleReference getNewValue() {
+ return newValue;
+ }
+
+ public synchronized void setOldOperation(IndexOperation oldOperation) {
+ this.oldOperation = oldOperation;
+ }
+
+ public synchronized IndexOperation getOldOperation() {
+ return oldOperation;
+ }
+
+ public synchronized void setOldValue(ITupleReference oldValue) {
+ this.oldValue = oldValue;
+ }
+
+ public synchronized ITupleReference getOldValue() {
+ return oldValue;
+ }
+ }
+
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLoggerRepository.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLoggerRepository.java
new file mode 100644
index 0000000..7eefe37
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLoggerRepository.java
@@ -0,0 +1,46 @@
+/*
+ * 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.transaction.management.service.logging;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.asterix.transaction.management.service.transaction.MutableResourceId;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+
+public class IndexLoggerRepository {
+
+ private final Map<MutableResourceId, IndexLogger> loggers = new HashMap<MutableResourceId, IndexLogger>();
+ private final TransactionSubsystem provider;
+ private MutableResourceId mutableResourceId;
+
+ public IndexLoggerRepository(TransactionSubsystem provider) {
+ this.provider = provider;
+ mutableResourceId = new MutableResourceId(0);
+ }
+
+ public synchronized IndexLogger getIndexLogger(long resourceId, byte resourceType) {
+ mutableResourceId.setId(resourceId);
+ IndexLogger logger = loggers.get(mutableResourceId);
+ if (logger == null) {
+ MutableResourceId newMutableResourceId = new MutableResourceId(resourceId);
+ IIndex index = (IIndex) provider.getTransactionalResourceRepository().getTransactionalResource(resourceId);
+ logger = new IndexLogger(resourceId, resourceType, index);
+ loggers.put(newMutableResourceId, logger);
+ }
+ return logger;
+ }
+}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexResourceManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexResourceManager.java
new file mode 100644
index 0000000..a70fbe2
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexResourceManager.java
@@ -0,0 +1,115 @@
+/*
+ * 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.transaction.management.service.logging;
+
+import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+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.tuples.SimpleTupleReference;
+import edu.uci.ics.hyracks.storage.am.common.tuples.SimpleTupleWriter;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+
+public class IndexResourceManager implements IResourceManager {
+
+ public final byte resourceType;
+
+ private final TransactionSubsystem provider;
+
+ public IndexResourceManager(byte resourceType, TransactionSubsystem provider) {
+ this.resourceType = resourceType;
+ this.provider = provider;
+ }
+
+ public byte getResourceManagerId() {
+ return resourceType;
+ }
+
+ public void undo(ILogRecordHelper logRecordHelper, LogicalLogLocator logLocator) throws ACIDException {
+ long resourceId = logRecordHelper.getResourceId(logLocator);
+ int offset = logRecordHelper.getLogContentBeginPos(logLocator);
+
+ //TODO
+ //replace TransactionResourceRepository with IndexLifeCycleManager
+ // look up the repository to obtain the resource object
+ IIndex index = (IIndex) provider.getTransactionalResourceRepository().getTransactionalResource(resourceId);
+
+ /* field count */
+ int fieldCount = logLocator.getBuffer().readInt(logLocator.getMemoryOffset() + offset);
+ offset += 4;
+
+ /* new operation */
+ byte newOperation = logLocator.getBuffer().getByte(logLocator.getMemoryOffset() + offset);
+ offset += 1;
+
+ /* new value size */
+ int newValueSize = logLocator.getBuffer().readInt(logLocator.getMemoryOffset() + offset);
+ offset += 4;
+
+ /* new value */
+ SimpleTupleWriter tupleWriter = new SimpleTupleWriter();
+ SimpleTupleReference newTuple = (SimpleTupleReference) tupleWriter.createTupleReference();
+ newTuple.setFieldCount(fieldCount);
+ newTuple.resetByTupleOffset(logLocator.getBuffer().getByteBuffer(), offset);
+ offset += newValueSize;
+
+ ILSMIndexAccessor indexAccessor = (ILSMIndexAccessor) index.createAccessor(NoOpOperationCallback.INSTANCE,
+ NoOpOperationCallback.INSTANCE);
+
+ try {
+ if (resourceType == ResourceType.LSM_BTREE) {
+
+ /* old operation */
+ byte oldOperation = logLocator.getBuffer().getByte(logLocator.getMemoryOffset() + offset);
+ offset += 1;
+
+ if (oldOperation != (byte) IndexOperation.NOOP.ordinal()) {
+ /* old value size */
+ int oldValueSize = logLocator.getBuffer().readInt(logLocator.getMemoryOffset() + offset);
+ offset += 4;
+
+ /* old value */
+ SimpleTupleReference oldTuple = (SimpleTupleReference) tupleWriter.createTupleReference();
+ oldTuple.setFieldCount(fieldCount);
+ oldTuple.resetByTupleOffset(logLocator.getBuffer().getByteBuffer(), offset);
+ offset += oldValueSize;
+
+ if (oldOperation == (byte) IndexOperation.DELETE.ordinal()) {
+ indexAccessor.delete(oldTuple);
+ } else {
+ indexAccessor.insert(oldTuple);
+ }
+ } else {
+ indexAccessor.physicalDelete(newTuple);
+ }
+ } else {
+ if (newOperation == (byte) IndexOperation.DELETE.ordinal()) {
+ indexAccessor.insert(newTuple);
+ } else {
+ indexAccessor.delete(newTuple);
+ }
+ }
+ } catch (Exception e) {
+ throw new ACIDException("Undo failed", e);
+ }
+ }
+
+ public void redo(ILogRecordHelper logRecordHelper, LogicalLogLocator logicalLogLocator) throws ACIDException {
+ throw new UnsupportedOperationException(" Redo logic will be implemented as part of crash recovery feature");
+ }
+
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogCursor.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogCursor.java
index 7ba9130..61c47f4 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogCursor.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogCursor.java
@@ -21,33 +21,34 @@
public class LogCursor implements ILogCursor {
- private final ILogManager logManager;
+ private final LogManager logManager;
private final ILogFilter logFilter;
- private IFileBasedBuffer readOnlyBuffer;
+ private IBuffer readOnlyBuffer;
private LogicalLogLocator logicalLogLocator = null;
- private int bufferIndex = 0;
+ private long bufferIndex = 0;
+ private boolean firstNext = true;
+ private boolean readMemory = false;
+ private long readLSN = 0;
+ private boolean needReloadBuffer = true;
/**
* @param logFilter
*/
- public LogCursor(final ILogManager logManager, ILogFilter logFilter) throws ACIDException {
+ public LogCursor(final LogManager logManager, ILogFilter logFilter) throws ACIDException {
this.logFilter = logFilter;
this.logManager = logManager;
}
- public LogCursor(final ILogManager logManager, PhysicalLogLocator startingPhysicalLogLocator, ILogFilter logFilter)
- throws IOException {
+ public LogCursor(final LogManager logManager, PhysicalLogLocator startingPhysicalLogLocator, ILogFilter logFilter)
+ throws IOException, ACIDException {
this.logFilter = logFilter;
this.logManager = logManager;
initialize(startingPhysicalLogLocator);
}
- private void initialize(final PhysicalLogLocator startingPhysicalLogLocator) throws IOException {
- readOnlyBuffer = getReadOnlyBuffer(startingPhysicalLogLocator.getLsn(), logManager.getLogManagerProperties()
- .getLogBufferSize());
- logicalLogLocator = new LogicalLogLocator(startingPhysicalLogLocator.getLsn(), readOnlyBuffer, 0, logManager);
-
+ private void initialize(final PhysicalLogLocator startingPhysicalLogLocator) throws IOException, ACIDException {
+ logicalLogLocator = new LogicalLogLocator(startingPhysicalLogLocator.getLsn(), null, 0, logManager);
}
private IFileBasedBuffer getReadOnlyBuffer(long lsn, int size) throws IOException {
@@ -55,7 +56,8 @@
String filePath = LogUtil.getLogFilePath(logManager.getLogManagerProperties(), fileId);
File file = new File(filePath);
if (file.exists()) {
- return FileUtil.getFileBasedBuffer(filePath, lsn, size);
+ return FileUtil.getFileBasedBuffer(filePath, lsn
+ % logManager.getLogManagerProperties().getLogPartitionSize(), size);
} else {
return null;
}
@@ -66,19 +68,43 @@
* filter. The parameter nextLogLocator is set to the point to the next log
* record.
*
- * @param nextLogicalLogLocator
+ * @param currentLogLocator
* @return true if the cursor was successfully moved to the next log record
* false if there are no more log records that satisfy the
* configured filter.
*/
@Override
- public boolean next(LogicalLogLocator nextLogicalLogLocator) throws IOException, ACIDException {
+ public boolean next(LogicalLogLocator currentLogLocator) throws IOException, ACIDException {
+ //TODO
+ //Test the correctness when multiple log files are created
int integerRead = -1;
boolean logRecordBeginPosFound = false;
long bytesSkipped = 0;
+
+ //if the lsn to read is greater than the most recent lsn, then return false
+ if (logicalLogLocator.getLsn() > logManager.getCurrentLsn().get()) {
+ return false;
+ }
+
+ //if the lsn to read is greater than the last flushed lsn, then read from memory
+ if (logicalLogLocator.getLsn() > logManager.getLastFlushedLsn().get()) {
+ return readFromMemory(currentLogLocator);
+ }
+
+ //if the readOnlyBuffer should be reloaded, then load the log page from the log file.
+ //needReloadBuffer is set to true if the log record is read from the memory log page.
+ if (needReloadBuffer) {
+ readOnlyBuffer = getReadOnlyBuffer(logicalLogLocator.getLsn(), logManager.getLogManagerProperties()
+ .getLogBufferSize());
+ logicalLogLocator.setBuffer(readOnlyBuffer);
+ needReloadBuffer = false;
+ }
+
+ //check whether the currentOffset has enough space to have new log record by comparing
+ //the smallest log record type(which is commit)'s log header.
while (logicalLogLocator.getMemoryOffset() <= readOnlyBuffer.getSize()
- - logManager.getLogManagerProperties().getLogHeaderSize()) {
+ - logManager.getLogRecordHelper().getLogHeaderSize(LogType.COMMIT)) {
integerRead = readOnlyBuffer.readInt(logicalLogLocator.getMemoryOffset());
if (integerRead == logManager.getLogManagerProperties().logMagicNumber) {
logRecordBeginPosFound = true;
@@ -93,36 +119,46 @@
// bytes without finding a log record, it
// indicates an absence of logs any further.
}
+
+ if (logicalLogLocator.getLsn() > logManager.getLastFlushedLsn().get()) {
+ return next(currentLogLocator); //should read from memory if there is any further log
+ }
}
if (!logRecordBeginPosFound) {
// need to reload the buffer
- long lsnpos = (++bufferIndex * logManager.getLogManagerProperties().getLogBufferSize());
+ // TODO
+ // reduce IO by reading more pages(equal to logBufferSize) at a time.
+ long lsnpos = ((logicalLogLocator.getLsn() / logManager.getLogManagerProperties().getLogPageSize()) + 1)
+ * logManager.getLogManagerProperties().getLogPageSize();
+
readOnlyBuffer = getReadOnlyBuffer(lsnpos, logManager.getLogManagerProperties().getLogBufferSize());
if (readOnlyBuffer != null) {
logicalLogLocator.setBuffer(readOnlyBuffer);
logicalLogLocator.setLsn(lsnpos);
logicalLogLocator.setMemoryOffset(0);
- return next(nextLogicalLogLocator);
+ return next(currentLogLocator);
} else {
return false;
}
}
- int logLength = logManager.getLogRecordHelper().getLogLength(logicalLogLocator);
- if (logManager.getLogRecordHelper().validateLogRecord(logManager.getLogManagerProperties(), logicalLogLocator)) {
- if (nextLogicalLogLocator == null) {
- nextLogicalLogLocator = new LogicalLogLocator(0, readOnlyBuffer, -1, logManager);
+ int logLength = logManager.getLogRecordHelper().getLogRecordSize(
+ logManager.getLogRecordHelper().getLogType(logicalLogLocator),
+ logManager.getLogRecordHelper().getLogContentSize(logicalLogLocator));
+ if (logManager.getLogRecordHelper().validateLogRecord(logicalLogLocator)) {
+ if (currentLogLocator == null) {
+ currentLogLocator = new LogicalLogLocator(0, readOnlyBuffer, -1, logManager);
}
- nextLogicalLogLocator.setLsn(logicalLogLocator.getLsn());
- nextLogicalLogLocator.setMemoryOffset(logicalLogLocator.getMemoryOffset());
- nextLogicalLogLocator.setBuffer(readOnlyBuffer);
+ currentLogLocator.setLsn(logicalLogLocator.getLsn());
+ currentLogLocator.setMemoryOffset(logicalLogLocator.getMemoryOffset());
+ currentLogLocator.setBuffer(readOnlyBuffer);
logicalLogLocator.incrementLsn(logLength);
logicalLogLocator.setMemoryOffset(logicalLogLocator.getMemoryOffset() + logLength);
} else {
throw new ACIDException("Invalid Log Record found ! checksums do not match :( ");
}
- return logFilter.accept(readOnlyBuffer, nextLogicalLogLocator.getMemoryOffset(), logLength);
+ return logFilter.accept(readOnlyBuffer, currentLogLocator.getMemoryOffset(), logLength);
}
/**
@@ -135,4 +171,99 @@
return logFilter;
}
+ private boolean readFromMemory(LogicalLogLocator currentLogLocator) throws ACIDException, IOException {
+ byte[] logRecord = null;
+ long lsn = logicalLogLocator.getLsn();
+
+ //set the needReloadBuffer to true
+ needReloadBuffer = true;
+
+ int pageIndex = logManager.getLogPageIndex(lsn);
+ //int pageOffset = logManager.getLogPageOffset(lsn);
+ logicalLogLocator.setMemoryOffset(logManager.getLogPageOffset(lsn));
+
+ // take a lock on the log page so that the page is not flushed to
+ // disk interim
+ IFileBasedBuffer logPage = logManager.getLogPage(pageIndex);
+ synchronized (logPage) {
+ // need to check again if the log record in the log buffer or has reached the disk
+ if (lsn > logManager.getLastFlushedLsn().get()) {
+
+ //find the magic number to identify the start of the log record
+ //----------------------------------------------------------------
+ int readNumber = -1;
+ int logPageSize = logManager.getLogManagerProperties().getLogPageSize();
+ int logMagicNumber = logManager.getLogManagerProperties().logMagicNumber;
+ int bytesSkipped = 0;
+ boolean logRecordBeginPosFound = false;
+ //check whether the currentOffset has enough space to have new log record by comparing
+ //the smallest log record type(which is commit)'s log header.
+ while (logicalLogLocator.getMemoryOffset() <= logPageSize
+ - logManager.getLogRecordHelper().getLogHeaderSize(LogType.COMMIT)) {
+ readNumber = logPage.readInt(logicalLogLocator.getMemoryOffset());
+ if (readNumber == logMagicNumber) {
+ logRecordBeginPosFound = true;
+ break;
+ }
+ logicalLogLocator.increaseMemoryOffset(1);
+ logicalLogLocator.incrementLsn();
+ bytesSkipped++;
+ if (bytesSkipped > logPageSize) {
+ return false; // the maximum size of a log record is limited to
+ // a log page size. If we have skipped as many
+ // bytes without finding a log record, it
+ // indicates an absence of logs any further.
+ }
+ }
+
+ if (!logRecordBeginPosFound) {
+ // need to read the next log page
+ readOnlyBuffer = null;
+ logicalLogLocator.setBuffer(null);
+ logicalLogLocator.setLsn(lsn / logPageSize + 1);
+ logicalLogLocator.setMemoryOffset(0);
+ return next(currentLogLocator);
+ }
+ //------------------------------------------------------
+
+ logicalLogLocator.setBuffer(logPage);
+ int logLength = logManager.getLogRecordHelper().getLogRecordSize(
+ logManager.getLogRecordHelper().getLogType(logicalLogLocator),
+ logManager.getLogRecordHelper().getLogContentSize(logicalLogLocator));
+ logRecord = new byte[logLength];
+
+ //copy the log record and set the buffer of logical log locator to the buffer of the copied log record.
+ System.arraycopy(logPage.getArray(), logicalLogLocator.getMemoryOffset(), logRecord, 0, logLength);
+ MemBasedBuffer memBuffer = new MemBasedBuffer(logRecord);
+ readOnlyBuffer = memBuffer;
+ logicalLogLocator.setBuffer(readOnlyBuffer);
+ logicalLogLocator.setMemoryOffset(0);
+
+ if (logManager.getLogRecordHelper().validateLogRecord(logicalLogLocator)) {
+ if (currentLogLocator == null) {
+ currentLogLocator = new LogicalLogLocator(0, readOnlyBuffer, -1, logManager);
+ }
+ currentLogLocator.setLsn(logicalLogLocator.getLsn());
+ currentLogLocator.setMemoryOffset(logicalLogLocator.getMemoryOffset());
+ currentLogLocator.setBuffer(readOnlyBuffer);
+ logicalLogLocator.incrementLsn(logLength);
+ logicalLogLocator.setMemoryOffset(logicalLogLocator.getMemoryOffset() + logLength);
+ } else {
+ //if the checksum doesn't match, there is two possible scenario.
+ //case1) the log file corrupted: there's nothing we can do for this case during abort.
+ //case2) the log record is partially written by another thread. So, we may ignore this log record
+ // and continue to read the next log record
+ //[NOTICE]
+ //Only case2 is handled here.
+ logicalLogLocator.incrementLsn(logLength);
+ logicalLogLocator.setMemoryOffset(logicalLogLocator.getMemoryOffset() + logLength);
+ return next(currentLogLocator);
+ }
+ return logFilter.accept(readOnlyBuffer, currentLogLocator.getMemoryOffset(), logLength);
+
+ } else {
+ return next(currentLogLocator);//read from disk
+ }
+ }
+ }
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
index 1d1e067..2221e86 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
@@ -21,7 +21,6 @@
import java.io.RandomAccessFile;
import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
-import java.util.Map;
import java.util.Properties;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.atomic.AtomicInteger;
@@ -30,22 +29,16 @@
import java.util.logging.Logger;
import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.logging.IndexLogger.ReusableLogContentObject;
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.asterix.transaction.management.service.transaction.TransactionSubsystem;
public class LogManager implements ILogManager {
- /*
- * Log Record Structure HEADER
- * <(log_magic_number,4)(log_length,8)(log_type,1
- * )(log_action_type,1)(log_timestamp
- * ,8)(log_transaction_id,8)(resource_manager_id
- * ,1)(page_id,8)(previous_lsn,8) <CONTENT> TAIL <(checksum,8)>
- */
-
+ public static final boolean IS_DEBUG_MODE = false;//true
private static final Logger LOGGER = Logger.getLogger(LogManager.class.getName());
- private TransactionProvider provider;
+ private TransactionSubsystem provider;
private LogManagerProperties logManagerProperties;
/*
@@ -162,7 +155,7 @@
}
public void addFlushRequest(int pageIndex) {
- pendingFlushRequests[pageIndex].add(Thread.currentThread());
+ pendingFlushRequests[pageIndex].add(pendingFlushRequests);
}
public AtomicLong getLastFlushedLsn() {
@@ -181,13 +174,13 @@
return lastFlushedLsn.addAndGet(delta);
}
- public LogManager(TransactionProvider provider) throws ACIDException {
+ public LogManager(TransactionSubsystem provider) throws ACIDException {
this.provider = provider;
initLogManagerProperties(null);
initLogManager();
}
- public LogManager(TransactionProvider provider, LogManagerProperties logConfiguration) throws ACIDException {
+ public LogManager(TransactionSubsystem provider, LogManagerProperties logConfiguration) throws ACIDException {
this.provider = provider;
initLogManagerProperties(logConfiguration);
initLogManager();
@@ -433,6 +426,9 @@
}
if (forwardPage) {
+ //TODO
+ //this is not safe since the incoming thread may reach the same page slot with this page
+ //(differ by the log buffer size)
logPageStatus[prevPage].set(PageState.INACTIVE); // mark
// previous
// page
@@ -495,35 +491,35 @@
}
}
- public void log(LogicalLogLocator logLocator, TransactionContext context, byte resourceMgrId, long pageId,
- byte logType, byte logActionType, int requestedSpaceForLog, ILogger logger,
- Map<Object, Object> loggerArguments) throws ACIDException {
+ @Override
+ public void log(byte logType, TransactionContext context, int datasetId, int PKHashValue, long resourceId,
+ byte resourceMgrId, int logContentSize, ReusableLogContentObject reusableLogContentObject, ILogger logger,
+ LogicalLogLocator logicalLogLocator) throws ACIDException {
/*
* logLocator is a re-usable object that is appropriately set in each
* invocation. If the reference is null, the log manager must throw an
* exception
*/
- if (logLocator == null) {
+ if (logicalLogLocator == null) {
throw new ACIDException(
" you need to pass in a non-null logLocator, if you dont have it, then pass in a dummy so that the +"
+ "log manager can set it approporiately for you");
}
// compute the total log size including the header and the checksum.
- int totalLogSize = logManagerProperties.getLogHeaderSize() + requestedSpaceForLog
- + logManagerProperties.getLogChecksumSize();
+ int totalLogSize = logRecordHelper.getLogRecordSize(logType, logContentSize);
// check for the total space requirement to be less than a log page.
if (totalLogSize > logManagerProperties.getLogPageSize()) {
throw new ACIDException(
" Maximum Log Content Size is "
- + (logManagerProperties.getLogPageSize() - logManagerProperties.getLogHeaderSize() - logManagerProperties
+ + (logManagerProperties.getLogPageSize() - logRecordHelper.getLogHeaderSize(LogType.UPDATE) - logRecordHelper
.getLogChecksumSize()));
}
- // all constraints checked and we are goot to go and acquire a lsn.
- long previousLogLocator = -1;
- long myLogLocator; // the will be set to the location (a long value)
+ // all constraints checked and we are good to go and acquire a lsn.
+ long previousLSN = -1;
+ long currentLSN; // the will be set to the location (a long value)
// where the log record needs to be placed.
/*
@@ -534,10 +530,13 @@
* the last log record written by (any thread of) the transaction.
*/
synchronized (context) {
- previousLogLocator = context.getLastLogLocator().getLsn();
- myLogLocator = getLsn(totalLogSize, logType);
- context.getLastLogLocator().setLsn(myLogLocator);
- logLocator.setLsn(myLogLocator);
+ previousLSN = context.getLastLogLocator().getLsn();
+ currentLSN = getLsn(totalLogSize, logType);
+ context.setLastLSN(currentLSN);
+ if (IS_DEBUG_MODE) {
+ System.out.println("--------------> LSN(" + currentLSN + ") is allocated");
+ }
+ logicalLogLocator.setLsn(currentLSN);
}
/*
@@ -555,7 +554,7 @@
// thread has submitted a flush
// request.
- int pageIndex = (int) getLogPageIndex(myLogLocator);
+ int pageIndex = (int) getLogPageIndex(currentLSN);
/*
* the lsn has been obtained for the log record. need to set the
@@ -564,44 +563,54 @@
try {
- logLocator.setBuffer(logPages[pageIndex]);
- int pageOffset = getLogPageOffset(myLogLocator);
- logLocator.setMemoryOffset(pageOffset);
+ logicalLogLocator.setBuffer(logPages[pageIndex]);
+ int pageOffset = getLogPageOffset(currentLSN);
+ logicalLogLocator.setMemoryOffset(pageOffset);
/*
* write the log header.
*/
- logRecordHelper.writeLogHeader(context, logLocator, resourceMgrId, pageId, logType, logActionType,
- requestedSpaceForLog, previousLogLocator);
+ logRecordHelper.writeLogHeader(logicalLogLocator, logType, context, datasetId, PKHashValue, previousLSN,
+ resourceId, resourceMgrId, logContentSize);
// increment the offset so that the transaction can fill up the
// content in the correct region of the allocated space.
- logLocator.increaseMemoryOffset(logManagerProperties.getLogHeaderSize());
+ logicalLogLocator.increaseMemoryOffset(logRecordHelper.getLogHeaderSize(logType));
// a COMMIT log record does not have any content
// and hence the logger (responsible for putting the log content) is
// not invoked.
- if (requestedSpaceForLog != 0) {
- logger.preLog(context, loggerArguments);
+ if (logContentSize != 0) {
+ logger.preLog(context, reusableLogContentObject);
}
- if (requestedSpaceForLog != 0) {
+ if (logContentSize != 0) {
// call the logger implementation and ask to fill in the log
// record content at the allocated space.
- logger.log(context, logLocator, requestedSpaceForLog, loggerArguments);
- logger.postLog(context, loggerArguments);
+ logger.log(context, logicalLogLocator, logContentSize, reusableLogContentObject);
+ logger.postLog(context, reusableLogContentObject);
+ if (IS_DEBUG_MODE) {
+ logicalLogLocator.setMemoryOffset(logicalLogLocator.getMemoryOffset()
+ - logRecordHelper.getLogHeaderSize(logType));
+ System.out.println(logRecordHelper.getLogRecordForDisplay(logicalLogLocator));
+ logicalLogLocator.increaseMemoryOffset(logRecordHelper.getLogHeaderSize(logType));
+ }
}
/*
* The log record has been written. For integrity checks, compute
* the checksum and put it at the end of the log record.
*/
- int startPosChecksum = logLocator.getMemoryOffset() - logManagerProperties.getLogHeaderSize();
- int length = totalLogSize - logManagerProperties.getLogChecksumSize();
+ int startPosChecksum = logicalLogLocator.getMemoryOffset() - logRecordHelper.getLogHeaderSize(logType);
+ int length = totalLogSize - logRecordHelper.getLogChecksumSize();
long checksum = DataUtil.getChecksum(logPages[pageIndex], startPosChecksum, length);
- logPages[pageIndex].writeLong(pageOffset + logManagerProperties.getLogHeaderSize() + requestedSpaceForLog,
+ logPages[pageIndex].writeLong(pageOffset + logRecordHelper.getLogHeaderSize(logType) + logContentSize,
checksum);
+ if (IS_DEBUG_MODE) {
+ System.out.println("--------------> LSN(" + currentLSN + ") is written");
+ }
+
/*
* release the ownership as the log record has been placed in
* created space.
@@ -615,8 +624,7 @@
* If the transaction thread happens to be the last owner of the log
* page the page must by marked as a candidate to be flushed.
*/
- if (pageDirtyCount == 0) {
- logPageStatus[pageIndex].set(PageState.INACTIVE);
+ if (pageDirtyCount == 0 && logPageStatus[pageIndex].get() == PageState.INACTIVE) {
addFlushRequest(pageIndex);
addedFlushRequest = true;
}
@@ -628,24 +636,13 @@
* been flushed to disk because the containing log page filled up.
*/
if (logType == LogType.COMMIT) {
- if (getLastFlushedLsn().get() < myLogLocator) {
- if (!addedFlushRequest) {
- addFlushRequest(pageIndex);
- }
-
- /*
- * the commit log record is still in log buffer. need to
- * wait until the containing log page is flushed. When the
- * log flusher thread does flush the page, it notifies all
- * waiting threads of the flush event.
- */
- synchronized (logPages[pageIndex]) {
- while (getLastFlushedLsn().get() < myLogLocator) {
- logPages[pageIndex].wait();
- }
+ synchronized (logPages[pageIndex]) {
+ while (getLastFlushedLsn().get() < currentLSN) {
+ logPages[pageIndex].wait();
}
}
}
+
} catch (Exception e) {
e.printStackTrace();
throw new ACIDException(context, "Thread: " + Thread.currentThread().getName()
@@ -670,6 +667,9 @@
logPages[pageIndex].reset(filePath, LogUtil.getFileOffset(this, nextWritePosition),
logManagerProperties.getLogPageSize());
+
+ //TODO Check if this is necessary
+ //Arrays.fill(logPages[pageIndex].getArray(), (byte) 0);
}
@Override
@@ -688,11 +688,9 @@
/*
* Read a log that is residing on the disk.
*/
- private LogicalLogLocator readDiskLog(PhysicalLogLocator physicalLogLocator) throws ACIDException {
- LogicalLogLocator logicalLogLocator;
- String filePath = LogUtil.getLogFilePath(logManagerProperties,
- LogUtil.getFileId(this, physicalLogLocator.getLsn()));
- long fileOffset = LogUtil.getFileOffset(this, physicalLogLocator.getLsn());
+ private void readDiskLog(long lsnValue, LogicalLogLocator logicalLogLocator) throws ACIDException {
+ String filePath = LogUtil.getLogFilePath(logManagerProperties, LogUtil.getFileId(this, lsnValue));
+ long fileOffset = LogUtil.getFileOffset(this, lsnValue);
ByteBuffer buffer = ByteBuffer.allocate(logManagerProperties.getLogPageSize());
RandomAccessFile raf = null;
try {
@@ -701,15 +699,25 @@
FileChannel fileChannel = raf.getChannel();
fileChannel.read(buffer);
buffer.position(0);
- buffer.limit(buffer.getInt(4));
+ byte logType = buffer.get(4);
+ int logHeaderSize = logRecordHelper.getLogHeaderSize(logType);
+ int logBodySize = buffer.getInt(logHeaderSize - 4);
+ int logRecordSize = logHeaderSize + logBodySize + logRecordHelper.getLogChecksumSize();
+ buffer.limit(logRecordSize);
MemBasedBuffer memBuffer = new MemBasedBuffer(buffer.slice());
- logicalLogLocator = new LogicalLogLocator(physicalLogLocator.getLsn(), memBuffer, 0, this);
- if (!logRecordHelper.validateLogRecord(logManagerProperties, logicalLogLocator)) {
- throw new ACIDException(" invalid log record at lsn " + physicalLogLocator.getLsn());
+ if (logicalLogLocator == null) {
+ logicalLogLocator = new LogicalLogLocator(lsnValue, memBuffer, 0, this);
+ } else {
+ logicalLogLocator.setLsn(lsnValue);
+ logicalLogLocator.setBuffer(memBuffer);
+ logicalLogLocator.setMemoryOffset(0);
+ }
+ if (!logRecordHelper.validateLogRecord(logicalLogLocator)) {
+ throw new ACIDException(" invalid log record at lsn " + lsnValue);
}
} catch (Exception fnfe) {
- throw new ACIDException(" unable to retrieve log record with lsn " + physicalLogLocator.getLsn()
- + " from the file system", fnfe);
+ throw new ACIDException(" unable to retrieve log record with lsn " + lsnValue + " from the file system",
+ fnfe);
} finally {
try {
if (raf != null) {
@@ -720,23 +728,24 @@
throw new ACIDException(" exception in closing " + raf, ioe);
}
}
- return logicalLogLocator;
}
@Override
- public LogicalLogLocator readLog(PhysicalLogLocator physicalLogLocator) throws ACIDException {
+ public void readLog(long lsnValue, LogicalLogLocator logicalLogLocator) throws ACIDException {
byte[] logRecord = null;
- long lsnValue = physicalLogLocator.getLsn();
+ //long lsnValue = physicalLogLocator.getLsn();
if (lsnValue > lsn.get()) {
- throw new ACIDException(" invalid lsn " + physicalLogLocator);
+ throw new ACIDException(" invalid lsn " + lsnValue);
}
- LogicalLogLocator logLocator = null;
-
/* check if the log record in the log buffer or has reached the disk. */
if (lsnValue > getLastFlushedLsn().get()) {
int pageIndex = getLogPageIndex(lsnValue);
int pageOffset = getLogPageOffset(lsnValue);
+
+ //TODO
+ //minimize memory allocation overhead. current code allocates the log page size per reading a log record.
+
byte[] pageContent = new byte[logManagerProperties.getLogPageSize()];
// take a lock on the log page so that the page is not flushed to
// disk interim
@@ -750,25 +759,33 @@
// get the log record length
logPages[pageIndex].getBytes(pageContent, 0, pageContent.length);
- int logRecordLength = DataUtil.byteArrayToInt(pageContent, pageOffset + 4);
- logRecord = new byte[logRecordLength];
+ byte logType = pageContent[pageOffset + 4];
+ int logHeaderSize = logRecordHelper.getLogHeaderSize(logType);
+ int logBodySize = DataUtil.byteArrayToInt(pageContent, pageOffset + logHeaderSize - 4);
+ int logRecordSize = logHeaderSize + logBodySize + logRecordHelper.getLogChecksumSize();
+ logRecord = new byte[logRecordSize];
/*
* copy the log record content
*/
- System.arraycopy(pageContent, pageOffset, logRecord, 0, logRecordLength);
+ System.arraycopy(pageContent, pageOffset, logRecord, 0, logRecordSize);
MemBasedBuffer memBuffer = new MemBasedBuffer(logRecord);
- logLocator = new LogicalLogLocator(lsnValue, memBuffer, 0, this);
+ if (logicalLogLocator == null) {
+ logicalLogLocator = new LogicalLogLocator(lsnValue, memBuffer, 0, this);
+ } else {
+ logicalLogLocator.setLsn(lsnValue);
+ logicalLogLocator.setBuffer(memBuffer);
+ logicalLogLocator.setMemoryOffset(0);
+ }
try {
// validate the log record by comparing checksums
- if (!logRecordHelper.validateLogRecord(logManagerProperties, logLocator)) {
- throw new ACIDException(" invalid log record at lsn " + physicalLogLocator);
+ if (!logRecordHelper.validateLogRecord(logicalLogLocator)) {
+ throw new ACIDException(" invalid log record at lsn " + lsnValue);
}
} catch (Exception e) {
- throw new ACIDException("exception encoutered in validating log record at lsn "
- + physicalLogLocator, e);
+ throw new ACIDException("exception encoutered in validating log record at lsn " + lsnValue, e);
}
- return logLocator;
+ return;
}
}
}
@@ -776,9 +793,10 @@
/*
* the log record is residing on the disk, read it from there.
*/
- return readDiskLog(physicalLogLocator);
+ readDiskLog(lsnValue, logicalLogLocator);
}
+ @Override
public ILogRecordHelper getLogRecordHelper() {
return logRecordHelper;
}
@@ -789,6 +807,7 @@
* logic to event based when log manager support is integrated with the
* Buffer Manager.
*/
+ @Override
public synchronized void flushLog(LogicalLogLocator logicalLogLocator) throws ACIDException {
if (logicalLogLocator.getLsn() > lsn.get()) {
throw new ACIDException(" invalid lsn " + logicalLogLocator.getLsn());
@@ -828,6 +847,7 @@
return pageNo == 0 ? numLogPages - 1 : pageNo - 1;
}
+ @Override
public LogManagerProperties getLogManagerProperties() {
return logManagerProperties;
}
@@ -861,7 +881,7 @@
}
@Override
- public TransactionProvider getTransactionProvider() {
+ public TransactionSubsystem getTransactionSubsystem() {
return provider;
}
}
@@ -975,6 +995,7 @@
// that got flushed.
logManager.getLogPages()[pageToFlush].notifyAll();
logManager.setLastFlushedPage(pageToFlush);
+
}
} catch (IOException ioe) {
ioe.printStackTrace();
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManagerProperties.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManagerProperties.java
index 14b45b6..177533d 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManagerProperties.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManagerProperties.java
@@ -40,22 +40,15 @@
private int logPageSize = 128 * 1024; // 128 KB
private int numLogPages = 8; // number of log pages in the log buffer.
- private long logPartitionSize = logPageSize * 250; // maximum size of each
- // log file
+
private long groupCommitWaitPeriod = 0; // time in milliseconds for which a
// commit record will wait before
// the housing page is marked for
// flushing.
private int logBufferSize = logPageSize * numLogPages;
+ // maximum size of each log file
+ private long logPartitionSize = logBufferSize * 1024 * 2; //2GB
- private final int logHeaderSize = 43; /*
- * ( magic number(4) + (length(4) +
- * type(1) + actionType(1) +
- * timestamp(8) + transacitonId(8) +
- * resourceMgrId(1) + pageId(8) +
- * prevLSN(8)
- */
- private int logTailSize = 8; /* checksum(8) */
public int logMagicNumber = 123456789;
public static final String LOG_PARTITION_SIZE_KEY = "log_partition_size";
@@ -104,18 +97,6 @@
return logDir;
}
- public int getLogHeaderSize() {
- return logHeaderSize;
- }
-
- public int getLogChecksumSize() {
- return logTailSize;
- }
-
- public int getTotalLogRecordLength(int logContentSize) {
- return logContentSize + logHeaderSize + logTailSize;
- }
-
public int getLogPageSize() {
return logPageSize;
}
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..2ab4152 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
@@ -21,18 +21,45 @@
* for writing/reading of log header and checksum as well as validating log
* record by checksum comparison. Every ILogManager implementation has an
* associated ILogRecordHelper implementation.
+ * == LogRecordFormat ==
+ * [Header]
+ * --------------------------- Header part1(17) : Both COMMIT and UPDATE log type have part1 fields
+ * LogMagicNumber(4)
+ * LogType(1)
+ * JobId(4)
+ * DatasetId(4) //stored in dataset_dataset in Metadata Node
+ * PKHashValue(4)
+ * --------------------------- Header part2(21) : Only UPDATE log type has part2 fields
+ * PrevLSN(8) //only for UPDATE
+ * ResourceId(8) //stored in .metadata of the corresponding index in NC node
+ * ResourceMgrId(1)
+ * LogRecordSize(4)
+ * --------------------------- COMMIT doesn't have Body fields.
+ * [Body] The Body size is given through the parameter reusableLogContentObjectLength
+ * TupleFieldCount(4)
+ * NewOp(1)
+ * NewValueLength(4)
+ * NewValue(NewValueLength)
+ * OldOp(1)
+ * OldValueLength(4)
+ * OldValue(OldValueLength)
+ * --------------------------- Both COMMIT and UPDATE have tail fields.
+ * [Tail]
+ * Checksum(8)
*/
public class LogRecordHelper implements ILogRecordHelper {
- private final int BEGIN_MAGIC_NO_POS = 0;
- private final int BEGING_LENGTH_POS = 4;
- private final int BEGIN_TYPE_POS = 8;
- 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 LOG_CHECKSUM_SIZE = 8;
+
+ private final int MAGIC_NO_POS = 0;
+ private final int LOG_TYPE_POS = 4;
+ private final int JOB_ID_POS = 5;
+ private final int DATASET_ID_POS = 9;
+ private final int PK_HASH_VALUE_POS = 13;
+ private final int PREV_LSN_POS = 17;
+ private final int RESOURCE_ID_POS = 25;
+ private final int RESOURCE_MGR_ID_POS = 33;
+ private final int LOG_RECORD_SIZE_POS = 34;
private ILogManager logManager;
@@ -40,59 +67,38 @@
this.logManager = logManager;
}
+ @Override
public byte getLogType(LogicalLogLocator logicalLogLocator) {
- return logicalLogLocator.getBuffer().getByte(logicalLogLocator.getMemoryOffset() + BEGIN_TYPE_POS);
+ return logicalLogLocator.getBuffer().getByte(logicalLogLocator.getMemoryOffset() + LOG_TYPE_POS);
}
- public byte getLogActionType(LogicalLogLocator logicalLogLocator) {
- return logicalLogLocator.getBuffer().getByte(logicalLogLocator.getMemoryOffset() + BEGIN_ACTION_TYPE_POS);
+ @Override
+ public int getJobId(LogicalLogLocator logicalLogLocator) {
+ return logicalLogLocator.getBuffer().readInt(logicalLogLocator.getMemoryOffset() + JOB_ID_POS);
}
- public int getLogLength(LogicalLogLocator logicalLogLocator) {
- return (logicalLogLocator.getBuffer()).readInt(logicalLogLocator.getMemoryOffset() + BEGING_LENGTH_POS);
+ @Override
+ public int getDatasetId(LogicalLogLocator logicalLogLocator) {
+ return logicalLogLocator.getBuffer().readInt(logicalLogLocator.getMemoryOffset() + DATASET_ID_POS);
}
- public long getLogTimestamp(LogicalLogLocator logicalLogLocator) {
- return (logicalLogLocator.getBuffer()).readLong(logicalLogLocator.getMemoryOffset() + BEGIN_TIMESTAMP_POS);
+ @Override
+ public int getPKHashValue(LogicalLogLocator logicalLogLocator) {
+ return logicalLogLocator.getBuffer().readInt(logicalLogLocator.getMemoryOffset() + PK_HASH_VALUE_POS);
}
- public long getLogChecksum(LogicalLogLocator logicalLogLocator) {
- return (logicalLogLocator.getBuffer()).readLong(logicalLogLocator.getMemoryOffset()
- + getLogLength(logicalLogLocator) - 8);
- }
-
- public long getLogTransactionId(LogicalLogLocator logicalLogLocator) {
- return (logicalLogLocator.getBuffer()).readLong(logicalLogLocator.getMemoryOffset() + BEGIN_TRANSACTION_ID_POS);
- }
-
- public byte getResourceMgrId(LogicalLogLocator logicalLogLocator) {
- return (logicalLogLocator.getBuffer()).getByte(logicalLogLocator.getMemoryOffset() + BEGIN_RESOURCE_MGR_ID_POS);
- }
-
- public long getPageId(LogicalLogLocator logicalLogLocator) {
- return (logicalLogLocator.getBuffer()).readLong(logicalLogLocator.getMemoryOffset() + BEGIN_PAGE_ID_POS);
- }
-
- public int getLogContentBeginPos(LogicalLogLocator logicalLogLocator) {
- return logicalLogLocator.getMemoryOffset() + logManager.getLogManagerProperties().getLogHeaderSize();
- }
-
- public int getLogContentEndPos(LogicalLogLocator logicalLogLocator) {
- return logicalLogLocator.getMemoryOffset() + getLogLength(logicalLogLocator)
- - logManager.getLogManagerProperties().getLogChecksumSize();
- }
-
- public PhysicalLogLocator getPreviousLsnByTransaction(LogicalLogLocator logicalLogLocator) {
- long prevLsnValue = (logicalLogLocator.getBuffer()).readLong(logicalLogLocator.getMemoryOffset()
- + BEGIN_PREV_LSN_POS);
+ @Override
+ public PhysicalLogLocator getPrevLSN(LogicalLogLocator logicalLogLocator) {
+ long prevLsnValue = (logicalLogLocator.getBuffer())
+ .readLong(logicalLogLocator.getMemoryOffset() + PREV_LSN_POS);
PhysicalLogLocator previousLogLocator = new PhysicalLogLocator(prevLsnValue, logManager);
return previousLogLocator;
}
- public boolean getPreviousLsnByTransaction(PhysicalLogLocator physicalLogLocator,
- LogicalLogLocator logicalLogLocator) {
- long prevLsnValue = (logicalLogLocator.getBuffer()).readLong(logicalLogLocator.getMemoryOffset()
- + BEGIN_PREV_LSN_POS);
+ @Override
+ public boolean getPrevLSN(PhysicalLogLocator physicalLogLocator, LogicalLogLocator logicalLogLocator) {
+ long prevLsnValue = (logicalLogLocator.getBuffer())
+ .readLong(logicalLogLocator.getMemoryOffset() + PREV_LSN_POS);
if (prevLsnValue == -1) {
return false;
}
@@ -100,6 +106,41 @@
return true;
}
+ @Override
+ public long getResourceId(LogicalLogLocator logicalLogLocator) {
+ return logicalLogLocator.getBuffer().readLong(logicalLogLocator.getMemoryOffset() + RESOURCE_ID_POS);
+ }
+
+ @Override
+ public byte getResourceMgrId(LogicalLogLocator logicalLogLocater) {
+ return logicalLogLocater.getBuffer().getByte(logicalLogLocater.getMemoryOffset() + RESOURCE_MGR_ID_POS);
+ }
+
+ @Override
+ public int getLogContentSize(LogicalLogLocator logicalLogLocater) {
+ return logicalLogLocater.getBuffer().readInt(logicalLogLocater.getMemoryOffset() + LOG_RECORD_SIZE_POS);
+ }
+
+ @Override
+ public long getLogChecksum(LogicalLogLocator logicalLogLocator) {
+ return (logicalLogLocator.getBuffer()).readLong(logicalLogLocator.getMemoryOffset()
+ + getLogRecordSize(getLogType(logicalLogLocator), getLogContentSize(logicalLogLocator))
+ - LOG_CHECKSUM_SIZE);
+ }
+
+ @Override
+ public int getLogContentBeginPos(LogicalLogLocator logicalLogLocator) {
+ return logicalLogLocator.getMemoryOffset() + getLogHeaderSize(getLogType(logicalLogLocator));
+ }
+
+ @Override
+ public int getLogContentEndPos(LogicalLogLocator logicalLogLocator) {
+ return logicalLogLocator.getMemoryOffset()
+ + getLogRecordSize(getLogType(logicalLogLocator), getLogContentSize(logicalLogLocator))
+ - LOG_CHECKSUM_SIZE;
+ }
+
+ @Override
public String getLogRecordForDisplay(LogicalLogLocator logicalLogLocator) {
StringBuilder builder = new StringBuilder();
byte logType = new Byte(getLogType(logicalLogLocator));
@@ -112,66 +153,97 @@
logTypeDisplay = "UPDATE";
break;
}
- builder.append(" Log Type :" + logTypeDisplay);
- builder.append(" Log Length :" + getLogLength(logicalLogLocator));
- builder.append(" Log Timestamp:" + getLogTimestamp(logicalLogLocator));
- builder.append(" Log Transaction Id:" + getLogTransactionId(logicalLogLocator));
- builder.append(" Log Resource Mgr Id:" + getResourceMgrId(logicalLogLocator));
- builder.append(" Page Id:" + getPageId(logicalLogLocator));
- builder.append(" Log Checksum:" + getLogChecksum(logicalLogLocator));
- builder.append(" Log Previous lsn: " + getPreviousLsnByTransaction(logicalLogLocator));
- return new String(builder);
- }
-
- public void writeLogHeader(TransactionContext context, LogicalLogLocator logicalLogLocator, byte resourceMgrId,
- long pageId, byte logType, byte logActionType, int logContentSize, long prevLogicalLogLocator) {
- /* magic no */
- (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + BEGIN_MAGIC_NO_POS,
- logManager.getLogManagerProperties().logMagicNumber);
-
- /* length */
- int length = logManager.getLogManagerProperties().getTotalLogRecordLength(logContentSize);
- (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + BEGING_LENGTH_POS, length);
-
- /* log type */
- (logicalLogLocator.getBuffer()).put(logicalLogLocator.getMemoryOffset() + BEGIN_TYPE_POS, logType);
-
- /* log action type */
- (logicalLogLocator.getBuffer()).put(logicalLogLocator.getMemoryOffset() + BEGIN_ACTION_TYPE_POS, logActionType);
-
- /* timestamp */
- long timestamp = System.currentTimeMillis();
- (logicalLogLocator.getBuffer()).writeLong(logicalLogLocator.getMemoryOffset() + BEGIN_TIMESTAMP_POS, timestamp);
-
- /* transaction id */
- (logicalLogLocator.getBuffer()).writeLong(logicalLogLocator.getMemoryOffset() + BEGIN_TRANSACTION_ID_POS,
- context.getTransactionID());
-
- /* resource Mgr id */
- (logicalLogLocator.getBuffer()).put(logicalLogLocator.getMemoryOffset() + BEGIN_RESOURCE_MGR_ID_POS,
- resourceMgrId);
-
- /* page id */
- (logicalLogLocator.getBuffer()).writeLong(logicalLogLocator.getMemoryOffset() + BEGIN_PAGE_ID_POS, pageId);
-
- /* previous LSN's File Id by the transaction */
- (logicalLogLocator.getBuffer()).writeLong(logicalLogLocator.getMemoryOffset() + BEGIN_PREV_LSN_POS,
- prevLogicalLogLocator);
- }
-
- public void writeLogTail(LogicalLogLocator logicalLogLocator, ILogManager logManager) {
- (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset(),
- logManager.getLogManagerProperties().logMagicNumber);
+ builder.append(" LSN : ").append(logicalLogLocator.getLsn());
+ builder.append(" Log Type : ").append(logTypeDisplay);
+ builder.append(" Job Id : ").append(getJobId(logicalLogLocator));
+ builder.append(" Dataset Id : ").append(getDatasetId(logicalLogLocator));
+ builder.append(" PK Hash Value : ").append(getPKHashValue(logicalLogLocator));
+ if (logType == LogType.UPDATE) {
+ builder.append(" PrevLSN : ").append(getPrevLSN(logicalLogLocator).getLsn());
+ builder.append(" Resource Id : ").append(getResourceId(logicalLogLocator));
+ builder.append(" ResourceMgr Id : ").append(getResourceMgrId(logicalLogLocator));
+ builder.append(" Log Record Size : ").append(
+ getLogRecordSize(logType, getLogContentSize(logicalLogLocator)));
+ }
+ return builder.toString();
}
@Override
- public boolean validateLogRecord(LogManagerProperties logManagerProperties, LogicalLogLocator logicalLogLocator) {
- int logLength = this.getLogLength(logicalLogLocator);
+ public void writeLogHeader(LogicalLogLocator logicalLogLocator, byte logType, TransactionContext context,
+ int datasetId, int PKHashValue, long prevLogicalLogLocator, long resourceId, byte resourceMgrId,
+ int logRecordSize) {
+
+ /* magic no */
+ (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + MAGIC_NO_POS,
+ logManager.getLogManagerProperties().logMagicNumber);
+
+ /* log type */
+ (logicalLogLocator.getBuffer()).put(logicalLogLocator.getMemoryOffset() + LOG_TYPE_POS, logType);
+
+ /* jobId */
+ (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + JOB_ID_POS, context.getJobId()
+ .getId());
+
+ /* datasetId */
+ (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + DATASET_ID_POS, datasetId);
+
+ /* PK hash value */
+ (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + PK_HASH_VALUE_POS, PKHashValue);
+
+ if (logType == LogType.UPDATE) {
+ /* prevLSN */
+ (logicalLogLocator.getBuffer()).writeLong(logicalLogLocator.getMemoryOffset() + PREV_LSN_POS,
+ prevLogicalLogLocator);
+
+ /* resourceId */
+ (logicalLogLocator.getBuffer())
+ .writeLong(logicalLogLocator.getMemoryOffset() + RESOURCE_ID_POS, resourceId);
+
+ /* resourceMgr id */
+ (logicalLogLocator.getBuffer()).put(logicalLogLocator.getMemoryOffset() + RESOURCE_MGR_ID_POS,
+ resourceMgrId);
+
+ /* log record size */
+ (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + LOG_RECORD_SIZE_POS,
+ logRecordSize);
+
+ }
+ }
+
+ @Override
+ public boolean validateLogRecord(LogicalLogLocator logicalLogLocator) {
+ int logLength = this.getLogRecordSize(getLogType(logicalLogLocator), getLogContentSize(logicalLogLocator));
long expectedChecksum = DataUtil.getChecksum(logicalLogLocator.getBuffer(),
- logicalLogLocator.getMemoryOffset(), logLength - logManagerProperties.getLogChecksumSize());
- long actualChecksum = logicalLogLocator.getBuffer().readLong(
- logicalLogLocator.getMemoryOffset() + logLength - logManagerProperties.getLogChecksumSize());
+ logicalLogLocator.getMemoryOffset(), logLength - LOG_CHECKSUM_SIZE);
+ long actualChecksum = getLogChecksum(logicalLogLocator);
return expectedChecksum == actualChecksum;
}
+ /**
+ * @param logType
+ * @param logBodySize
+ * @return
+ */
+ @Override
+ public int getLogRecordSize(byte logType, int logBodySize) {
+ if (logType == LogType.UPDATE) {
+ return 46 + logBodySize;
+ } else {
+ return 25;
+ }
+ }
+
+ @Override
+ public int getLogHeaderSize(byte logType) {
+ if (logType == LogType.UPDATE) {
+ return 38;
+ } else {
+ return 17;
+ }
+ }
+
+ @Override
+ public int getLogChecksumSize() {
+ return LOG_CHECKSUM_SIZE;
+ }
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogType.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogType.java
index 72b393d..dd276c6 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogType.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogType.java
@@ -18,8 +18,5 @@
public static final byte UPDATE = 0;
public static final byte COMMIT = 1;
- public static final byte CLR = 2;
- public static final byte BGN_CHPKT = 3;
- public static final byte END_CHPKT = 4;
}
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
deleted file mode 100644
index 3b923dd..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeLogger.java
+++ /dev/null
@@ -1,200 +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.transaction.management.service.logging;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-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.service.transaction.TransactionContext;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
-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;
-
-/**
- * Represents a utility class for generating log records corresponding to
- * operations on a ITreeIndex implementation. A TreeLogger instance is thread
- * safe and can be shared across multiple threads that may belong to same or
- * different transactions.
- */
-class TransactionState {
-
- private final Map<Long, TxnThreadState> transactionThreads = new HashMap<Long, TxnThreadState>();
-
- public synchronized TxnThreadState getTransactionThreadState(long threadId) {
- return transactionThreads.get(threadId);
- }
-
- public synchronized void putTransactionThreadState(long threadId, TxnThreadState txnThreadState) {
- this.transactionThreads.put(threadId, txnThreadState);
- }
-
- public synchronized void remove(long threadId) {
- transactionThreads.remove(threadId);
- }
-}
-
-/**
- * Represents the state of a transaction thread. The state contains information
- * that includes the tuple being operated, the operation and the location of the
- * log record corresponding to the operation.
- */
-class TxnThreadState {
-
- private ITupleReference tuple;
- private IndexOp indexOperation;
- private LogicalLogLocator logicalLogLocator;
-
- public TxnThreadState(LogicalLogLocator logicalLogLocator, IndexOp indexOperation, ITupleReference tupleReference) {
- this.tuple = tupleReference;
- this.indexOperation = indexOperation;
- this.logicalLogLocator = logicalLogLocator;
- }
-
- public synchronized ITupleReference getTuple() {
- return tuple;
- }
-
- public synchronized void setTuple(ITupleReference tuple) {
- this.tuple = tuple;
- }
-
- public synchronized IndexOp getIndexOperation() {
- return indexOperation;
- }
-
- public synchronized void setIndexOperation(IndexOp indexOperation) {
- this.indexOperation = indexOperation;
- }
-
- public synchronized LogicalLogLocator getLogicalLogLocator() {
- return logicalLogLocator;
- }
-
- public synchronized void setLogicalLogLocator(LogicalLogLocator logicalLogLocator) {
- this.logicalLogLocator = logicalLogLocator;
- }
-
-}
-
-public class TreeLogger implements ILogger, ICloseable {
-
- private static final byte resourceMgrId = TreeResourceManager.ID;
- private final Map<Object, Object> arguments = new ConcurrentHashMap<Object, Object>();
-
- public static final String TREE_INDEX = "TREE_INDEX";
- public static final String TUPLE_REFERENCE = "TUPLE_REFERENCE";
- public static final String TUPLE_WRITER = "TUPLE_WRITER";
- public static final String INDEX_OPERATION = "INDEX_OPERATION";
- public static final String RESOURCE_ID = "RESOURCE_ID";
-
- private final ITreeIndex treeIndex;
- private final ITreeIndexTupleWriter treeIndexTupleWriter;
- private final byte[] resourceIdBytes;
- private final byte[] resourceIdLengthBytes;
-
- public class BTreeOperationCodes {
- public static final byte INSERT = 0;
- public static final byte DELETE = 1;
- }
-
- public TreeLogger(byte[] resourceIdBytes, ITreeIndex treeIndex) {
- this.resourceIdBytes = resourceIdBytes;
- this.treeIndex = treeIndex;
- treeIndexTupleWriter = treeIndex.getLeafFrameFactory().getTupleWriterFactory().createTupleWriter();
- this.resourceIdLengthBytes = DataUtil.intToByteArray(resourceIdBytes.length);
- }
-
- public synchronized void close(TransactionContext context) {
- TransactionState txnState = (TransactionState) arguments.get(context.getTransactionID());
- txnState.remove(Thread.currentThread().getId());
- arguments.remove(context.getTransactionID());
- }
-
- public void generateLogRecord(TransactionProvider provider, TransactionContext context, IndexOp 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) {
- throw new ACIDException("Loging for Operation " + operation + " not supported");
-
- }
-
- TxnThreadState txnThreadState = null;
- TransactionState txnState;
- txnState = (TransactionState) arguments.get(context.getTransactionID());
- 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);
- }
- }
- }
-
- txnThreadState = txnState.getTransactionThreadState(Thread.currentThread().getId());
- if (txnThreadState == null) {
- LogicalLogLocator logicalLogLocator = LogUtil.getDummyLogicalLogLocator(provider.getLogManager());
- txnThreadState = new TxnThreadState(logicalLogLocator, operation, tuple);
- txnState.putTransactionThreadState(Thread.currentThread().getId(), txnThreadState);
- }
- txnThreadState.setIndexOperation(operation);
- txnThreadState.setTuple(tuple);
- int tupleSize = treeIndexTupleWriter.bytesRequired(tuple);
- // Below 4 is for the int representing the length of resource id and 1
- // is for
- // the byte representing the operation
- int logContentLength = 4 + resourceIdBytes.length + 1 + tupleSize;
- provider.getLogManager().log(txnThreadState.getLogicalLogLocator(), context, resourceMgrId, 0L, LogType.UPDATE,
- LogActionType.REDO_UNDO, logContentLength, (ILogger) this, arguments);
- }
-
- @Override
- public void log(TransactionContext context, LogicalLogLocator logicalLogLocator, int logRecordSize,
- Map<Object, Object> loggerArguments) throws ACIDException {
- TransactionState txnState = (TransactionState) loggerArguments.get(context.getTransactionID());
- TxnThreadState state = (TxnThreadState) txnState.getTransactionThreadState(Thread.currentThread().getId());
- int count = 0;
- byte[] logBuffer = logicalLogLocator.getBuffer().getArray();
- System.arraycopy(resourceIdLengthBytes, 0, logBuffer, logicalLogLocator.getMemoryOffset(), 4);
- count += 4; // count is incremented by 4 because we wrote the length
- // that is an int and hence 4 bytes
- System.arraycopy(resourceIdBytes, 0, logBuffer, logicalLogLocator.getMemoryOffset() + count,
- resourceIdBytes.length);
- count += resourceIdBytes.length;
- logBuffer[logicalLogLocator.getMemoryOffset() + count] = (byte) state.getIndexOperation().ordinal();
- count += 1; // count is incremented by 1 to account for the byte
- // written.
- treeIndexTupleWriter.writeTuple(state.getTuple(), logicalLogLocator.getBuffer().getArray(),
- logicalLogLocator.getMemoryOffset() + count);
- }
-
- @Override
- public void postLog(TransactionContext context, Map<Object, Object> loggerArguments) throws ACIDException {
- }
-
- @Override
- public void preLog(TransactionContext context, Map<Object, Object> loggerArguments) throws ACIDException {
- }
-
-}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeLoggerRepository.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeLoggerRepository.java
deleted file mode 100644
index 34e1466..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeLoggerRepository.java
+++ /dev/null
@@ -1,44 +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.transaction.management.service.logging;
-
-import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.Map;
-
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
-
-public class TreeLoggerRepository {
-
- private final Map<ByteBuffer, TreeLogger> loggers = new HashMap<ByteBuffer, TreeLogger>();
- private final TransactionProvider provider;
-
- public TreeLoggerRepository(TransactionProvider provider) {
- this.provider = provider;
- }
-
- public synchronized TreeLogger getTreeLogger(byte[] resourceIdBytes) {
- ByteBuffer resourceId = ByteBuffer.wrap(resourceIdBytes);
- TreeLogger logger = loggers.get(resourceId);
- if (logger == null) {
- ITreeIndex treeIndex = (ITreeIndex) provider.getTransactionalResourceRepository().getTransactionalResource(
- resourceIdBytes);
- logger = new TreeLogger(resourceIdBytes, treeIndex);
- loggers.put(resourceId, logger);
- }
- return logger;
- }
-}
\ No newline at end of file
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
deleted file mode 100644
index 7c6d05a..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeResourceManager.java
+++ /dev/null
@@ -1,80 +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.transaction.management.service.logging;
-
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
-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;
-
-public class TreeResourceManager implements IResourceManager {
-
- public static final byte ID = (byte) 1;
-
- private final TransactionProvider provider;
-
- public TreeResourceManager(TransactionProvider provider) {
- this.provider = provider;
- }
-
- public byte getResourceManagerId() {
- return ID;
- }
-
- 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
- int resourceIdLength = DataUtil.byteArrayToInt(logBufferContent, logContentBeginPos);
- byte[] resourceIdBytes = new byte[resourceIdLength];
-
- // copy the resource if bytes
- System.arraycopy(logBufferContent, logContentBeginPos + 4, resourceIdBytes, 0, resourceIdLength);
-
- // look up the repository to obtain the resource object
- ITreeIndex treeIndex = (ITreeIndex) provider.getTransactionalResourceRepository().getTransactionalResource(
- resourceIdBytes);
- int operationOffset = logContentBeginPos + 4 + resourceIdLength;
- int tupleBeginPos = operationOffset + 1;
-
- ITreeIndexTupleReference tupleReference = treeIndex.getLeafFrameFactory().getTupleWriterFactory()
- .createTupleWriter().createTupleReference();
- // TODO: remove this call.
- tupleReference.setFieldCount(tupleReference.getFieldCount());
- tupleReference.resetByTupleOffset(logLocator.getBuffer().getByteBuffer(), tupleBeginPos);
- byte operation = logBufferContent[operationOffset];
- IIndexAccessor treeIndexAccessor = treeIndex.createAccessor();
- try {
- switch (operation) {
- case TreeLogger.BTreeOperationCodes.INSERT:
- treeIndexAccessor.delete(tupleReference);
- break;
- case TreeLogger.BTreeOperationCodes.DELETE:
- treeIndexAccessor.insert(tupleReference);
- break;
- }
- } catch (Exception e) {
- throw new ACIDException(" could not rollback ", e);
- }
- }
-
- public void redo(ILogRecordHelper logRecordHelper, LogicalLogLocator logicalLogLocator) throws ACIDException {
- throw new UnsupportedOperationException(" Redo logic will be implemented as part of crash recovery feature");
- }
-
-}
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..0905907 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
@@ -20,21 +20,26 @@
import java.io.FileReader;
import java.io.FileWriter;
import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
import java.util.HashMap;
+import java.util.Iterator;
import java.util.List;
import java.util.Map;
+import java.util.Map.Entry;
import java.util.logging.Level;
import java.util.logging.Logger;
import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.resource.TransactionalResourceRepository;
import edu.uci.ics.asterix.transaction.management.service.logging.FileUtil;
import edu.uci.ics.asterix.transaction.management.service.logging.IBuffer;
import edu.uci.ics.asterix.transaction.management.service.logging.ILogCursor;
import edu.uci.ics.asterix.transaction.management.service.logging.ILogFilter;
import edu.uci.ics.asterix.transaction.management.service.logging.ILogManager;
import edu.uci.ics.asterix.transaction.management.service.logging.ILogRecordHelper;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogActionType;
+import edu.uci.ics.asterix.transaction.management.service.logging.IndexResourceManager;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogManager;
import edu.uci.ics.asterix.transaction.management.service.logging.LogRecordHelper;
import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
import edu.uci.ics.asterix.transaction.management.service.logging.LogUtil;
@@ -43,7 +48,7 @@
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.asterix.transaction.management.service.transaction.TransactionSubsystem;
/**
* This is the Recovery Manager and is responsible for rolling back a
@@ -53,8 +58,9 @@
*/
public class RecoveryManager implements IRecoveryManager {
+ public static final boolean IS_DEBUG_MODE = false;//true
private static final Logger LOGGER = Logger.getLogger(RecoveryManager.class.getName());
- private TransactionProvider transactionProvider;
+ private TransactionSubsystem txnSubsystem;
/**
* A file at a known location that contains the LSN of the last log record
@@ -65,8 +71,8 @@
private Map<Long, TransactionTableEntry> transactionTable;
private Map<Long, List<PhysicalLogLocator>> dirtyPagesTable;
- public RecoveryManager(TransactionProvider TransactionProvider) throws ACIDException {
- this.transactionProvider = TransactionProvider;
+ public RecoveryManager(TransactionSubsystem TransactionProvider) throws ACIDException {
+ this.txnSubsystem = TransactionProvider;
try {
FileUtil.createFileIfNotExists(checkpoint_record_file);
} catch (IOException ioe) {
@@ -79,14 +85,14 @@
}
private PhysicalLogLocator getBeginRecoveryLSN() throws ACIDException {
- return new PhysicalLogLocator(0, transactionProvider.getLogManager());
+ return new PhysicalLogLocator(0, txnSubsystem.getLogManager());
}
/**
* TODO:This method is currently not implemented completely.
*/
public SystemState startRecovery(boolean synchronous) throws IOException, ACIDException {
- ILogManager logManager = transactionProvider.getLogManager();
+ ILogManager logManager = txnSubsystem.getLogManager();
state = SystemState.RECOVERING;
transactionTable = new HashMap<Long, TransactionTableEntry>();
dirtyPagesTable = new HashMap<Long, List<PhysicalLogLocator>>();
@@ -110,21 +116,16 @@
break;
}
byte resourceMgrId = parser.getResourceMgrId(memLSN);
- IResourceManager resourceMgr = transactionProvider.getTransactionalResourceRepository()
+ IResourceManager resourceMgr = txnSubsystem.getTransactionalResourceRepository()
.getTransactionalResourceMgr(resourceMgrId);
+ //register resourceMgr if it is not registered.
if (resourceMgr == null) {
- throw new ACIDException("unknown resource mgr with id " + resourceMgrId);
- } else {
- byte actionType = parser.getLogActionType(memLSN);
- switch (actionType) {
- case LogActionType.REDO:
- resourceMgr.redo(parser, memLSN);
- break;
- case LogActionType.UNDO: /* skip these records */
- break;
- default: /* do nothing */
- }
+ resourceMgr = new IndexResourceManager(resourceMgrId, txnSubsystem);
+ txnSubsystem.getTransactionalResourceRepository().registerTransactionalResourceManager(
+ resourceMgrId, resourceMgr);
}
+ resourceMgr.redo(parser, memLSN);
+
writeCheckpointRecord(memLSN.getLsn());
}
state = SystemState.HEALTHY;
@@ -164,97 +165,222 @@
}
/**
- * Rollback a transaction (non-Javadoc)
+ * Rollback a transaction
*
- * @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 {
- ILogManager logManager = transactionProvider.getLogManager();
- ILogRecordHelper parser = logManager.getLogRecordHelper();
+ ILogManager logManager = txnSubsystem.getLogManager();
+ ILogRecordHelper logRecordHelper = logManager.getLogRecordHelper();
+ Map<TxnId, List<Long>> loserTxnTable = new HashMap<TxnId, List<Long>>();
+ TxnId tempKeyTxnId = new TxnId(-1, -1, -1);
- // Obtain the last log record written by the transaction
- PhysicalLogLocator lsn = txnContext.getLastLogLocator();
+ int updateLogCount = 0;
+ int commitLogCount = 0;
+
+ // Obtain the first log record written by the Job
+ PhysicalLogLocator firstLSNLogLocator = txnContext.getFirstLogLocator();
+ PhysicalLogLocator lastLSNLogLocator = txnContext.getLastLogLocator();
if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(" rollbacking transaction log records at lsn " + lsn.getLsn());
+ LOGGER.info(" rollbacking transaction log records from " + firstLSNLogLocator.getLsn() + " to "
+ + lastLSNLogLocator.getLsn());
}
// check if the transaction actually wrote some logs.
- if (lsn.getLsn() == TransactionManagementConstants.LogManagerConstants.TERMINAL_LSN) {
+ if (firstLSNLogLocator.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;
}
- // a dummy logLocator instance that is re-used during rollback
- LogicalLogLocator logLocator = LogUtil.getDummyLogicalLogLocator(logManager);
+ // While reading log records from firstLSN to lastLSN, collect uncommitted txn's LSNs
+ ILogCursor logCursor;
+ try {
+ logCursor = logManager.readLog(firstLSNLogLocator, new ILogFilter() {
+ @Override
+ public boolean accept(IBuffer buffer, long startOffset, int length) {
+ return true;
+ }
+ });
+ } catch (IOException e) {
+ throw new ACIDException("Failed to create LogCursor with LSN:" + firstLSNLogLocator.getLsn(), e);
+ }
- while (true) {
+ LogicalLogLocator currentLogLocator = LogUtil.getDummyLogicalLogLocator(logManager);
+ boolean valid;
+ byte logType;
+ List<Long> undoLSNSet = null;
+
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(" collecting loser transaction's LSNs from " + firstLSNLogLocator.getLsn() + " to "
+ + +lastLSNLogLocator.getLsn());
+ }
+
+ while (currentLogLocator.getLsn() != lastLSNLogLocator.getLsn()) {
try {
- // read the log record at the given position
- logLocator = logManager.readLog(lsn);
- } catch (Exception e) {
- e.printStackTrace();
- state = SystemState.CORRUPTED;
- throw new ACIDException(" could not read log at lsn :" + lsn, e);
+ valid = logCursor.next(currentLogLocator);
+ } catch (IOException e) {
+ throw new ACIDException("Failed to read log at LSN:" + currentLogLocator.getLsn(), e);
+ }
+ if (!valid) {
+ if (currentLogLocator.getLsn() != lastLSNLogLocator.getLsn()) {
+ throw new ACIDException("Log File Corruption: lastLSN mismatch");
+ } else {
+ break;//End of Log File
+ }
}
- byte logType = parser.getLogType(logLocator);
- if (LOGGER.isLoggable(Level.FINE)) {
- LOGGER.fine(" reading LSN value inside rollback transaction method " + txnContext.getLastLogLocator()
- + " txn id " + parser.getLogTransactionId(logLocator) + " log type " + logType);
+ if (LogManager.IS_DEBUG_MODE) {
+ System.out.println(logManager.getLogRecordHelper().getLogRecordForDisplay(currentLogLocator));
}
+ tempKeyTxnId.setTxnId(logRecordHelper.getJobId(currentLogLocator),
+ logRecordHelper.getDatasetId(currentLogLocator), logRecordHelper.getPKHashValue(currentLogLocator));
+ logType = logRecordHelper.getLogType(currentLogLocator);
+
switch (logType) {
case LogType.UPDATE:
-
- // extract the resource manager id from the log record.
- byte resourceMgrId = parser.getResourceMgrId(logLocator);
- if (LOGGER.isLoggable(Level.FINE)) {
- LOGGER.fine(parser.getLogRecordForDisplay(logLocator));
+ undoLSNSet = loserTxnTable.get(tempKeyTxnId);
+ if (undoLSNSet == null) {
+ TxnId txnId = new TxnId(logRecordHelper.getJobId(currentLogLocator),
+ logRecordHelper.getDatasetId(currentLogLocator),
+ logRecordHelper.getPKHashValue(currentLogLocator));
+ undoLSNSet = new ArrayList<Long>();
+ loserTxnTable.put(txnId, undoLSNSet);
}
-
- // look up the repository to get the resource manager
- IResourceManager resourceMgr = transactionProvider.getTransactionalResourceRepository()
- .getTransactionalResourceMgr(resourceMgrId);
- if (resourceMgr == null) {
- throw new ACIDException(txnContext, " unknown resource manager " + resourceMgrId);
- } else {
- byte actionType = parser.getLogActionType(logLocator);
- switch (actionType) {
- case LogActionType.REDO: // no need to do anything
- break;
- case LogActionType.UNDO: // undo the log record
- resourceMgr.undo(parser, logLocator);
- break;
- case LogActionType.REDO_UNDO: // undo log record
- resourceMgr.undo(parser, logLocator);
- break;
- default:
- }
+ undoLSNSet.add(currentLogLocator.getLsn());
+ if (IS_DEBUG_MODE) {
+ updateLogCount++;
+ System.out.println("" + Thread.currentThread().getId() + "======> update["
+ + currentLogLocator.getLsn() + "]:" + tempKeyTxnId);
}
- case LogType.CLR: // skip the CLRs as they are not undone
break;
+
case LogType.COMMIT:
- throw new ACIDException(txnContext, " cannot rollback commmitted transaction");
+ undoLSNSet = loserTxnTable.get(tempKeyTxnId);
+ if (undoLSNSet != null) {
+ loserTxnTable.remove(tempKeyTxnId);
+ }
+ if (IS_DEBUG_MODE) {
+ commitLogCount++;
+ System.out.println("" + Thread.currentThread().getId() + "======> commit["
+ + currentLogLocator.getLsn() + "]" + tempKeyTxnId);
+ }
+ break;
- }
-
- // follow the previous LSN pointer to get the previous log record
- // written by the transaction
- // If the return value is true, the logLocator, it indicates that
- // the logLocator object has been
- // appropriately set to the location of the next log record to be
- // processed as part of the roll back
- boolean moreLogs = parser.getPreviousLsnByTransaction(lsn, logLocator);
- if (!moreLogs) {
- // no more logs to process
- break;
+ default:
+ throw new ACIDException("Unsupported LogType: " + logType);
}
}
+ //undo loserTxn's effect
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(" undoing loser transaction's effect");
+ }
+
+ TxnId txnId = null;
+ Iterator<Entry<TxnId, List<Long>>> iter = loserTxnTable.entrySet().iterator();
+ byte resourceMgrId;
+ int undoCount = 0;
+ while (iter.hasNext()) {
+
+ Map.Entry<TxnId, List<Long>> loserTxn = (Map.Entry<TxnId, List<Long>>) iter.next();
+ txnId = loserTxn.getKey();
+
+ undoLSNSet = loserTxn.getValue();
+ Comparator<Long> comparator = Collections.reverseOrder();
+ Collections.sort(undoLSNSet, comparator);
+
+ for (long undoLSN : undoLSNSet) {
+ // here, all the log records are UPDATE type. So, we don't need to check the type again.
+
+ //read the corresponding log record to be undone.
+ logManager.readLog(undoLSN, currentLogLocator);
+
+ if (LogManager.IS_DEBUG_MODE) {
+ System.out.println(logManager.getLogRecordHelper().getLogRecordForDisplay(currentLogLocator));
+ }
+
+ // extract the resource manager id from the log record.
+ resourceMgrId = logRecordHelper.getResourceMgrId(currentLogLocator);
+ if (LOGGER.isLoggable(Level.FINE)) {
+ LOGGER.fine(logRecordHelper.getLogRecordForDisplay(currentLogLocator));
+ }
+
+ // look up the repository to get the resource manager
+ IResourceManager resourceMgr = txnSubsystem.getTransactionalResourceRepository()
+ .getTransactionalResourceMgr(resourceMgrId);
+
+ // register resourceMgr if it is not registered.
+ if (resourceMgr == null) {
+ resourceMgr = new IndexResourceManager(resourceMgrId, txnSubsystem);
+ txnSubsystem.getTransactionalResourceRepository().registerTransactionalResourceManager(
+ resourceMgrId, resourceMgr);
+ }
+ resourceMgr.undo(logRecordHelper, currentLogLocator);
+
+ if (IS_DEBUG_MODE) {
+ undoCount++;
+ }
+ }
+ }
+
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(" undone loser transaction's effect");
+ }
+ if (IS_DEBUG_MODE) {
+ System.out.println("UpdateLogCount/CommitLogCount/UndoCount:" + updateLogCount + "/" + commitLogCount + "/"
+ + undoCount);
+ }
+ }
+}
+
+class TxnId {
+ public int jobId;
+ public int datasetId;
+ public int pkHashVal;
+
+ public TxnId(int jobId, int datasetId, int pkHashVal) {
+ this.jobId = jobId;
+ this.datasetId = datasetId;
+ this.pkHashVal = pkHashVal;
}
+ public void setTxnId(int jobId, int datasetId, int pkHashVal) {
+ this.jobId = jobId;
+ this.datasetId = datasetId;
+ this.pkHashVal = pkHashVal;
+ }
+
+ public void setTxnId(TxnId txnId) {
+ this.jobId = txnId.jobId;
+ this.datasetId = txnId.datasetId;
+ this.pkHashVal = txnId.pkHashVal;
+ }
+
+ @Override
+ public String toString() {
+ return "[" + jobId + "," + datasetId + "," + pkHashVal + "]";
+ }
+
+ @Override
+ public int hashCode() {
+ return pkHashVal;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (o == this) {
+ return true;
+ }
+ if (!(o instanceof TxnId)) {
+ return false;
+ }
+ TxnId txnId = (TxnId) o;
+
+ return (txnId.pkHashVal == pkHashVal && txnId.datasetId == datasetId && txnId.jobId == jobId);
+ }
}
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/FieldsHashValueGenerator.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/FieldsHashValueGenerator.java
new file mode 100644
index 0000000..d4f15d9
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/FieldsHashValueGenerator.java
@@ -0,0 +1,36 @@
+/*
+ * 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.transaction;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public class FieldsHashValueGenerator {
+ public static int computeFieldsHashValue(ITupleReference tuple, int[] fieldIndexes,
+ IBinaryHashFunction[] fieldHashFunctions) {
+ int h = 0;
+ for (int i = 0; i < fieldIndexes.length; i++) {
+ int primaryKeyFieldIdx = fieldIndexes[i];
+ int fh = fieldHashFunctions[i].hash(tuple.getFieldData(primaryKeyFieldIdx),
+ tuple.getFieldStart(primaryKeyFieldIdx), tuple.getFieldLength(primaryKeyFieldIdx));
+ h = h * 31 + fh;
+ if (h < 0) {
+ h = h*(-1);
+ }
+ }
+ return h;
+ }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/IResourceManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/IResourceManager.java
index 200527f..f7715e8 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/IResourceManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/IResourceManager.java
@@ -22,6 +22,12 @@
* Provides APIs for undo or redo of an operation on a resource.
*/
public interface IResourceManager {
+
+ public class ResourceType {
+ public static final byte LSM_BTREE = 1;
+ public static final byte LSM_RTREE = 2;
+ public static final byte LSM_INVERTED_INDEX = 3;
+ }
/**
* Returns the unique identifier for the resource manager.
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..3f55ac9 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,46 +40,50 @@
* 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.
*
* @param txnContext
* the transaction context associated with the transaction
+ * @param datasetId TODO
+ * @param PKHashVal TODO
* @throws ACIDException
- * @see transactionContext
+ * @see transactionContextimport edu.uci.ics.hyracks.api.job.JobId;
* @see ACIDException
*/
- public void commitTransaction(TransactionContext txnContext) throws ACIDException;
+ public void commitTransaction(TransactionContext txnContext, DatasetId datasetId, int PKHashVal) throws ACIDException;
/**
* Aborts a transaction.
*
* @param txnContext
* the transaction context associated with the transaction
+ * @param datasetId TODO
+ * @param PKHashVal TODO
* @throws ACIDException
* @see transactionContext
* @see ACIDException
*/
- public void abortTransaction(TransactionContext txnContext) throws ACIDException;
+ public void abortTransaction(TransactionContext txnContext, DatasetId datasetId, int PKHashVal) throws ACIDException;
/**
* Indicates end of all activity for a transaction. In other words, all
@@ -88,36 +92,23 @@
*
* @param txnContext
* the transaction context associated with the transaction
+ * @param datasetId TODO
+ * @param PKHashVal TODO
* @param success
* indicates the success or failure. The transaction is committed
* or aborted accordingly.
* @throws ACIDException
*/
- 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;
+ public void completedTransaction(TransactionContext txnContext, DatasetId datasetId, int PKHashVal, boolean success) 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.
*
- * @see TransactionProvider
+ * @see TransactionSubsystem
* @return TransactionProvider
*/
- public TransactionProvider getTransactionProvider();
+ public TransactionSubsystem getTransactionProvider();
}
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/ITransactionSubsystemProvider.java
similarity index 70%
copy from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionIDFactory.java
copy to asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/ITransactionSubsystemProvider.java
index e5475f4..857d8ae 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/ITransactionSubsystemProvider.java
@@ -12,17 +12,13 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
+
package edu.uci.ics.asterix.transaction.management.service.transaction;
-import java.util.concurrent.atomic.AtomicLong;
+import java.io.Serializable;
-/**
- * Represents a factory to generate unique transaction IDs.
- */
-public class TransactionIDFactory {
- private static final AtomicLong ID = new AtomicLong();
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
- public static long generateTransactionId() {
- return ID.incrementAndGet();
- }
-}
\ No newline at end of file
+public interface ITransactionSubsystemProvider extends Serializable{
+ public TransactionSubsystem getTransactionSubsystem(IHyracksTaskContext ctx);
+}
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/MutableResourceId.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/MutableResourceId.java
new file mode 100644
index 0000000..5552930
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/MutableResourceId.java
@@ -0,0 +1,30 @@
+package edu.uci.ics.asterix.transaction.management.service.transaction;
+
+public class MutableResourceId{
+ long id;
+
+ public MutableResourceId(long id) {
+ this.id = id;
+ }
+
+ public void setId(long id) {
+ this.id = id;
+ }
+
+ public long getId() {
+ return id;
+ }
+
+ @Override
+ public int hashCode() {
+ return (int)id;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if ((o == null) || !(o instanceof MutableResourceId)) {
+ return false;
+ }
+ return ((MutableResourceId) o).id == this.id;
+ }
+}
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..a88ed5f 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
@@ -15,14 +15,21 @@
package edu.uci.ics.asterix.transaction.management.service.transaction;
import java.io.Serializable;
+import java.util.ArrayList;
import java.util.HashSet;
+import java.util.List;
import java.util.Set;
import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.AbstractOperationCallback;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
import edu.uci.ics.asterix.transaction.management.resource.ICloseable;
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.ITransactionManager.TransactionState;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
/**
* Represents a holder object that contains all information related to a
@@ -36,7 +43,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,
@@ -44,14 +51,61 @@
}
private static final long serialVersionUID = -6105616785783310111L;
- private TransactionProvider transactionProvider;
- private long transactionID;
- private LogicalLogLocator lastLogLocator;
- private TransactionState txnState;
+ private TransactionSubsystem transactionSubsystem;
+ private LogicalLogLocator firstLogLocator;//firstLSN of the Job
+ private LogicalLogLocator lastLogLocator;//lastLSN of the Job
+ private TransactionState txnState;
private long startWaitTime;
private int status;
private Set<ICloseable> resources = new HashSet<ICloseable>();
private TransactionType transactionType = TransactionType.READ;
+ private JobId jobId;
+
+ // List of indexes on which operations were performed on behalf of this transaction.
+ private final List<ILSMIndex> indexes = new ArrayList<ILSMIndex>();
+
+ // List of operation callbacks corresponding to the operand indexes. In particular, needed to track
+ // the number of active operations contributed by this transaction.
+ private final List<AbstractOperationCallback> callbacks = new ArrayList<AbstractOperationCallback>();
+
+ public TransactionContext(JobId jobId, TransactionSubsystem transactionSubsystem) throws ACIDException {
+ this.jobId = jobId;
+ this.transactionSubsystem = transactionSubsystem;
+ init();
+ }
+
+ private void init() throws ACIDException {
+ firstLogLocator = LogUtil.getDummyLogicalLogLocator(transactionSubsystem.getLogManager());
+ lastLogLocator = LogUtil.getDummyLogicalLogLocator(transactionSubsystem.getLogManager());
+ txnState = TransactionState.ACTIVE;
+ startWaitTime = INVALID_TIME;
+ status = ACTIVE_STATUS;
+ }
+
+ public void registerIndexAndCallback(ILSMIndex index, AbstractOperationCallback callback) {
+ synchronized (indexes) {
+ indexes.add(index);
+ callbacks.add(callback);
+ }
+ }
+
+ public void setLastLSNToIndexes(long lastLSN) {
+ synchronized (indexes) {
+ for (ILSMIndex index : indexes) {
+ ((IndexOperationTracker) index.getOperationTracker()).setLastLSN(lastLSN);
+ }
+ }
+ }
+
+ public void decreaseActiveTransactionCountOnIndexes() throws HyracksDataException {
+ synchronized (indexes) {
+ for (int i = 0; i < indexes.size(); i++) {
+ ILSMIndex index = indexes.get(i);
+ IModificationOperationCallback modificationCallback = (IModificationOperationCallback) callbacks.get(i);
+ ((IndexOperationTracker) index.getOperationTracker()).completeOperation(null, modificationCallback);
+ }
+ }
+ }
public void setTransactionType(TransactionType transactionType) {
this.transactionType = transactionType;
@@ -65,29 +119,23 @@
resources.add(resource);
}
- public TransactionContext(long transactionId, TransactionProvider transactionProvider) throws ACIDException {
- this.transactionID = transactionId;
- this.transactionProvider = transactionProvider;
- init();
+ public LogicalLogLocator getFirstLogLocator() {
+ return firstLogLocator;
}
-
- private void init() throws ACIDException {
- lastLogLocator = LogUtil.getDummyLogicalLogLocator(transactionProvider.getLogManager());
- txnState = TransactionState.ACTIVE;
- startWaitTime = INVALID_TIME;
- status = ACTIVE_STATUS;
- }
-
+
public LogicalLogLocator getLastLogLocator() {
return lastLogLocator;
}
- public void setLastLSN(LogicalLogLocator lastLogLocator) {
- this.lastLogLocator = lastLogLocator;
+ public void setLastLSN(long lsn) {
+ if (firstLogLocator.getLsn() == -1) {
+ firstLogLocator.setLsn(lsn);
+ }
+ lastLogLocator.setLsn(lsn);
}
- 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..bafa753 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
@@ -20,8 +20,8 @@
import java.util.logging.Logger;
import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogActionType;
import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
/**
* An implementation of the @see ITransactionManager interface that provides
@@ -29,15 +29,16 @@
*/
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 final TransactionSubsystem transactionProvider;
+ private Map<JobId, TransactionContext> transactionContextRepository = new HashMap<JobId, TransactionContext>();
- public TransactionManager(TransactionProvider provider) {
+ public TransactionManager(TransactionSubsystem provider) {
this.transactionProvider = provider;
}
@Override
- public void abortTransaction(TransactionContext txnContext) throws ACIDException {
+ public void abortTransaction(TransactionContext txnContext, DatasetId datasetId, int PKHashVal)
+ throws ACIDException {
synchronized (txnContext) {
if (txnContext.getTxnState().equals(TransactionState.ABORTED)) {
return;
@@ -50,88 +51,95 @@
if (LOGGER.isLoggable(Level.SEVERE)) {
LOGGER.severe(msg);
}
+ ae.printStackTrace();
throw new Error(msg);
} 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;
}
}
@Override
- public void commitTransaction(TransactionContext txnContext) throws ACIDException {
+ public void commitTransaction(TransactionContext txnContext, DatasetId datasetId, int PKHashVal)
+ throws ACIDException {
synchronized (txnContext) {
if ((txnContext.getTxnState().equals(TransactionState.COMMITTED))) {
return;
}
+ //There is either job-level commit or entity-level commit.
+ //The job-level commit will have -1 value both for datasetId and PKHashVal.
+
+ //for entity-level commit
+ if (PKHashVal != -1) {
+ transactionProvider.getLockManager().unlock(datasetId, PKHashVal, txnContext, true);
+ try {
+ //decrease the transaction reference count on index
+ txnContext.decreaseActiveTransactionCountOnIndexes();
+ } catch (HyracksDataException e) {
+ throw new ACIDException("failed to complete index operation", e);
+ }
+ return;
+ }
+
+ //for job-level commit
try {
- if (txnContext.getTransactionType().equals(TransactionContext.TransactionType.READ_WRITE)) { // conditionally
- // write
- // commit
- // log
- // record
- transactionProvider.getLogManager().log(txnContext.getLastLogLocator(), txnContext, (byte) (-1), 0,
- LogType.COMMIT, LogActionType.NO_OP, 0, null, null);
+ if (txnContext.getTransactionType().equals(TransactionContext.TransactionType.READ_WRITE)) {
+ transactionProvider.getLogManager().log(LogType.COMMIT, txnContext, -1, -1, -1, (byte) 0, 0, null,
+ null, txnContext.getLastLogLocator());
}
} 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 {
+ public void completedTransaction(TransactionContext txnContext, DatasetId datasetId, int PKHashVal, boolean success)
+ throws ACIDException {
if (!success) {
- abortTransaction(txnContext);
+ abortTransaction(txnContext, datasetId, PKHashVal);
} else {
- commitTransaction(txnContext);
+ commitTransaction(txnContext, datasetId, PKHashVal);
}
}
@Override
- public TransactionProvider getTransactionProvider() {
+ public TransactionSubsystem getTransactionProvider() {
return transactionProvider;
}
-
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionProvider.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionSubsystem.java
similarity index 89%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionProvider.java
rename to asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionSubsystem.java
index ef843f4..88082b0 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionProvider.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionSubsystem.java
@@ -20,7 +20,7 @@
import edu.uci.ics.asterix.transaction.management.service.locking.LockManager;
import edu.uci.ics.asterix.transaction.management.service.logging.ILogManager;
import edu.uci.ics.asterix.transaction.management.service.logging.LogManager;
-import edu.uci.ics.asterix.transaction.management.service.logging.TreeLoggerRepository;
+import edu.uci.ics.asterix.transaction.management.service.logging.IndexLoggerRepository;
import edu.uci.ics.asterix.transaction.management.service.recovery.IRecoveryManager;
import edu.uci.ics.asterix.transaction.management.service.recovery.RecoveryManager;
@@ -28,22 +28,22 @@
* Provider for all the sub-systems (transaction/lock/log/recovery) managers.
* Users of transaction sub-systems must obtain them from the provider.
*/
-public class TransactionProvider {
+public class TransactionSubsystem {
private final String id;
private final ILogManager logManager;
private final ILockManager lockManager;
private final ITransactionManager transactionManager;
private final IRecoveryManager recoveryManager;
private final TransactionalResourceRepository resourceRepository;
- private final TreeLoggerRepository loggerRepository;
+ private final IndexLoggerRepository loggerRepository;
- public TransactionProvider(String id) throws ACIDException {
+ public TransactionSubsystem(String id) throws ACIDException {
this.id = id;
this.transactionManager = new TransactionManager(this);
this.logManager = new LogManager(this);
this.lockManager = new LockManager(this);
this.recoveryManager = new RecoveryManager(this);
- this.loggerRepository = new TreeLoggerRepository(this);
+ this.loggerRepository = new IndexLoggerRepository(this);
this.resourceRepository = new TransactionalResourceRepository();
}
@@ -67,7 +67,7 @@
return resourceRepository;
}
- public TreeLoggerRepository getTreeLoggerRepository() {
+ public IndexLoggerRepository getTreeLoggerRepository() {
return loggerRepository;
}
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/BasicLogger.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/BasicLogger.java
index 3e95d6a..c9d01a0 100644
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/BasicLogger.java
+++ b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/BasicLogger.java
@@ -14,12 +14,12 @@
*/
package edu.uci.ics.asterix.transaction.management.logging;
-import java.util.Map;
import java.util.Random;
import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
import edu.uci.ics.asterix.transaction.management.service.logging.IBuffer;
import edu.uci.ics.asterix.transaction.management.service.logging.ILogger;
+import edu.uci.ics.asterix.transaction.management.service.logging.IndexLogger.ReusableLogContentObject;
import edu.uci.ics.asterix.transaction.management.service.logging.LogicalLogLocator;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
@@ -29,7 +29,7 @@
private static long count = 0;
public void log(TransactionContext context, LogicalLogLocator wMemLSN, int length,
- Map<Object, Object> loggerArguments) throws ACIDException {
+ ReusableLogContentObject reusableLogContentObject) throws ACIDException {
byte[] logContent = getRandomBytes(length);
try {
@@ -66,12 +66,12 @@
return averageContentCreationTime;
}
- public void postLog(TransactionContext context, Map<Object, Object> loggerArguments) throws ACIDException {
+ public void postLog(TransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException {
// TODO Auto-generated method stub
}
- public void preLog(TransactionContext context, Map<Object, Object> loggerArguments) throws ACIDException {
+ public void preLog(TransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException {
// TODO Auto-generated method stub
}
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/RecoverySimulator.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/RecoverySimulator.java
index 3a288a2..c086954 100644
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/RecoverySimulator.java
+++ b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/RecoverySimulator.java
@@ -18,7 +18,7 @@
import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
import edu.uci.ics.asterix.transaction.management.service.recovery.IRecoveryManager;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
public class RecoverySimulator {
@@ -31,7 +31,7 @@
public static void main(String args[]) throws IOException, ACIDException {
String id = "nc1";
try {
- TransactionProvider factory = new TransactionProvider(id);
+ TransactionSubsystem factory = new TransactionSubsystem(id);
IRecoveryManager recoveryManager = factory.getRecoveryManager();
recoveryManager.startRecovery(true);
} catch (ACIDException acide) {
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..5d68911 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,16 +27,18 @@
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.IResourceManager.ResourceType;
+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;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
public class TransactionWorkloadSimulator {
public static ILogManager logManager;
public static ILockManager lockManager;
- TransactionProvider provider;
+ TransactionSubsystem provider;
public static WorkloadProperties workload;
Transaction[] transactions;
@@ -48,7 +49,7 @@
}
public void beginWorkload() throws ACIDException {
- provider = new TransactionProvider("nc1");
+ provider = new TransactionSubsystem("nc1");
logManager = provider.getLogManager();
lockManager = provider.getLockManager();
provider.getTransactionalResourceRepository().registerTransactionalResourceManager(DummyResourceMgr.id,
@@ -67,7 +68,7 @@
}
for (int i = 0; i < workload.numActiveThreads; i++) {
- provider.getTransactionManager().commitTransaction(transactions[i].getContext());
+ provider.getTransactionManager().commitTransaction(transactions[i].getContext(), new DatasetId(-1), -1);
}
long endTime = System.nanoTime();
@@ -97,9 +98,9 @@
class SingleTransactionContextFactory {
private static TransactionContext context;
- public static TransactionContext getContext(TransactionProvider provider) throws ACIDException {
+ public static TransactionContext getContext(TransactionSubsystem provider) throws ACIDException {
if (context == null) {
- context = new TransactionContext(TransactionIDFactory.generateTransactionId(), provider);
+ context = new TransactionContext(JobIdFactory.generateJobId(), provider);
}
return context;
}
@@ -107,8 +108,8 @@
class MultipleTransactionContextFactory {
- public static TransactionContext getContext(TransactionProvider provider) throws ACIDException {
- return new TransactionContext(TransactionIDFactory.generateTransactionId(), provider);
+ public static TransactionContext getContext(TransactionSubsystem provider) throws ACIDException {
+ return new TransactionContext(JobIdFactory.generateJobId(), provider);
}
}
@@ -121,12 +122,14 @@
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 TransactionSubsystem transactionProvider;
private ILogManager logManager;
+ private DatasetId tempDatasetId = new DatasetId(-1);
- public Transaction(TransactionProvider provider, String name, boolean singleTransaction) throws ACIDException {
+ public Transaction(TransactionSubsystem provider, String name, boolean singleTransaction) throws ACIDException {
this.name = name;
this.transactionProvider = provider;
if (singleTransaction) {
@@ -157,12 +160,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,21 +177,18 @@
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;
- }
- TransactionWorkloadSimulator.logManager.log(memLSN, context, ResourceMgrInfo.BTreeResourceMgrId,
- pageId, logType, logActionType, logSize, logger, null);
+ tempDatasetId.setId(resourceID);
+ TransactionWorkloadSimulator.lockManager.lock(tempDatasetId, -1, lockMode, context);
+ TransactionWorkloadSimulator.logManager.log(logType, context, resourceID,
+ -1, resourceID, ResourceType.LSM_BTREE, logSize, null, logger, memLSN);
retry = false;
Thread.currentThread().sleep(TransactionWorkloadSimulator.workload.thinkTime);
logCount.incrementAndGet();
logByteCount.addAndGet(logSize
- + TransactionWorkloadSimulator.logManager.getLogManagerProperties().getLogHeaderSize()
- + TransactionWorkloadSimulator.logManager.getLogManagerProperties().getLogChecksumSize());
+ + TransactionWorkloadSimulator.logManager.getLogRecordHelper().getLogHeaderSize(logType)
+ + TransactionWorkloadSimulator.logManager.getLogRecordHelper().getLogChecksumSize());
myLogCount++;
}
} catch (ACIDException acide) {
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileLogger.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileLogger.java
index 16c5d6b..54bb0b8 100644
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileLogger.java
+++ b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileLogger.java
@@ -14,11 +14,11 @@
*/
package edu.uci.ics.asterix.transaction.management.test;
-import java.util.Map;
import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
import edu.uci.ics.asterix.transaction.management.logging.IResource;
import edu.uci.ics.asterix.transaction.management.service.logging.ILogger;
+import edu.uci.ics.asterix.transaction.management.service.logging.IndexLogger.ReusableLogContentObject;
import edu.uci.ics.asterix.transaction.management.service.logging.LogicalLogLocator;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
@@ -39,14 +39,14 @@
}
@Override
- public void preLog(TransactionContext context, Map<Object, Object> loggerArguments) throws ACIDException {
+ public void preLog(TransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException {
// TODO Auto-generated method stub
}
@Override
- public void log(TransactionContext context, final LogicalLogLocator memLSN, int logRecordSize,
- Map<Object, Object> loggerArguments) throws ACIDException {
+ public void log(TransactionContext context, final LogicalLogLocator memLSN, int logContentSize,
+ ReusableLogContentObject reusableLogContentObject) throws ACIDException {
byte[] buffer = memLSN.getBuffer().getArray();
byte[] content = logRecordContent.getBytes();
for (int i = 0; i < resource.getId().length; i++) {
@@ -58,7 +58,7 @@
}
@Override
- public void postLog(TransactionContext context, Map<Object, Object> loggerArguments) throws ACIDException {
+ public void postLog(TransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException {
// TODO Auto-generated method stub
}
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/LogRecordReader.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/LogRecordReader.java
index 899dbde..eb22dd7 100644
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/LogRecordReader.java
+++ b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/LogRecordReader.java
@@ -28,13 +28,13 @@
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.logging.PhysicalLogLocator;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
public class LogRecordReader {
ILogManager logManager;
- public LogRecordReader(TransactionProvider factory) throws ACIDException {
+ public LogRecordReader(TransactionSubsystem factory) throws ACIDException {
logManager = factory.getLogManager();
}
@@ -64,7 +64,8 @@
}
public void readLogRecord(long lsnValue) throws IOException, ACIDException {
- LogicalLogLocator memLSN = logManager.readLog(new PhysicalLogLocator(lsnValue, logManager));
+ LogicalLogLocator memLSN = null;
+ logManager.readLog(lsnValue, memLSN);
System.out.println(logManager.getLogRecordHelper().getLogRecordForDisplay(memLSN));
}
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..98585b5 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
@@ -19,21 +19,22 @@
import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
import edu.uci.ics.asterix.transaction.management.logging.IResource;
-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.ILogger;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogActionType;
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.recovery.IRecoveryManager;
import edu.uci.ics.asterix.transaction.management.service.recovery.IRecoveryManager.SystemState;
+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.IResourceManager.ResourceType;
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.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;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
public class TransactionSimulator {
@@ -45,11 +46,11 @@
private ILogger logger;
private IResource resource;
private LogicalLogLocator memLSN;
- private TransactionProvider transactionProvider;
+ private TransactionSubsystem transactionProvider;
public TransactionSimulator(IResource resource, IResourceManager resourceMgr) throws ACIDException {
String id = "nc1";
- transactionProvider = new TransactionProvider(id);
+ transactionProvider = new TransactionSubsystem(id);
transactionManager = transactionProvider.getTransactionManager();
logManager = transactionProvider.getLogManager();
lockManager = transactionProvider.getLockManager();
@@ -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)
@@ -77,15 +78,13 @@
case INCREMENT:
finalValue = currentValue + 1;
int logRecordLength = ((FileLogger) logger).generateLogRecordContent(currentValue, finalValue);
- logManager.log(memLSN, txnContext, FileResourceManager.id, 0, LogType.UPDATE, LogActionType.REDO_UNDO,
- logRecordLength, logger, null);
+ logManager.log(LogType.UPDATE, txnContext, 1, -1, 1, ResourceType.LSM_BTREE, 0, null, logger, memLSN);
((FileResource) resource).increment();
break;
case DECREMENT:
finalValue = currentValue - 1;
logRecordLength = ((FileLogger) logger).generateLogRecordContent(currentValue, finalValue);
- logManager.log(memLSN, txnContext, FileResourceManager.id, 0, LogType.UPDATE, LogActionType.REDO_UNDO,
- logRecordLength, logger, null);
+ logManager.log(LogType.UPDATE, txnContext, 1, -1, 1, ResourceType.LSM_BTREE, 0, null, logger, memLSN);
((FileResource) resource).decrement();
break;
}
@@ -93,7 +92,7 @@
}
public void commitTransaction(TransactionContext context) throws ACIDException {
- transactionManager.commitTransaction(context);
+ transactionManager.commitTransaction(context, new DatasetId(-1), -1);
}
public SystemState recover() throws ACIDException, IOException {
diff --git a/pom.xml b/pom.xml
index 506d57d..62ed003 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>