Merge external indexes into master
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntime.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntime.java
index 269526b..6dd11bd 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntime.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntime.java
@@ -19,11 +19,11 @@
import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.DatasetId;
+import edu.uci.ics.asterix.common.transactions.ILogManager;
import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.ITransactionContext.TransactionType;
import edu.uci.ics.asterix.common.transactions.ITransactionManager;
import edu.uci.ics.asterix.common.transactions.JobId;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogRecord;
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;
@@ -35,16 +35,18 @@
import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.MurmurHash128Bit;
public class CommitRuntime implements IPushRuntime {
-
+
private final static long SEED = 0L;
private final IHyracksTaskContext hyracksTaskCtx;
private final ITransactionManager transactionManager;
+ private final ILogManager logMgr;
private final JobId jobId;
- private final DatasetId datasetId;
+ private final int datasetId;
private final int[] primaryKeyFields;
private final boolean isWriteTransaction;
- private final long[] longHashes;
+ private final long[] longHashes;
+ private final LogRecord logRecord;
private ITransactionContext transactionContext;
private RecordDescriptor inputRecordDesc;
@@ -54,23 +56,24 @@
public CommitRuntime(IHyracksTaskContext ctx, JobId jobId, int datasetId, int[] primaryKeyFields,
boolean isWriteTransaction) {
this.hyracksTaskCtx = ctx;
- IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext()
- .getApplicationObject();
+ IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext()
+ .getApplicationContext().getApplicationObject();
this.transactionManager = runtimeCtx.getTransactionSubsystem().getTransactionManager();
+ this.logMgr = runtimeCtx.getTransactionSubsystem().getLogManager();
this.jobId = jobId;
- this.datasetId = new DatasetId(datasetId);
+ this.datasetId = datasetId;
this.primaryKeyFields = primaryKeyFields;
this.frameTupleReference = new FrameTupleReference();
this.isWriteTransaction = isWriteTransaction;
- this.longHashes= new long[2];
+ this.longHashes = new long[2];
+ this.logRecord = new LogRecord();
}
@Override
public void open() throws HyracksDataException {
try {
transactionContext = transactionManager.getTransactionContext(jobId);
- transactionContext.setTransactionType(isWriteTransaction ? TransactionType.READ_WRITE
- : TransactionType.READ);
+ transactionContext.setWriteTxn(isWriteTransaction);
} catch (ACIDException e) {
throw new HyracksDataException(e);
}
@@ -84,17 +87,15 @@
for (int t = 0; t < nTuple; t++) {
frameTupleReference.reset(frameTupleAccessor, t);
pkHash = computePrimaryKeyHashValue(frameTupleReference, primaryKeyFields);
- try {
- transactionManager.commitTransaction(transactionContext, datasetId, pkHash);
- } catch (ACIDException e) {
- throw new HyracksDataException(e);
- }
+ logRecord.formEntityCommitLogRecord(transactionContext, datasetId, pkHash, frameTupleReference,
+ primaryKeyFields);
+ logMgr.log(logRecord);
}
}
-
+
private int computePrimaryKeyHashValue(ITupleReference tuple, int[] primaryKeyFields) {
MurmurHash128Bit.hash3_x64_128(tuple, primaryKeyFields, SEED, longHashes);
- return Math.abs((int) longHashes[0]);
+ return Math.abs((int) longHashes[0]);
}
@Override
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 23f1d56..56fc0e7 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
@@ -20,6 +20,7 @@
import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
import edu.uci.ics.asterix.common.dataflow.IAsterixApplicationContextInfo;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
import edu.uci.ics.asterix.metadata.MetadataException;
import edu.uci.ics.asterix.metadata.MetadataManager;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
@@ -38,6 +39,7 @@
import edu.uci.ics.asterix.optimizer.rules.am.InvertedIndexAccessMethod;
import edu.uci.ics.asterix.optimizer.rules.am.InvertedIndexAccessMethod.SearchModifierType;
import edu.uci.ics.asterix.optimizer.rules.am.InvertedIndexJobGenParams;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -217,18 +219,18 @@
if (!isPartitioned) {
dataflowHelperFactory = new LSMInvertedIndexDataflowHelperFactory(
new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new SecondaryIndexOperationTrackerProvider(
+ LSMInvertedIndexIOOperationCallbackFactory.INSTANCE, dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
storageProperties.getBloomFilterFalsePositiveRate());
} else {
dataflowHelperFactory = new PartitionedLSMInvertedIndexDataflowHelperFactory(
new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new SecondaryIndexOperationTrackerProvider(
+ LSMInvertedIndexIOOperationCallbackFactory.INSTANCE, dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
storageProperties.getBloomFilterFalsePositiveRate());
}
LSMInvertedIndexSearchOperatorDescriptor invIndexSearchOp = new LSMInvertedIndexSearchOperatorDescriptor(
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 a24a459..ffde764 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
@@ -33,7 +33,7 @@
import edu.uci.ics.asterix.optimizer.rules.IntroduceEnforcedListTypeRule;
import edu.uci.ics.asterix.optimizer.rules.IntroduceInstantLockSearchCallbackRule;
import edu.uci.ics.asterix.optimizer.rules.IntroduceMaterializationForInsertWithSelfScanRule;
-import edu.uci.ics.asterix.optimizer.rules.IntroduceRapidFrameFlushProjectRule;
+import edu.uci.ics.asterix.optimizer.rules.IntroduceRapidFrameFlushProjectAssignRule;
import edu.uci.ics.asterix.optimizer.rules.IntroduceSecondaryIndexInsertDeleteRule;
import edu.uci.ics.asterix.optimizer.rules.IntroduceStaticTypeCastForInsertRule;
import edu.uci.ics.asterix.optimizer.rules.IntroduceUnnestForCollectionToSequenceRule;
@@ -264,7 +264,7 @@
physicalRewritesTopLevel.add(new PushLimitDownRule());
physicalRewritesTopLevel.add(new IntroduceProjectsRule());
physicalRewritesTopLevel.add(new SetAlgebricksPhysicalOperatorsRule());
- physicalRewritesTopLevel.add(new IntroduceRapidFrameFlushProjectRule());
+ physicalRewritesTopLevel.add(new IntroduceRapidFrameFlushProjectAssignRule());
physicalRewritesTopLevel.add(new SetExecutionModeRule());
return physicalRewritesTopLevel;
}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceRapidFrameFlushProjectRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceRapidFrameFlushProjectAssignRule.java
similarity index 67%
rename from asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceRapidFrameFlushProjectRule.java
rename to asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceRapidFrameFlushProjectAssignRule.java
index d3e11ed2..98f7eeb 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceRapidFrameFlushProjectRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceRapidFrameFlushProjectAssignRule.java
@@ -22,21 +22,23 @@
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.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AssignPOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.StreamProjectPOperator;
import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
/**
- * This rule will search for project operators in an insert/delete/update plan and
- * pass a hint to all those projects between the first "insert" and the commit
- * operator. This hint is used by the project operator so that frames are pushed to
+ * This rule will search for project and assign operators in an insert/delete/update plan and
+ * pass a hint to all of them. This hint is used by the project and assign operators so that frames are pushed to
* the next operator without waiting until they get full. The purpose of this is to
- * reduce the time of holding exclusive locks on the keys that have been inserted.
+ * reduce the time of holding exclusive locks on the keys that have been inserted. Also to allow feeds batching
+ * to work correctly.
*
* @author salsubaiee
*/
-public class IntroduceRapidFrameFlushProjectRule implements IAlgebraicRewriteRule {
+public class IntroduceRapidFrameFlushProjectAssignRule implements IAlgebraicRewriteRule {
@Override
public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
@@ -52,44 +54,49 @@
return false;
}
- AbstractLogicalOperator descendantOp = op;
- while (descendantOp != null) {
- if (descendantOp.getOperatorTag() == LogicalOperatorTag.PROJECT) {
+ for (int i = 0; i < op.getInputs().size(); ++i) {
+ AbstractLogicalOperator descendantOp = (AbstractLogicalOperator) op.getInputs().get(i).getValue();
+
+ if (descendantOp.getOperatorTag() == LogicalOperatorTag.PROJECT
+ || descendantOp.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
if (descendantOp.getPhysicalOperator() == null) {
return false;
}
- } else if (descendantOp.getOperatorTag() == LogicalOperatorTag.INSERT_DELETE) {
- break;
}
- descendantOp = (AbstractLogicalOperator) descendantOp.getInputs().get(0).getValue();
+ checkIfRuleIsApplicable(descendantOp);
}
return true;
}
+ private boolean changeRule(AbstractLogicalOperator op) {
+ boolean planModified = false;
+ for (int i = 0; i < op.getInputs().size(); ++i) {
+ AbstractLogicalOperator descendantOp = (AbstractLogicalOperator) op.getInputs().get(i).getValue();
+
+ if (descendantOp.getOperatorTag() == LogicalOperatorTag.PROJECT) {
+ ProjectOperator projectOp = (ProjectOperator) descendantOp;
+ StreamProjectPOperator physicalOp = (StreamProjectPOperator) projectOp.getPhysicalOperator();
+ physicalOp.setRapidFrameFlush(true);
+ planModified = true;
+ } else if (descendantOp.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+ AssignOperator assignOp = (AssignOperator) descendantOp;
+ AssignPOperator physicalOp = (AssignPOperator) assignOp.getPhysicalOperator();
+ physicalOp.setRapidFrameFlush(true);
+ planModified = true;
+ }
+ changeRule(descendantOp);
+ }
+ return planModified;
+ }
+
@Override
public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
throws AlgebricksException {
AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
-
if (!checkIfRuleIsApplicable(op)) {
return false;
}
- AbstractLogicalOperator descendantOp = op;
- ProjectOperator projectOp = null;
-
- boolean planModified = false;
- while (descendantOp != null) {
- if (descendantOp.getOperatorTag() == LogicalOperatorTag.PROJECT) {
- projectOp = (ProjectOperator) descendantOp;
- StreamProjectPOperator physicalOp = (StreamProjectPOperator) projectOp.getPhysicalOperator();
- physicalOp.setRapidFrameFlush(true);
- planModified = true;
- } else if (descendantOp.getOperatorTag() == LogicalOperatorTag.INSERT_DELETE) {
- break;
- }
- descendantOp = (AbstractLogicalOperator) descendantOp.getInputs().get(0).getValue();
- }
- return planModified;
+ return changeRule(op);
}
}
\ No newline at end of file
diff --git a/asterix-app/data/csv/sample_01.csv b/asterix-app/data/csv/sample_01.csv
new file mode 100644
index 0000000..4dd437a
--- /dev/null
+++ b/asterix-app/data/csv/sample_01.csv
@@ -0,0 +1,8 @@
+1,0.899682764,5.6256,2013-08-07,07:22:35,1979-02-25T23:48:27.034
+2,0.669052398,,-1923-03-29,19:33:34,-1979-02-25T23:48:27.002
+3,0.572733058,192674,-1923-03-28,19:33:34,-1979-02-25T23:48:27.001
+4,,192674,-1923-03-27,19:33:34,-1979-02-25T23:48:27.001
+5,0.572733058,192674,,19:33:34,-1979-02-25T23:48:27.001
+6,0.572733058,192674,-1923-03-25,,-1979-02-25T23:48:27.001
+7,0.572733058,192674,-1923-03-24,19:33:34,
+8,,,,,
\ No newline at end of file
diff --git a/asterix-app/data/tpch0.001/partsupp.tbl b/asterix-app/data/tpch0.001/partsupp.tbl
index d8e5856..d7a631f 100644
--- a/asterix-app/data/tpch0.001/partsupp.tbl
+++ b/asterix-app/data/tpch0.001/partsupp.tbl
@@ -120,44 +120,44 @@
30|3|7945|583.84| sleep. bold, regular deposits hang doggedly furiously bold requests. slyly bold excuses detect busily above the even gifts. blithely express courts are carefully. blithely final packages until th|
31|2|9685|620.84|he blithely regular ideas. blithely unusual requests haggle fluffily. platelets|
31|7|1951|120.99|refully regular pinto beans. ironic requests integrate furiously since the quickly ruthless platelets. quickly ironic attainments ha|
-31|2|1402|761.64|r platelets nag blithely regular deposits. ironic, bold requests |
-31|7|137|849.11|blithely ironic accounts. slyly ironic asymptotes sleep ironic, even accounts. regular accounts thrash quickly|
+31|8|1402|761.64|r platelets nag blithely regular deposits. ironic, bold requests |
+31|10|137|849.11|blithely ironic accounts. slyly ironic asymptotes sleep ironic, even accounts. regular accounts thrash quickly|
32|3|2203|406.03|es? slyly enticing dugouts haggle carefully. regular packages alongside of the asymptotes are carefull|
32|8|467|109.34|ainst the unusual braids nod fluffily packages. regular packages nod among the slyly express|
-32|3|7975|747.14|final foxes boost furiously pending packages. quickly regular depths promise blithely accoun|
-32|8|7938|856.09|s integrate according to the even dependencies. carefully regular reque|
+32|9|7975|747.14|final foxes boost furiously pending packages. quickly regular depths promise blithely accoun|
+32|10|7938|856.09|s integrate according to the even dependencies. carefully regular reque|
33|4|4028|891.46|, pending requests affix slyly. slyly ironic deposits wake accounts. express accounts sleep slowly. ironic, express accounts run carefully fluffily final dependencies. furiously unusual ideas|
-33|9|4410|929.05| packages sleep carefully. slyly final instructions boost. slyly even requests among the carefully pending platelets wake along the final accounts. quickly expre|
-33|4|1287|310.76|dolites above the slyly express deposits try to haggle blithely special gifts. blithely ironic reque|
-33|9|6006|327.19|ly. ironic dependencies haggle carefully silent instructions. furiously ironic dolphins are fluffily furiously even theo|
+33|8|4410|929.05| packages sleep carefully. slyly final instructions boost. slyly even requests among the carefully pending platelets wake along the final accounts. quickly expre|
+33|9|1287|310.76|dolites above the slyly express deposits try to haggle blithely special gifts. blithely ironic reque|
+33|10|6006|327.19|ly. ironic dependencies haggle carefully silent instructions. furiously ironic dolphins are fluffily furiously even theo|
34|5|9934|848.75|ven instructions besides the gifts are furiously among the slyly regular packages! instructions use carefully. even requests sleep quickl|
-34|10|4749|265.31|ckly regular theodolites eat above the bravely regular courts. ironic requests wake slyly.|
-34|5|5459|824.69|ong the slyly silent requests. express, even requests haggle slyly|
+34|6|4749|265.31|ckly regular theodolites eat above the bravely regular courts. ironic requests wake slyly.|
+34|7|5459|824.69|ong the slyly silent requests. express, even requests haggle slyly|
34|10|5884|609.69|ully final tithes. slyly ironic deposits hang furiously about the regular, regular deposits|
35|6|2500|451.58|nic packages boost carefully carefully even theodolites. blithely fina|
35|1|8875|537.72|ully regular deposits: special accounts use. slyly final deposits wake slyly unusual, special ideas. asymptotes |
-35|6|596|669.19|slyly against the daring, pending accounts. fluffily special pinto beans integrate slyly after the carefully unusual packages. slyly bold accounts besides|
-35|1|2025|411.17|s cajole fluffily final deposits. furiously express packages after the blithely special realms boost evenly even requests. slow requests use above the unusual accoun|
-36|7|3907|630.91|al deposits detect fluffily fluffily unusual sauternes. carefully regular requests against the car|
-36|2|174|434.47|permanently express instructions. unusual accounts nag toward the accou|
-36|7|2625|569.91|ctions. pending requests are fluffily across the furiously regular notornis. unusu|
-36|2|8209|289.15|arefully regular requests cajole. special, express foxes sleep slowly. quickly unusual in|
+35|3|596|669.19|slyly against the daring, pending accounts. fluffily special pinto beans integrate slyly after the carefully unusual packages. slyly bold accounts besides|
+35|7|2025|411.17|s cajole fluffily final deposits. furiously express packages after the blithely special realms boost evenly even requests. slow requests use above the unusual accoun|
+36|8|3907|630.91|al deposits detect fluffily fluffily unusual sauternes. carefully regular requests against the car|
+36|9|174|434.47|permanently express instructions. unusual accounts nag toward the accou|
+36|10|2625|569.91|ctions. pending requests are fluffily across the furiously regular notornis. unusu|
+36|11|8209|289.15|arefully regular requests cajole. special, express foxes sleep slowly. quickly unusual in|
37|8|7171|824.96|usly into the slyly final requests. ironic accounts are furiously furiously ironic i|
37|3|5542|126.59|ven deposits. ironic foxes cajole. slyly final deposits are furiously after the furiously even packages. slyly ironic platelets toward the slyl|
-37|8|7113|15.72|re bravely along the furiously express requests. blithely special asymptotes are quickly. fluffily regular packages alo|
-37|3|1449|745.64|y after the ironic accounts. blithely final instructions affix blithely. bold packages sleep carefully regular instructions. regular packages affix carefully. stealthy fo|
+37|7|7113|15.72|re bravely along the furiously express requests. blithely special asymptotes are quickly. fluffily regular packages alo|
+37|9|1449|745.64|y after the ironic accounts. blithely final instructions affix blithely. bold packages sleep carefully regular instructions. regular packages affix carefully. stealthy fo|
38|9|1226|570.11| slyly even pinto beans. blithely special requests nag slyly about the ironic packages. |
38|4|4237|662.75|lar warhorses cajole evenly against the attainments. requests cajole furiously furiously express requests. carefully regular platelets use fluffily after the silent, unusual ideas: bl|
-38|9|1135|160.70|express accounts haggle. carefully even pinto beans according to the slyly final foxes nag slyly about the enticingly express dol|
-38|4|3516|847.09|nal accounts. furiously pending hockey players solve slyly after the furiously final dependencies. deposits are blithely. carefully regular packages unwind busily at the deposits. fluffily |
+38|7|1135|160.70|express accounts haggle. carefully even pinto beans according to the slyly final foxes nag slyly about the enticingly express dol|
+38|8|3516|847.09|nal accounts. furiously pending hockey players solve slyly after the furiously final dependencies. deposits are blithely. carefully regular packages unwind busily at the deposits. fluffily |
39|10|3633|463.10|kages are slyly above the slyly pending pinto beans. bold, ironic pinto beans sleep against the blithely regular requests. fluffily even pinto beans use. regular theodolites haggle against the quic|
39|5|3682|300.43|ng requests are according to the packages. regular packages boost quickly. express Tiresias sleep silently across the even, regular ideas! blithely iro|
-39|10|5475|532.26| beans cajole carefully carefully express requests. instructions sleep furiously bold deposits. furiously regular depos|
-39|5|6259|737.86|y. special, even asymptotes cajole carefully ironic accounts. regular, final pinto beans cajole quickly. regular requests use warhorses. special, special accounts hinder boldly across the|
+39|6|5475|532.26| beans cajole carefully carefully express requests. instructions sleep furiously bold deposits. furiously regular depos|
+39|3|6259|737.86|y. special, even asymptotes cajole carefully ironic accounts. regular, final pinto beans cajole quickly. regular requests use warhorses. special, special accounts hinder boldly across the|
40|1|7690|776.13|lets use fluffily carefully final deposits. blithely ironic instructions sublate against the furiously final ideas; slyly bold courts x-ray silent foxes. regular foxes wake blithely. slyl|
40|6|1704|565.82|riously furiously silent asymptotes. final deposits cajole blithely ironic requests. furiously special pains into the blithely final instru|
-40|1|4521|374.71|ptotes haggle. slyly even requests nag fluffily silent packages. blith|
-40|6|6617|196.64|he slyly unusual epitaphs? ironic deposits at the furiously unusual instructions thrash blithely requests. requests are carefully blithely pending waters.|
+40|7|4521|374.71|ptotes haggle. slyly even requests nag fluffily silent packages. blith|
+40|8|6617|196.64|he slyly unusual epitaphs? ironic deposits at the furiously unusual instructions thrash blithely requests. requests are carefully blithely pending waters.|
41|2|9040|488.55|ss the dinos wake along the blithely regular theodolites. foxes cajole quickly ironic, final foxes. blithely ironic packages haggle against |
41|8|5946|391.81| slyly slyly regular requests. final deposits sleep fluffily. blithely bold instructions detect carefully. blithely pending requests are furiously ironically final ideas. regul|
41|4|1550|916.55| the blithely final ideas. furiously regular asymptotes could cajole furious|
@@ -319,45 +319,45 @@
80|9|5385|945.72|cial asymptotes believe after the blithely unusual deposits. furiously silent pinto beans cajole quickly inside the slyly even deposits. regular, f|
80|8|4034|797.05|ptotes cajole carefully. express ideas cajole carefully even somas. final pinto beans print fluffily across the |
81|2|1605|550.29|es haggle blithely fluffily final requests. furiously regular foxes use. furiously unusual requests outside the furiously regular requests|
-81|2|5923|220.23|the final, quick accounts are blithely above the s|
-81|2|2942|409.73|accounts boost. fluffily unusual requests cajole fluffily slyly ironic requests. foxes cajole quick|
-81|2|58|492.19| instructions boost furiously across the foxes-- final depo|
+81|3|5923|220.23|the final, quick accounts are blithely above the s|
+81|4|2942|409.73|accounts boost. fluffily unusual requests cajole fluffily slyly ironic requests. foxes cajole quick|
+81|5|58|492.19| instructions boost furiously across the foxes-- final depo|
82|3|7793|697.31|he accounts cajole quickly after the even patterns. ironic platelets sublate regular, even asymptotes. quick courts affix according to|
-82|3|7698|585.86|pinto beans. slyly express excuses haggle. blithely even pinto beans about the quick inst|
-82|3|8268|604.25|e after the carefully even theodolites. regular, pending accounts boost. quickly final asymptotes haggle slyly. requests use final, bold pinto beans. bold, ruthle|
-82|3|5532|900.07| slyly? fluffily special dependencies haggle among the slyly special requests. regular, bold packages after the blithely ironic packages are slyly ironic packages. slyly final deposits w|
+82|4|7698|585.86|pinto beans. slyly express excuses haggle. blithely even pinto beans about the quick inst|
+82|5|8268|604.25|e after the carefully even theodolites. regular, pending accounts boost. quickly final asymptotes haggle slyly. requests use final, bold pinto beans. bold, ruthle|
+82|6|5532|900.07| slyly? fluffily special dependencies haggle among the slyly special requests. regular, bold packages after the blithely ironic packages are slyly ironic packages. slyly final deposits w|
83|4|3010|745.51|l foxes along the bold, regular packages integrate carefully express courts! final excuses sleep carefully ironic|
-83|4|8200|399.64|y final platelets are carefully carefully special platelets. carefully ironic requests wake blithely alongside of the slyly even accounts. bold, regular requests sleep |
-83|4|5974|657.22| even packages boost furiously. slyly regular gifts above the accounts are quickly express packages. slyly pending deposits besides the express, even asymptotes haggle after the ironic ins|
-83|4|3890|24.73|deposits. carefully even dependencies across the dependencies haggl|
+83|5|8200|399.64|y final platelets are carefully carefully special platelets. carefully ironic requests wake blithely alongside of the slyly even accounts. bold, regular requests sleep |
+83|6|5974|657.22| even packages boost furiously. slyly regular gifts above the accounts are quickly express packages. slyly pending deposits besides the express, even asymptotes haggle after the ironic ins|
+83|7|3890|24.73|deposits. carefully even dependencies across the dependencies haggl|
84|5|5711|233.61|arefully final platelets cajole blithely; quickly final accounts use furiously. furiously reg|
-84|5|208|469.80|carefully express dolphins nag about the slyly bold requests. slyly even packages wake among the furiously special attainments.|
-84|5|2909|969.44|silent requests cajole slowly bold ideas. special, special deposits according to the always silent packages are against the furiously silent packages. even, blithe accounts sleep slyly across |
-84|5|903|707.77|gly regular dependencies boost. slyly even accounts sleep. furiously final hockey players wake carefully with the reg|
+84|6|208|469.80|carefully express dolphins nag about the slyly bold requests. slyly even packages wake among the furiously special attainments.|
+84|7|2909|969.44|silent requests cajole slowly bold ideas. special, special deposits according to the always silent packages are against the furiously silent packages. even, blithe accounts sleep slyly across |
+84|8|903|707.77|gly regular dependencies boost. slyly even accounts sleep. furiously final hockey players wake carefully with the reg|
85|6|2628|608.77|xes wake furiously after the carefully even platelets. blithe theodolites are furi|
-85|6|118|917.83| against the even deposits. furiously bold ideas along the furious requ|
-85|6|2074|491.20|encies-- slyly regular requests about the quiet accounts detect quickly at the |
-85|6|8289|73.81|s cajole slyly along the slyly special accounts. regular, special deposits wake. furiously special foxes boost. blithely even packa|
+85|5|118|917.83| against the even deposits. furiously bold ideas along the furious requ|
+85|4|2074|491.20|encies-- slyly regular requests about the quiet accounts detect quickly at the |
+85|3|8289|73.81|s cajole slyly along the slyly special accounts. regular, special deposits wake. furiously special foxes boost. blithely even packa|
86|7|806|65.98|ackages. blithely pending accounts are slyly furiously pending theodolites. furiously eve|
-86|7|2773|250.04|ding accounts. slyly special requests will have to affix carefully along the furiously unusual packages. regular theodol|
-86|7|5546|816.53|s. slyly final requests wake. furious deposits must wake blithely among the blithely ironic instructions. special hockey players try to are bli|
-86|7|1418|332.65|press theodolites sleep carefully about the blithely unusual requests. quickly final deposits breach slyly |
+86|1|2773|250.04|ding accounts. slyly special requests will have to affix carefully along the furiously unusual packages. regular theodol|
+86|2|5546|816.53|s. slyly final requests wake. furious deposits must wake blithely among the blithely ironic instructions. special hockey players try to are bli|
+86|3|1418|332.65|press theodolites sleep carefully about the blithely unusual requests. quickly final deposits breach slyly |
87|8|5679|688.33|t the carefully regular asymptotes. blithely stealthy pinto beans within the furiously expres|
-87|8|1272|435.42|ronic foxes sleep along the special foxes. final ideas wake quickly about the carefully special theodolites. blithely ironic packages are blithely. regular, regular pint|
-87|8|9041|617.20|furiously final deposits. furiously special dependencies solve across the regular, special ideas. carefully silent requests haggle furiously after the special, specia|
-87|8|1892|868.60|arhorses are. unusual requests use blithely furiously final ideas. final requests sleep theodoli|
+87|2|1272|435.42|ronic foxes sleep along the special foxes. final ideas wake quickly about the carefully special theodolites. blithely ironic packages are blithely. regular, regular pint|
+87|3|9041|617.20|furiously final deposits. furiously special dependencies solve across the regular, special ideas. carefully silent requests haggle furiously after the special, specia|
+87|4|1892|868.60|arhorses are. unusual requests use blithely furiously final ideas. final requests sleep theodoli|
88|9|6116|334.58|ect furiously around the regular deposits. special, final platelets boost furiously. blithely unusu|
-88|9|395|71.50| the regular accounts-- furiously even accounts use quickly after the regular, regular deposits. furiously e|
-88|9|9979|81.82|f the regular, regular requests believe fluffily along the final, quiet decoys. furiously even accounts cajole. carefully express requests wake quickly among the ideas. quickly silent |
-88|9|276|821.43|gular pinto beans. slyly pending excuses breach blithely express accounts. thin deposits sleep slyly around the even accounts; fluffily busy patterns kindle. slyly final deposits along the |
+88|1|395|71.50| the regular accounts-- furiously even accounts use quickly after the regular, regular deposits. furiously e|
+88|2|9979|81.82|f the regular, regular requests believe fluffily along the final, quiet decoys. furiously even accounts cajole. carefully express requests wake quickly among the ideas. quickly silent |
+88|3|276|821.43|gular pinto beans. slyly pending excuses breach blithely express accounts. thin deposits sleep slyly around the even accounts; fluffily busy patterns kindle. slyly final deposits along the |
89|10|3430|744.87| integrate slyly dolphins. bold, final frets use beside the carefully even accounts. slyly close dependencies sleep quickly carefully final pinto beans. foxes promi|
-89|10|8599|776.53|ress packages use furiously. furiously regular packages thrash blithely about the slyly pe|
-89|10|7876|417.61|nstructions: furiously even requests are quietly unusual accounts. regular requests are after the blithely regular deposits. sl|
-89|10|924|920.02|ickly unusual asymptotes after the slyly unusual accounts are carefully doggedly ironic accounts. even, final accounts use furiousl|
+89|1|8599|776.53|ress packages use furiously. furiously regular packages thrash blithely about the slyly pe|
+89|2|7876|417.61|nstructions: furiously even requests are quietly unusual accounts. regular requests are after the blithely regular deposits. sl|
+89|3|924|920.02|ickly unusual asymptotes after the slyly unusual accounts are carefully doggedly ironic accounts. even, final accounts use furiousl|
90|1|8037|409.38|eas. unusual, pending packages boost quietly final accounts. slyly final packages serve. slyly even instructions sleep carefully. quickly even foxes wake quickly. |
-90|1|9683|498.43| accounts! fluffily regular deposits x-ray about the unusual, final packages. furiously final deposits alongside of the caref|
-90|1|7849|666.13|carefully ironic accounts are around the slyly bold asymptotes. carefully regular packages use furiously. ironic platelets affix carefully final accounts-- fluffily final pinto beans across the fina|
-90|1|7629|50.84|onic requests wake fluffily unusual packages. furiously even frays after the daringly pending requests wake furiously alongside of the bold requests. fluffily ironic ideas nag. ironic,|
+90|2|9683|498.43| accounts! fluffily regular deposits x-ray about the unusual, final packages. furiously final deposits alongside of the caref|
+90|3|7849|666.13|carefully ironic accounts are around the slyly bold asymptotes. carefully regular packages use furiously. ironic platelets affix carefully final accounts-- fluffily final pinto beans across the fina|
+90|4|7629|50.84|onic requests wake fluffily unusual packages. furiously even frays after the daringly pending requests wake furiously alongside of the bold requests. fluffily ironic ideas nag. ironic,|
91|2|7986|528.64|luffily final instructions. furiously unusual foxes haggle |
91|3|3257|906.20|ackages cajole slyly. blithely bold deposits cajole. blithely |
91|4|483|823.21|n: slyly ironic foxes nag blithely according to the furiously bold foxes. regular, regular accounts a|
@@ -520,44 +520,44 @@
130|3|7387|883.99|aggle furiously. even ideas hinder deposits. even, final ideas are. unusual theodolites after the special, express foxes haggle carefully pending accou|
131|2|3263|211.70|sits sleep quickly regular multipliers. slyly even platelets cajole after the furiously ironic deposits. slyly ironic requests should have to cajole: bl|
131|7|125|861.84|l accounts grow quickly-- slyly ironic requests haggle? quickly express pinto bean|
-131|2|5138|572.43|grouches run with the carefully even packages. ironic, even deposits run slyly along the packages. special dependencies among the regular |
-131|7|8945|613.09| are carefully along the quickly final theodolites. packages after the quickly pending package|
+131|3|5138|572.43|grouches run with the carefully even packages. ironic, even deposits run slyly along the packages. special dependencies among the regular |
+131|4|8945|613.09| are carefully along the quickly final theodolites. packages after the quickly pending package|
132|3|3092|687.29|y special decoys against the ideas affix against the sly|
132|8|1904|925.73|the regular foxes wake ironic deposits. ironic, special requests use blithely instructions! final requests hang. blithely regular deposits haggle. ir|
-132|3|7441|357.06|ests. furiously unusual requests wake furiously. quickly unusual depos|
-132|8|5303|353.06|ep blithely after the sly accounts. slyly express dolphins cajole amon|
+132|4|7441|357.06|ests. furiously unusual requests wake furiously. quickly unusual depos|
+132|5|5303|353.06|ep blithely after the sly accounts. slyly express dolphins cajole amon|
133|4|5727|49.17|boost blithely across the ironic, regular instructions. packages use slyly unusual requests. bold accounts above the fu|
133|9|404|478.18|ly ironic requests run instead of the blithely ironic accounts? regular ideas use fluffily: even, express packages sleep abov|
-133|4|4568|57.48|dolites. ironic accounts are blithely pinto beans. regular pinto beans haggle beneath|
-133|9|2813|277.26|s. pending, final accounts haggle blithely furiously pending deposits! carefully unusual attainments integrate. blithely bo|
+133|5|4568|57.48|dolites. ironic accounts are blithely pinto beans. regular pinto beans haggle beneath|
+133|6|2813|277.26|s. pending, final accounts haggle blithely furiously pending deposits! carefully unusual attainments integrate. blithely bo|
134|5|8879|848.14|lites. slyly final foxes after the bold requests cajole carefu|
134|10|9013|102.99|pendencies. furiously express warthogs cajole furiously ironic, regular asymptotes. bold deposits boost among the furiously even theodolites. regular instructions integrate carefully |
-134|5|852|927.45| sleep unusual, express packages. unusual sentiments are furio|
-134|10|6270|388.28| to the furiously pending deposits nag along the slyly express asymptotes. slyly silent accounts shal|
+134|6|852|927.45| sleep unusual, express packages. unusual sentiments are furio|
+134|7|6270|388.28| to the furiously pending deposits nag along the slyly express asymptotes. slyly silent accounts shal|
135|6|6940|465.82|ding foxes cajole. even dugouts haggle busily. fluffily pending packages about the express excuses boost slyly final packages. blithely express ideas cajole about the carefu|
135|1|2443|9.83|atterns. pending, special deposits are furiously. express, regular deposits integrate quickly. unusual gifts cajole blithely stealthily pending deposit|
-135|6|7453|698.42|ven accounts. slyly final instructions nag slyly around the regular, unusual packages. slyly sp|
-135|1|2771|306.43|old deposits. furiously express instructions boost. pending dolphins use requests. slyly regular packages cajole quickly final ideas. pending, regular ideas nag carefully even, express pla|
+135|2|7453|698.42|ven accounts. slyly final instructions nag slyly around the regular, unusual packages. slyly sp|
+135|3|2771|306.43|old deposits. furiously express instructions boost. pending dolphins use requests. slyly regular packages cajole quickly final ideas. pending, regular ideas nag carefully even, express pla|
136|7|2237|548.19|ond the silent accounts haggle above the blithely regular packages|
136|2|6068|806.19|structions. ironic theodolites haggle according to the final, daring pearls. carefully ironic somas are silently requests. express pa|
-136|7|8979|387.57|ans. express pinto beans wake carefully among the slyly ironic foxes: carefully final pinto beans haggle blithely. pending, final deposits promise furiously|
-136|2|9617|525.81| across the carefully pending warthogs. close, regular packages are quickly after the never ironic foxes. accounts sleep quickly along the furiously regular re|
+136|1|8979|387.57|ans. express pinto beans wake carefully among the slyly ironic foxes: carefully final pinto beans haggle blithely. pending, final deposits promise furiously|
+136|3|9617|525.81| across the carefully pending warthogs. close, regular packages are quickly after the never ironic foxes. accounts sleep quickly along the furiously regular re|
137|8|9057|302.26|slyly about the regular instructions. even, ironic theodolites use carefully around the even decoys. unusual, pending dolphin|
137|3|4078|441.11|packages. blithely unusual sentiments should are. furiously regular accounts nag quickly carefully special asymptotes! idly ironic requests dazzle bold requests. carefully expres|
-137|8|467|371.85|ly special accounts detect carefully. furiously ironic deposits nag express packages. slyly quiet |
-137|3|7850|187.31|atelets sublate fluffily. enticingly unusual packages boost according to the blithely ironic foxes. pending requests mold sly|
+137|2|467|371.85|ly special accounts detect carefully. furiously ironic deposits nag express packages. slyly quiet |
+137|4|7850|187.31|atelets sublate fluffily. enticingly unusual packages boost according to the blithely ironic foxes. pending requests mold sly|
138|9|133|576.96|regular, final deposits maintain slyly even requests. regularly furious deposits use above the stealthy requests. ironic deposits are. carefully final frays are carefully. carefu|
138|4|2535|885.35|lar deposits. courts sleep carefully. furiously express ideas boost furiously after the final, regular foxes. furiously bold deposits are. express accounts haggle blithely. |
-138|9|7907|119.83|epitaphs? quickly express foxes use pending accounts. special packages cajole blithely among the quickly unusual accounts? boldly ironic packages across the slyly ironic senti|
-138|4|967|309.03|pendencies integrate against the unusual pains. carefully unusual theodolites wake quickly across the deposits. blithely regular deposits alongside of the carefully regular deposits|
+138|1|7907|119.83|epitaphs? quickly express foxes use pending accounts. special packages cajole blithely among the quickly unusual accounts? boldly ironic packages across the slyly ironic senti|
+138|2|967|309.03|pendencies integrate against the unusual pains. carefully unusual theodolites wake quickly across the deposits. blithely regular deposits alongside of the carefully regular deposits|
139|10|2886|285.75|fully ironic requests according to the quickly final idea|
139|5|9255|684.61|ickly furiously regular excuses. boldly express deposits sleep. ideas nag above the silent dependencies. slyly regular packages wake furiously. requests are carefully. quickly final fox|
-139|10|1042|972.23|gular, regular theodolites. regular asymptotes haggle carefully according to the permanently even deposits. slyly special account|
-139|5|3285|690.00|xpress pains. quickly regular ideas after the special, bold excuses wake furiously final ideas. slyly bold accounts nag packages. ironically regular|
+139|1|1042|972.23|gular, regular theodolites. regular asymptotes haggle carefully according to the permanently even deposits. slyly special account|
+139|3|3285|690.00|xpress pains. quickly regular ideas after the special, bold excuses wake furiously final ideas. slyly bold accounts nag packages. ironically regular|
140|1|2379|501.05|of the silent, bold courts. slyly regular dependencies haggle. fluffily special deposits cajole carefully. quickly ironic depos|
140|6|3533|781.45|ayers. carefully ironic pinto beans nod carefully furiously regular pinto beans. slyly ironic requests after the carefully regular packages are about the blithel|
-140|1|304|45.84|ing requests. carefully unusual foxes are final requests. slyly regular accounts wake permanently. quickly ironic theodolites hagg|
-140|6|7346|429.52| special pinto beans wake carefully unusual warthogs! furi|
+140|2|304|45.84|ing requests. carefully unusual foxes are final requests. slyly regular accounts wake permanently. quickly ironic theodolites hagg|
+140|3|7346|429.52| special pinto beans wake carefully unusual warthogs! furi|
141|2|6776|293.63|fluffily unusual courts sleep. close pinto beans haggle quickly after the carefully ir|
141|8|1660|139.18|egular accounts. enticingly bold theodolites eat slyly across the never ironic platelets. theodolites wake bli|
141|4|7628|838.08|sly about the pinto beans. blithely ironic ideas sleep. foxes are quietly among the pinto beans. carefu|
@@ -719,45 +719,45 @@
180|9|724|426.16|e, regular accounts. furiously final ideas are furiously above the bold, silent asymptotes. sly instructions are carefully quickly final sentiments. furiously ironic foxes cajole bold, exp|
180|8|5899|864.83|hin the carefully furious pinto beans. furiously ironic pinto beans use slyly above the even instructio|
181|2|2416|844.44|ully. theodolites throughout the blithely unusual pinto bea|
-181|2|3242|886.53| express ideas nag carefully brave accounts. slyly express deposits would affix. final, special requests against the slyl|
-181|2|215|938.29| accounts boost furiously furiously blithe theodolites. slyly bold requests unwind special, unusual requests. furious ideas boost quickly pending |
-181|2|1122|657.25|lyly fluffily pending foxes. fluffily ironic pains haggle. thinly regular requests against the deposits affix after the never ev|
+181|3|3242|886.53| express ideas nag carefully brave accounts. slyly express deposits would affix. final, special requests against the slyl|
+181|4|215|938.29| accounts boost furiously furiously blithe theodolites. slyly bold requests unwind special, unusual requests. furious ideas boost quickly pending |
+181|5|1122|657.25|lyly fluffily pending foxes. fluffily ironic pains haggle. thinly regular requests against the deposits affix after the never ev|
182|3|9699|535.27|ound the furiously regular foxes. pending requests dazzle along |
-182|3|960|519.36|arefully pending dependencies are always slyly unusual pin|
-182|3|6243|741.46|accounts are slyly. furiously ironic requests haggle. express, special instructions against the ironic theodolites use s|
-182|3|6146|365.00|s. blithely express theodolites sleep blithely alongside of the requests?|
+182|4|960|519.36|arefully pending dependencies are always slyly unusual pin|
+182|5|6243|741.46|accounts are slyly. furiously ironic requests haggle. express, special instructions against the ironic theodolites use s|
+182|6|6146|365.00|s. blithely express theodolites sleep blithely alongside of the requests?|
183|4|30|875.44|slyly. furiously regular instructions cajole slyly about the pending, final theodolites. blithely final deposits cajole fluffily alo|
-183|4|4482|424.86|es. depths affix fluffily. bold instructions haggle. ruthless instructions must have to boost|
-183|4|8707|884.26|posits wake. blithely pending requests nag furiously alongside of the p|
-183|4|333|678.16|ost final, final theodolites. slyly bold foxes dazzle carefully furiously regular accounts. regular, sly instructions about the furiously regular excuses nag blithely abou|
+183|5|4482|424.86|es. depths affix fluffily. bold instructions haggle. ruthless instructions must have to boost|
+183|6|8707|884.26|posits wake. blithely pending requests nag furiously alongside of the p|
+183|7|333|678.16|ost final, final theodolites. slyly bold foxes dazzle carefully furiously regular accounts. regular, sly instructions about the furiously regular excuses nag blithely abou|
184|5|7069|449.45|nal ideas. blithely final ideas haggle against the pinto beans. qu|
-184|5|9193|576.88|uickly quick dependencies could detect furiously. final packages p|
-184|5|6400|551.90|ss dependencies. quickly even pinto beans are. express accounts a|
-184|5|831|186.84|kages cajole carefully furiously ironic instructions. deposits use bl|
+184|1|9193|576.88|uickly quick dependencies could detect furiously. final packages p|
+184|2|6400|551.90|ss dependencies. quickly even pinto beans are. express accounts a|
+184|3|831|186.84|kages cajole carefully furiously ironic instructions. deposits use bl|
185|6|1475|538.58|unts hinder slyly. quickly express ideas sleep carefully |
-185|6|6244|213.04|ly unusual decoys are furiously quickly regular packages. bold, ironic foxes cajole fluffily around|
-185|6|7245|426.74|sleep blithely alongside of the regular excuses. even, regular|
-185|6|8014|510.23|lithely even ideas. regular platelets wake carefully ironic, special instructions! final pearls above the fluffily quiet ideas use furiously about the |
+185|1|6244|213.04|ly unusual decoys are furiously quickly regular packages. bold, ironic foxes cajole fluffily around|
+185|2|7245|426.74|sleep blithely alongside of the regular excuses. even, regular|
+185|3|8014|510.23|lithely even ideas. regular platelets wake carefully ironic, special instructions! final pearls above the fluffily quiet ideas use furiously about the |
186|7|1095|252.84|. carefully regular pinto beans according to the blithely close asymptotes haggle carefully special requests. packages cajole up the furi|
-186|7|1945|18.75|nic foxes boost carefully careful packages: express, fluffy dolphins nag quickly ironic packages. slyly bold requests nag amon|
-186|7|8838|729.42|ing asymptotes. enticingly regular theodolites mai|
-186|7|7898|812.37|ctions sleep silently carefully bold platelets. furiously ironic dependencies boost. regular de|
+186|1|1945|18.75|nic foxes boost carefully careful packages: express, fluffy dolphins nag quickly ironic packages. slyly bold requests nag amon|
+186|2|8838|729.42|ing asymptotes. enticingly regular theodolites mai|
+186|3|7898|812.37|ctions sleep silently carefully bold platelets. furiously ironic dependencies boost. regular de|
187|8|8656|238.66|tes use along the even foxes? final foxes haggle pinto beans. slyly ironic theodolites are according to the deposits. furiously pending reques|
-187|8|4945|316.64|eposits boost quickly bold requests. furiously regular ideas boost boldly. special, express dependencies are fluffily slyly reg|
-187|8|3183|362.75|t the bold platelets. fluffily express platelets cajole fluffily along the always bold requests. blith|
-187|8|7440|989.71|e slyly against the slyly regular pinto beans. requests haggle carefully around the asymptotes. regular, regular asymptotes use furiously some|
+187|1|4945|316.64|eposits boost quickly bold requests. furiously regular ideas boost boldly. special, express dependencies are fluffily slyly reg|
+187|2|3183|362.75|t the bold platelets. fluffily express platelets cajole fluffily along the always bold requests. blith|
+187|3|7440|989.71|e slyly against the slyly regular pinto beans. requests haggle carefully around the asymptotes. regular, regular asymptotes use furiously some|
188|9|4835|771.95|pains are fluffily about the fluffily pending asymptot|
-188|9|2620|331.70|elets nag slyly regular pinto beans. slyly even dugouts above the blithely unusual theodolites su|
-188|9|730|713.62|nag against the final accounts. blithely pending attainments lose. silent requests wake quickly. careful|
-188|9|5430|920.20|uriously. special, regular instructions sleep along the accounts. quickly even foxes across the regular theodolites hang u|
+188|1|2620|331.70|elets nag slyly regular pinto beans. slyly even dugouts above the blithely unusual theodolites su|
+188|2|730|713.62|nag against the final accounts. blithely pending attainments lose. silent requests wake quickly. careful|
+188|3|5430|920.20|uriously. special, regular instructions sleep along the accounts. quickly even foxes across the regular theodolites hang u|
189|10|1305|392.50|packages. regular, unusual accounts lose furiously fluffily regular platelets. requests sleep carefully dependenc|
-189|10|8777|573.22|beans cajole slyly ironic requests. requests are quickly unusual, even packages. ironic frays haggle. blithely pending requests nod slyly. express, silent requests against the slyly unusual |
-189|10|6369|946.07|ts hinder slyly regular, unusual foxes. final sentiments use above the slyly r|
-189|10|2505|593.23| the deposits. special deposits sleep-- furiously regular sauternes solve furiously across the furiously regular pack|
+189|1|8777|573.22|beans cajole slyly ironic requests. requests are quickly unusual, even packages. ironic frays haggle. blithely pending requests nod slyly. express, silent requests against the slyly unusual |
+189|2|6369|946.07|ts hinder slyly regular, unusual foxes. final sentiments use above the slyly r|
+189|3|2505|593.23| the deposits. special deposits sleep-- furiously regular sauternes solve furiously across the furiously regular pack|
190|1|535|621.53|unts must have to haggle; slyly ironic accounts affix slyly alongside of the carefully even accounts. furious deposits haggle quietly among the packages. blithely |
-190|1|5845|608.91| haggle along the carefully unusual pinto beans. quickly final accounts sleep a|
-190|1|4579|396.60|inal, final foxes. regular, even deposits wake blithely! silent, regular packages integrate according to the slyly regular deposits. ironic, ironic notornis ha|
-190|1|2861|458.00|s cajole slyly across the daring, final pinto beans. carefully quiet requests affix along the a|
+190|2|5845|608.91| haggle along the carefully unusual pinto beans. quickly final accounts sleep a|
+190|3|4579|396.60|inal, final foxes. regular, even deposits wake blithely! silent, regular packages integrate according to the slyly regular deposits. ironic, ironic notornis ha|
+190|4|2861|458.00|s cajole slyly across the daring, final pinto beans. carefully quiet requests affix along the a|
191|2|8310|521.06|the slowly regular deposits. special accounts along the quickly unusual|
191|3|1693|464.46|y. slyly unusual waters across the special pinto beans nag blithely according to the busy deposits. carefully regular accounts are against the regular accounts; perman|
191|4|597|126.96|ly final accounts should have to boost above the doggedly express pinto beans. blithely regular packages cajole furiously bold requests. fluf|
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
index e56ed92..18a81e5 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
@@ -15,6 +15,7 @@
package edu.uci.ics.asterix.api.common;
import java.io.IOException;
+import java.util.List;
import java.util.logging.Logger;
import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
@@ -45,7 +46,7 @@
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
-import edu.uci.ics.hyracks.storage.am.lsm.common.impls.SynchronousScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AsynchronousScheduler;
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.DelayPageCleanerPolicy;
@@ -107,7 +108,8 @@
storageProperties.getBufferCachePageSize(), storageProperties.getBufferCacheNumPages(),
storageProperties.getBufferCacheMaxOpenFiles(), ncApplicationContext.getThreadFactory());
- lsmIOScheduler = SynchronousScheduler.INSTANCE;
+ AsynchronousScheduler.INSTANCE.init(ncApplicationContext.getThreadFactory());
+ lsmIOScheduler = AsynchronousScheduler.INSTANCE;
mergePolicy = new ConstantMergePolicy(storageProperties.getLSMIndexMergeThreshold(), this);
ILocalResourceRepositoryFactory persistentLocalResourceRepositoryFactory = new PersistentLocalResourceRepositoryFactory(
@@ -212,8 +214,8 @@
}
@Override
- public IVirtualBufferCache getVirtualBufferCache(int datasetID) {
- return indexLifecycleManager.getVirtualBufferCache(datasetID);
+ public List<IVirtualBufferCache> getVirtualBufferCaches(int datasetID) {
+ return indexLifecycleManager.getVirtualBufferCaches(datasetID);
}
@Override
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContextProviderForRecovery.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContextProviderForRecovery.java
index 712d993..ee9dfae 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContextProviderForRecovery.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContextProviderForRecovery.java
@@ -14,6 +14,8 @@
*/
package edu.uci.ics.asterix.api.common;
+import java.util.List;
+
import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
@@ -88,32 +90,28 @@
}
@Override
- public IVirtualBufferCache getVirtualBufferCache(int datasetID) {
- return asterixAppRuntimeContext.getVirtualBufferCache(datasetID);
+ public List<IVirtualBufferCache> getVirtualBufferCaches(int datasetID) {
+ return asterixAppRuntimeContext.getVirtualBufferCaches(datasetID);
}
@Override
public ILSMIOOperationCallbackProvider getNoOpIOOperationCallbackProvider() {
- return AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER;
+ return AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
}
@Override
public ILSMIOOperationCallbackProvider getLSMBTreeIOOperationCallbackProvider(boolean isPrimary) {
- if (isPrimary) {
- return AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER;
- } else {
- return AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER;
- }
+ return AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
}
@Override
public ILSMIOOperationCallbackProvider getLSMRTreeIOOperationCallbackProvider() {
- return AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER;
+ return AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
}
@Override
public ILSMIOOperationCallbackProvider getLSMInvertedIndexIOOperationCallbackProvider() {
- return AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER;
+ return AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
}
@Override
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
index 6acf6c1..9c3a853 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
@@ -128,6 +128,7 @@
ADDED_PENDINGOP_RECORD_TO_METADATA
}
+ public static final boolean IS_DEBUG_MODE = false;//true
private final List<Statement> aqlStatements;
private final PrintWriter out;
private final SessionConfig sessionConfig;
@@ -1283,6 +1284,7 @@
acquireReadLatch();
try {
+ metadataProvider.setWriteTransaction(true);
BeginFeedStatement bfs = (BeginFeedStatement) stmt;
String dataverseName = getActiveDataverseName(bfs.getDataverseName());
@@ -1520,6 +1522,9 @@
private void abort(Exception rootE, Exception parentE, MetadataTransactionContext mdTxnCtx) {
try {
+ if (IS_DEBUG_MODE) {
+ rootE.printStackTrace();
+ }
MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
} catch (Exception e2) {
parentE.addSuppressed(e2);
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 a0872d2..3e41a77 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
@@ -123,12 +123,12 @@
datasetName);
AsterixStorageProperties storageProperties = AsterixAppContextInfo.getInstance().getStorageProperties();
IndexDropOperatorDescriptor primaryBtreeDrop = new IndexDropOperatorDescriptor(specPrimary,
- AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
splitsAndConstraint.first, new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
- dataset.getDatasetId()), AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
storageProperties.getBloomFilterFalsePositiveRate()));
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(specPrimary, primaryBtreeDrop,
splitsAndConstraint.second);
@@ -175,13 +175,12 @@
localResourceMetadata, LocalResource.LSMBTreeResource);
TreeIndexCreateOperatorDescriptor indexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
- AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
splitsAndConstraint.first, typeTraits, comparatorFactories, blooFilterKeyFields,
new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
- new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER, storageProperties
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new PrimaryIndexOperationTrackerProvider(
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, storageProperties
.getBloomFilterFalsePositiveRate()), localResourceFactoryProvider,
NoOpOperationCallbackFactory.INSTANCE);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, indexCreateOp,
@@ -260,15 +259,15 @@
TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad;
if (!loadStmt.alreadySorted()) {
btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
- AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, splitsAndConstraint.first, typeTraits,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, splitsAndConstraint.first, typeTraits,
comparatorFactories, blooFilterKeyFields, fieldPermutation, GlobalConfig.DEFAULT_BTREE_FILL_FACTOR,
true, numElementsHint, true, new LSMBTreeDataflowHelperFactory(
new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
storageProperties.getBloomFilterFalsePositiveRate()), NoOpOperationCallbackFactory.INSTANCE);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, btreeBulkLoad,
splitsAndConstraint.second);
@@ -286,15 +285,15 @@
spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0, btreeBulkLoad, 0);
} else {
btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
- AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, splitsAndConstraint.first, typeTraits,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, splitsAndConstraint.first, typeTraits,
comparatorFactories, blooFilterKeyFields, fieldPermutation, GlobalConfig.DEFAULT_BTREE_FILL_FACTOR,
- false, numElementsHint, true, new LSMBTreeDataflowHelperFactory(
+ true, numElementsHint, true, new LSMBTreeDataflowHelperFactory(
new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
storageProperties.getBloomFilterFalsePositiveRate()), NoOpOperationCallbackFactory.INSTANCE);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, btreeBulkLoad,
splitsAndConstraint.second);
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 e2767cd..de4d075 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
@@ -20,12 +20,14 @@
import edu.uci.ics.asterix.common.config.OptimizationConfUtil;
import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
import edu.uci.ics.asterix.metadata.MetadataException;
import edu.uci.ics.asterix.metadata.MetadataManager;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
import edu.uci.ics.asterix.metadata.entities.Dataset;
import edu.uci.ics.asterix.metadata.entities.ExternalFile;
import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
import edu.uci.ics.asterix.translator.CompiledStatements.CompiledCreateIndexStatement;
import edu.uci.ics.asterix.translator.CompiledStatements.CompiledIndexDropStatement;
@@ -81,12 +83,12 @@
.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverseName, datasetName, indexName);
AsterixStorageProperties storageProperties = AsterixAppContextInfo.getInstance().getStorageProperties();
IndexDropOperatorDescriptor btreeDrop = new IndexDropOperatorDescriptor(spec,
- AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
splitsAndConstraint.first, new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
- dataset.getDatasetId()), AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ new SecondaryIndexOperationTrackerProvider(LSMBTreeIOOperationCallbackFactory.INSTANCE, dataset
+ .getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
storageProperties.getBloomFilterFalsePositiveRate()));
AlgebricksPartitionConstraintHelper
.setPartitionConstraintInJobSpec(spec, btreeDrop, splitsAndConstraint.second);
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 fc407eb..144131f 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
@@ -14,13 +14,15 @@
*/
package edu.uci.ics.asterix.file;
-
import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
+
import edu.uci.ics.asterix.external.data.operator.ExternalDataIndexingOperatorDescriptor;
import edu.uci.ics.asterix.external.util.ExternalIndexHashPartitionComputerFactory;
import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
@@ -53,10 +55,10 @@
public class SecondaryBTreeCreator extends SecondaryIndexCreator {
- protected SecondaryBTreeCreator(PhysicalOptimizationConfig physOptConf,
- IAsterixPropertiesProvider propertiesProvider) {
- super(physOptConf, propertiesProvider);
- }
+ protected SecondaryBTreeCreator(PhysicalOptimizationConfig physOptConf,
+ IAsterixPropertiesProvider propertiesProvider) {
+ super(physOptConf, propertiesProvider);
+ }
@Override
public JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException {
@@ -68,23 +70,23 @@
dataset.getDatasetId());
ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
localResourceMetadata, LocalResource.LSMBTreeResource);
+ TreeIndexCreateOperatorDescriptor secondaryIndexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
+ secondaryBloomFilterKeyFields, new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ new SecondaryIndexOperationTrackerProvider(LSMBTreeIOOperationCallbackFactory.INSTANCE, dataset
+ .getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ storageProperties.getBloomFilterFalsePositiveRate()), localResourceFactoryProvider,
+ NoOpOperationCallbackFactory.INSTANCE);
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,
+ secondaryPartitionConstraint);
+ spec.addRoot(secondaryIndexCreateOp);
+ spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+ return spec;
+ }
- TreeIndexCreateOperatorDescriptor secondaryIndexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
- AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
- secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
- secondaryBloomFilterKeyFields, new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
- dataset.getDatasetId()), AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- storageProperties.getBloomFilterFalsePositiveRate()), localResourceFactoryProvider,
- NoOpOperationCallbackFactory.INSTANCE);
- AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,
- secondaryPartitionConstraint);
- spec.addRoot(secondaryIndexCreateOp);
- spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
- return spec;
- }
@Override
public JobSpecification buildLoadingJobSpec() throws AsterixException, AlgebricksException{
@@ -116,16 +118,15 @@
// Sort by secondary keys.
ExternalSortOperatorDescriptor sortOp = createSortOp(spec, secondaryComparatorFactories, secondaryRecDesc,RIDScanOpAndConstraints.second);
// Create secondary BTree bulk load op.
- AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
- TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(
- spec,
- numSecondaryKeys,
- new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER, storageProperties
- .getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
+ TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(
+ spec,
+ numSecondaryKeys,
+ new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new SecondaryIndexOperationTrackerProvider(
+ LSMBTreeIOOperationCallbackFactory.INSTANCE, dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, storageProperties
+ .getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
IBinaryHashFunctionFactory[] hashFactories = DatasetUtils.computeExternalDataKeysBinaryHashFunFactories(dataset, NonTaggedDataFormat.INSTANCE.getBinaryHashFunctionFactoryProvider());
//select partitioning keys (always the first 2 after secondary keys)
@@ -172,15 +173,15 @@
AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
// Create secondary BTree bulk load op.
- TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(
- spec,
- numSecondaryKeys,
- new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER, storageProperties
- .getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
+ TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(
+ spec,
+ numSecondaryKeys,
+ new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new SecondaryIndexOperationTrackerProvider(
+ LSMBTreeIOOperationCallbackFactory.INSTANCE, dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, storageProperties
+ .getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
// Connect the operators.
spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
@@ -197,4 +198,4 @@
return spec;
}
}
-}
\ No newline at end of file
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java
index e55ae5a..60a4451 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java
@@ -19,12 +19,15 @@
import java.io.IOException;
import java.util.List;
-
import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
+import edu.uci.ics.asterix.common.context.ITransactionSubsystemProvider;
+import edu.uci.ics.asterix.common.context.TransactionSubsystemProvider;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.transactions.IRecoveryManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.JobId;
import edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory;
import edu.uci.ics.asterix.external.adapter.factory.HiveAdapterFactory;
import edu.uci.ics.asterix.external.data.operator.ExternalDataIndexingOperatorDescriptor;
@@ -46,9 +49,12 @@
import edu.uci.ics.asterix.runtime.evaluators.functions.AndDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.IsNullDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.NotDescriptor;
+import edu.uci.ics.asterix.runtime.job.listener.JobEventListenerFactory;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexInstantSearchOperationCallbackFactory;
import edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat;
import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexOperationTrackerProvider;
import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobIdFactory;
import edu.uci.ics.asterix.translator.CompiledStatements.CompiledCreateIndexStatement;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
@@ -71,6 +77,7 @@
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.job.IJobletEventListenerFactory;
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.marshalling.IntegerSerializerDeserializer;
@@ -79,6 +86,7 @@
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.BTreeSearchOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
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.NoOpOperationCallbackFactory;
@@ -89,564 +97,298 @@
// possible index job specs,
// not just for creation.
public abstract class SecondaryIndexCreator {
- protected final PhysicalOptimizationConfig physOptConf;
- protected int numPrimaryKeys;
- protected int numSecondaryKeys;
- protected AqlMetadataProvider metadataProvider;
- protected String dataverseName;
- protected String datasetName;
- protected Dataset dataset;
- protected ARecordType itemType;
- protected ISerializerDeserializer payloadSerde;
- protected IFileSplitProvider primaryFileSplitProvider;
- protected AlgebricksPartitionConstraint primaryPartitionConstraint;
- protected List<String> secondaryKeyFields;
- protected IFileSplitProvider secondaryFileSplitProvider;
- protected AlgebricksPartitionConstraint secondaryPartitionConstraint;
- protected String secondaryIndexName;
- protected boolean anySecondaryKeyIsNullable = false;
- protected long numElementsHint;
- protected IBinaryComparatorFactory[] primaryComparatorFactories;
- protected int[] primaryBloomFilterKeyFields;
- protected RecordDescriptor primaryRecDesc;
- protected IBinaryComparatorFactory[] secondaryComparatorFactories;
- protected int[] secondaryBloomFilterKeyFields;
- protected RecordDescriptor secondaryRecDesc;
- protected ICopyEvaluatorFactory[] secondaryFieldAccessEvalFactories;
- protected IAsterixPropertiesProvider propertiesProvider;
+ protected final PhysicalOptimizationConfig physOptConf;
- // Prevent public construction. Should be created via createIndexCreator().
- protected SecondaryIndexCreator(PhysicalOptimizationConfig physOptConf,
- IAsterixPropertiesProvider propertiesProvider) {
- this.physOptConf = physOptConf;
- this.propertiesProvider = propertiesProvider;
- }
+ protected int numPrimaryKeys;
+ protected int numSecondaryKeys;
+ protected AqlMetadataProvider metadataProvider;
+ protected String dataverseName;
+ protected String datasetName;
+ protected Dataset dataset;
+ protected ARecordType itemType;
+ protected ISerializerDeserializer payloadSerde;
+ protected IFileSplitProvider primaryFileSplitProvider;
+ protected AlgebricksPartitionConstraint primaryPartitionConstraint;
+ protected IFileSplitProvider secondaryFileSplitProvider;
+ protected AlgebricksPartitionConstraint secondaryPartitionConstraint;
+ protected String secondaryIndexName;
+ protected boolean anySecondaryKeyIsNullable = false;
- public static SecondaryIndexCreator createIndexCreator(CompiledCreateIndexStatement createIndexStmt,
- AqlMetadataProvider metadataProvider, PhysicalOptimizationConfig physOptConf) throws AsterixException,
- AlgebricksException {
- IAsterixPropertiesProvider asterixPropertiesProvider = AsterixAppContextInfo.getInstance();
- SecondaryIndexCreator indexCreator = null;
- switch (createIndexStmt.getIndexType()) {
- case BTREE: {
- indexCreator = new SecondaryBTreeCreator(physOptConf, asterixPropertiesProvider);
- break;
- }
- case RTREE: {
- indexCreator = new SecondaryRTreeCreator(physOptConf, asterixPropertiesProvider);
- break;
- }
- case SINGLE_PARTITION_WORD_INVIX:
- case SINGLE_PARTITION_NGRAM_INVIX:
- case LENGTH_PARTITIONED_WORD_INVIX:
- case LENGTH_PARTITIONED_NGRAM_INVIX: {
- indexCreator = new SecondaryInvertedIndexCreator(physOptConf, asterixPropertiesProvider);
- break;
- }
- default: {
- throw new AsterixException("Unknown Index Type: " + createIndexStmt.getIndexType());
- }
- }
- indexCreator.init(createIndexStmt, metadataProvider);
- return indexCreator;
- }
+ protected long numElementsHint;
+ protected IBinaryComparatorFactory[] primaryComparatorFactories;
+ protected int[] primaryBloomFilterKeyFields;
+ protected RecordDescriptor primaryRecDesc;
+ protected IBinaryComparatorFactory[] secondaryComparatorFactories;
+ protected int[] secondaryBloomFilterKeyFields;
+ protected RecordDescriptor secondaryRecDesc;
+ protected ICopyEvaluatorFactory[] secondaryFieldAccessEvalFactories;
- public abstract JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException;
+ protected IAsterixPropertiesProvider propertiesProvider;
- public abstract JobSpecification buildLoadingJobSpec() throws AsterixException, AlgebricksException;
+ // Prevent public construction. Should be created via createIndexCreator().
+ protected SecondaryIndexCreator(PhysicalOptimizationConfig physOptConf,
+ IAsterixPropertiesProvider propertiesProvider) {
+ this.physOptConf = physOptConf;
+ this.propertiesProvider = propertiesProvider;
+ }
- protected void init(CompiledCreateIndexStatement createIndexStmt, AqlMetadataProvider metadataProvider)
- throws AsterixException, AlgebricksException {
- this.metadataProvider = metadataProvider;
- dataverseName = createIndexStmt.getDataverseName() == null ? metadataProvider.getDefaultDataverseName()
- : createIndexStmt.getDataverseName();
- datasetName = createIndexStmt.getDatasetName();
- secondaryIndexName = createIndexStmt.getIndexName();
- dataset = metadataProvider.findDataset(dataverseName, datasetName);
- if (dataset == null) {
- throw new AsterixException("Unknown dataset " + datasetName);
- }
- if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
- //get external dataset details
- ExternalDatasetDetails edsd = ((ExternalDatasetDetails)dataset.getDatasetDetails());
- //get adapter name
- String adapter = edsd.getAdapter();
- //if not an hdfs adapter, throw an exception
- if(!adapter.equals(HDFSAdapterFactory.HDFS_ADAPTER_NAME) && !adapter.equals(HiveAdapter.class.getName()))
- {
- throw new AsterixException("Cannot index an external dataset with adapter type(" + adapter + ").");
- }
- //get the item type
- ARecordType externalItemType = (ARecordType) metadataProvider.findType(dataset.getDataverseName(), dataset.getItemTypeName());
- //number of primary keys here depends on the file input, 3 for rcfiles and 2 for text and sequence files.
- numPrimaryKeys = DatasetUtils.getExternalRIDSize(dataset);
- itemType = createExternalItemTypeWithRID(externalItemType);
- payloadSerde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType);
- numSecondaryKeys = createIndexStmt.getKeyFields().size();
- //splits and constraints <--They don't exist-->
- primaryFileSplitProvider = null;
- primaryPartitionConstraint = null;
- //create secondary split and constraints
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadataProvider
- .splitProviderAndPartitionConstraintsForExternalDataset(dataverseName, datasetName,
- secondaryIndexName);
- secondaryFileSplitProvider = secondarySplitsAndConstraint.first;
- secondaryPartitionConstraint = secondarySplitsAndConstraint.second;
- // Must be called in this order.
- setExternalRIDDescAndComparators();
- setExternalSecondaryRecDescAndComparators(createIndexStmt, metadataProvider);
- numElementsHint = metadataProvider.getCardinalityPerPartitionHint(dataset);
- }
- else
- {
- itemType = (ARecordType) metadataProvider.findType(dataset.getDataverseName(), dataset.getItemTypeName());
- payloadSerde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType);
- numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
- numSecondaryKeys = createIndexStmt.getKeyFields().size();
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> primarySplitsAndConstraint = metadataProvider
- .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverseName, datasetName, datasetName);
- primaryFileSplitProvider = primarySplitsAndConstraint.first;
- primaryPartitionConstraint = primarySplitsAndConstraint.second;
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadataProvider
- .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverseName, datasetName,
- secondaryIndexName);
- secondaryFileSplitProvider = secondarySplitsAndConstraint.first;
- secondaryPartitionConstraint = secondarySplitsAndConstraint.second;
- // Must be called in this order.
- setPrimaryRecDescAndComparators();
- setSecondaryRecDescAndComparators(createIndexStmt, metadataProvider);
- numElementsHint = metadataProvider.getCardinalityPerPartitionHint(dataset);
- }
- }
+ public static SecondaryIndexCreator createIndexCreator(CompiledCreateIndexStatement createIndexStmt,
+ AqlMetadataProvider metadataProvider, PhysicalOptimizationConfig physOptConf) throws AsterixException,
+ AlgebricksException {
+ IAsterixPropertiesProvider asterixPropertiesProvider = AsterixAppContextInfo.getInstance();
+ SecondaryIndexCreator indexCreator = null;
+ switch (createIndexStmt.getIndexType()) {
+ case BTREE: {
+ indexCreator = new SecondaryBTreeCreator(physOptConf, asterixPropertiesProvider);
+ break;
+ }
+ case RTREE: {
+ indexCreator = new SecondaryRTreeCreator(physOptConf, asterixPropertiesProvider);
+ break;
+ }
+ case SINGLE_PARTITION_WORD_INVIX:
+ case SINGLE_PARTITION_NGRAM_INVIX:
+ case LENGTH_PARTITIONED_WORD_INVIX:
+ case LENGTH_PARTITIONED_NGRAM_INVIX: {
+ indexCreator = new SecondaryInvertedIndexCreator(physOptConf, asterixPropertiesProvider);
+ break;
+ }
+ default: {
+ throw new AsterixException("Unknown Index Type: " + createIndexStmt.getIndexType());
+ }
+ }
+ indexCreator.init(createIndexStmt, metadataProvider);
+ return indexCreator;
+ }
- protected ARecordType createSecondaryItemType(ARecordType externalItemType, boolean isRCFile) throws AsterixException
- {
+ public abstract JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException;
- String[] fieldsNames = new String[numSecondaryKeys+numPrimaryKeys];
- IAType[] fieldsTypes = new IAType[numSecondaryKeys+numPrimaryKeys];
+ public abstract JobSpecification buildLoadingJobSpec() throws AsterixException, AlgebricksException;
- //first create the secondary index fields
- for(int i=0; i<numSecondaryKeys; i++)
- {
- fieldsNames[i] = secondaryKeyFields.get(i);
- try {
- fieldsTypes[i] = externalItemType.getFieldType(fieldsNames[i]);
- } catch (IOException e) {
- // TODO Auto-generated catch block
- e.printStackTrace();
- }
- }
+ protected void init(CompiledCreateIndexStatement createIndexStmt, AqlMetadataProvider metadataProvider)
+ throws AsterixException, AlgebricksException {
+ this.metadataProvider = metadataProvider;
+ dataverseName = createIndexStmt.getDataverseName() == null ? metadataProvider.getDefaultDataverseName()
+ : createIndexStmt.getDataverseName();
+ datasetName = createIndexStmt.getDatasetName();
+ secondaryIndexName = createIndexStmt.getIndexName();
+ dataset = metadataProvider.findDataset(dataverseName, datasetName);
+ if (dataset == null) {
+ throw new AsterixException("Unknown dataset " + datasetName);
+ }
+ if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+ throw new AsterixException("Cannot index an external dataset (" + datasetName + ").");
+ }
+ itemType = (ARecordType) metadataProvider.findType(dataset.getDataverseName(), dataset.getItemTypeName());
+ payloadSerde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType);
+ numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
+ numSecondaryKeys = createIndexStmt.getKeyFields().size();
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> primarySplitsAndConstraint = metadataProvider
+ .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverseName, datasetName, datasetName);
+ primaryFileSplitProvider = primarySplitsAndConstraint.first;
+ primaryPartitionConstraint = primarySplitsAndConstraint.second;
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadataProvider
+ .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverseName, datasetName,
+ secondaryIndexName);
+ secondaryFileSplitProvider = secondarySplitsAndConstraint.first;
+ secondaryPartitionConstraint = secondarySplitsAndConstraint.second;
+ // Must be called in this order.
+ setPrimaryRecDescAndComparators();
+ setSecondaryRecDescAndComparators(createIndexStmt, metadataProvider);
+ numElementsHint = metadataProvider.getCardinalityPerPartitionHint(dataset);
+ }
- //second add RID fields (File name or number and byte location)
- if(AqlMetadataProvider.isOptimizeExternalIndexes())
- {
- fieldsNames[numSecondaryKeys] = "_file-number";
- fieldsTypes[numSecondaryKeys] = BuiltinType.ASTRING;
- }
- else
- {
- fieldsNames[numSecondaryKeys] = "_file-name";
- fieldsTypes[numSecondaryKeys] = BuiltinType.ASTRING;
- }
- fieldsNames[numSecondaryKeys+1] = "_byte-location";
- fieldsTypes[numSecondaryKeys+1] = BuiltinType.AINT64;
+ protected void setPrimaryRecDescAndComparators() throws AlgebricksException {
+ List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+ int numPrimaryKeys = partitioningKeys.size();
+ ISerializerDeserializer[] primaryRecFields = new ISerializerDeserializer[numPrimaryKeys + 1];
+ ITypeTraits[] primaryTypeTraits = new ITypeTraits[numPrimaryKeys + 1];
+ primaryComparatorFactories = new IBinaryComparatorFactory[numPrimaryKeys];
+ primaryBloomFilterKeyFields = new int[numPrimaryKeys];
+ ISerializerDeserializerProvider serdeProvider = metadataProvider.getFormat().getSerdeProvider();
+ for (int i = 0; i < numPrimaryKeys; i++) {
+ IAType keyType;
+ try {
+ keyType = itemType.getFieldType(partitioningKeys.get(i));
+ } catch (IOException e) {
+ throw new AlgebricksException(e);
+ }
+ primaryRecFields[i] = serdeProvider.getSerializerDeserializer(keyType);
+ primaryComparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(
+ keyType, true);
+ primaryTypeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
+ primaryBloomFilterKeyFields[i] = i;
+ }
+ primaryRecFields[numPrimaryKeys] = payloadSerde;
+ primaryTypeTraits[numPrimaryKeys] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(itemType);
+ primaryRecDesc = new RecordDescriptor(primaryRecFields, primaryTypeTraits);
+ }
- if(isRCFile)
- {
- fieldsNames[numSecondaryKeys+2] = "_row-Number";
- fieldsTypes[numSecondaryKeys+2] = BuiltinType.AINT32;
- }
+ protected void setSecondaryRecDescAndComparators(CompiledCreateIndexStatement createIndexStmt,
+ AqlMetadataProvider metadataProvider) throws AlgebricksException, AsterixException {
+ List<String> secondaryKeyFields = createIndexStmt.getKeyFields();
+ secondaryFieldAccessEvalFactories = new ICopyEvaluatorFactory[numSecondaryKeys];
+ secondaryComparatorFactories = new IBinaryComparatorFactory[numSecondaryKeys + numPrimaryKeys];
+ secondaryBloomFilterKeyFields = new int[numSecondaryKeys];
+ ISerializerDeserializer[] secondaryRecFields = new ISerializerDeserializer[numPrimaryKeys + numSecondaryKeys];
+ ITypeTraits[] secondaryTypeTraits = new ITypeTraits[numSecondaryKeys + numPrimaryKeys];
+ ISerializerDeserializerProvider serdeProvider = metadataProvider.getFormat().getSerdeProvider();
+ ITypeTraitProvider typeTraitProvider = metadataProvider.getFormat().getTypeTraitProvider();
+ IBinaryComparatorFactoryProvider comparatorFactoryProvider = metadataProvider.getFormat()
+ .getBinaryComparatorFactoryProvider();
+ for (int i = 0; i < numSecondaryKeys; i++) {
+ secondaryFieldAccessEvalFactories[i] = metadataProvider.getFormat().getFieldAccessEvaluatorFactory(
+ itemType, secondaryKeyFields.get(i), numPrimaryKeys);
+ Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(secondaryKeyFields.get(i), itemType);
+ IAType keyType = keyTypePair.first;
+ anySecondaryKeyIsNullable = anySecondaryKeyIsNullable || keyTypePair.second;
+ ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(keyType);
+ secondaryRecFields[i] = keySerde;
+ secondaryComparatorFactories[i] = comparatorFactoryProvider.getBinaryComparatorFactory(keyType, true);
+ secondaryTypeTraits[i] = typeTraitProvider.getTypeTrait(keyType);
+ secondaryBloomFilterKeyFields[i] = i;
+ }
+ // Add serializers and comparators for primary index fields.
+ for (int i = 0; i < numPrimaryKeys; i++) {
+ secondaryRecFields[numSecondaryKeys + i] = primaryRecDesc.getFields()[i];
+ secondaryTypeTraits[numSecondaryKeys + i] = primaryRecDesc.getTypeTraits()[i];
+ secondaryComparatorFactories[numSecondaryKeys + i] = primaryComparatorFactories[i];
+ }
+ secondaryRecDesc = new RecordDescriptor(secondaryRecFields, secondaryTypeTraits);
+ }
- //return type
- return new ARecordType(externalItemType.getTypeName(), fieldsNames, fieldsTypes, externalItemType.isOpen());
- }
- protected ARecordType createExternalItemTypeWithRID(
- ARecordType externalItemType) throws AsterixException {
+ protected AbstractOperatorDescriptor createDummyKeyProviderOp(JobSpecification spec) throws AsterixException,
+ AlgebricksException {
+ // Build dummy tuple containing one field with a dummy value inside.
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(1);
+ DataOutput dos = tb.getDataOutput();
+ tb.reset();
+ try {
+ // Serialize dummy value into a field.
+ IntegerSerializerDeserializer.INSTANCE.serialize(0, dos);
+ } catch (HyracksDataException e) {
+ throw new AsterixException(e);
+ }
+ // Add dummy field.
+ tb.addFieldEndOffset();
+ ISerializerDeserializer[] keyRecDescSers = { IntegerSerializerDeserializer.INSTANCE };
+ RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
+ ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
+ keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, keyProviderOp,
+ primaryPartitionConstraint);
+ return keyProviderOp;
+ }
- String[] fieldsNames = new String[externalItemType.getFieldNames().length+numPrimaryKeys];
- IAType[] fieldsTypes = new IAType[externalItemType.getFieldTypes().length+numPrimaryKeys];
+ protected BTreeSearchOperatorDescriptor createPrimaryIndexScanOp(JobSpecification spec) throws AlgebricksException {
+ // -Infinity
+ int[] lowKeyFields = null;
+ // +Infinity
+ int[] highKeyFields = null;
+ AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
+ BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
+ AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+ primaryFileSplitProvider, primaryRecDesc.getTypeTraits(), primaryComparatorFactories,
+ primaryBloomFilterKeyFields, lowKeyFields, highKeyFields, true, true,
+ new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+ new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER, storageProperties
+ .getBloomFilterFalsePositiveRate()), false,
+ NoOpOperationCallbackFactory.INSTANCE);
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, primarySearchOp,
+ primaryPartitionConstraint);
+ return primarySearchOp;
+ }
- //add RID fields names and types
- if(AqlMetadataProvider.isOptimizeExternalIndexes())
- {
- fieldsNames[0] = "_file-number";
- fieldsTypes[0] = BuiltinType.AINT32;
- }
- else
- {
- fieldsNames[0] = "_file-name";
- fieldsTypes[0] = BuiltinType.ASTRING;
- }
- fieldsNames[1] = "_byte-location";
- fieldsTypes[1] = BuiltinType.AINT64;
- if(numPrimaryKeys == 3)
- {
- //add the row number for rc files
- fieldsNames[2] = "_row-number";
- fieldsTypes[2] = BuiltinType.AINT32;
- }
-
- //add the original fields names and types
- for(int i=0; i < externalItemType.getFieldNames().length; i++)
- {
- fieldsNames[i+numPrimaryKeys] = externalItemType.getFieldNames()[i];
- fieldsTypes[i+numPrimaryKeys] = externalItemType.getFieldTypes()[i];
- }
- return new ARecordType(externalItemType.getTypeName(), fieldsNames, fieldsTypes, externalItemType.isOpen());
- }
+ protected AlgebricksMetaOperatorDescriptor createAssignOp(JobSpecification spec,
+ BTreeSearchOperatorDescriptor primaryScanOp, int numSecondaryKeyFields) throws AlgebricksException {
+ int[] outColumns = new int[numSecondaryKeyFields];
+ int[] projectionList = new int[numSecondaryKeyFields + numPrimaryKeys];
+ for (int i = 0; i < numSecondaryKeyFields; i++) {
+ outColumns[i] = numPrimaryKeys + i + 1;
+ }
+ int projCount = 0;
+ for (int i = 0; i < numSecondaryKeyFields; i++) {
+ projectionList[projCount++] = numPrimaryKeys + i + 1;
+ }
+ for (int i = 0; i < numPrimaryKeys; i++) {
+ projectionList[projCount++] = i;
+ }
+ IScalarEvaluatorFactory[] sefs = new IScalarEvaluatorFactory[secondaryFieldAccessEvalFactories.length];
+ for (int i = 0; i < secondaryFieldAccessEvalFactories.length; ++i) {
+ sefs[i] = new LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.ScalarEvaluatorFactoryAdapter(
+ secondaryFieldAccessEvalFactories[i]);
+ }
+ AssignRuntimeFactory assign = new AssignRuntimeFactory(outColumns, sefs, projectionList);
+ AlgebricksMetaOperatorDescriptor asterixAssignOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
+ new IPushRuntimeFactory[] { assign }, new RecordDescriptor[] { secondaryRecDesc });
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, asterixAssignOp,
+ primaryPartitionConstraint);
+ return asterixAssignOp;
+ }
- protected void setExternalRIDDescAndComparators() throws AlgebricksException {
+ protected ExternalSortOperatorDescriptor createSortOp(JobSpecification spec,
+ IBinaryComparatorFactory[] secondaryComparatorFactories, RecordDescriptor secondaryRecDesc) {
+ int[] sortFields = new int[secondaryComparatorFactories.length];
+ for (int i = 0; i < secondaryComparatorFactories.length; i++) {
+ sortFields[i] = i;
+ }
+ ExternalSortOperatorDescriptor sortOp = new ExternalSortOperatorDescriptor(spec,
+ physOptConf.getMaxFramesExternalSort(), sortFields, secondaryComparatorFactories, secondaryRecDesc);
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, sortOp, primaryPartitionConstraint);
+ return sortOp;
+ }
- ISerializerDeserializer[] externalRecFields = new ISerializerDeserializer[itemType.getFieldNames().length];
- ITypeTraits[] externalTypeTraits = new ITypeTraits[itemType.getFieldNames().length];
+ protected TreeIndexBulkLoadOperatorDescriptor createTreeIndexBulkLoadOp(JobSpecification spec,
+ int numSecondaryKeyFields, IIndexDataflowHelperFactory dataflowHelperFactory, float fillFactor)
+ throws MetadataException, AlgebricksException {
+ int[] fieldPermutation = new int[numSecondaryKeyFields + numPrimaryKeys];
+ for (int i = 0; i < numSecondaryKeyFields + numPrimaryKeys; i++) {
+ fieldPermutation[i] = i;
+ }
+ TreeIndexBulkLoadOperatorDescriptor treeIndexBulkLoadOp = new TreeIndexBulkLoadOperatorDescriptor(spec,
+ AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
+ AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER, secondaryFileSplitProvider,
+ secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields,
+ fieldPermutation, fillFactor, false, numElementsHint, false, dataflowHelperFactory,
+ NoOpOperationCallbackFactory.INSTANCE);
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, treeIndexBulkLoadOp,
+ secondaryPartitionConstraint);
+ return treeIndexBulkLoadOp;
+ }
- primaryComparatorFactories = new IBinaryComparatorFactory[numPrimaryKeys];
- primaryBloomFilterKeyFields = new int[numPrimaryKeys];
- ISerializerDeserializerProvider serdeProvider = metadataProvider.getFormat().getSerdeProvider();
-
- if(AqlMetadataProvider.isOptimizeExternalIndexes())
- {
- primaryComparatorFactories[0] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(BuiltinType.AINT32, true);
- }
- else
- {
- primaryComparatorFactories[0] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(BuiltinType.ASTRING, true);
- }
- primaryComparatorFactories[1] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(BuiltinType.AINT64, true);
-
- primaryBloomFilterKeyFields[0]=0;
- primaryBloomFilterKeyFields[1]=1;
-
- if(numPrimaryKeys == 3)
- {
- primaryComparatorFactories[2] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(BuiltinType.AINT32, true);
- primaryBloomFilterKeyFields[2]=2;
- }
-
- for(int i=0; i < itemType.getFieldNames().length; i++)
- {
- externalRecFields[i] = serdeProvider.getSerializerDeserializer(itemType.getFieldTypes()[i]);
- externalTypeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(itemType.getFieldTypes()[i]);
- }
- primaryRecDesc = new RecordDescriptor(externalRecFields, externalTypeTraits);
- }
-
- protected void setPrimaryRecDescAndComparators() throws AlgebricksException {
- List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
- int numPrimaryKeys = partitioningKeys.size();
- ISerializerDeserializer[] primaryRecFields = new ISerializerDeserializer[numPrimaryKeys + 1];
- ITypeTraits[] primaryTypeTraits = new ITypeTraits[numPrimaryKeys + 1];
- primaryComparatorFactories = new IBinaryComparatorFactory[numPrimaryKeys];
- primaryBloomFilterKeyFields = new int[numPrimaryKeys];
- ISerializerDeserializerProvider serdeProvider = metadataProvider.getFormat().getSerdeProvider();
- for (int i = 0; i < numPrimaryKeys; i++) {
- IAType keyType;
- try {
- keyType = itemType.getFieldType(partitioningKeys.get(i));
- } catch (IOException e) {
- throw new AlgebricksException(e);
- }
- primaryRecFields[i] = serdeProvider.getSerializerDeserializer(keyType);
- primaryComparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(
- keyType, true);
- primaryTypeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
- primaryBloomFilterKeyFields[i] = i;
- }
- primaryRecFields[numPrimaryKeys] = payloadSerde;
- primaryTypeTraits[numPrimaryKeys] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(itemType);
- primaryRecDesc = new RecordDescriptor(primaryRecFields, primaryTypeTraits);
- }
-
- protected void setSecondaryRecDescAndComparators(CompiledCreateIndexStatement createIndexStmt,
- AqlMetadataProvider metadataProvider) throws AlgebricksException, AsterixException {
- secondaryKeyFields = createIndexStmt.getKeyFields();
- secondaryFieldAccessEvalFactories = new ICopyEvaluatorFactory[numSecondaryKeys];
- secondaryComparatorFactories = new IBinaryComparatorFactory[numSecondaryKeys + numPrimaryKeys];
- secondaryBloomFilterKeyFields = new int[numSecondaryKeys];
- ISerializerDeserializer[] secondaryRecFields = new ISerializerDeserializer[numPrimaryKeys + numSecondaryKeys];
- ITypeTraits[] secondaryTypeTraits = new ITypeTraits[numSecondaryKeys + numPrimaryKeys];
- ISerializerDeserializerProvider serdeProvider = metadataProvider.getFormat().getSerdeProvider();
- ITypeTraitProvider typeTraitProvider = metadataProvider.getFormat().getTypeTraitProvider();
- IBinaryComparatorFactoryProvider comparatorFactoryProvider = metadataProvider.getFormat()
- .getBinaryComparatorFactoryProvider();
- for (int i = 0; i < numSecondaryKeys; i++) {
- secondaryFieldAccessEvalFactories[i] = metadataProvider.getFormat().getFieldAccessEvaluatorFactory(
- itemType, secondaryKeyFields.get(i), numPrimaryKeys);
- Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(secondaryKeyFields.get(i), itemType);
- IAType keyType = keyTypePair.first;
- anySecondaryKeyIsNullable = anySecondaryKeyIsNullable || keyTypePair.second;
- ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(keyType);
- secondaryRecFields[i] = keySerde;
- secondaryComparatorFactories[i] = comparatorFactoryProvider.getBinaryComparatorFactory(keyType, true);
- secondaryTypeTraits[i] = typeTraitProvider.getTypeTrait(keyType);
- secondaryBloomFilterKeyFields[i] = i;
- }
- // Add serializers and comparators for primary index fields.
- for (int i = 0; i < numPrimaryKeys; i++) {
- secondaryRecFields[numSecondaryKeys + i] = primaryRecDesc.getFields()[i];
- secondaryTypeTraits[numSecondaryKeys + i] = primaryRecDesc.getTypeTraits()[i];
- secondaryComparatorFactories[numSecondaryKeys + i] = primaryComparatorFactories[i];
- }
- secondaryRecDesc = new RecordDescriptor(secondaryRecFields, secondaryTypeTraits);
- }
-
- protected void setExternalSecondaryRecDescAndComparators(CompiledCreateIndexStatement createIndexStmt,
- AqlMetadataProvider metadataProvider) throws AlgebricksException, AsterixException {
- secondaryKeyFields = createIndexStmt.getKeyFields();
- secondaryFieldAccessEvalFactories = new ICopyEvaluatorFactory[numSecondaryKeys+ numPrimaryKeys];
- secondaryComparatorFactories = new IBinaryComparatorFactory[numSecondaryKeys + numPrimaryKeys];
- secondaryBloomFilterKeyFields = new int[numSecondaryKeys];
- ISerializerDeserializer[] secondaryRecFields = new ISerializerDeserializer[numPrimaryKeys + numSecondaryKeys];
- ITypeTraits[] secondaryTypeTraits = new ITypeTraits[numSecondaryKeys + numPrimaryKeys];
- ISerializerDeserializerProvider serdeProvider = metadataProvider.getFormat().getSerdeProvider();
- ITypeTraitProvider typeTraitProvider = metadataProvider.getFormat().getTypeTraitProvider();
- IBinaryComparatorFactoryProvider comparatorFactoryProvider = metadataProvider.getFormat()
- .getBinaryComparatorFactoryProvider();
-
- for (int i = 0; i < numSecondaryKeys; i++) {
- secondaryFieldAccessEvalFactories[i] = metadataProvider.getFormat().getFieldAccessEvaluatorFactory(
- itemType, secondaryKeyFields.get(i), 0);
- Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(secondaryKeyFields.get(i), itemType);
- IAType keyType = keyTypePair.first;
- anySecondaryKeyIsNullable = anySecondaryKeyIsNullable || keyTypePair.second;
- ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(keyType);
- secondaryRecFields[i] = keySerde;
- secondaryComparatorFactories[i] = comparatorFactoryProvider.getBinaryComparatorFactory(keyType, true);
- secondaryTypeTraits[i] = typeTraitProvider.getTypeTrait(keyType);
- secondaryBloomFilterKeyFields[i] = i;
- }
-
- if(AqlMetadataProvider.isOptimizeExternalIndexes())
- {
- secondaryFieldAccessEvalFactories[numSecondaryKeys] = metadataProvider.getFormat().getFieldAccessEvaluatorFactory(
- itemType, "_file-number", 0);
- }
- else
- {
- secondaryFieldAccessEvalFactories[numSecondaryKeys] = metadataProvider.getFormat().getFieldAccessEvaluatorFactory(
- itemType, "_file-name", 0);
- }
- secondaryFieldAccessEvalFactories[numSecondaryKeys+1] = metadataProvider.getFormat().getFieldAccessEvaluatorFactory(
- itemType, "_byte-location", 0);
- if(numPrimaryKeys == 3)
- {
- secondaryFieldAccessEvalFactories[numSecondaryKeys+2] = metadataProvider.getFormat().getFieldAccessEvaluatorFactory(
- itemType, "_row-number", 0);
- }
-
- for (int i = 0; i < numPrimaryKeys; i++) {
- secondaryRecFields[numSecondaryKeys + i] = primaryRecDesc.getFields()[i];
- secondaryTypeTraits[numSecondaryKeys + i] = primaryRecDesc.getTypeTraits()[i];
- secondaryComparatorFactories[numSecondaryKeys + i] = primaryComparatorFactories[i];
- }
- secondaryRecDesc = new RecordDescriptor(secondaryRecFields, secondaryTypeTraits);
- }
-
- protected AbstractOperatorDescriptor createDummyKeyProviderOp(JobSpecification spec) throws AsterixException,
- AlgebricksException {
- // Build dummy tuple containing one field with a dummy value inside.
- ArrayTupleBuilder tb = new ArrayTupleBuilder(1);
- DataOutput dos = tb.getDataOutput();
- tb.reset();
- try {
- // Serialize dummy value into a field.
- IntegerSerializerDeserializer.INSTANCE.serialize(0, dos);
- } catch (HyracksDataException e) {
- throw new AsterixException(e);
- }
- // Add dummy field.
- tb.addFieldEndOffset();
- ISerializerDeserializer[] keyRecDescSers = { IntegerSerializerDeserializer.INSTANCE };
- RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
- ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
- keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
- AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, keyProviderOp,
- primaryPartitionConstraint);
- return keyProviderOp;
- }
-
- protected Pair<ExternalDataIndexingOperatorDescriptor, AlgebricksPartitionConstraint> createExternalIndexingOp(JobSpecification spec) throws Exception {
- Pair<ExternalDataIndexingOperatorDescriptor,AlgebricksPartitionConstraint> indexingOpAndConstraints = metadataProvider.buildExternalDataIndexingRuntime(spec, itemType, dataset, NonTaggedDataFormat.INSTANCE);
- AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, indexingOpAndConstraints.first,
- indexingOpAndConstraints.second);
- return indexingOpAndConstraints;
- }
-
- protected BTreeSearchOperatorDescriptor createPrimaryIndexScanOp(JobSpecification spec) throws AlgebricksException {
- // -Infinity
- int[] lowKeyFields = null;
- // +Infinity
- int[] highKeyFields = null;
- AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
- BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
- AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
- primaryFileSplitProvider, primaryRecDesc.getTypeTraits(), primaryComparatorFactories,
- primaryBloomFilterKeyFields, lowKeyFields, highKeyFields, true, true,
- new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
- new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER, storageProperties
- .getBloomFilterFalsePositiveRate()), false,
- NoOpOperationCallbackFactory.INSTANCE);
- AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, primarySearchOp,
- primaryPartitionConstraint);
- return primarySearchOp;
- }
-
- protected AlgebricksMetaOperatorDescriptor createAssignOp(JobSpecification spec,
- BTreeSearchOperatorDescriptor primaryScanOp, int numSecondaryKeyFields) throws AlgebricksException {
- int[] outColumns = new int[numSecondaryKeyFields];
- int[] projectionList = new int[numSecondaryKeyFields + numPrimaryKeys];
- for (int i = 0; i < numSecondaryKeyFields; i++) {
- outColumns[i] = numPrimaryKeys + i + 1;
- }
- int projCount = 0;
- for (int i = 0; i < numSecondaryKeyFields; i++) {
- projectionList[projCount++] = numPrimaryKeys + i + 1;
- }
- for (int i = 0; i < numPrimaryKeys; i++) {
- projectionList[projCount++] = i;
- }
- IScalarEvaluatorFactory[] sefs = new IScalarEvaluatorFactory[secondaryFieldAccessEvalFactories.length];
- for (int i = 0; i < secondaryFieldAccessEvalFactories.length; ++i) {
- sefs[i] = new LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.ScalarEvaluatorFactoryAdapter(
- secondaryFieldAccessEvalFactories[i]);
- }
- AssignRuntimeFactory assign = new AssignRuntimeFactory(outColumns, sefs, projectionList);
- AlgebricksMetaOperatorDescriptor asterixAssignOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
- new IPushRuntimeFactory[] { assign }, new RecordDescriptor[] { secondaryRecDesc });
- AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, asterixAssignOp,
- primaryPartitionConstraint);
- return asterixAssignOp;
- }
-
- protected AlgebricksMetaOperatorDescriptor createExternalAssignOp(JobSpecification spec) throws AlgebricksException {
- int[] outColumns = new int[numSecondaryKeys + numPrimaryKeys];
- int[] projectionList = new int[numSecondaryKeys + numPrimaryKeys];
- for (int i = 0; i < numSecondaryKeys + numPrimaryKeys; i++) {
- outColumns[i] = i;
- projectionList[i] = i;
- }
-
- IScalarEvaluatorFactory[] sefs = new IScalarEvaluatorFactory[secondaryFieldAccessEvalFactories.length];
- for (int i = 0; i < secondaryFieldAccessEvalFactories.length; ++i) {
- sefs[i] = new LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.ScalarEvaluatorFactoryAdapter(
- secondaryFieldAccessEvalFactories[i]);
- }
- AssignRuntimeFactory assign = new AssignRuntimeFactory(outColumns, sefs, projectionList);
- AlgebricksMetaOperatorDescriptor asterixAssignOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
- new IPushRuntimeFactory[] { assign }, new RecordDescriptor[] { secondaryRecDesc });
- return asterixAssignOp;
- }
-
- protected ExternalSortOperatorDescriptor createSortOp(JobSpecification spec,
- IBinaryComparatorFactory[] secondaryComparatorFactories, RecordDescriptor secondaryRecDesc) {
- int[] sortFields = new int[secondaryComparatorFactories.length];
- for (int i = 0; i < secondaryComparatorFactories.length; i++) {
- sortFields[i] = i;
- }
- ExternalSortOperatorDescriptor sortOp = new ExternalSortOperatorDescriptor(spec,
- physOptConf.getMaxFramesExternalSort(), sortFields, secondaryComparatorFactories, secondaryRecDesc);
- AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, sortOp, primaryPartitionConstraint);
- return sortOp;
- }
-
- protected ExternalSortOperatorDescriptor createSortOp(JobSpecification spec,
- IBinaryComparatorFactory[] secondaryComparatorFactories, RecordDescriptor secondaryRecDesc,
- AlgebricksPartitionConstraint partitionConstraints) {
- int[] sortFields = new int[secondaryComparatorFactories.length];
- for (int i = 0; i < secondaryComparatorFactories.length; i++) {
- sortFields[i] = i;
- }
- ExternalSortOperatorDescriptor sortOp = new ExternalSortOperatorDescriptor(spec,
- physOptConf.getMaxFramesExternalSort(), sortFields, secondaryComparatorFactories, secondaryRecDesc);
- AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, sortOp, partitionConstraints);
- return sortOp;
- }
-
- protected TreeIndexBulkLoadOperatorDescriptor createTreeIndexBulkLoadOp(JobSpecification spec,
- int numSecondaryKeyFields, IIndexDataflowHelperFactory dataflowHelperFactory, float fillFactor)
- throws MetadataException, AlgebricksException {
- int[] fieldPermutation = new int[numSecondaryKeyFields + numPrimaryKeys];
- for (int i = 0; i < numSecondaryKeyFields + numPrimaryKeys; i++) {
- fieldPermutation[i] = i;
- }
- TreeIndexBulkLoadOperatorDescriptor treeIndexBulkLoadOp = new TreeIndexBulkLoadOperatorDescriptor(spec,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER, secondaryFileSplitProvider,
- secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields,
- fieldPermutation, fillFactor, false, numElementsHint, false, dataflowHelperFactory,
- NoOpOperationCallbackFactory.INSTANCE);
- AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, treeIndexBulkLoadOp,
- secondaryPartitionConstraint);
- return treeIndexBulkLoadOp;
- }
-
- public AlgebricksMetaOperatorDescriptor createFilterNullsSelectOp(JobSpecification spec, int numSecondaryKeyFields,AlgebricksPartitionConstraint partitionConstraints)
- throws AlgebricksException {
- ICopyEvaluatorFactory[] andArgsEvalFactories = new ICopyEvaluatorFactory[numSecondaryKeyFields];
- NotDescriptor notDesc = new NotDescriptor();
- IsNullDescriptor isNullDesc = new IsNullDescriptor();
- for (int i = 0; i < numSecondaryKeyFields; i++) {
- // Access column i, and apply 'is not null'.
- ColumnAccessEvalFactory columnAccessEvalFactory = new ColumnAccessEvalFactory(i);
- ICopyEvaluatorFactory isNullEvalFactory = isNullDesc
- .createEvaluatorFactory(new ICopyEvaluatorFactory[] { columnAccessEvalFactory });
- ICopyEvaluatorFactory notEvalFactory = notDesc
- .createEvaluatorFactory(new ICopyEvaluatorFactory[] { isNullEvalFactory });
- andArgsEvalFactories[i] = notEvalFactory;
- }
- ICopyEvaluatorFactory selectCond = null;
- if (numSecondaryKeyFields > 1) {
- // Create conjunctive condition where all secondary index keys must
- // satisfy 'is not null'.
- AndDescriptor andDesc = new AndDescriptor();
- selectCond = andDesc.createEvaluatorFactory(andArgsEvalFactories);
- } else {
- selectCond = andArgsEvalFactories[0];
- }
- StreamSelectRuntimeFactory select = new StreamSelectRuntimeFactory(
- new LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.ScalarEvaluatorFactoryAdapter(selectCond),
- null, AqlBinaryBooleanInspectorImpl.FACTORY);
- AlgebricksMetaOperatorDescriptor asterixSelectOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
- new IPushRuntimeFactory[] { select }, new RecordDescriptor[] { secondaryRecDesc });
- AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, asterixSelectOp,
- partitionConstraints);
- return asterixSelectOp;
- }
-
-
- public AlgebricksMetaOperatorDescriptor createFilterNullsSelectOp(JobSpecification spec, int numSecondaryKeyFields)
- throws AlgebricksException {
- ICopyEvaluatorFactory[] andArgsEvalFactories = new ICopyEvaluatorFactory[numSecondaryKeyFields];
- NotDescriptor notDesc = new NotDescriptor();
- IsNullDescriptor isNullDesc = new IsNullDescriptor();
- for (int i = 0; i < numSecondaryKeyFields; i++) {
- // Access column i, and apply 'is not null'.
- ColumnAccessEvalFactory columnAccessEvalFactory = new ColumnAccessEvalFactory(i);
- ICopyEvaluatorFactory isNullEvalFactory = isNullDesc
- .createEvaluatorFactory(new ICopyEvaluatorFactory[] { columnAccessEvalFactory });
- ICopyEvaluatorFactory notEvalFactory = notDesc
- .createEvaluatorFactory(new ICopyEvaluatorFactory[] { isNullEvalFactory });
- andArgsEvalFactories[i] = notEvalFactory;
- }
- ICopyEvaluatorFactory selectCond = null;
- if (numSecondaryKeyFields > 1) {
- // Create conjunctive condition where all secondary index keys must
- // satisfy 'is not null'.
- AndDescriptor andDesc = new AndDescriptor();
- selectCond = andDesc.createEvaluatorFactory(andArgsEvalFactories);
- } else {
- selectCond = andArgsEvalFactories[0];
- }
- StreamSelectRuntimeFactory select = new StreamSelectRuntimeFactory(
- new LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.ScalarEvaluatorFactoryAdapter(selectCond),
- null, AqlBinaryBooleanInspectorImpl.FACTORY);
- AlgebricksMetaOperatorDescriptor asterixSelectOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
- new IPushRuntimeFactory[] { select }, new RecordDescriptor[] { secondaryRecDesc });
- AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, asterixSelectOp,
- primaryPartitionConstraint);
- return asterixSelectOp;
- }
-}
\ No newline at end of file
+ public AlgebricksMetaOperatorDescriptor createFilterNullsSelectOp(JobSpecification spec, int numSecondaryKeyFields)
+ throws AlgebricksException {
+ ICopyEvaluatorFactory[] andArgsEvalFactories = new ICopyEvaluatorFactory[numSecondaryKeyFields];
+ NotDescriptor notDesc = new NotDescriptor();
+ IsNullDescriptor isNullDesc = new IsNullDescriptor();
+ for (int i = 0; i < numSecondaryKeyFields; i++) {
+ // Access column i, and apply 'is not null'.
+ ColumnAccessEvalFactory columnAccessEvalFactory = new ColumnAccessEvalFactory(i);
+ ICopyEvaluatorFactory isNullEvalFactory = isNullDesc
+ .createEvaluatorFactory(new ICopyEvaluatorFactory[] { columnAccessEvalFactory });
+ ICopyEvaluatorFactory notEvalFactory = notDesc
+ .createEvaluatorFactory(new ICopyEvaluatorFactory[] { isNullEvalFactory });
+ andArgsEvalFactories[i] = notEvalFactory;
+ }
+ ICopyEvaluatorFactory selectCond = null;
+ if (numSecondaryKeyFields > 1) {
+ // Create conjunctive condition where all secondary index keys must
+ // satisfy 'is not null'.
+ AndDescriptor andDesc = new AndDescriptor();
+ selectCond = andDesc.createEvaluatorFactory(andArgsEvalFactories);
+ } else {
+ selectCond = andArgsEvalFactories[0];
+ }
+ StreamSelectRuntimeFactory select = new StreamSelectRuntimeFactory(
+ new LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.ScalarEvaluatorFactoryAdapter(selectCond),
+ null, AqlBinaryBooleanInspectorImpl.FACTORY);
+ AlgebricksMetaOperatorDescriptor asterixSelectOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
+ new IPushRuntimeFactory[] { select }, new RecordDescriptor[] { secondaryRecDesc });
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, asterixSelectOp,
+ primaryPartitionConstraint);
+ return asterixSelectOp;
+ }
+}
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 70124e8..6f64aa2 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
@@ -22,11 +22,13 @@
import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
import edu.uci.ics.asterix.metadata.entities.Index;
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
import edu.uci.ics.asterix.runtime.formats.FormatUtils;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
import edu.uci.ics.asterix.transaction.management.resource.LSMInvertedIndexLocalResourceMetadata;
import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
@@ -176,8 +178,8 @@
IIndexDataflowHelperFactory dataflowHelperFactory = createDataflowHelperFactory();
LSMInvertedIndexCreateOperatorDescriptor invIndexCreateOp = new LSMInvertedIndexCreateOperatorDescriptor(spec,
- AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, secondaryFileSplitProvider,
- AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, tokenTypeTraits, tokenComparatorFactories,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, tokenTypeTraits, tokenComparatorFactories,
invListsTypeTraits, primaryComparatorFactories, tokenizerFactory, dataflowHelperFactory,
localResourceFactoryProvider, NoOpOperationCallbackFactory.INSTANCE);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, invIndexCreateOp,
@@ -267,8 +269,8 @@
IIndexDataflowHelperFactory dataflowHelperFactory = createDataflowHelperFactory();
LSMInvertedIndexBulkLoadOperatorDescriptor invIndexBulkLoadOp = new LSMInvertedIndexBulkLoadOperatorDescriptor(
spec, fieldPermutation, false, numElementsHint, false,
- AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, secondaryFileSplitProvider,
- AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, tokenTypeTraits, tokenComparatorFactories,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, tokenTypeTraits, tokenComparatorFactories,
invListsTypeTraits, primaryComparatorFactories, tokenizerFactory, dataflowHelperFactory,
NoOpOperationCallbackFactory.INSTANCE);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, invIndexBulkLoadOp,
@@ -280,17 +282,17 @@
AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
if (!isPartitioned) {
return new LSMInvertedIndexDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
- dataset.getDatasetId()), AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ new SecondaryIndexOperationTrackerProvider(LSMInvertedIndexIOOperationCallbackFactory.INSTANCE,
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
storageProperties.getBloomFilterFalsePositiveRate());
} else {
return new PartitionedLSMInvertedIndexDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
- dataset.getDatasetId()), AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ new SecondaryIndexOperationTrackerProvider(LSMInvertedIndexIOOperationCallbackFactory.INSTANCE,
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
storageProperties.getBloomFilterFalsePositiveRate());
}
}
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 cc8eb1b..c91903f 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
@@ -22,6 +22,7 @@
import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
import edu.uci.ics.asterix.dataflow.data.nontagged.valueproviders.AqlPrimitiveValueProviderFactory;
import edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory;
import edu.uci.ics.asterix.external.data.operator.ExternalDataIndexingOperatorDescriptor;
@@ -37,6 +38,7 @@
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
import edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
import edu.uci.ics.asterix.transaction.management.resource.LSMRTreeLocalResourceMetadata;
import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
@@ -95,14 +97,14 @@
localResourceMetadata, LocalResource.LSMRTreeResource);
TreeIndexCreateOperatorDescriptor secondaryIndexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
- AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, null,
new LSMRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
primaryComparatorFactories, new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, AqlMetadataProvider.proposeLinearizer(
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new SecondaryIndexOperationTrackerProvider(
+ LSMRTreeIOOperationCallbackFactory.INSTANCE, dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AqlMetadataProvider.proposeLinearizer(
keyType, secondaryComparatorFactories.length), storageProperties
.getBloomFilterFalsePositiveRate()), localResourceFactoryProvider,
NoOpOperationCallbackFactory.INSTANCE);
@@ -298,10 +300,10 @@
numNestedSecondaryKeyFields,
new LSMRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
primaryComparatorFactories, new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, AqlMetadataProvider.proposeLinearizer(
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new SecondaryIndexOperationTrackerProvider(
+ LSMRTreeIOOperationCallbackFactory.INSTANCE, dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AqlMetadataProvider.proposeLinearizer(
keyType, secondaryComparatorFactories.length), storageProperties
.getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
diff --git a/asterix-app/src/main/resources/asterix-build-configuration.xml b/asterix-app/src/main/resources/asterix-build-configuration.xml
index 6a6332d..9c4d15b 100644
--- a/asterix-app/src/main/resources/asterix-build-configuration.xml
+++ b/asterix-app/src/main/resources/asterix-build-configuration.xml
@@ -1,50 +1,40 @@
-<!--
- ! Copyright 2009-2013 by The Regents of the University of California
- ! Licensed under the Apache License, Version 2.0 (the "License");
- ! you may not use this file except in compliance with the License.
- ! you may obtain a copy of the License from
- !
- ! http://www.apache.org/licenses/LICENSE-2.0
- !
- ! Unless required by applicable law or agreed to in writing, software
- ! distributed under the License is distributed on an "AS IS" BASIS,
- ! WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- ! See the License for the specific language governing permissions and
- ! limitations under the License.
- !-->
+<!-- ! Copyright 2009-2013 by The Regents of the University of California
+ ! Licensed under the Apache License, Version 2.0 (the "License"); ! you may
+ not use this file except in compliance with the License. ! you may obtain
+ a copy of the License from ! ! http://www.apache.org/licenses/LICENSE-2.0
+ ! ! Unless required by applicable law or agreed to in writing, software !
+ distributed under the License is distributed on an "AS IS" BASIS, ! WITHOUT
+ WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. ! See the
+ License for the specific language governing permissions and ! limitations
+ under the License. ! -->
<asterixConfiguration xmlns="asterixconf">
- <metadataNode>nc1</metadataNode>
- <store>
- <ncId>nc1</ncId>
- <storeDirs>nc1data</storeDirs>
- </store>
- <store>
- <ncId>nc2</ncId>
- <storeDirs>nc2data</storeDirs>
- </store>
- <transactionLogDir>
- <ncId>nc1</ncId>
- <txnLogDirPath>target/txnLogDir/nc1</txnLogDirPath>
- </transactionLogDir>
- <transactionLogDir>
- <ncId>nc2</ncId>
- <txnLogDirPath>target/txnLogDir/nc2</txnLogDirPath>
- </transactionLogDir>
- <property>
- <name>log.level</name>
- <value>WARNING</value>
- <description>Log level for running tests/build</description>
- </property>
- <property>
+ <metadataNode>nc1</metadataNode>
+ <store>
+ <ncId>nc1</ncId>
+ <storeDirs>nc1data</storeDirs>
+ </store>
+ <store>
+ <ncId>nc2</ncId>
+ <storeDirs>nc2data</storeDirs>
+ </store>
+ <transactionLogDir>
+ <ncId>nc1</ncId>
+ <txnLogDirPath>target/txnLogDir/nc1</txnLogDirPath>
+ </transactionLogDir>
+ <transactionLogDir>
+ <ncId>nc2</ncId>
+ <txnLogDirPath>target/txnLogDir/nc2</txnLogDirPath>
+ </transactionLogDir>
+ <property>
+ <name>log.level</name>
+ <value>WARNING</value>
+ <description>Log level for running tests/build</description>
+ </property>
+ <property>
<name>storage.memorycomponent.numpages</name>
<value>8</value>
<description>The number of pages to allocate for a memory component.
(Default = 8)
</description>
</property>
- <property>
- <name>txn.log.groupcommitinterval</name>
- <value>1</value>
- <description>The group commit wait time in milliseconds.</description>
- </property>
</asterixConfiguration>
diff --git a/asterix-app/src/test/resources/runtimets/queries/custord/denorm-cust-order_02/denorm-cust-order_02.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/custord/denorm-cust-order_02/denorm-cust-order_02.2.update.aql
index 8faaa98..a3f070b 100644
--- a/asterix-app/src/test/resources/runtimets/queries/custord/denorm-cust-order_02/denorm-cust-order_02.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/custord/denorm-cust-order_02/denorm-cust-order_02.2.update.aql
@@ -2,11 +2,11 @@
load dataset Customers2
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/custord-tiny/customer-tiny.adm"),("format"="adm")) pre-sorted;
+(("path"="nc1://data/custord-tiny/customer-tiny.adm"),("format"="adm"));
load dataset Orders2
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/custord-tiny/order-tiny.adm"),("format"="adm")) pre-sorted;
+(("path"="nc1://data/custord-tiny/order-tiny.adm"),("format"="adm"));
insert into dataset CustomerOrders2 (
diff --git a/asterix-app/src/test/resources/runtimets/queries/custord/load-test/load-test.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/custord/load-test/load-test.2.update.aql
index a395f03..c0ab30f 100644
--- a/asterix-app/src/test/resources/runtimets/queries/custord/load-test/load-test.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/custord/load-test/load-test.2.update.aql
@@ -2,5 +2,5 @@
load dataset c1
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/custord-tiny/customer-tiny.adm"),("format"="adm")) pre-sorted;
+(("path"="nc1://data/custord-tiny/customer-tiny.adm"),("format"="adm"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.2.update.aql
index cf7b309..29f6242 100644
--- a/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.2.update.aql
@@ -8,8 +8,6 @@
use dataverse test;
-use dataverse test;
-
load dataset test.employee
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
(("path"="nc1://data/names.adm"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/load/csv_01/csv_01.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/load/csv_01/csv_01.1.ddl.aql
new file mode 100644
index 0000000..e890942
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/load/csv_01/csv_01.1.ddl.aql
@@ -0,0 +1,22 @@
+/**
+ *
+ * CSV file loading test
+ * Expected result: success
+ *
+ */
+
+drop dataverse temp if exists;
+create dataverse temp
+use dataverse temp;
+
+create type test as closed {
+ id: int32,
+ float: float?,
+ double: double?,
+ date: string?,
+ time: string?,
+ datetime: string?
+};
+
+create dataset testds (test)
+primary key id;
diff --git a/asterix-app/src/test/resources/runtimets/queries/load/csv_01/csv_01.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/load/csv_01/csv_01.2.update.aql
new file mode 100644
index 0000000..c3161d5
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/load/csv_01/csv_01.2.update.aql
@@ -0,0 +1,12 @@
+/**
+ *
+ * CSV file loading test
+ * Expected result: success
+ *
+ */
+
+use dataverse temp;
+
+load dataset testds
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/csv/sample_01.csv"),("format"="delimited-text"),("delimiter"=","));
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/load/csv_01/csv_01.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/load/csv_01/csv_01.3.query.aql
new file mode 100644
index 0000000..efa6dbc
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/load/csv_01/csv_01.3.query.aql
@@ -0,0 +1,18 @@
+/**
+ *
+ * CSV file loading test
+ * Expected result: success
+ *
+ */
+
+use dataverse temp;
+
+for $i in dataset testds
+order by $i.id
+return { "id": $i.id,
+ "float": $i.float,
+ "double": $i.double,
+ "date-before": $i.date, "date-after": date($i.date),
+ "time-before": $i.time, "time-after": time($i.time),
+ "datetime-before": $i.datetime, "datetime-after": datetime($i.datetime)
+ }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/load/csv_02/csv_02.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/load/csv_02/csv_02.1.ddl.aql
new file mode 100644
index 0000000..b6884a8
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/load/csv_02/csv_02.1.ddl.aql
@@ -0,0 +1,22 @@
+/**
+ *
+ * CSV file loading test
+ * Expected result: success
+ *
+ */
+
+drop dataverse temp if exists;
+create dataverse temp
+use dataverse temp;
+
+create type test as closed {
+ id: int32,
+ float: float?,
+ double: double?,
+ date: string,
+ time: string,
+ datetime: string
+};
+
+create dataset testds (test)
+primary key id;
diff --git a/asterix-app/src/test/resources/runtimets/queries/load/csv_02/csv_02.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/load/csv_02/csv_02.2.update.aql
new file mode 100644
index 0000000..c3161d5
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/load/csv_02/csv_02.2.update.aql
@@ -0,0 +1,12 @@
+/**
+ *
+ * CSV file loading test
+ * Expected result: success
+ *
+ */
+
+use dataverse temp;
+
+load dataset testds
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/csv/sample_01.csv"),("format"="delimited-text"),("delimiter"=","));
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/load/csv_02/csv_02.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/load/csv_02/csv_02.3.query.aql
new file mode 100644
index 0000000..1299235
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/load/csv_02/csv_02.3.query.aql
@@ -0,0 +1,18 @@
+/**
+ *
+ * CSV file loading test
+ * Expected result: success
+ *
+ */
+
+use dataverse temp;
+
+for $i in dataset testds
+order by $i.id
+return { "id": $i.id,
+ "float": $i.float,
+ "double": $i.double,
+ "date-string": $i.date,
+ "time-string": $i.time,
+ "datetime-string": $i.datetime
+ }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/load/issue289_query/issue289_query.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/load/issue289_query/issue289_query.2.update.aql
index f0671ca..8a52ab9 100644
--- a/asterix-app/src/test/resources/runtimets/queries/load/issue289_query/issue289_query.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/load/issue289_query/issue289_query.2.update.aql
@@ -8,4 +8,4 @@
load dataset Customers
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/custord-tiny/customer-tiny-neg.adm"),("format"="adm")) pre-sorted;
\ No newline at end of file
+(("path"="nc1://data/custord-tiny/customer-tiny-neg.adm"),("format"="adm"));
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/parse_01/parse_01.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/parse_01/parse_01.1.ddl.aql
new file mode 100644
index 0000000..b899dc2
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/parse_01/parse_01.1.ddl.aql
@@ -0,0 +1,7 @@
+/**
+ * Test case for parsing temporal strings with format strings
+ * Expected Result: Success
+ **/
+
+drop dataverse test if exists;
+create dataverse test;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/parse_01/parse_01.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/parse_01/parse_01.2.update.aql
new file mode 100644
index 0000000..ec16dd3
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/parse_01/parse_01.2.update.aql
@@ -0,0 +1,4 @@
+/**
+ * Test case for parsing temporal strings with format strings
+ * Expected Result: Success
+ **/
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/parse_01/parse_01.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/parse_01/parse_01.3.query.aql
new file mode 100644
index 0000000..9f9f0d2
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/parse_01/parse_01.3.query.aql
@@ -0,0 +1,26 @@
+/**
+ * Test case for parsing temporal strings with format strings
+ * Expected Result: Success
+ **/
+
+use dataverse test;
+
+{ "date1": parse-date("2013-8-23", "YY-M-D"),
+ "date2": parse-date("Aug 12 -12", "MMM D Y"),
+ "date3": parse-date("-1234-01-01", "YYYY-MM-DD"),
+ "date4": parse-date("09/11/-1980", "D/M/Y"),
+ "date5": parse-date("09/11/-1990", "YY-M-D|MMM D Y|D/M/Y"),
+ "date6": parse-date("Mon Aug 19 2013", "W MMM D Y"),
+ "data7": parse-date("SKIPMEPLEASE Mon Aug SKIPME1ALSO 19 2013", "O W MMM O D Y"),
+ "time1": parse-time("8:23:49", "h:m:s"),
+ "time2": parse-time("8.19.23:32", "h.m.s:nn"),
+ "time3": parse-time("08.19.23:32 pm", "h.m.s:nn a"),
+ "time4": parse-time("6:30:40.948 pm PST", "h:mm:ss.nnn a z"),
+ "time5": parse-time("6:30:40.948 pm PST", "h:m:s|h.m.s:nn|h.m.s:nn a|h:mm:ss.nnn a z"),
+ "datetime1": parse-datetime("Dec 30 -1203 3:48:27 PM", "MMM DD YYYY h:m:s a"),
+ "datetime2": parse-datetime("12/30/-1203 03:48:27.392 PM Asia/Shanghai", "MM/DD/YYY hh:mm:ss.nnn a z"),
+ "datetime3": parse-datetime("1723-12-03T23:59:23.392Z", "YYYY-MM-DDThh:mm:ss.nnnz"),
+ "datetime4": parse-datetime("1723-12-03T23:59:23.392-04:00", "YYYY-MM-DDThh:mm:ss.nnnz"),
+ "datetime5": parse-datetime("1723-12-03T23:59:23.392-04:00", "MMM DD YYYY h:m:s a|MM/DD/YYY hh:mm:ss.nnn a z|YYYY-MM-DDThh:mm:ss.nnnz"),
+ "datetime6": parse-datetime("1970-01-01 Thu 23:59:23.392-04:00", "MMM DD YYYY h:m:s a|MM/DD/YYY hh:mm:ss.nnn a z|YYYY-MM-DD W hh:mm:ss.nnnz"),
+ "datetime7": parse-datetime("1723-12-03 What3v3r STRINGHERE 23:59:23.392-04:00", "MMM DD YYYY h:m:s a|MM/DD/YYY hh:mm:ss.nnn a z|YYYY-MM-DD O O hh:mm:ss.nnnz") }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/parse_02/parse_02.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/parse_02/parse_02.1.ddl.aql
new file mode 100644
index 0000000..b899dc2
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/parse_02/parse_02.1.ddl.aql
@@ -0,0 +1,7 @@
+/**
+ * Test case for parsing temporal strings with format strings
+ * Expected Result: Success
+ **/
+
+drop dataverse test if exists;
+create dataverse test;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/parse_02/parse_02.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/parse_02/parse_02.2.update.aql
new file mode 100644
index 0000000..ec16dd3
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/parse_02/parse_02.2.update.aql
@@ -0,0 +1,4 @@
+/**
+ * Test case for parsing temporal strings with format strings
+ * Expected Result: Success
+ **/
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/parse_02/parse_02.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/parse_02/parse_02.3.query.aql
new file mode 100644
index 0000000..7617d5f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/parse_02/parse_02.3.query.aql
@@ -0,0 +1,20 @@
+/**
+ * Test case for parsing temporal strings with format strings
+ * Expected Result: Success
+ **/
+
+use dataverse test;
+
+let $date := date("-0123-01-30")
+let $time := time("08:07:29.030Z")
+let $datetime := datetime("0137-12-31T23:59:59.999+08:00")
+return {
+ "date-string-1": print-date($date, "YY/M/D"),
+ "date-string-2": print-date($date, "MMM DD, YYYY"),
+ "date-string-3": print-date($date, "YYYY/MM/DD"),
+ "time-string-1": print-time($time, "h.m.s.nn a z"),
+ "time-string-2": print-time($time, "hh.mm.ss.nnn a z"),
+ "datetime-string-1": print-datetime($datetime, "MMM DD h:m:s.nnn a YY z"),
+ "datetime-string-2": print-datetime($datetime, "YYYY/MMM/DD h:m:s.nnnz a"),
+ "datetime-string-3": print-datetime($datetime, "YYYY-MM-DDThh:mm:ss.nnnz")
+ }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q02_minimum_cost_supplier/q02_minimum_cost_supplier.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q02_minimum_cost_supplier/q02_minimum_cost_supplier.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q02_minimum_cost_supplier/q02_minimum_cost_supplier.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q02_minimum_cost_supplier/q02_minimum_cost_supplier.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q04_order_priority/q04_order_priority.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q04_order_priority/q04_order_priority.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q04_order_priority/q04_order_priority.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q04_order_priority/q04_order_priority.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q05_local_supplier_volume/q05_local_supplier_volume.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q05_local_supplier_volume/q05_local_supplier_volume.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q05_local_supplier_volume/q05_local_supplier_volume.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q05_local_supplier_volume/q05_local_supplier_volume.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q06_forecast_revenue_change/q06_forecast_revenue_change.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q06_forecast_revenue_change/q06_forecast_revenue_change.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q06_forecast_revenue_change/q06_forecast_revenue_change.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q06_forecast_revenue_change/q06_forecast_revenue_change.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q07_volume_shipping/q07_volume_shipping.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q07_volume_shipping/q07_volume_shipping.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q07_volume_shipping/q07_volume_shipping.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q07_volume_shipping/q07_volume_shipping.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q08_national_market_share/q08_national_market_share.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q08_national_market_share/q08_national_market_share.2.update.aql
index b23ff7f..ff66e4b 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q08_national_market_share/q08_national_market_share.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q08_national_market_share/q08_national_market_share.2.update.aql
@@ -2,32 +2,32 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q09_product_type_profit_nt/q09_product_type_profit_nt.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q09_product_type_profit_nt/q09_product_type_profit_nt.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q09_product_type_profit_nt/q09_product_type_profit_nt.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q09_product_type_profit_nt/q09_product_type_profit_nt.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q10_returned_item/q10_returned_item.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q10_returned_item/q10_returned_item.2.update.aql
index 0adc8e1..22a7590 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q10_returned_item/q10_returned_item.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q10_returned_item/q10_returned_item.2.update.aql
@@ -6,29 +6,29 @@
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q10_returned_item_int64/q10_returned_item_int64.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q10_returned_item_int64/q10_returned_item_int64.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q10_returned_item_int64/q10_returned_item_int64.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q10_returned_item_int64/q10_returned_item_int64.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q11_important_stock/q11_important_stock.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q11_important_stock/q11_important_stock.2.update.aql
index b23ff7f..ff66e4b 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q11_important_stock/q11_important_stock.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q11_important_stock/q11_important_stock.2.update.aql
@@ -2,32 +2,32 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q12_shipping/q12_shipping.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q12_shipping/q12_shipping.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q12_shipping/q12_shipping.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q12_shipping/q12_shipping.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q13_customer_distribution/q13_customer_distribution.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q13_customer_distribution/q13_customer_distribution.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q13_customer_distribution/q13_customer_distribution.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q13_customer_distribution/q13_customer_distribution.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q14_promotion_effect/q14_promotion_effect.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q14_promotion_effect/q14_promotion_effect.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q14_promotion_effect/q14_promotion_effect.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q14_promotion_effect/q14_promotion_effect.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q15_top_supplier/q15_top_supplier.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q15_top_supplier/q15_top_supplier.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q15_top_supplier/q15_top_supplier.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q15_top_supplier/q15_top_supplier.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q16_parts_supplier_relationship/q16_parts_supplier_relationship.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q16_parts_supplier_relationship/q16_parts_supplier_relationship.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q16_parts_supplier_relationship/q16_parts_supplier_relationship.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q16_parts_supplier_relationship/q16_parts_supplier_relationship.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q18_large_volume_customer/q18_large_volume_customer.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q18_large_volume_customer/q18_large_volume_customer.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q18_large_volume_customer/q18_large_volume_customer.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q18_large_volume_customer/q18_large_volume_customer.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q19_discounted_revenue/q19_discounted_revenue.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q19_discounted_revenue/q19_discounted_revenue.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q19_discounted_revenue/q19_discounted_revenue.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q19_discounted_revenue/q19_discounted_revenue.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q20_potential_part_promotion/q20_potential_part_promotion.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q20_potential_part_promotion/q20_potential_part_promotion.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q20_potential_part_promotion/q20_potential_part_promotion.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q20_potential_part_promotion/q20_potential_part_promotion.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/results/load/csv_01/csv_01.1.adm b/asterix-app/src/test/resources/runtimets/results/load/csv_01/csv_01.1.adm
new file mode 100644
index 0000000..b8d4151
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/load/csv_01/csv_01.1.adm
@@ -0,0 +1,8 @@
+{ "id": 1, "float": 0.89968276f, "double": 5.6256d, "date-before": "2013-08-07", "date-after": date("2013-08-07"), "time-before": "07:22:35", "time-after": time("07:22:35.000Z"), "datetime-before": "1979-02-25T23:48:27.034", "datetime-after": datetime("1979-02-25T23:48:27.034Z") }
+{ "id": 2, "float": 0.6690524f, "double": null, "date-before": "-1923-03-29", "date-after": date("-1923-03-29"), "time-before": "19:33:34", "time-after": time("19:33:34.000Z"), "datetime-before": "-1979-02-25T23:48:27.002", "datetime-after": datetime("-1979-02-25T23:48:27.002Z") }
+{ "id": 3, "float": 0.57273304f, "double": 192674.0d, "date-before": "-1923-03-28", "date-after": date("-1923-03-28"), "time-before": "19:33:34", "time-after": time("19:33:34.000Z"), "datetime-before": "-1979-02-25T23:48:27.001", "datetime-after": datetime("-1979-02-25T23:48:27.001Z") }
+{ "id": 4, "float": null, "double": 192674.0d, "date-before": "-1923-03-27", "date-after": date("-1923-03-27"), "time-before": "19:33:34", "time-after": time("19:33:34.000Z"), "datetime-before": "-1979-02-25T23:48:27.001", "datetime-after": datetime("-1979-02-25T23:48:27.001Z") }
+{ "id": 5, "float": 0.57273304f, "double": 192674.0d, "date-before": null, "date-after": null, "time-before": "19:33:34", "time-after": time("19:33:34.000Z"), "datetime-before": "-1979-02-25T23:48:27.001", "datetime-after": datetime("-1979-02-25T23:48:27.001Z") }
+{ "id": 6, "float": 0.57273304f, "double": 192674.0d, "date-before": "-1923-03-25", "date-after": date("-1923-03-25"), "time-before": null, "time-after": null, "datetime-before": "-1979-02-25T23:48:27.001", "datetime-after": datetime("-1979-02-25T23:48:27.001Z") }
+{ "id": 7, "float": 0.57273304f, "double": 192674.0d, "date-before": "-1923-03-24", "date-after": date("-1923-03-24"), "time-before": "19:33:34", "time-after": time("19:33:34.000Z"), "datetime-before": null, "datetime-after": null }
+{ "id": 8, "float": null, "double": null, "date-before": null, "date-after": null, "time-before": null, "time-after": null, "datetime-before": null, "datetime-after": null }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/load/csv_02/csv_02.1.adm b/asterix-app/src/test/resources/runtimets/results/load/csv_02/csv_02.1.adm
new file mode 100644
index 0000000..44240dd
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/load/csv_02/csv_02.1.adm
@@ -0,0 +1,8 @@
+{ "id": 1, "float": 0.89968276f, "double": 5.6256d, "date-string": "2013-08-07", "time-string": "07:22:35", "datetime-string": "1979-02-25T23:48:27.034" }
+{ "id": 2, "float": 0.6690524f, "double": null, "date-string": "-1923-03-29", "time-string": "19:33:34", "datetime-string": "-1979-02-25T23:48:27.002" }
+{ "id": 3, "float": 0.57273304f, "double": 192674.0d, "date-string": "-1923-03-28", "time-string": "19:33:34", "datetime-string": "-1979-02-25T23:48:27.001" }
+{ "id": 4, "float": null, "double": 192674.0d, "date-string": "-1923-03-27", "time-string": "19:33:34", "datetime-string": "-1979-02-25T23:48:27.001" }
+{ "id": 5, "float": 0.57273304f, "double": 192674.0d, "date-string": "", "time-string": "19:33:34", "datetime-string": "-1979-02-25T23:48:27.001" }
+{ "id": 6, "float": 0.57273304f, "double": 192674.0d, "date-string": "-1923-03-25", "time-string": "", "datetime-string": "-1979-02-25T23:48:27.001" }
+{ "id": 7, "float": 0.57273304f, "double": 192674.0d, "date-string": "-1923-03-24", "time-string": "19:33:34", "datetime-string": "" }
+{ "id": 8, "float": null, "double": null, "date-string": "", "time-string": "", "datetime-string": "" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/temporal/parse_01/parse_01.1.adm b/asterix-app/src/test/resources/runtimets/results/temporal/parse_01/parse_01.1.adm
new file mode 100644
index 0000000..dee2631
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/temporal/parse_01/parse_01.1.adm
@@ -0,0 +1 @@
+{ "date1": date("2013-08-23"), "date2": date("-0012-08-12"), "date3": date("-1234-01-01"), "date4": date("-1980-11-09"), "date5": date("-1990-11-09"), "date6": date("2013-08-19"), "data7": date("2013-08-19"), "time1": time("08:23:49.000Z"), "time2": time("08:19:23.320Z"), "time3": time("20:19:23.320Z"), "time4": time("10:30:40.948Z"), "time5": time("10:30:40.948Z"), "datetime1": datetime("-1203-12-30T15:48:27.000Z"), "datetime2": datetime("-1203-12-30T23:48:27.392Z"), "datetime3": datetime("1723-12-03T23:59:23.392Z"), "datetime4": datetime("1723-12-04T03:59:23.392Z"), "datetime5": datetime("1723-12-04T03:59:23.392Z"), "datetime6": datetime("1970-01-02T03:59:23.392Z"), "datetime7": datetime("1723-12-04T03:59:23.392Z") }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/temporal/parse_02/parse_02.1.adm b/asterix-app/src/test/resources/runtimets/results/temporal/parse_02/parse_02.1.adm
new file mode 100644
index 0000000..963e8d0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/temporal/parse_02/parse_02.1.adm
@@ -0,0 +1 @@
+{ "date-string-1": "-123/1/30", "date-string-2": "JAN 30, -0123", "date-string-3": "-0123/01/30", "time-string-1": "8.7.29.03 AM Z", "time-string-2": "08.07.29.030 AM Z", "datetime-string-1": "DEC 31 3:59:59.999 PM 137 Z", "datetime-string-2": "0137/DEC/31 3:59:59.999Z PM", "datetime-string-3": "0137-12-31T15:59:59.999Z" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/tpch/q02_minimum_cost_supplier/q02_minimum_cost_supplier.1.adm b/asterix-app/src/test/resources/runtimets/results/tpch/q02_minimum_cost_supplier/q02_minimum_cost_supplier.1.adm
index af60345..a7d5b93 100644
--- a/asterix-app/src/test/resources/runtimets/results/tpch/q02_minimum_cost_supplier/q02_minimum_cost_supplier.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/tpch/q02_minimum_cost_supplier/q02_minimum_cost_supplier.1.adm
@@ -1,6 +1,8 @@
{ "s_acctbal": 6820.35d, "s_name": "Supplier#000000007", "n_name": "UNITED KINGDOM", "p_partkey": 2, "p_mfgr": "Manufacturer#1", "s_address": "s,4TicNGB4uO6PaSqNBUq", "s_phone": "33-990-965-2201", "s_comment": "s unwind silently furiously regular courts. final requests are deposits. requests wake quietly blit" }
{ "s_acctbal": 6820.35d, "s_name": "Supplier#000000007", "n_name": "UNITED KINGDOM", "p_partkey": 4, "p_mfgr": "Manufacturer#3", "s_address": "s,4TicNGB4uO6PaSqNBUq", "s_phone": "33-990-965-2201", "s_comment": "s unwind silently furiously regular courts. final requests are deposits. requests wake quietly blit" }
{ "s_acctbal": 6820.35d, "s_name": "Supplier#000000007", "n_name": "UNITED KINGDOM", "p_partkey": 22, "p_mfgr": "Manufacturer#4", "s_address": "s,4TicNGB4uO6PaSqNBUq", "s_phone": "33-990-965-2201", "s_comment": "s unwind silently furiously regular courts. final requests are deposits. requests wake quietly blit" }
+{ "s_acctbal": 6820.35d, "s_name": "Supplier#000000007", "n_name": "UNITED KINGDOM", "p_partkey": 35, "p_mfgr": "Manufacturer#4", "s_address": "s,4TicNGB4uO6PaSqNBUq", "s_phone": "33-990-965-2201", "s_comment": "s unwind silently furiously regular courts. final requests are deposits. requests wake quietly blit" }
+{ "s_acctbal": 6820.35d, "s_name": "Supplier#000000007", "n_name": "UNITED KINGDOM", "p_partkey": 38, "p_mfgr": "Manufacturer#4", "s_address": "s,4TicNGB4uO6PaSqNBUq", "s_phone": "33-990-965-2201", "s_comment": "s unwind silently furiously regular courts. final requests are deposits. requests wake quietly blit" }
{ "s_acctbal": 6820.35d, "s_name": "Supplier#000000007", "n_name": "UNITED KINGDOM", "p_partkey": 62, "p_mfgr": "Manufacturer#3", "s_address": "s,4TicNGB4uO6PaSqNBUq", "s_phone": "33-990-965-2201", "s_comment": "s unwind silently furiously regular courts. final requests are deposits. requests wake quietly blit" }
{ "s_acctbal": 6820.35d, "s_name": "Supplier#000000007", "n_name": "UNITED KINGDOM", "p_partkey": 79, "p_mfgr": "Manufacturer#4", "s_address": "s,4TicNGB4uO6PaSqNBUq", "s_phone": "33-990-965-2201", "s_comment": "s unwind silently furiously regular courts. final requests are deposits. requests wake quietly blit" }
{ "s_acctbal": 6820.35d, "s_name": "Supplier#000000007", "n_name": "UNITED KINGDOM", "p_partkey": 94, "p_mfgr": "Manufacturer#3", "s_address": "s,4TicNGB4uO6PaSqNBUq", "s_phone": "33-990-965-2201", "s_comment": "s unwind silently furiously regular courts. final requests are deposits. requests wake quietly blit" }
diff --git a/asterix-app/src/test/resources/runtimets/results/tpch/q09_product_type_profit_nt/q09_product_type_profit_nt.1.adm b/asterix-app/src/test/resources/runtimets/results/tpch/q09_product_type_profit_nt/q09_product_type_profit_nt.1.adm
index aec4b5e..e9f3f47 100644
--- a/asterix-app/src/test/resources/runtimets/results/tpch/q09_product_type_profit_nt/q09_product_type_profit_nt.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/tpch/q09_product_type_profit_nt/q09_product_type_profit_nt.1.adm
@@ -5,7 +5,7 @@
{ "nation": "ARGENTINA", "o_year": 1993, "sum_profit": 35857.08d }
{ "nation": "ARGENTINA", "o_year": 1992, "sum_profit": 35740.0d }
{ "nation": "ETHIOPIA", "o_year": 1998, "sum_profit": 2758.7801999999992d }
-{ "nation": "ETHIOPIA", "o_year": 1997, "sum_profit": 19419.294599999997d }
+{ "nation": "ETHIOPIA", "o_year": 1997, "sum_profit": 19419.294599999994d }
{ "nation": "ETHIOPIA", "o_year": 1995, "sum_profit": 51231.87439999999d }
{ "nation": "ETHIOPIA", "o_year": 1994, "sum_profit": 3578.9478999999974d }
{ "nation": "ETHIOPIA", "o_year": 1992, "sum_profit": 1525.8234999999986d }
@@ -22,38 +22,38 @@
{ "nation": "IRAQ", "o_year": 1994, "sum_profit": 36010.728599999995d }
{ "nation": "IRAQ", "o_year": 1993, "sum_profit": 33221.9399d }
{ "nation": "IRAQ", "o_year": 1992, "sum_profit": 47755.05900000001d }
-{ "nation": "KENYA", "o_year": 1998, "sum_profit": 47441.812d }
-{ "nation": "KENYA", "o_year": 1997, "sum_profit": 118144.65099999998d }
-{ "nation": "KENYA", "o_year": 1996, "sum_profit": 75090.5437d }
-{ "nation": "KENYA", "o_year": 1995, "sum_profit": 87626.6791d }
-{ "nation": "KENYA", "o_year": 1994, "sum_profit": 130302.4071d }
-{ "nation": "KENYA", "o_year": 1993, "sum_profit": 185952.9548d }
-{ "nation": "KENYA", "o_year": 1992, "sum_profit": 310009.1845d }
-{ "nation": "MOROCCO", "o_year": 1998, "sum_profit": 172158.9448d }
-{ "nation": "MOROCCO", "o_year": 1997, "sum_profit": 92247.75379999999d }
-{ "nation": "MOROCCO", "o_year": 1996, "sum_profit": 273187.9262d }
-{ "nation": "MOROCCO", "o_year": 1995, "sum_profit": 380050.2938d }
-{ "nation": "MOROCCO", "o_year": 1994, "sum_profit": 262538.8032d }
-{ "nation": "MOROCCO", "o_year": 1993, "sum_profit": 211233.76599999992d }
-{ "nation": "MOROCCO", "o_year": 1992, "sum_profit": 336202.0852d }
-{ "nation": "PERU", "o_year": 1998, "sum_profit": 143951.22939999998d }
-{ "nation": "PERU", "o_year": 1997, "sum_profit": 197680.59720000002d }
-{ "nation": "PERU", "o_year": 1996, "sum_profit": 236156.92649999997d }
-{ "nation": "PERU", "o_year": 1995, "sum_profit": 190426.59480000002d }
-{ "nation": "PERU", "o_year": 1994, "sum_profit": 118171.16209999999d }
-{ "nation": "PERU", "o_year": 1993, "sum_profit": 102292.5426d }
-{ "nation": "PERU", "o_year": 1992, "sum_profit": 135197.25370000003d }
-{ "nation": "UNITED KINGDOM", "o_year": 1998, "sum_profit": 115488.22360000003d }
-{ "nation": "UNITED KINGDOM", "o_year": 1997, "sum_profit": 224769.43809999994d }
-{ "nation": "UNITED KINGDOM", "o_year": 1996, "sum_profit": 367799.3432d }
-{ "nation": "UNITED KINGDOM", "o_year": 1995, "sum_profit": 563229.4064d }
-{ "nation": "UNITED KINGDOM", "o_year": 1994, "sum_profit": 305695.22239999997d }
-{ "nation": "UNITED KINGDOM", "o_year": 1993, "sum_profit": 762202.0801000001d }
-{ "nation": "UNITED KINGDOM", "o_year": 1992, "sum_profit": 109066.47399999999d }
+{ "nation": "KENYA", "o_year": 1998, "sum_profit": 44194.831999999995d }
+{ "nation": "KENYA", "o_year": 1997, "sum_profit": 57578.3626d }
+{ "nation": "KENYA", "o_year": 1996, "sum_profit": 59195.9021d }
+{ "nation": "KENYA", "o_year": 1995, "sum_profit": 79262.6278d }
+{ "nation": "KENYA", "o_year": 1994, "sum_profit": 102360.66609999999d }
+{ "nation": "KENYA", "o_year": 1993, "sum_profit": 128422.01959999999d }
+{ "nation": "KENYA", "o_year": 1992, "sum_profit": 181517.20890000003d }
+{ "nation": "MOROCCO", "o_year": 1998, "sum_profit": 41797.823199999984d }
+{ "nation": "MOROCCO", "o_year": 1997, "sum_profit": 23685.801799999997d }
+{ "nation": "MOROCCO", "o_year": 1996, "sum_profit": 62115.19579999999d }
+{ "nation": "MOROCCO", "o_year": 1995, "sum_profit": 42442.64300000001d }
+{ "nation": "MOROCCO", "o_year": 1994, "sum_profit": 48655.87800000001d }
+{ "nation": "MOROCCO", "o_year": 1993, "sum_profit": 22926.744400000003d }
+{ "nation": "MOROCCO", "o_year": 1992, "sum_profit": 32239.8088d }
+{ "nation": "PERU", "o_year": 1998, "sum_profit": 86999.36459999997d }
+{ "nation": "PERU", "o_year": 1997, "sum_profit": 121110.41070000001d }
+{ "nation": "PERU", "o_year": 1996, "sum_profit": 177040.40759999998d }
+{ "nation": "PERU", "o_year": 1995, "sum_profit": 122247.94519999999d }
+{ "nation": "PERU", "o_year": 1994, "sum_profit": 88046.2533d }
+{ "nation": "PERU", "o_year": 1993, "sum_profit": 49379.813799999996d }
+{ "nation": "PERU", "o_year": 1992, "sum_profit": 80646.86050000001d }
+{ "nation": "UNITED KINGDOM", "o_year": 1998, "sum_profit": 50577.25560000001d }
+{ "nation": "UNITED KINGDOM", "o_year": 1997, "sum_profit": 114288.86049999998d }
+{ "nation": "UNITED KINGDOM", "o_year": 1996, "sum_profit": 147684.46480000002d }
+{ "nation": "UNITED KINGDOM", "o_year": 1995, "sum_profit": 225267.6576d }
+{ "nation": "UNITED KINGDOM", "o_year": 1994, "sum_profit": 140595.58639999997d }
+{ "nation": "UNITED KINGDOM", "o_year": 1993, "sum_profit": 322548.49210000003d }
+{ "nation": "UNITED KINGDOM", "o_year": 1992, "sum_profit": 67747.88279999999d }
{ "nation": "UNITED STATES", "o_year": 1998, "sum_profit": 3957.0431999999996d }
{ "nation": "UNITED STATES", "o_year": 1997, "sum_profit": 94729.5704d }
{ "nation": "UNITED STATES", "o_year": 1996, "sum_profit": 79297.8567d }
{ "nation": "UNITED STATES", "o_year": 1995, "sum_profit": 62201.23360000001d }
-{ "nation": "UNITED STATES", "o_year": 1994, "sum_profit": 43075.6299d }
-{ "nation": "UNITED STATES", "o_year": 1993, "sum_profit": 27168.486199999992d }
+{ "nation": "UNITED STATES", "o_year": 1994, "sum_profit": 43075.62989999999d }
+{ "nation": "UNITED STATES", "o_year": 1993, "sum_profit": 27168.486199999996d }
{ "nation": "UNITED STATES", "o_year": 1992, "sum_profit": 34092.366d }
diff --git a/asterix-app/src/test/resources/runtimets/results/tpch/q11_important_stock/q11_important_stock.1.adm b/asterix-app/src/test/resources/runtimets/results/tpch/q11_important_stock/q11_important_stock.1.adm
index 42b3861..0fc28fb 100644
--- a/asterix-app/src/test/resources/runtimets/results/tpch/q11_important_stock/q11_important_stock.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/tpch/q11_important_stock/q11_important_stock.1.adm
@@ -1,13 +1,13 @@
{ "partkey": 25, "part_value": 2.832302068E7d }
{ "partkey": 124, "part_value": 2.59627599E7d }
-{ "partkey": 175, "part_value": 2.3853953630000003E7d }
+{ "partkey": 175, "part_value": 2.385395363E7d }
{ "partkey": 197, "part_value": 2.248551967E7d }
{ "partkey": 163, "part_value": 2.099460571E7d }
{ "partkey": 160, "part_value": 2.00232846E7d }
-{ "partkey": 82, "part_value": 1.9919213349999998E7d }
+{ "partkey": 82, "part_value": 1.991921335E7d }
{ "partkey": 169, "part_value": 1.898734723E7d }
{ "partkey": 29, "part_value": 1.867279344E7d }
-{ "partkey": 26, "part_value": 1.8612458270000003E7d }
+{ "partkey": 26, "part_value": 1.861245827E7d }
{ "partkey": 73, "part_value": 1.827170729E7d }
{ "partkey": 161, "part_value": 1.7987463009999998E7d }
{ "partkey": 75, "part_value": 1.7959598009999998E7d }
@@ -16,28 +16,28 @@
{ "partkey": 69, "part_value": 1.728526943E7d }
{ "partkey": 111, "part_value": 1.7083882619999997E7d }
{ "partkey": 171, "part_value": 1.635442066E7d }
-{ "partkey": 166, "part_value": 1.6351893739999998E7d }
+{ "partkey": 166, "part_value": 1.6351893740000002E7d }
{ "partkey": 77, "part_value": 1.598059909E7d }
{ "partkey": 78, "part_value": 1.58768992E7d }
{ "partkey": 143, "part_value": 1.585686159E7d }
-{ "partkey": 17, "part_value": 1.5474261120000001E7d }
-{ "partkey": 109, "part_value": 1.505468262E7d }
+{ "partkey": 17, "part_value": 1.547426112E7d }
+{ "partkey": 109, "part_value": 1.5054682620000001E7d }
{ "partkey": 105, "part_value": 1.5053163809999999E7d }
{ "partkey": 96, "part_value": 1.495213259E7d }
{ "partkey": 146, "part_value": 1.481075944E7d }
{ "partkey": 136, "part_value": 1.465496775E7d }
{ "partkey": 116, "part_value": 1.4432091339999998E7d }
-{ "partkey": 128, "part_value": 1.439355526E7d }
-{ "partkey": 142, "part_value": 1.4220399040000001E7d }
+{ "partkey": 128, "part_value": 1.4393555259999998E7d }
+{ "partkey": 142, "part_value": 1.422039904E7d }
{ "partkey": 121, "part_value": 1.420032605E7d }
{ "partkey": 30, "part_value": 1.416313241E7d }
{ "partkey": 16, "part_value": 1.413646503E7d }
-{ "partkey": 198, "part_value": 1.4135353350000001E7d }
+{ "partkey": 198, "part_value": 1.413535335E7d }
{ "partkey": 79, "part_value": 1.38652287E7d }
{ "partkey": 90, "part_value": 1.373279748E7d }
{ "partkey": 32, "part_value": 1.369962979E7d }
{ "partkey": 74, "part_value": 1.338871111E7d }
-{ "partkey": 1, "part_value": 1.3378707239999998E7d }
+{ "partkey": 1, "part_value": 1.337870724E7d }
{ "partkey": 89, "part_value": 1.337148041E7d }
{ "partkey": 22, "part_value": 1.3354991740000002E7d }
{ "partkey": 186, "part_value": 1.317604077E7d }
@@ -45,13 +45,13 @@
{ "partkey": 14, "part_value": 1.299397721E7d }
{ "partkey": 93, "part_value": 1.299298218E7d }
{ "partkey": 168, "part_value": 1.299041501E7d }
-{ "partkey": 99, "part_value": 1.275004679E7d }
+{ "partkey": 99, "part_value": 1.2750046790000001E7d }
{ "partkey": 167, "part_value": 1.268255069E7d }
{ "partkey": 2, "part_value": 1.258471636E7d }
{ "partkey": 182, "part_value": 1.256239411E7d }
{ "partkey": 61, "part_value": 1.253677656E7d }
{ "partkey": 112, "part_value": 1.234957975E7d }
-{ "partkey": 178, "part_value": 1.226030174E7d }
+{ "partkey": 178, "part_value": 1.2260301739999998E7d }
{ "partkey": 172, "part_value": 1.219775193E7d }
{ "partkey": 165, "part_value": 1.219746506E7d }
{ "partkey": 184, "part_value": 1.216784393E7d }
@@ -59,14 +59,14 @@
{ "partkey": 153, "part_value": 1.2119354219999999E7d }
{ "partkey": 95, "part_value": 1.20468895E7d }
{ "partkey": 11, "part_value": 1.2007151559999999E7d }
-{ "partkey": 125, "part_value": 1.200347611E7d }
-{ "partkey": 154, "part_value": 1.1851133850000001E7d }
+{ "partkey": 125, "part_value": 1.2003476109999998E7d }
+{ "partkey": 154, "part_value": 1.185113385E7d }
{ "partkey": 15, "part_value": 1.1798438790000001E7d }
{ "partkey": 67, "part_value": 1.178579951E7d }
{ "partkey": 8, "part_value": 1.1707892620000001E7d }
{ "partkey": 87, "part_value": 1.168637671E7d }
-{ "partkey": 134, "part_value": 1.168358693E7d }
-{ "partkey": 130, "part_value": 1.168246149E7d }
+{ "partkey": 134, "part_value": 1.1683586929999998E7d }
+{ "partkey": 130, "part_value": 1.1682461489999998E7d }
{ "partkey": 43, "part_value": 1.161150462E7d }
{ "partkey": 102, "part_value": 1.151554211E7d }
{ "partkey": 21, "part_value": 1.141066856E7d }
@@ -76,7 +76,7 @@
{ "partkey": 173, "part_value": 1.102677486E7d }
{ "partkey": 94, "part_value": 1.092440116E7d }
{ "partkey": 3, "part_value": 1.075814545E7d }
-{ "partkey": 103, "part_value": 1.0691221600000001E7d }
+{ "partkey": 103, "part_value": 1.06912216E7d }
{ "partkey": 158, "part_value": 1.067861635E7d }
{ "partkey": 49, "part_value": 1.06445572E7d }
{ "partkey": 139, "part_value": 1.044045371E7d }
@@ -89,30 +89,30 @@
{ "partkey": 33, "part_value": 1.005296264E7d }
{ "partkey": 28, "part_value": 1.005234286E7d }
{ "partkey": 40, "part_value": 9927827.77d }
-{ "partkey": 199, "part_value": 9907803.56d }
+{ "partkey": 199, "part_value": 9907803.559999999d }
{ "partkey": 193, "part_value": 9869674.77d }
{ "partkey": 106, "part_value": 9869361.73d }
-{ "partkey": 108, "part_value": 9868370.31d }
+{ "partkey": 108, "part_value": 9868370.309999999d }
{ "partkey": 183, "part_value": 9855564.82d }
-{ "partkey": 70, "part_value": 9700431.940000001d }
-{ "partkey": 48, "part_value": 9655921.879999999d }
+{ "partkey": 70, "part_value": 9700431.94d }
+{ "partkey": 48, "part_value": 9655921.88d }
{ "partkey": 118, "part_value": 9622756.15d }
{ "partkey": 13, "part_value": 9592610.32d }
{ "partkey": 83, "part_value": 9543465.08d }
{ "partkey": 159, "part_value": 9519909.44d }
{ "partkey": 147, "part_value": 9513932.18d }
-{ "partkey": 45, "part_value": 9423874.469999999d }
+{ "partkey": 45, "part_value": 9423874.47d }
{ "partkey": 117, "part_value": 9408426.72d }
-{ "partkey": 135, "part_value": 9311247.28d }
+{ "partkey": 135, "part_value": 9311247.280000001d }
{ "partkey": 185, "part_value": 9305341.780000001d }
{ "partkey": 131, "part_value": 9223742.49d }
{ "partkey": 7, "part_value": 9175528.21d }
{ "partkey": 71, "part_value": 9167712.04d }
-{ "partkey": 100, "part_value": 9131099.53d }
+{ "partkey": 100, "part_value": 9131099.530000001d }
{ "partkey": 76, "part_value": 9092927.11d }
{ "partkey": 53, "part_value": 8979121.97d }
-{ "partkey": 141, "part_value": 8686511.12d }
-{ "partkey": 64, "part_value": 8627897.29d }
+{ "partkey": 141, "part_value": 8686511.120000001d }
+{ "partkey": 64, "part_value": 8627897.290000001d }
{ "partkey": 101, "part_value": 8521762.0d }
{ "partkey": 176, "part_value": 8510175.88d }
{ "partkey": 19, "part_value": 8481679.5d }
@@ -120,44 +120,43 @@
{ "partkey": 91, "part_value": 8460636.52d }
{ "partkey": 132, "part_value": 8416851.239999998d }
{ "partkey": 113, "part_value": 8405217.96d }
-{ "partkey": 51, "part_value": 8247118.5d }
+{ "partkey": 51, "part_value": 8247118.499999999d }
{ "partkey": 41, "part_value": 8187897.16d }
{ "partkey": 55, "part_value": 8092552.890000001d }
-{ "partkey": 72, "part_value": 8007155.300000001d }
+{ "partkey": 72, "part_value": 8007155.3d }
{ "partkey": 115, "part_value": 7954624.0d }
{ "partkey": 170, "part_value": 7895241.609999999d }
{ "partkey": 114, "part_value": 7832023.28d }
-{ "partkey": 37, "part_value": 7809598.66d }
-{ "partkey": 54, "part_value": 7578243.789999999d }
+{ "partkey": 37, "part_value": 7809598.659999999d }
+{ "partkey": 54, "part_value": 7578243.79d }
{ "partkey": 180, "part_value": 7531794.4799999995d }
-{ "partkey": 60, "part_value": 7508961.6899999995d }
-{ "partkey": 31, "part_value": 7433034.24d }
+{ "partkey": 60, "part_value": 7508961.69d }
+{ "partkey": 31, "part_value": 7433034.240000001d }
{ "partkey": 35, "part_value": 7132671.49d }
{ "partkey": 140, "part_value": 7122050.08d }
{ "partkey": 150, "part_value": 7106237.92d }
{ "partkey": 107, "part_value": 7082828.68d }
{ "partkey": 123, "part_value": 7049500.720000001d }
-{ "partkey": 190, "part_value": 7017966.899999999d }
+{ "partkey": 190, "part_value": 7017966.9d }
{ "partkey": 120, "part_value": 6920857.090000001d }
{ "partkey": 196, "part_value": 6905182.43d }
{ "partkey": 177, "part_value": 6887257.27d }
{ "partkey": 126, "part_value": 6813302.029999999d }
{ "partkey": 122, "part_value": 6812763.34d }
-{ "partkey": 200, "part_value": 6780024.529999999d }
+{ "partkey": 200, "part_value": 6780024.53d }
{ "partkey": 157, "part_value": 6766365.680000001d }
{ "partkey": 63, "part_value": 6724960.14d }
{ "partkey": 38, "part_value": 6667789.55d }
-{ "partkey": 58, "part_value": 6640619.38d }
-{ "partkey": 145, "part_value": 6633786.590000001d }
+{ "partkey": 58, "part_value": 6640619.380000001d }
+{ "partkey": 145, "part_value": 6633786.59d }
{ "partkey": 144, "part_value": 6546945.92d }
{ "partkey": 20, "part_value": 6533101.39d }
{ "partkey": 127, "part_value": 6483139.620000001d }
{ "partkey": 10, "part_value": 6433776.51d }
-{ "partkey": 36, "part_value": 6410209.249999999d }
-{ "partkey": 47, "part_value": 6407355.37d }
+{ "partkey": 47, "part_value": 6407355.369999999d }
{ "partkey": 191, "part_value": 6347187.43d }
{ "partkey": 137, "part_value": 6180452.85d }
-{ "partkey": 56, "part_value": 6145826.600000001d }
+{ "partkey": 56, "part_value": 6145826.6d }
{ "partkey": 104, "part_value": 6134341.85d }
{ "partkey": 44, "part_value": 6038126.66d }
{ "partkey": 97, "part_value": 6036047.1899999995d }
@@ -167,7 +166,7 @@
{ "partkey": 52, "part_value": 5680644.4799999995d }
{ "partkey": 155, "part_value": 5552007.57d }
{ "partkey": 92, "part_value": 5489588.279999999d }
-{ "partkey": 5, "part_value": 5461046.93d }
+{ "partkey": 5, "part_value": 5461046.930000001d }
{ "partkey": 18, "part_value": 5456316.21d }
{ "partkey": 149, "part_value": 5367514.63d }
{ "partkey": 110, "part_value": 5261352.11d }
@@ -175,26 +174,27 @@
{ "partkey": 6, "part_value": 5120392.470000001d }
{ "partkey": 148, "part_value": 5061589.27d }
{ "partkey": 42, "part_value": 4957032.47d }
-{ "partkey": 119, "part_value": 4954403.48d }
+{ "partkey": 119, "part_value": 4954403.4799999995d }
{ "partkey": 84, "part_value": 4891082.38d }
-{ "partkey": 65, "part_value": 4834763.090000001d }
+{ "partkey": 65, "part_value": 4834763.09d }
{ "partkey": 66, "part_value": 4719253.369999999d }
-{ "partkey": 179, "part_value": 4610607.92d }
+{ "partkey": 179, "part_value": 4610607.919999999d }
{ "partkey": 23, "part_value": 4531731.12d }
{ "partkey": 68, "part_value": 4504770.61d }
{ "partkey": 27, "part_value": 4371849.52d }
-{ "partkey": 129, "part_value": 3997604.7800000003d }
+{ "partkey": 36, "part_value": 4036576.8999999994d }
+{ "partkey": 129, "part_value": 3997604.78d }
{ "partkey": 195, "part_value": 3817436.31d }
{ "partkey": 59, "part_value": 3765210.2100000004d }
{ "partkey": 57, "part_value": 3739347.12d }
{ "partkey": 138, "part_value": 3567425.75d }
-{ "partkey": 174, "part_value": 3484708.31d }
+{ "partkey": 174, "part_value": 3484708.3100000005d }
{ "partkey": 164, "part_value": 3462215.0d }
{ "partkey": 81, "part_value": 3421610.42d }
{ "partkey": 46, "part_value": 3398443.33d }
{ "partkey": 85, "part_value": 3338711.3899999997d }
-{ "partkey": 50, "part_value": 3145791.97d }
+{ "partkey": 50, "part_value": 3145791.9699999997d }
{ "partkey": 88, "part_value": 3117730.2399999998d }
{ "partkey": 151, "part_value": 2727444.22d }
{ "partkey": 152, "part_value": 1837809.1700000002d }
-{ "partkey": 133, "part_value": 1517282.33d }
+{ "partkey": 133, "part_value": 1517282.3299999998d }
diff --git a/asterix-app/src/test/resources/runtimets/results/tpch/q16_parts_supplier_relationship/q16_parts_supplier_relationship.1.adm b/asterix-app/src/test/resources/runtimets/results/tpch/q16_parts_supplier_relationship/q16_parts_supplier_relationship.1.adm
index 4e06408..8247e9f 100644
--- a/asterix-app/src/test/resources/runtimets/results/tpch/q16_parts_supplier_relationship/q16_parts_supplier_relationship.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/tpch/q16_parts_supplier_relationship/q16_parts_supplier_relationship.1.adm
@@ -2,10 +2,12 @@
{ "p_brand": "Brand#11", "p_type": "SMALL PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#11", "p_type": "STANDARD POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#13", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#14", "p_type": "SMALL ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#15", "p_type": "LARGE ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#23", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#31", "p_type": "PROMO POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
@@ -16,19 +18,17 @@
{ "p_brand": "Brand#34", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#34", "p_type": "SMALL PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#43", "p_type": "SMALL BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#44", "p_type": "SMALL PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#52", "p_type": "MEDIUM BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#53", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#55", "p_type": "STANDARD ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#55", "p_type": "STANDARD BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
-{ "p_brand": "Brand#13", "p_type": "SMALL BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 2i64 }
-{ "p_brand": "Brand#25", "p_type": "SMALL BURNISHED COPPER", "p_size": 3, "supplier_cnt": 2i64 }
-{ "p_brand": "Brand#43", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 14, "supplier_cnt": 2i64 }
-{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED STEEL", "p_size": 45, "supplier_cnt": 2i64 }
-{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED STEEL", "p_size": 19, "supplier_cnt": 1i64 }
-{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED STEEL", "p_size": 49, "supplier_cnt": 1i64 }
-{ "p_brand": "Brand#53", "p_type": "LARGE BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 1i64 }
-{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 9, "supplier_cnt": 1i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BURNISHED COPPER", "p_size": 3, "supplier_cnt": 3i64 }
diff --git a/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterix-app/src/test/resources/runtimets/testsuite.xml
index 3a2a44e..d65da69 100644
--- a/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -4253,6 +4253,16 @@
</test-group>
<test-group name="load">
<test-case FilePath="load">
+ <compilation-unit name="csv_01">
+ <output-dir compare="Text">csv_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
+ <compilation-unit name="csv_02">
+ <output-dir compare="Text">csv_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
<compilation-unit name="issue14_query">
<output-dir compare="Text">none</output-dir>
<expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error>
@@ -4344,6 +4354,16 @@
</test-group>
<test-group name="temporal">
<test-case FilePath="temporal">
+ <compilation-unit name="parse_02">
+ <output-dir compare="Text">parse_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="temporal">
+ <compilation-unit name="parse_01">
+ <output-dir compare="Text">parse_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="temporal">
<compilation-unit name="day_of_week_01">
<output-dir compare="Text">day_of_week_01</output-dir>
</compilation-unit>
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixThreadExecutor.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixThreadExecutor.java
index 14975ff..edd4b2a 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixThreadExecutor.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixThreadExecutor.java
@@ -14,12 +14,15 @@
*/
package edu.uci.ics.asterix.common.api;
+import java.util.concurrent.Callable;
import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
public class AsterixThreadExecutor implements Executor {
public final static AsterixThreadExecutor INSTANCE = new AsterixThreadExecutor();
- private final Executor executor = Executors.newCachedThreadPool(AsterixThreadFactory.INSTANCE);
+ private final ExecutorService executorService = Executors.newCachedThreadPool(AsterixThreadFactory.INSTANCE);
private AsterixThreadExecutor() {
@@ -27,6 +30,10 @@
@Override
public void execute(Runnable command) {
- executor.execute(command);
+ executorService.execute(command);
+ }
+
+ public Future<Object> submit(Callable command) {
+ return (Future<Object>) executorService.submit(command);
}
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixAppRuntimeContext.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixAppRuntimeContext.java
index d035303..4287212 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixAppRuntimeContext.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixAppRuntimeContext.java
@@ -15,6 +15,7 @@
package edu.uci.ics.asterix.common.api;
import java.io.IOException;
+import java.util.List;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
@@ -62,5 +63,5 @@
public double getBloomFilterFalsePositiveRate();
- public IVirtualBufferCache getVirtualBufferCache(int datasetID);
+ public List<IVirtualBufferCache> getVirtualBufferCaches(int datasetID);
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixStorageProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixStorageProperties.java
index 85f90e4..d7ff15d 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixStorageProperties.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixStorageProperties.java
@@ -31,6 +31,9 @@
private static final String STORAGE_MEMORYCOMPONENT_NUMPAGES_KEY = "storage.memorycomponent.numpages";
private static final int STORAGE_MEMORYCOMPONENT_NUMPAGES_DEFAULT = 1024; // ... so 32MB components
+ private static final String STORAGE_MEMORYCOMPONENT_NUMCOMPONENTS_KEY = "storage.memorycomponent.numcomponents";
+ private static final int STORAGE_MEMORYCOMPONENT_NUMCOMPONENTS_DEFAULT = 2; // 2 components
+
private static final String STORAGE_MEMORYCOMPONENT_GLOBALBUDGET_KEY = "storage.memorycomponent.globalbudget";
private static final long STORAGE_MEMORYCOMPONENT_GLOBALBUDGET_DEFAULT = 536870912; // 512MB
@@ -53,7 +56,7 @@
return accessor.getProperty(STORAGE_BUFFERCACHE_SIZE_KEY, STORAGE_BUFFERCACHE_SIZE_DEFAULT,
PropertyInterpreters.getLongPropertyInterpreter());
}
-
+
public int getBufferCacheNumPages() {
return (int) (getBufferCacheSize() / getBufferCachePageSize());
}
@@ -73,6 +76,11 @@
PropertyInterpreters.getIntegerPropertyInterpreter());
}
+ public int getMemoryComponentsNum() {
+ return accessor.getProperty(STORAGE_MEMORYCOMPONENT_NUMCOMPONENTS_KEY,
+ STORAGE_MEMORYCOMPONENT_NUMCOMPONENTS_DEFAULT, PropertyInterpreters.getIntegerPropertyInterpreter());
+ }
+
public long getMemoryComponentGlobalBudget() {
return accessor.getProperty(STORAGE_MEMORYCOMPONENT_GLOBALBUDGET_KEY,
STORAGE_MEMORYCOMPONENT_GLOBALBUDGET_DEFAULT, PropertyInterpreters.getLongPropertyInterpreter());
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixTransactionProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixTransactionProperties.java
index 5a40ece..a1dd52a 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixTransactionProperties.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixTransactionProperties.java
@@ -25,12 +25,6 @@
private static final String TXN_LOG_PARTITIONSIZE_KEY = "txn.log.partitionsize";
private static final long TXN_LOG_PARTITIONSIZE_DEFAULT = ((long)2 << 30); // 2GB
- private static final String TXN_LOG_DISKSECTORSIZE_KEY = "txn.log.disksectorsize";
- private static final int TXN_LOG_DISKSECTORSIZE_DEFAULT = 4096;
-
- private static final String TXN_LOG_GROUPCOMMITINTERVAL_KEY = "txn.log.groupcommitinterval";
- private static int TXN_LOG_GROUPCOMMITINTERVAL_DEFAULT = 10; // 0.1ms
-
private static final String TXN_LOG_CHECKPOINT_LSNTHRESHOLD_KEY = "txn.log.checkpoint.lsnthreshold";
private static final int TXN_LOG_CHECKPOINT_LSNTHRESHOLD_DEFAULT = (64 << 20); // 64M
@@ -75,16 +69,6 @@
PropertyInterpreters.getLongPropertyInterpreter());
}
- public int getLogDiskSectorSize() {
- return accessor.getProperty(TXN_LOG_DISKSECTORSIZE_KEY, TXN_LOG_DISKSECTORSIZE_DEFAULT,
- PropertyInterpreters.getIntegerPropertyInterpreter());
- }
-
- public int getGroupCommitInterval() {
- return accessor.getProperty(TXN_LOG_GROUPCOMMITINTERVAL_KEY, TXN_LOG_GROUPCOMMITINTERVAL_DEFAULT,
- PropertyInterpreters.getIntegerPropertyInterpreter());
- }
-
public int getCheckpointLSNThreshold() {
return accessor.getProperty(TXN_LOG_CHECKPOINT_LSNTHRESHOLD_KEY, TXN_LOG_CHECKPOINT_LSNTHRESHOLD_DEFAULT,
PropertyInterpreters.getIntegerPropertyInterpreter());
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixVirtualBufferCacheProvider.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixVirtualBufferCacheProvider.java
index bd2828d..9efc9fd 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixVirtualBufferCacheProvider.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixVirtualBufferCacheProvider.java
@@ -14,6 +14,8 @@
*/
package edu.uci.ics.asterix.common.context;
+import java.util.List;
+
import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
@@ -29,9 +31,9 @@
}
@Override
- public IVirtualBufferCache getVirtualBufferCache(IHyracksTaskContext ctx) {
+ public List<IVirtualBufferCache> getVirtualBufferCaches(IHyracksTaskContext ctx) {
return ((IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
- .getVirtualBufferCache(datasetID);
+ .getVirtualBufferCaches(datasetID);
}
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/BaseOperationTracker.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/BaseOperationTracker.java
index c48115c..3610478 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/BaseOperationTracker.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/BaseOperationTracker.java
@@ -26,13 +26,18 @@
public class BaseOperationTracker implements ILSMOperationTracker {
+ protected final DatasetLifecycleManager datasetLifecycleManager;
protected final ILSMIOOperationCallback ioOpCallback;
protected long lastLSN;
protected long firstLSN;
+ protected final int datasetID;
- public BaseOperationTracker(ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
+ public BaseOperationTracker(DatasetLifecycleManager datasetLifecycleManager,
+ ILSMIOOperationCallbackFactory ioOpCallbackFactory, int datasetID) {
+ this.datasetLifecycleManager = datasetLifecycleManager;
this.ioOpCallback = ioOpCallbackFactory == null ? NoOpIOOperationCallback.INSTANCE : ioOpCallbackFactory
.createIOOperationCallback(this);
+ this.datasetID = datasetID;
resetLSNs();
}
@@ -63,11 +68,17 @@
@Override
public void beforeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
IModificationOperationCallback modificationCallback) throws HyracksDataException {
+ if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
+ datasetLifecycleManager.declareActiveIOOperation(datasetID);
+ }
}
@Override
public void afterOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
IModificationOperationCallback modificationCallback) throws HyracksDataException {
+ if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
+ datasetLifecycleManager.undeclareActiveIOOperation(datasetID);
+ }
}
@Override
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/ConstantMergePolicy.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/ConstantMergePolicy.java
index 3ffa73d..cf69bfe 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/ConstantMergePolicy.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/ConstantMergePolicy.java
@@ -15,10 +15,13 @@
package edu.uci.ics.asterix.common.context;
+import java.util.List;
+
import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
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.ILSMMergePolicy;
@@ -34,9 +37,10 @@
this.ctx = ctx;
}
- public void diskComponentAdded(final ILSMIndex index, int totalNumDiskComponents) throws HyracksDataException,
- IndexException {
- if (!ctx.isShuttingdown() && totalNumDiskComponents >= threshold) {
+ @Override
+ public void diskComponentAdded(final ILSMIndex index) throws HyracksDataException, IndexException {
+ List<ILSMComponent> immutableComponents = index.getImmutableComponents();
+ if (!ctx.isShuttingdown() && immutableComponents.size() >= threshold) {
ILSMIndexAccessor accessor = (ILSMIndexAccessor) index.createAccessor(NoOpOperationCallback.INSTANCE,
NoOpOperationCallback.INSTANCE);
accessor.scheduleMerge(NoOpIOOperationCallback.INSTANCE);
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java
index 7232f01..617b6ff 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java
@@ -31,7 +31,9 @@
import edu.uci.ics.hyracks.api.lifecycle.ILifeCycleComponent;
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.impls.NoOpOperationCallback;
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;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.MultitenantVirtualBufferCache;
@@ -42,7 +44,7 @@
public class DatasetLifecycleManager implements IIndexLifecycleManager, ILifeCycleComponent {
private final AsterixStorageProperties storageProperties;
- private final Map<Integer, MultitenantVirtualBufferCache> datasetVirtualBufferCaches;
+ private final Map<Integer, List<IVirtualBufferCache>> datasetVirtualBufferCaches;
private final Map<Integer, ILSMOperationTracker> datasetOpTrackers;
private final Map<Integer, DatasetInfo> datasetInfos;
private final ILocalResourceRepository resourceRepository;
@@ -53,7 +55,7 @@
ILocalResourceRepository resourceRepository) {
this.storageProperties = storageProperties;
this.resourceRepository = resourceRepository;
- datasetVirtualBufferCaches = new HashMap<Integer, MultitenantVirtualBufferCache>();
+ datasetVirtualBufferCaches = new HashMap<Integer, List<IVirtualBufferCache>>();
datasetOpTrackers = new HashMap<Integer, ILSMOperationTracker>();
datasetInfos = new HashMap<Integer, DatasetInfo>();
capacity = storageProperties.getMemoryComponentGlobalBudget();
@@ -98,29 +100,67 @@
public synchronized void unregister(long resourceID) throws HyracksDataException {
int did = getDIDfromRID(resourceID);
DatasetInfo dsInfo = datasetInfos.get(did);
- IndexInfo iInfo = dsInfo.indexes.remove(resourceID);
+ IndexInfo iInfo = dsInfo.indexes.get(resourceID);
+
if (dsInfo == null || iInfo == null) {
throw new HyracksDataException("Index with resource ID " + resourceID + " does not exist.");
}
- if (iInfo.referenceCount != 0) {
- dsInfo.indexes.put(resourceID, iInfo);
+ PrimaryIndexOperationTracker opTracker = (PrimaryIndexOperationTracker) datasetOpTrackers.get(dsInfo.datasetID);
+ if (iInfo.referenceCount != 0 || (opTracker != null && opTracker.getNumActiveOperations() != 0)) {
throw new HyracksDataException("Cannot remove index while it is open.");
}
+ // TODO: use fine-grained counters, one for each index instead of a single counter per dataset.
+
+ // First wait for any ongoing IO operations
+ while (dsInfo.numActiveIOOps > 0) {
+ try {
+ wait();
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ // Flush and wait for it to finish, it is separated from the above wait so they don't deadlock each other.
+ // TODO: Find a better way to do this.
+ flushAndWaitForIO(dsInfo, iInfo);
+
if (iInfo.isOpen) {
- iInfo.index.deactivate(true);
+ iInfo.index.deactivate(false);
}
+ dsInfo.indexes.remove(resourceID);
if (dsInfo.referenceCount == 0 && dsInfo.isOpen && dsInfo.indexes.isEmpty()) {
- IVirtualBufferCache vbc = getVirtualBufferCache(did);
- assert vbc != null;
- used -= (vbc.getNumPages() * vbc.getPageSize());
+ List<IVirtualBufferCache> vbcs = getVirtualBufferCaches(did);
+ assert vbcs != null;
+ for (IVirtualBufferCache vbc : vbcs) {
+ used -= (vbc.getNumPages() * vbc.getPageSize());
+ }
datasetInfos.remove(did);
+ datasetVirtualBufferCaches.remove(did);
+ datasetOpTrackers.remove(did);
}
}
+ public synchronized void declareActiveIOOperation(int datasetID) throws HyracksDataException {
+ DatasetInfo dsInfo = datasetInfos.get(datasetID);
+ if (dsInfo == null) {
+ throw new HyracksDataException("Failed to find a dataset with ID " + datasetID);
+ }
+ dsInfo.incrementActiveIOOps();
+ }
+
+ public synchronized void undeclareActiveIOOperation(int datasetID) throws HyracksDataException {
+ DatasetInfo dsInfo = datasetInfos.get(datasetID);
+ if (dsInfo == null) {
+ throw new HyracksDataException("Failed to find a dataset with ID " + datasetID);
+ }
+ dsInfo.decrementActiveIOOps();
+ notifyAll();
+ }
+
@Override
public synchronized void open(long resourceID) throws HyracksDataException {
int did = getDIDfromRID(resourceID);
@@ -137,9 +177,12 @@
}
if (!dsInfo.isOpen) {
- IVirtualBufferCache vbc = getVirtualBufferCache(did);
- assert vbc != null;
- long additionalSize = vbc.getNumPages() * vbc.getPageSize();
+ List<IVirtualBufferCache> vbcs = getVirtualBufferCaches(did);
+ assert vbcs != null;
+ long additionalSize = 0;
+ for (IVirtualBufferCache vbc : vbcs) {
+ additionalSize += vbc.getNumPages() * vbc.getPageSize();
+ }
while (used + additionalSize > capacity) {
if (!evictCandidateDataset()) {
throw new HyracksDataException("Cannot activate index since memory budget would be exceeded.");
@@ -165,26 +208,63 @@
List<DatasetInfo> datasetInfosList = new ArrayList<DatasetInfo>(datasetInfos.values());
Collections.sort(datasetInfosList);
for (DatasetInfo dsInfo : datasetInfosList) {
- ILSMOperationTracker opTracker = datasetOpTrackers.get(dsInfo.datasetID);
- if (opTracker != null && ((PrimaryIndexOperationTracker) opTracker).getNumActiveOperations() == 0
- && dsInfo.referenceCount == 0 && dsInfo.isOpen) {
+ PrimaryIndexOperationTracker opTracker = (PrimaryIndexOperationTracker) datasetOpTrackers
+ .get(dsInfo.datasetID);
+ if (opTracker != null && opTracker.getNumActiveOperations() == 0 && dsInfo.referenceCount == 0
+ && dsInfo.isOpen) {
+
+ // First wait for any ongoing IO operations
+ while (dsInfo.numActiveIOOps > 0) {
+ try {
+ wait();
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ for (IndexInfo iInfo : dsInfo.indexes.values()) {
+ // TODO: This is not efficient since we flush the indexes sequentially.
+ // Think of a way to allow submitting the flush requests concurrently. We don't do them concurrently because this
+ // may lead to a deadlock scenario between the DatasetLifeCycleManager and the PrimaryIndexOperationTracker.
+ flushAndWaitForIO(dsInfo, iInfo);
+ }
+
for (IndexInfo iInfo : dsInfo.indexes.values()) {
if (iInfo.isOpen) {
- iInfo.index.deactivate(true);
+ iInfo.index.deactivate(false);
iInfo.isOpen = false;
}
assert iInfo.referenceCount == 0;
}
-
- IVirtualBufferCache vbc = getVirtualBufferCache(dsInfo.datasetID);
- used -= vbc.getNumPages() * vbc.getPageSize();
dsInfo.isOpen = false;
+
+ List<IVirtualBufferCache> vbcs = getVirtualBufferCaches(dsInfo.datasetID);
+ for (IVirtualBufferCache vbc : vbcs) {
+ used -= vbc.getNumPages() * vbc.getPageSize();
+ }
return true;
+
}
}
return false;
}
+ private void flushAndWaitForIO(DatasetInfo dsInfo, IndexInfo iInfo) throws HyracksDataException {
+ if (iInfo.isOpen) {
+ ILSMIndexAccessor accessor = (ILSMIndexAccessor) iInfo.index.createAccessor(NoOpOperationCallback.INSTANCE,
+ NoOpOperationCallback.INSTANCE);
+ accessor.scheduleFlush(((BaseOperationTracker) iInfo.index.getOperationTracker()).getIOOperationCallback());
+ }
+ // Wait for the above flush op.
+ while (dsInfo.numActiveIOOps > 0) {
+ try {
+ wait();
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ }
+
@Override
public synchronized void close(long resourceID) throws HyracksDataException {
int did = getDIDfromRID(resourceID);
@@ -213,15 +293,22 @@
return openIndexes;
}
- public IVirtualBufferCache getVirtualBufferCache(int datasetID) {
+ public List<IVirtualBufferCache> getVirtualBufferCaches(int datasetID) {
synchronized (datasetVirtualBufferCaches) {
- MultitenantVirtualBufferCache vbc = datasetVirtualBufferCaches.get(datasetID);
- if (vbc == null) {
- vbc = new MultitenantVirtualBufferCache(new VirtualBufferCache(new HeapBufferAllocator(),
- storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages()));
- datasetVirtualBufferCaches.put(datasetID, vbc);
+ List<IVirtualBufferCache> vbcs = datasetVirtualBufferCaches.get(datasetID);
+ if (vbcs == null) {
+ vbcs = new ArrayList<IVirtualBufferCache>();
+ for (int i = 0; i < storageProperties.getMemoryComponentsNum(); i++) {
+ MultitenantVirtualBufferCache vbc = new MultitenantVirtualBufferCache(
+ new VirtualBufferCache(new HeapBufferAllocator(),
+ storageProperties.getMemoryComponentPageSize(),
+ storageProperties.getMemoryComponentNumPages()
+ / storageProperties.getMemoryComponentsNum()));
+ vbcs.add(vbc);
+ }
+ datasetVirtualBufferCaches.put(datasetID, vbcs);
}
- return vbc;
+ return vbcs;
}
}
@@ -245,7 +332,9 @@
}
Set<ILSMIndex> datasetIndexes = new HashSet<ILSMIndex>();
for (IndexInfo iInfo : dsInfo.indexes.values()) {
- datasetIndexes.add(iInfo.index);
+ if (iInfo.isOpen) {
+ datasetIndexes.add(iInfo.index);
+ }
}
return datasetIndexes;
}
@@ -280,6 +369,7 @@
private final Map<Long, IndexInfo> indexes;
private final int datasetID;
private long lastAccess;
+ private int numActiveIOOps;
public DatasetInfo(int datasetID) {
this.indexes = new HashMap<Long, IndexInfo>();
@@ -297,6 +387,14 @@
lastAccess = System.currentTimeMillis();
}
+ public void incrementActiveIOOps() {
+ numActiveIOOps++;
+ }
+
+ public void decrementActiveIOOps() {
+ numActiveIOOps--;
+ }
+
@Override
public int compareTo(DatasetInfo i) {
// sort by (isOpen, referenceCount, lastAccess) ascending, where true < false
@@ -376,4 +474,4 @@
}
outputStream.write(sb.toString().getBytes());
}
-}
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/PrimaryIndexOperationTracker.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/PrimaryIndexOperationTracker.java
index 53b9192..2ed4b0ec 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/PrimaryIndexOperationTracker.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/PrimaryIndexOperationTracker.java
@@ -18,7 +18,6 @@
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
-import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
@@ -26,35 +25,27 @@
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.IVirtualBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexInternal;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMOperationType;
public class PrimaryIndexOperationTracker extends BaseOperationTracker {
- private final DatasetLifecycleManager datasetLifecycleManager;
- private final IVirtualBufferCache datasetBufferCache;
- private final int datasetID;
- // Number of active operations on a ILSMIndex instance.
- private AtomicInteger numActiveOperations;
+ // Number of active operations on an ILSMIndex instance.
+ private final AtomicInteger numActiveOperations;
public PrimaryIndexOperationTracker(DatasetLifecycleManager datasetLifecycleManager, int datasetID,
ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
- super(ioOpCallbackFactory);
- this.datasetLifecycleManager = datasetLifecycleManager;
- this.numActiveOperations = new AtomicInteger(0);
- this.datasetID = datasetID;
- datasetBufferCache = datasetLifecycleManager.getVirtualBufferCache(datasetID);
+ super(datasetLifecycleManager, ioOpCallbackFactory, datasetID);
+ this.numActiveOperations = new AtomicInteger();
}
@Override
public void beforeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
IModificationOperationCallback modificationCallback) throws HyracksDataException {
- numActiveOperations.incrementAndGet();
-
- // Increment transactor-local active operations count.
- AbstractOperationCallback opCallback = getOperationCallback(searchCallback, modificationCallback);
- if (opCallback != null) {
- opCallback.incrementLocalNumActiveOperations();
+ if (opType == LSMOperationType.MODIFICATION || opType == LSMOperationType.FORCE_MODIFICATION) {
+ numActiveOperations.incrementAndGet();
+ } else if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
+ datasetLifecycleManager.declareActiveIOOperation(datasetID);
}
}
@@ -62,8 +53,7 @@
public void afterOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
IModificationOperationCallback modificationCallback) throws HyracksDataException {
// Searches are immediately considered complete, because they should not prevent the execution of flushes.
- if ((searchCallback != null && searchCallback != NoOpOperationCallback.INSTANCE)
- || opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
+ if (opType == LSMOperationType.SEARCH || opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
completeOperation(index, opType, searchCallback, modificationCallback);
}
}
@@ -71,50 +61,46 @@
@Override
public void completeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
IModificationOperationCallback modificationCallback) throws HyracksDataException {
- int nActiveOps = numActiveOperations.decrementAndGet();
- // Decrement transactor-local active operations count.
- AbstractOperationCallback opCallback = getOperationCallback(searchCallback, modificationCallback);
- if (opCallback != null) {
- opCallback.decrementLocalNumActiveOperations();
+ if (opType == LSMOperationType.MODIFICATION || opType == LSMOperationType.FORCE_MODIFICATION) {
+ numActiveOperations.decrementAndGet();
+ } else if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
+ datasetLifecycleManager.undeclareActiveIOOperation(datasetID);
}
- if (opType != LSMOperationType.FLUSH) {
- flushIfFull(nActiveOps);
+
+ if (opType == LSMOperationType.MODIFICATION || opType == LSMOperationType.FORCE_MODIFICATION) {
+ flushIfFull();
}
}
- private void flushIfFull(int nActiveOps) throws HyracksDataException {
- // If we need a flush, and this is the last completing operation, then schedule the flush.
- if (datasetBufferCache.isFull() && nActiveOps == 0) {
- Set<ILSMIndex> indexes = datasetLifecycleManager.getDatasetIndexes(datasetID);
- for (ILSMIndex lsmIndex : indexes) {
- ILSMIndexAccessor accessor = (ILSMIndexAccessor) lsmIndex.createAccessor(
- NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
- accessor.scheduleFlush(((BaseOperationTracker) lsmIndex.getOperationTracker()).getIOOperationCallback());
+ private void flushIfFull() throws HyracksDataException {
+ // If we need a flush, and this is the last completing operation, then schedule the flush.
+ // TODO: Is there a better way to check if we need to flush instead of communicating with the datasetLifecycleManager each time?
+ Set<ILSMIndex> indexes = datasetLifecycleManager.getDatasetIndexes(datasetID);
+ boolean needsFlush = false;
+ for (ILSMIndex lsmIndex : indexes) {
+ if (((ILSMIndexInternal) lsmIndex).hasFlushRequestForCurrentMutableComponent()) {
+ needsFlush = true;
+ break;
}
-
+ }
+ synchronized (this) {
+ if (needsFlush && numActiveOperations.get() == 0) {
+ for (ILSMIndex lsmIndex : indexes) {
+ ILSMIndexAccessor accessor = (ILSMIndexAccessor) lsmIndex.createAccessor(
+ NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+ accessor.scheduleFlush(((BaseOperationTracker) lsmIndex.getOperationTracker())
+ .getIOOperationCallback());
+ }
+ }
}
}
public void exclusiveJobCommitted() throws HyracksDataException {
numActiveOperations.set(0);
- flushIfFull(0);
- }
-
- private AbstractOperationCallback getOperationCallback(ISearchOperationCallback searchCallback,
- IModificationOperationCallback modificationCallback) {
-
- if (modificationCallback == NoOpOperationCallback.INSTANCE || modificationCallback == null) {
- return null;
- }
- if (searchCallback != null && searchCallback != NoOpOperationCallback.INSTANCE) {
- return (AbstractOperationCallback) searchCallback;
- } else {
- return (AbstractOperationCallback) modificationCallback;
- }
+ flushIfFull();
}
public int getNumActiveOperations() {
return numActiveOperations.get();
}
-
-}
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
index 129a1a7..bc19543 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
@@ -47,7 +47,6 @@
if (tupleFilter != null) {
frameTuple.reset(accessor, i);
if (!tupleFilter.accept(frameTuple)) {
- lsmAccessor.noOp();
continue;
}
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java
index fcc6dad..da08cd8 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java
@@ -45,7 +45,7 @@
opTracker.resetLSNs();
}
- protected abstract long getComponentLSN(List<ILSMComponent> oldComponents) throws HyracksDataException;
+ public abstract long getComponentLSN(List<ILSMComponent> oldComponents) throws HyracksDataException;
protected void putLSNIntoMetadata(ITreeIndex treeIndex, List<ILSMComponent> oldComponents)
throws HyracksDataException {
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
index 40377f4..b6025cb 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
@@ -20,7 +20,7 @@
import edu.uci.ics.asterix.common.context.BaseOperationTracker;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTreeImmutableComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTreeDiskComponent;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
public class LSMBTreeIOOperationCallback extends AbstractLSMIOOperationCallback {
@@ -33,21 +33,21 @@
public void afterOperation(List<ILSMComponent> oldComponents, ILSMComponent newComponent)
throws HyracksDataException {
if (oldComponents != null && newComponent != null) {
- LSMBTreeImmutableComponent btreeComponent = (LSMBTreeImmutableComponent) newComponent;
+ LSMBTreeDiskComponent btreeComponent = (LSMBTreeDiskComponent) newComponent;
putLSNIntoMetadata(btreeComponent.getBTree(), oldComponents);
}
}
@Override
- protected long getComponentLSN(List<ILSMComponent> oldComponents) throws HyracksDataException {
- if (oldComponents == null) {
+ public long getComponentLSN(List<ILSMComponent> diskComponents) throws HyracksDataException {
+ if (diskComponents == null) {
// Implies a flush IO operation.
return opTracker.getLastLSN();
}
- // Get max LSN from the oldComponents. Implies a merge IO operation.
+ // Get max LSN from the diskComponents. Implies a merge IO operation or Recovery operation.
long maxLSN = -1;
- for (ILSMComponent c : oldComponents) {
- BTree btree = ((LSMBTreeImmutableComponent) c).getBTree();
+ for (ILSMComponent c : diskComponents) {
+ BTree btree = ((LSMBTreeDiskComponent) c).getBTree();
maxLSN = Math.max(getTreeIndexLSN(btree), maxLSN);
}
return maxLSN;
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
index eb9878c..4f99ae6 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
@@ -20,7 +20,7 @@
import edu.uci.ics.asterix.common.context.BaseOperationTracker;
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.invertedindex.impls.LSMInvertedIndexImmutableComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexDiskComponent;
public class LSMInvertedIndexIOOperationCallback extends AbstractLSMIOOperationCallback {
@@ -32,21 +32,21 @@
public void afterOperation(List<ILSMComponent> oldComponents, ILSMComponent newComponent)
throws HyracksDataException {
if (oldComponents != null && newComponent != null) {
- LSMInvertedIndexImmutableComponent invIndexComponent = (LSMInvertedIndexImmutableComponent) newComponent;
+ LSMInvertedIndexDiskComponent invIndexComponent = (LSMInvertedIndexDiskComponent) newComponent;
putLSNIntoMetadata(invIndexComponent.getDeletedKeysBTree(), oldComponents);
}
}
@Override
- protected long getComponentLSN(List<ILSMComponent> oldComponents) throws HyracksDataException {
- if (oldComponents == null) {
+ public long getComponentLSN(List<ILSMComponent> diskComponents) throws HyracksDataException {
+ if (diskComponents == null) {
// Implies a flush IO operation.
return opTracker.getLastLSN();
}
- // Get max LSN from the oldComponents. Implies a merge IO operation.
+ // Get max LSN from the diskComponents. Implies a merge IO operation or Recovery operation.
long maxLSN = -1;
- for (Object o : oldComponents) {
- LSMInvertedIndexImmutableComponent invIndexComponent = (LSMInvertedIndexImmutableComponent) o;
+ for (Object o : diskComponents) {
+ LSMInvertedIndexDiskComponent invIndexComponent = (LSMInvertedIndexDiskComponent) o;
maxLSN = Math.max(getTreeIndexLSN(invIndexComponent.getDeletedKeysBTree()), maxLSN);
}
return maxLSN;
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java
index 63016f1..cd7b7a0 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java
@@ -20,7 +20,7 @@
import edu.uci.ics.asterix.common.context.BaseOperationTracker;
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.rtree.impls.LSMRTreeImmutableComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.LSMRTreeDiskComponent;
public class LSMRTreeIOOperationCallback extends AbstractLSMIOOperationCallback {
@@ -32,22 +32,22 @@
public void afterOperation(List<ILSMComponent> oldComponents, ILSMComponent newComponent)
throws HyracksDataException {
if (oldComponents != null && newComponent != null) {
- LSMRTreeImmutableComponent rtreeComponent = (LSMRTreeImmutableComponent) newComponent;
+ LSMRTreeDiskComponent rtreeComponent = (LSMRTreeDiskComponent) newComponent;
putLSNIntoMetadata(rtreeComponent.getRTree(), oldComponents);
putLSNIntoMetadata(rtreeComponent.getBTree(), oldComponents);
}
}
@Override
- protected long getComponentLSN(List<ILSMComponent> oldComponents) throws HyracksDataException {
- if (oldComponents == null) {
+ public long getComponentLSN(List<ILSMComponent> diskComponents) throws HyracksDataException {
+ if (diskComponents == null) {
// Implies a flush IO operation.
return opTracker.getLastLSN();
}
- // Get max LSN from the oldComponents. Implies a merge IO operation.
+ // Get max LSN from the diskComponents. Implies a merge IO operation or Recovery operation.
long maxLSN = -1;
- for (Object o : oldComponents) {
- LSMRTreeImmutableComponent rtreeComponent = (LSMRTreeImmutableComponent) o;
+ for (Object o : diskComponents) {
+ LSMRTreeDiskComponent rtreeComponent = (LSMRTreeDiskComponent) o;
maxLSN = Math.max(getTreeIndexLSN(rtreeComponent.getRTree()), maxLSN);
}
return maxLSN;
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/AbstractOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/AbstractOperationCallback.java
index c951826..d4b26f7 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/AbstractOperationCallback.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/AbstractOperationCallback.java
@@ -15,8 +15,6 @@
package edu.uci.ics.asterix.common.transactions;
-import java.util.concurrent.atomic.AtomicInteger;
-
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.MurmurHash128Bit;
@@ -28,7 +26,6 @@
protected final int[] primaryKeyFields;
protected final ITransactionContext txnCtx;
protected final ILockManager lockManager;
- protected final AtomicInteger transactorLocalNumActiveOperations;
protected final long[] longHashes;
public AbstractOperationCallback(int datasetId, int[] primaryKeyFields, ITransactionContext txnCtx,
@@ -37,7 +34,6 @@
this.primaryKeyFields = primaryKeyFields;
this.txnCtx = txnCtx;
this.lockManager = lockManager;
- this.transactorLocalNumActiveOperations = new AtomicInteger(0);
this.longHashes = new long[2];
}
@@ -45,16 +41,4 @@
MurmurHash128Bit.hash3_x64_128(tuple, primaryKeyFields, SEED, longHashes);
return Math.abs((int) longHashes[0]);
}
-
- public int getLocalNumActiveOperations() {
- return transactorLocalNumActiveOperations.get();
- }
-
- public void incrementLocalNumActiveOperations() {
- transactorLocalNumActiveOperations.incrementAndGet();
- }
-
- public void decrementLocalNumActiveOperations() {
- transactorLocalNumActiveOperations.decrementAndGet();
- }
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/Buffer.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/Buffer.java
deleted file mode 100644
index a10bf08..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/Buffer.java
+++ /dev/null
@@ -1,143 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.common.transactions;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-
-/**
- * Represents a wrapper over @see ByteBuffer supporting some custom APIs for
- * transaction support. This class is not "thread-safe". For performance
- * concerns, it is required for multiple writers to be able to write to the
- * buffer concurrently and that a writer is never blocked by another writer. The
- * users of this class must ensure that two concurrent writers get to write in
- * exclusive areas in the buffer. A reader and writer may or may not conflict
- * with each other. For example, reading of logs during roll back of a
- * transaction t1 does not conflict with writing of logs by another transaction
- * t2 as they are concerned with exclusive areas of the buffer. On the contrary,
- * a flushing the buffer to disk conflicts with a reader reading the buffer.
- * Appropriate locks are taken on the Buffer in the application logic and not
- * directly imposed by synchronized methods.
- */
-
-public class Buffer implements IBuffer {
-
- ByteBuffer buffer;
-
- public Buffer(ByteBuffer buffer) {
- this.buffer = buffer;
- }
-
- protected Buffer() {
- }
-
- public void setBuffer(ByteBuffer buffer) {
- this.buffer = buffer;
- }
-
- @Override
- public byte getByte(int offset) {
- return buffer.get(offset);
- }
-
- @Override
- public byte getByte() {
- return buffer.get();
- }
-
- @Override
- public void getBytes(byte[] bytes, int offset, int size) {
- System.arraycopy(buffer.array(), offset, bytes, 0, size);
- }
-
- @Override
- public int getSize() {
- return buffer.capacity();
- }
-
- @Override
- public int readInt() {
- return buffer.getInt();
- }
-
- @Override
- public int readInt(int offset) {
- return buffer.getInt(offset);
- }
-
- @Override
- public long readLong(int offset) {
- return buffer.getLong(offset);
- }
-
- @Override
- public void put(byte b) {
- buffer.put(b);
- }
-
- @Override
- public void put(int offset, byte b) {
- buffer.put(offset, b);
- }
-
- @Override
- public void put(byte[] bytes, int start, int length) {
- buffer.put(bytes, start, length);
-
- }
-
- @Override
- public void put(byte[] bytes) {
- buffer.put(bytes);
- }
-
- @Override
- public void writeInt(int value) {
- buffer.putInt(value);
- }
-
- @Override
- public void writeInt(int offset, int value) {
- buffer.putInt(offset, value);
-
- }
-
- @Override
- public void writeLong(long value) {
- buffer.putLong(value);
- }
-
- @Override
- public void writeLong(int offset, long value) {
- buffer.putLong(offset, value);
-
- }
-
- @Override
- public byte[] getArray() {
- return buffer.array();
- }
-
- @Override
- public void erase() {
- Arrays.fill(buffer.array(), (byte) 0);
- }
-
- @Override
- public ByteBuffer getByteBuffer() {
- return buffer;
- }
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileBasedBuffer.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileBasedBuffer.java
deleted file mode 100644
index fbe9b4c..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileBasedBuffer.java
+++ /dev/null
@@ -1,245 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.common.transactions;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-/**
- * Represent a buffer that is backed by a physical file. Provider custom APIs
- * for accessing a chunk of the underlying file.
- */
-public class FileBasedBuffer extends Buffer implements IFileBasedBuffer {
-
- private String filePath;
- private FileChannel fileChannel;
- private RandomAccessFile raf;
- private int bufferSize;
-
- private int bufferLastFlushOffset;
- private int bufferNextWriteOffset;
- private final int diskSectorSize;
-
- private final ReadWriteLock latch;
- private final AtomicInteger referenceCount;
-
- public FileBasedBuffer(String filePath, long offset, int bufferSize, int diskSectorSize) throws IOException {
- this.filePath = filePath;
- buffer = ByteBuffer.allocate(bufferSize);
- raf = new RandomAccessFile(new File(filePath), "rw");
- fileChannel = raf.getChannel();
- fileChannel.position(offset);
- fileChannel.read(buffer);
- buffer.position(0);
- this.bufferSize = bufferSize;
- buffer.limit(bufferSize);
- bufferLastFlushOffset = 0;
- bufferNextWriteOffset = 0;
- this.diskSectorSize = diskSectorSize;
- latch = new ReentrantReadWriteLock(true);
- referenceCount = new AtomicInteger(0);
- }
-
- public String getFilePath() {
- return filePath;
- }
-
- public void setFilePath(String filePath) {
- this.filePath = filePath;
- }
-
- @Override
- public int getSize() {
- return bufferSize;
- }
-
- public void clear() {
- buffer.clear();
- }
-
- @Override
- public void flush() throws IOException {
- //flush
- int pos = bufferLastFlushOffset;
- int limit = (((bufferNextWriteOffset - 1) / diskSectorSize) + 1) * diskSectorSize;
- buffer.position(pos);
- buffer.limit(limit);
- fileChannel.write(buffer);
- fileChannel.force(false);
-
- //update variables
- bufferLastFlushOffset = limit;
- bufferNextWriteOffset = limit;
- buffer.limit(bufferSize);
- }
-
- @Override
- public void put(byte b) {
- buffer.put(b);
- }
-
- @Override
- public void put(int offset, byte b) {
- buffer.put(offset, b);
- }
-
- @Override
- public void put(byte[] bytes, int start, int length) {
- buffer.put(bytes, start, length);
- }
-
- @Override
- public void put(byte[] bytes) {
- buffer.put(bytes);
- }
-
- @Override
- public void writeInt(int value) {
- buffer.putInt(value);
- }
-
- @Override
- public void writeInt(int index, int value) {
- buffer.putInt(index, value);
- }
-
- @Override
- public void writeLong(long value) {
- buffer.putLong(value);
- }
-
- @Override
- public void writeLong(int index, long value) {
- buffer.putLong(index, value);
- }
-
- /**
- * Resets the buffer with content (size as specified) from a given file
- * starting at offset.
- */
- @Override
- public void reset(String filePath, long diskNextWriteOffset, int bufferSize) throws IOException {
- if (!filePath.equals(this.filePath)) {
- raf.close();//required?
- fileChannel.close();
- raf = new RandomAccessFile(filePath, "rw");
- this.filePath = filePath;
- }
- fileChannel = raf.getChannel();
- fileChannel.position(diskNextWriteOffset);
- erase();
- buffer.position(0);
- buffer.limit(bufferSize);
- this.bufferSize = bufferSize;
-
- bufferLastFlushOffset = 0;
- bufferNextWriteOffset = 0;
- }
-
- @Override
- public void close() throws IOException {
- fileChannel.close();
- }
-
- @Override
- public void open(String filePath, long offset, int bufferSize) throws IOException {
- raf = new RandomAccessFile(filePath, "rw");
- fileChannel = raf.getChannel();
- fileChannel.position(offset);
- erase();
- buffer.position(0);
- buffer.limit(bufferSize);
- this.bufferSize = bufferSize;
- bufferLastFlushOffset = 0;
- bufferNextWriteOffset = 0;
- }
-
- @Override
- public long getDiskNextWriteOffset() throws IOException {
- return fileChannel.position();
- }
-
- @Override
- public void setDiskNextWriteOffset(long offset) throws IOException {
- fileChannel.position(offset);
- }
-
- @Override
- public int getBufferLastFlushOffset() {
- return bufferLastFlushOffset;
- }
-
- @Override
- public void setBufferLastFlushOffset(int offset) {
- this.bufferLastFlushOffset = offset;
- }
-
- @Override
- public int getBufferNextWriteOffset() {
- synchronized (fileChannel) {
- return bufferNextWriteOffset;
- }
- }
-
- @Override
- public void setBufferNextWriteOffset(int offset) {
- synchronized (fileChannel) {
- if (bufferNextWriteOffset < offset) {
- bufferNextWriteOffset = offset;
- }
- }
- }
-
- @Override
- public void acquireWriteLatch() {
- latch.writeLock().lock();
- }
-
- @Override
- public void releaseWriteLatch() {
- latch.writeLock().unlock();
- }
-
- @Override
- public void acquireReadLatch() {
- latch.readLock().lock();
- }
-
- @Override
- public void releaseReadLatch() {
- latch.readLock().unlock();
- }
-
- @Override
- public void incRefCnt() {
- referenceCount.incrementAndGet();
- }
-
- @Override
- public void decRefCnt() {
- referenceCount.decrementAndGet();
- }
-
- @Override
- public int getRefCnt() {
- return referenceCount.get();
- }
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileUtil.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileUtil.java
deleted file mode 100644
index 64beb86..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileUtil.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.common.transactions;
-
-import java.io.File;
-import java.io.IOException;
-
-/**
- * A utility class that provides operations on files such as creation and
- * loading content in a buffer. It also provides API for creating directories
- */
-public class FileUtil {
-
- public static final String lineSeparator = System.getProperty("line.separator");
-
- public static boolean createFileIfNotExists(String path) throws IOException {
- File file = new File(path);
- File parentFile = file.getParentFile();
- if (parentFile != null) {
- parentFile.mkdirs();
- }
- return file.createNewFile();
- }
-
- public static boolean createNewDirectory(String path) throws IOException {
- return (new File(path)).mkdir();
- }
-
- public static IFileBasedBuffer getFileBasedBuffer(String filePath, long offset, int bufferSize, int diskSectorSize) throws IOException {
- IFileBasedBuffer fileBasedBuffer = new FileBasedBuffer(filePath, offset, bufferSize, diskSectorSize);
- return fileBasedBuffer;
- }
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IAsterixAppRuntimeContextProvider.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IAsterixAppRuntimeContextProvider.java
index 05ac025..2f522b9 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IAsterixAppRuntimeContextProvider.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IAsterixAppRuntimeContextProvider.java
@@ -14,6 +14,8 @@
*/
package edu.uci.ics.asterix.common.transactions;
+import java.util.List;
+
import edu.uci.ics.hyracks.api.io.IIOManager;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
@@ -58,5 +60,5 @@
public IIOManager getIOManager();
- public IVirtualBufferCache getVirtualBufferCache(int datasetID);
+ public List<IVirtualBufferCache> getVirtualBufferCaches(int datasetID);
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IBuffer.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IBuffer.java
deleted file mode 100644
index fd4af81..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IBuffer.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.common.transactions;
-
-import java.nio.ByteBuffer;
-
-/**
- * Represents a wrapper over @see ByteBuffer supporting some custom APIs for
- * transaction support.
- */
-public interface IBuffer {
-
- public byte getByte(int offset);
-
- public int getSize();
-
- public byte getByte();
-
- public void getBytes(byte[] bytes, int start, int size);
-
- public int readInt();
-
- public int readInt(int offset);
-
- public long readLong(int offset);
-
- public void put(byte b);
-
- public void put(int offset, byte b);
-
- public void put(byte[] bytes, int start, int length);
-
- public void put(byte[] bytes);
-
- public void writeInt(int value);
-
- public void writeLong(long value);
-
- public void writeInt(int offset, int value);
-
- public void writeLong(int offset, long value);
-
- public byte[] getArray();
-
- public void erase();
-
- public ByteBuffer getByteBuffer();
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ICloseable.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ICloseable.java
deleted file mode 100644
index 1c8cb76..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ICloseable.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.asterix.common.transactions;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-
-/**
- * Represents a closeable resource that implements a close(@see
- * TransactionContext) method. When a transaction commits/aborts, the close
- * method is called on each of the ICloseable resources that were involved in
- * the transaction. This gives an opportunity to release all resources and do a
- * cleanup. An example of ICloseable is the @see TreeLogger.
- */
-public interface ICloseable {
-
- /**
- * This method is invoked at the commit/abort of a transaction that involved
- * a ICloseable resource. It is used to do a clean up by the involved
- * resource before the transaction ends.
- *
- * @param context
- * @throws ACIDException
- */
- public void close(ITransactionContext context) throws ACIDException;
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IFileBasedBuffer.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IFileBasedBuffer.java
deleted file mode 100644
index 1d466e3..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IFileBasedBuffer.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.common.transactions;
-
-import java.io.IOException;
-
-/**
- * Represent a buffer that is backed by a physical file. Provides custom APIs
- * for accessing a chunk of the underlying file.
- */
-
-public interface IFileBasedBuffer extends IBuffer {
-
- public void flush() throws IOException;
-
- /**
- * Resets the buffer with content (size as specified) from a given file
- * starting at offset.
- */
- public void reset(String filePath, long offset, int size) throws IOException;
-
- public long getDiskNextWriteOffset() throws IOException;
-
- public void setDiskNextWriteOffset(long writePosition) throws IOException;
-
- public void close() throws IOException;
-
- public void open(String filePath, long offset, int size) throws IOException;
-
- public int getBufferLastFlushOffset();
-
- public void setBufferLastFlushOffset(int offset);
-
- public int getBufferNextWriteOffset();
-
- public void setBufferNextWriteOffset(int offset);
-
- public void acquireWriteLatch();
-
- public void releaseWriteLatch();
-
- public void acquireReadLatch();
-
- public void releaseReadLatch();
-
- public void incRefCnt();
-
- public void decRefCnt();
-
- public int getRefCnt();
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILockManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILockManager.java
index 698a788..a752afa 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILockManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILockManager.java
@@ -64,20 +64,9 @@
* @param txnContext
* @throws ACIDException
* TODO
- * @return true if the lock count is 0, false otherwise.
+ * @return
*/
- public boolean unlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext)
- throws ACIDException;
-
- /**
- * @param datasetId
- * @param entityHashValue
- * @param txnContext
- * @throws ACIDException
- * TODO
- * @return true if the lock count is 0, false otherwise.
- */
- public boolean unlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext, boolean commitFlag)
+ public void unlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext)
throws ACIDException;
/**
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogFilter.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogFilter.java
deleted file mode 100644
index 5c5c149..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogFilter.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.common.transactions;
-
-import edu.uci.ics.asterix.common.transactions.IBuffer;
-
-/**
- * Represents a predicate or a filter that needs to be applied when selectively
- * retrieving logs.
- */
-public interface ILogFilter {
-
- public boolean accept(IBuffer buffer, long startOffset, int length);
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java
index 9cc4d0d..8913f8a 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java
@@ -14,73 +14,10 @@
*/
package edu.uci.ics.asterix.common.transactions;
-import java.io.IOException;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-
public interface ILogManager {
- /**
- * @param logType
- * @param context
- * @param datasetId
- * @param PKHashValue
- * @param resourceId
- * @param resourceMgrId
- * @param logContentSize
- * @param reusableLogContentObject
- * @param logger
- * @param logicalLogLocator
- * @throws ACIDException
- */
- void log(byte logType, ITransactionContext context, int datasetId, int PKHashValue, long resourceId,
- byte resourceMgrId, int logContentSize, ReusableLogContentObject reusableLogContentObject, ILogger logger,
- LogicalLogLocator logicalLogLocator) throws ACIDException;
+ public void log(ILogRecord logRecord);
- /**
- * @param physicalLogLocator
- * specifies the physical location from where the logs need to be
- * read
- * @param logFilter
- * specifies the filtering criteria for the retrieved logs
- * @return LogCursor an iterator for the retrieved logs
- * @throws ACIDException
- */
- public ILogCursor readLog(PhysicalLogLocator physicalLogLocator, ILogFilter logFilter) throws IOException,
- ACIDException;
-
- /**
- * @param lsnValue
- * TODO
- * @param logicalLogLocator
- * TODO
- * @throws ACIDException
- */
- public void readLog(long lsnValue, LogicalLogLocator logicalLogLocator) throws ACIDException;
-
- /**
- * Retrieves the configuration parameters of the ILogManager
- *
- * @return LogManagerProperties: the configuration parameters for the
- * ILogManager
- */
- public LogManagerProperties getLogManagerProperties();
-
- /**
- * Returns the ILogRecordHelper instance associated with this ILogManager
- * instance
- *
- * @return ILogRecordHelper: the utility (class) for writing/reading log
- * header.
- */
- public ILogRecordHelper getLogRecordHelper();
-
- /**
- * Returns the Transaction Provider associated with this ILogManager
- * instance
- *
- * @return TransactionSubsystem
- */
- public ITransactionSubsystem getTransactionSubsystem();
+ public ILogReader getLogReader(boolean isRecoveryMode);
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILoggerRepository.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogPage.java
similarity index 78%
rename from asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILoggerRepository.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogPage.java
index 08940a7..be205a9 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILoggerRepository.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogPage.java
@@ -14,9 +14,10 @@
*/
package edu.uci.ics.asterix.common.transactions;
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
+public interface ILogPage {
-public interface ILoggerRepository {
+ public void append(ILogRecord logRecord, long appendLsn);
- public ILogger getIndexLogger(long resourceId, byte resourceType) throws ACIDException;
-}
+ public void flush();
+
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogCursor.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogReader.java
similarity index 70%
rename from asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogCursor.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogReader.java
index e455d3cc..fa92865 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogCursor.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogReader.java
@@ -14,18 +14,18 @@
*/
package edu.uci.ics.asterix.common.transactions;
-import java.io.IOException;
-
import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-/**
- * Provides a cursor over the logs created to date.
- */
-public interface ILogCursor {
+public interface ILogReader {
- public boolean next(LogicalLogLocator currentLogLocator) throws IOException, ACIDException;
+ public void initializeScan(long beginLSN) throws ACIDException;
- public ILogFilter getLogFilter();
+ //for scanning
+ public ILogRecord next() throws ACIDException;
-}
+ //for random reading
+ public ILogRecord read(long readLSN) throws ACIDException;
+
+ public void close() throws ACIDException;
+
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecord.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecord.java
new file mode 100644
index 0000000..d13ef6c
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecord.java
@@ -0,0 +1,122 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.common.transactions;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public interface ILogRecord {
+
+ public static final int JOB_COMMIT_LOG_SIZE = 13;
+ public static final int ENTITY_COMMIT_LOG_BASE_SIZE = 29;
+ public static final int UPDATE_LOG_BASE_SIZE = 64;
+
+ public boolean readLogRecord(ByteBuffer buffer);
+
+ public void writeLogRecord(ByteBuffer buffer);
+
+ public void formJobCommitLogRecord(ITransactionContext txnCtx);
+
+ public void formEntityCommitLogRecord(ITransactionContext txnCtx, int datasetId, int PKHashValue,
+ ITupleReference tupleReference, int[] primaryKeyFields);
+
+ public ITransactionContext getTxnCtx();
+
+ public void setTxnCtx(ITransactionContext txnCtx);
+
+ public boolean isFlushed();
+
+ public void isFlushed(boolean isFlushed);
+
+ public byte getLogType();
+
+ public void setLogType(byte logType);
+
+ public int getJobId();
+
+ public void setJobId(int jobId);
+
+ public int getDatasetId();
+
+ public void setDatasetId(int datasetId);
+
+ public int getPKHashValue();
+
+ public void setPKHashValue(int PKHashValue);
+
+ public long getPrevLSN();
+
+ public void setPrevLSN(long prevLsn);
+
+ public long getResourceId();
+
+ public void setResourceId(long resourceId);
+
+ public byte getResourceType();
+
+ public void setResourceType(byte resourceType);
+
+ public int getLogSize();
+
+ public void setLogSize(int logSize);
+
+ public byte getNewOp();
+
+ public void setNewOp(byte newOp);
+
+ public int getNewValueSize();
+
+ public void setNewValueSize(int newValueSize);
+
+ public ITupleReference getNewValue();
+
+ public void setNewValue(ITupleReference newValue);
+
+ public byte getOldOp();
+
+ public void setOldOp(byte oldOp);
+
+ public int getOldValueSize();
+
+ public void setOldValueSize(int oldValueSize);
+
+ public ITupleReference getOldValue();
+
+ public void setOldValue(ITupleReference oldValue);
+
+ public long getChecksum();
+
+ public void setChecksum(long checksum);
+
+ public long getLSN();
+
+ public void setLSN(long LSN);
+
+ public String getLogRecordForDisplay();
+
+ public void computeAndSetLogSize();
+
+ public int getPKValueSize();
+
+ public ITupleReference getPKValue();
+
+ public void setPKFields(int[] primaryKeyFields);
+
+ public void computeAndSetPKValueSize();
+
+ public void setPKValue(ITupleReference PKValue);
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecordHelper.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecordHelper.java
deleted file mode 100644
index a299a0d..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecordHelper.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.asterix.common.transactions;
-
-/**
- * Helper class 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.
- */
-
-public interface ILogRecordHelper {
-
- public byte getLogType(LogicalLogLocator logicalLogLocator);
-
- public int getJobId(LogicalLogLocator logicalLogLocator);
-
- public int getDatasetId(LogicalLogLocator logicalLogLocator);
-
- public int getPKHashValue(LogicalLogLocator logicalLogLocator);
-
- public PhysicalLogLocator getPrevLSN(LogicalLogLocator logicalLogLocator);
-
- public boolean getPrevLSN(PhysicalLogLocator physicalLogLocator, LogicalLogLocator logicalLogLocator);
-
- public long getResourceId(LogicalLogLocator logicalLogLocator);
-
- public byte getResourceMgrId(LogicalLogLocator logicalLogLocater);
-
- public int getLogContentSize(LogicalLogLocator logicalLogLocater);
-
- public long getLogChecksum(LogicalLogLocator logicalLogLocator);
-
- public int getLogContentBeginPos(LogicalLogLocator logicalLogLocator);
-
- public int getLogContentEndPos(LogicalLogLocator logicalLogLocator);
-
- public String getLogRecordForDisplay(LogicalLogLocator logicalLogLocator);
-
- public void writeLogHeader(LogicalLogLocator logicalLogLocator, byte logType, ITransactionContext context,
- int datasetId, int PKHashValue, long prevLogicalLogLocator, long resourceId, byte resourceMgrId,
- int logRecordSize);
-
- public boolean validateLogRecord(LogicalLogLocator logicalLogLocator);
-
- public int getLogRecordSize(byte logType, int logBodySize);
-
- public int getLogHeaderSize(byte logType);
-
- public int getLogChecksumSize();
-
- public int getCommitLogSize();
-
-}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogger.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogger.java
deleted file mode 100644
index 939f8e5..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogger.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.common.transactions;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-
-/**
- * An interface providing call back APIs that are invoked {@link ILogManager} for providing the content for the log record and doing any pre/post
- * processing.
- */
-public interface ILogger {
-
- public void preLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject)
- throws ACIDException;
-
- public void log(ITransactionContext context, final LogicalLogLocator logicalLogLocator, int logContentSize,
- ReusableLogContentObject reusableLogContentObject) throws ACIDException;
-
- public void postLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject)
- throws ACIDException;
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IRecoveryManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IRecoveryManager.java
index be41a6b..6d2f3cb 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IRecoveryManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IRecoveryManager.java
@@ -17,7 +17,7 @@
import java.io.IOException;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
/**
* Provides API for failure recovery. Failure could be at application level and
@@ -33,6 +33,12 @@
HEALTHY,
CORRUPTED
}
+
+ public class ResourceType {
+ public static final byte LSM_BTREE = 0;
+ public static final byte LSM_RTREE = 1;
+ public static final byte LSM_INVERTED_INDEX = 2;
+ }
/**
* Returns the state of the system.
@@ -69,5 +75,5 @@
*/
public void rollbackTransaction(ITransactionContext txnContext) throws ACIDException;
- public void checkpoint(boolean isSharpCheckpoint) throws ACIDException;
+ public void checkpoint(boolean isSharpCheckpoint) throws ACIDException, HyracksDataException;
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IResourceManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IResourceManager.java
deleted file mode 100644
index 36ce317..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IResourceManager.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.common.transactions;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-
-/**
- * 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.
- *
- * @return a unique identifier for the resource manager. The number of
- * resource managers in the system are expected to be handful and
- * can be uniquely identified by using a single byte as an id.
- */
- public byte getResourceManagerId();
-
- /**
- * Undo the operation corresponding to a given log record.
- *
- * @param logRecordHelper
- * (@see ILogRecordHelper) An implementation of the
- * ILogRecordHelper interface that is used to parse the log
- * record and extract useful information from the content.
- * @param LogicalLogLocator
- * (@see LogicalLogLocator) The locationof the log record that
- * needs to be undone.
- * @throws ACIDException
- */
- public void undo(ILogRecordHelper logRecordHelper, LogicalLogLocator logicalLogLocator) throws ACIDException;
-
- /**
- * Redo the operation corresponding to a given log record.
- *
- * @param logRecordHelper
- * (@see ILogRecordHelper) An implementation of the
- * ILogRecordHelper interface that is used to parse the log
- * record and extract useful information from the content.
- * @param LogicalLogLocator
- * (@see LogicalLogLocator) The locationof the log record that
- * needs to be undone.
- * @throws ACIDException
- */
- public void redo(ILogRecordHelper logRecordHelper, LogicalLogLocator memLSN) throws ACIDException;
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionContext.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionContext.java
index 3ebb963..dc33e69 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionContext.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionContext.java
@@ -14,61 +14,38 @@
*/
package edu.uci.ics.asterix.common.transactions;
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.ITransactionManager.TransactionState;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
public interface ITransactionContext {
- public void registerIndexAndCallback(ILSMIndex index, AbstractOperationCallback callback);
-
- public void updateLastLSNForIndexes(long lastLSN);
-
- public void decreaseActiveTransactionCountOnIndexes() throws HyracksDataException;
-
- public int getActiveOperationCountOnIndexes() throws HyracksDataException;
-
- public LogicalLogLocator getFirstLogLocator();
-
- public LogicalLogLocator getLastLogLocator();
-
- public void addCloseableResource(ICloseable resource);
+ public void registerIndexAndCallback(long resourceId, ILSMIndex index, AbstractOperationCallback callback,
+ boolean isPrimaryIndex);
public JobId getJobId();
- public void setStartWaitTime(long time);
+ public void setTimeout(boolean isTimeout);
- public long getStartWaitTime();
+ public boolean isTimeout();
- public void setStatus(int status);
+ public void setTxnState(int txnState);
- public int getStatus();
+ public int getTxnState();
- public void setTxnState(TransactionState txnState);
+ public long getFirstLSN();
- public TransactionState getTxnState();
+ public long getLastLSN();
- public void releaseResources() throws ACIDException;
+ public void setLastLSN(long resourceId, long LSN);
- public void setLastLSN(long lsn);
+ public boolean isWriteTxn();
- public TransactionType getTransactionType();
-
- public void setTransactionType(TransactionType transactionType);
+ public void setWriteTxn(boolean isWriterTxn);
public String prettyPrint();
- 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_STATUS = 1;
+ public void setMetadataTransaction(boolean isMetadataTxn);
- public enum TransactionType {
- READ,
- READ_WRITE
- }
+ public boolean isMetadataTransaction();
- public void setExclusiveJobLevelCommit();
-
+ public void notifyOptracker(boolean isJobLevelCommit);
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionManager.java
index 9a99dc9..77e960b 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionManager.java
@@ -29,12 +29,10 @@
* transaction has committed. ABORTED: The transaction has aborted.
* TIMED_OUT: The transaction has timed out waiting to acquire a lock.
*/
- public enum TransactionState {
- ACTIVE,
- COMMITTED,
- ABORTED,
- TIMED_OUT,
- };
+ public static final int ACTIVE = 0;
+ public static final int COMMITTED = 1;
+ public static final int ABORTED = 2;
+ public static final int TIMED_OUT = 3;
/**
* Begins a transaction identified by a transaction id and returns the
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionSubsystem.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionSubsystem.java
index 63ee5d0..b6c934a 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionSubsystem.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionSubsystem.java
@@ -14,7 +14,6 @@
*/
package edu.uci.ics.asterix.common.transactions;
-
public interface ITransactionSubsystem {
public ILogManager getLogManager();
@@ -25,10 +24,6 @@
public IRecoveryManager getRecoveryManager();
- public TransactionalResourceManagerRepository getTransactionalResourceRepository();
-
- public ILoggerRepository getTreeLoggerRepository();
-
public IAsterixAppRuntimeContextProvider getAsterixAppRuntimeContextProvider();
public String getId();
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogManagerProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogManagerProperties.java
index 4b5eb9b..dd1e7b4 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogManagerProperties.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogManagerProperties.java
@@ -22,6 +22,7 @@
private static final long serialVersionUID = 2084227360840799662L;
+ public static final String lineSeparator = System.getProperty("line.separator");
public static final int LOG_MAGIC_NUMBER = 123456789;
public static final String LOG_DIR_SUFFIX = ".txnLogDir";
private static final String DEFAULT_LOG_FILE_PREFIX = "asterix_transaction_log";
@@ -35,14 +36,10 @@
private final int logPageSize;
// number of log pages in the log buffer.
private final int numLogPages;
- // time in milliseconds
- private final long groupCommitWaitPeriod;
// logBufferSize = logPageSize * numLogPages;
private final int logBufferSize;
// maximum size of each log file
private final long logPartitionSize;
- // default disk sector size
- private final int diskSectorSize;
public LogManagerProperties(AsterixTransactionProperties txnProperties, String nodeId) {
this.logDirKey = new String(nodeId + LOG_DIR_SUFFIX);
@@ -51,12 +48,9 @@
long logPartitionSize = txnProperties.getLogPartitionSize();
this.logDir = txnProperties.getLogDirectory(nodeId);
this.logFilePrefix = DEFAULT_LOG_FILE_PREFIX;
- this.groupCommitWaitPeriod = txnProperties.getGroupCommitInterval();
-
this.logBufferSize = logPageSize * numLogPages;
//make sure that the log partition size is the multiple of log buffer size.
this.logPartitionSize = (logPartitionSize / logBufferSize) * logBufferSize;
- this.diskSectorSize = txnProperties.getLogDiskSectorSize();
}
public long getLogPartitionSize() {
@@ -83,27 +77,17 @@
return logBufferSize;
}
- public long getGroupCommitWaitPeriod() {
- return groupCommitWaitPeriod;
- }
-
public String getLogDirKey() {
return logDirKey;
}
- public int getDiskSectorSize() {
- return diskSectorSize;
- }
-
public String toString() {
StringBuilder builder = new StringBuilder();
- builder.append("log_dir_ : " + logDir + FileUtil.lineSeparator);
- builder.append("log_file_prefix" + logFilePrefix + FileUtil.lineSeparator);
- builder.append("log_page_size : " + logPageSize + FileUtil.lineSeparator);
- builder.append("num_log_pages : " + numLogPages + FileUtil.lineSeparator);
- builder.append("log_partition_size : " + logPartitionSize + FileUtil.lineSeparator);
- builder.append("group_commit_wait_period : " + groupCommitWaitPeriod + FileUtil.lineSeparator);
- builder.append("disk_sector_size : " + diskSectorSize + FileUtil.lineSeparator);
+ builder.append("log_dir_ : " + logDir + lineSeparator);
+ builder.append("log_file_prefix" + logFilePrefix + lineSeparator);
+ builder.append("log_page_size : " + logPageSize + lineSeparator);
+ builder.append("num_log_pages : " + numLogPages + lineSeparator);
+ builder.append("log_partition_size : " + logPartitionSize + lineSeparator);
return builder.toString();
}
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogUtil.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogUtil.java
deleted file mode 100644
index 58fc62a..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogUtil.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.common.transactions;
-
-import java.io.File;
-import java.io.FilenameFilter;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-
-/**
- * A utility class providing helper methods for the {@link ILogManager}
- */
-public class LogUtil {
-
- private static final Logger LOGGER = Logger.getLogger(LogUtil.class.getName());
-
- // read the log directory and initialize log anchor to point to the
- // current log partition file and the offset where the log manager shall
- // continue to insert log records.
-
- public static PhysicalLogLocator initializeLogAnchor(ILogManager logManager) throws ACIDException {
- int fileId = 0;
- long offset = 0;
- LogManagerProperties logManagerProperties = logManager.getLogManagerProperties();
- File logDir = new File(logManagerProperties.getLogDir());
- try {
- if (logDir.exists()) {
- List<String> logFiles = getLogFiles(logManagerProperties);
- if (logFiles == null || logFiles.size() == 0) {
- FileUtil.createFileIfNotExists(getLogFilePath(logManagerProperties, 0));
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("created a log file: " + getLogFilePath(logManagerProperties, 0));
- }
- } else {
- File logFile = new File(LogUtil.getLogFilePath(logManagerProperties,
- Long.parseLong(logFiles.get(logFiles.size() - 1))));
- fileId = logFiles.size() - 1;
- offset = logFile.length();
- }
- } else {
- FileUtil.createNewDirectory(logManagerProperties.getLogDir());
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("created the log directory: " + logManagerProperties.getLogDir());
- }
- FileUtil.createFileIfNotExists(getLogFilePath(logManagerProperties, 0));
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("created a log file: " + getLogFilePath(logManagerProperties, 0));
- }
- }
- } catch (IOException ioe) {
- throw new ACIDException("Unable to initialize log anchor", ioe);
- }
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(" file id :" + fileId + " offset " + offset);
- }
- return new PhysicalLogLocator(fileId, offset, logManager);
- }
-
- public static List<String> getLogFiles(final LogManagerProperties logManagerProperties) {
- File logDir = new File(logManagerProperties.getLogDir());
- String[] logFiles = new String[0];
- List<String> logFileCollection = new ArrayList<String>();
- if (logDir.exists()) {
- logFiles = logDir.list(new FilenameFilter() {
- public boolean accept(File dir, String name) {
- if (name.startsWith(logManagerProperties.getLogFilePrefix())) {
- return true;
- }
- return false;
- }
- });
- }
- for (String logFile : logFiles) {
- logFileCollection.add(logFile.substring(logManagerProperties.getLogFilePrefix().length() + 1));
- }
- Collections.sort(logFileCollection, new Comparator<String>() {
- @Override
- public int compare(String arg0, String arg1) {
- return Integer.parseInt(arg0) - Integer.parseInt(arg1);
- }
- });
- return logFileCollection;
- }
-
- public static long getFileId(String logFilePath, LogManagerProperties logManagerProperties) {
- String logFileName = logFilePath;
- if (logFilePath.contains(File.separator)) {
- logFileName = logFilePath.substring(logFilePath.lastIndexOf(File.separator));
- }
- return Long.parseLong(logFileName.substring(logFileName.indexOf(logManagerProperties.getLogFilePrefix())));
- }
-
- public static String getLogFilePath(LogManagerProperties logManagerProperties, long fileId) {
- return logManagerProperties.getLogDir() + File.separator + logManagerProperties.getLogFilePrefix() + "_"
- + fileId;
- }
-
- public static LogicalLogLocator getDummyLogicalLogLocator(ILogManager logManager) {
- LogicalLogLocator logicalLogLocator = new LogicalLogLocator(-1, null, -1, logManager);
- return logicalLogLocator;
- }
-
- /*
- * given a lsn, get the offset within the log file where the corresponding
- * log record is (to be) placed.
- */
- public static long getFileOffset(ILogManager logManager, long lsn) {
- return lsn % logManager.getLogManagerProperties().getLogPartitionSize();
- }
-
- /*
- * given a lsn, get the file id that contains the log record.
- */
- public static long getFileId(ILogManager logManager, long lsn) {
- return lsn / logManager.getLogManagerProperties().getLogPartitionSize();
- }
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogicalLogLocator.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogicalLogLocator.java
deleted file mode 100644
index 9ba7682..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogicalLogLocator.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.common.transactions;
-
-import java.util.concurrent.atomic.AtomicInteger;
-
-/**
- * Represents a location of a log record. The location has two parts to it. A
- * LogicalLogLocator extends PhyscialLogLocator and hence can also be used to
- * determine the physical location of the log record on the local filesystem. In
- * addition to the physical location, a LogicalLogLocator also contains a handle
- * to an in-memory buffer and an offset within the buffer where the log record
- * resides.
- */
-public class LogicalLogLocator extends PhysicalLogLocator {
-
- private IBuffer buffer;
- private AtomicInteger memoryOffset;
-
- public LogicalLogLocator(long lsnValue, IBuffer buffer, int bufferOffset, ILogManager logManager) {
- super(lsnValue, logManager);
- this.buffer = buffer;
- this.memoryOffset = new AtomicInteger(bufferOffset);
-
- }
-
- public IBuffer getBuffer() {
- return buffer;
- }
-
- public void setBuffer(IBuffer buffer) {
- this.buffer = buffer;
- }
-
- public int getMemoryOffset() {
- return memoryOffset.get();
- }
-
- public void setMemoryOffset(int memoryOffset) {
- this.memoryOffset.set(memoryOffset);
- }
-
- @Override
- public String toString() {
- return super.toString() + " " + "memoryOffset:" + memoryOffset;
- }
-
- public boolean checkValidity() {
- return true;
- }
-
- public long increaseMemoryOffset(int delta) {
- return memoryOffset.addAndGet(delta);
- }
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/MutableResourceId.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/MutableLong.java
similarity index 66%
rename from asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/MutableResourceId.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/MutableLong.java
index fb865aa..4480aba 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/MutableResourceId.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/MutableLong.java
@@ -14,31 +14,39 @@
*/
package edu.uci.ics.asterix.common.transactions;
-public class MutableResourceId{
- long id;
+public class MutableLong {
+ private long val;
- public MutableResourceId(long id) {
- this.id = id;
+
+ public MutableLong() {
+ }
+
+ public MutableLong(long val) {
+ this.val = val;
+ }
+
+ public void set(long val) {
+ this.val = val;
}
- public void setId(long id) {
- this.id = id;
+ public long get() {
+ return val;
}
-
- public long getId() {
- return id;
- }
-
+
@Override
public int hashCode() {
- return (int)id;
+ return (int)val;
}
@Override
public boolean equals(Object o) {
- if ((o == null) || !(o instanceof MutableResourceId)) {
+ if (o == this) {
+ return true;
+ }
+ if (!(o instanceof MutableLong)) {
return false;
}
- return ((MutableResourceId) o).id == this.id;
+ return ((MutableLong) o).val == val;
}
+
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/PhysicalLogLocator.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/PhysicalLogLocator.java
deleted file mode 100644
index b83a3f3..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/PhysicalLogLocator.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.common.transactions;
-
-import java.util.concurrent.atomic.AtomicLong;
-
-/**
- * Represents the physical location of a log record. The physical location i
- * deciphered from the contained lsn that is broken down into a file id and an
- * offset within the file. The mapping between fileId and the path on the local
- * file system is maintained by the log manager (@see ILogManager) path on the
- * local file system.
- */
-public class PhysicalLogLocator {
-
- // The log sequence number corresponding to the log record that is being
- // referred to.
- private final AtomicLong lsn;
-
- // Handle to the log manager that wrote the log record.
- private final ILogManager logManager;
-
- public static long getLsnValue(long fileId, long offset, ILogManager logManager) {
- return fileId * logManager.getLogManagerProperties().getLogPartitionSize() + offset;
- }
-
- public PhysicalLogLocator(long lsn, ILogManager logManager) {
- this.lsn = new AtomicLong(lsn);
- this.logManager = logManager;
- }
-
- public PhysicalLogLocator(long fileId, long offset, ILogManager logManager) {
- this.lsn = new AtomicLong(getLsnValue(fileId, offset, logManager));
- this.logManager = logManager;
- }
-
- @Override
- public String toString() {
- return "lsn :" + lsn.get();
- }
-
- public long getFileId() {
- return LogUtil.getFileId(logManager, lsn.get());
- }
-
- public boolean compareAndSet(long expect, long update) {
- return lsn.compareAndSet(expect, update);
- }
-
- public long getFileOffset() {
- return LogUtil.getFileOffset(logManager, lsn.get());
- }
-
- public long getLsn() {
- return lsn.get();
- }
-
- public long incrementLsn() {
- return lsn.incrementAndGet();
- }
-
- public long incrementLsn(long delta) {
- return lsn.addAndGet(delta);
- }
-
- public void setLsn(long lsn) {
- this.lsn.set(lsn);
- }
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ReusableLogContentObject.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ReusableLogContentObject.java
deleted file mode 100644
index b513fad..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ReusableLogContentObject.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.common.transactions;
-
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
-
-/**
- * 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-common/src/main/java/edu/uci/ics/asterix/common/transactions/TransactionalResourceManagerRepository.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/TransactionalResourceManagerRepository.java
deleted file mode 100644
index 64c90f4..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/TransactionalResourceManagerRepository.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.asterix.common.transactions;
-
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * Represents a repository containing Resource Managers and Resources in the
- * transaction eco-system. Operations on a resource require acquiring
- * appropriate locks (for isolation) and writing logs (durability). Every
- * resource is managed by an associated resource manager that contains the logic
- * to interpret the logs and take necessary action(s) during roll back or
- * recovery. An example of resource is a @see ITreeIndex that is managed by a
- * resource manager @see TreeResourceManager
- */
-public class TransactionalResourceManagerRepository {
-
- private Map<Byte, IResourceManager> resourceMgrRepository = new HashMap<Byte, IResourceManager>(); // repository
-
- public void registerTransactionalResourceManager(byte id, IResourceManager resourceMgr) {
- synchronized (resourceMgrRepository) {
- if (resourceMgrRepository.get(id) == null) {
- resourceMgrRepository.put(id, resourceMgr);
- }
- }
- }
-
- public IResourceManager getTransactionalResourceMgr(byte id) {
- synchronized (resourceMgrRepository) {
- return resourceMgrRepository.get(id);
- }
- }
-}
diff --git a/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java b/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java
index 88c65ff..861cce5 100644
--- a/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java
+++ b/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java
@@ -325,6 +325,8 @@
List<CompilationUnit> cUnits = testCaseCtx.getTestCase().getCompilationUnit();
for (CompilationUnit cUnit : cUnits) {
+ LOGGER.info("Starting [TEST]: " + testCaseCtx.getTestCase().getFilePath() + "/"
+ + cUnit.getName() + " ... ");
testFileCtxs = testCaseCtx.getTestFiles(cUnit);
expectedResultFileCtxs = testCaseCtx.getExpectedResultFiles(cUnit);
diff --git a/asterix-doc/src/site/markdown/install.md b/asterix-doc/src/site/markdown/install.md
index 4d53f75..9ba8ea3 100644
--- a/asterix-doc/src/site/markdown/install.md
+++ b/asterix-doc/src/site/markdown/install.md
@@ -880,3 +880,32 @@
##### Answer #####
It is recommended that MANAGIX_HOME is not on the NFS. Managix produces artifacts/logs on disk which are not required to be shared.
As such an overhead in creating the artifacts/logs on the NFS should be avoided.
+
+##### Question #####
+
+Question: How do we change the underlying code (apply a code patch) for an 'active' asterix instance?
+
+##### Answer #####
+
+At times, end-user (particularly asterix developer) may run into the need to altering the underlying code that is being run by an asterix instance. In the current version of managix, this can be achieved as follows:-
+
+Assume that you have an 'active' instance by the name a1 that is running version v1 of asterix.
+You have a revised version of asterix - v2 that fixes some bug(s).
+
+To upgrade asterix from v1 to v2:-
+
+step 1) managix stop -n a1
+
+step 2) managix shutdown
+
+step 3) copy asterix-server zip (version v2) to $MANAGIX_HOME/asterix/
+
+step 4) managix start -n a1
+
+a1 now is running on version v2.
+
+Limitations:-
+
+a) Obviously this wont work in a situation where v2 has made a change that is incompatible with earlier version, such altering schema.
+
+b) A change in asterix zip applies to all existing instances (after a restart) and subsequent instances that user creates.
diff --git a/asterix-doc/src/site/site.xml b/asterix-doc/src/site/site.xml
index 8c2a8a8..150544e 100644
--- a/asterix-doc/src/site/site.xml
+++ b/asterix-doc/src/site/site.xml
@@ -59,7 +59,7 @@
</links>
<menu name="Documentation">
- <item name="Installing AsterixDB using Managix" href="install.html"/>
+ <item name="Installing and Managing AsterixDB using Managix" href="install.html"/>
<item name="AsterixDB 101: An ADM and AQL Primer" href="aql/primer.html"/>
<item name="Asterix Data Model (ADM)" href="aql/datamodel.html"/>
<item name="Asterix Functions" href="aql/functions.html"/>
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/FileSystemBasedAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/FileSystemBasedAdapter.java
index 38903ec..33ee11f 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/FileSystemBasedAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/FileSystemBasedAdapter.java
@@ -16,6 +16,7 @@
import java.io.IOException;
import java.io.InputStream;
+import java.util.List;
import java.util.Map;
import java.util.logging.Level;
import java.util.logging.Logger;
@@ -26,6 +27,7 @@
import edu.uci.ics.asterix.external.util.INodeResolverFactory;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.AUnionType;
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.asterix.runtime.operators.file.AdmSchemafullRecordParserFactory;
import edu.uci.ics.asterix.runtime.operators.file.NtDelimitedDataTupleParserFactory;
@@ -89,7 +91,7 @@
throw new IllegalArgumentException(" Unspecified data format");
} else if (FORMAT_DELIMITED_TEXT.equalsIgnoreCase(specifiedFormat)) {
parserFactory = getDelimitedDataTupleParserFactory((ARecordType) atype);
- } else if (FORMAT_ADM.equalsIgnoreCase((String)configuration.get(KEY_FORMAT))) {
+ } else if (FORMAT_ADM.equalsIgnoreCase((String) configuration.get(KEY_FORMAT))) {
parserFactory = getADMDataTupleParserFactory((ARecordType) atype);
} else {
throw new IllegalArgumentException(" format " + configuration.get(KEY_FORMAT) + " not supported");
@@ -104,7 +106,19 @@
int n = recordType.getFieldTypes().length;
IValueParserFactory[] fieldParserFactories = new IValueParserFactory[n];
for (int i = 0; i < n; i++) {
- ATypeTag tag = recordType.getFieldTypes()[i].getTypeTag();
+ ATypeTag tag = null;
+ if (recordType.getFieldTypes()[i].getTypeTag() == ATypeTag.UNION) {
+ List<IAType> unionTypes = ((AUnionType) recordType.getFieldTypes()[i]).getUnionList();
+ if (unionTypes.size() != 2 && unionTypes.get(0).getTypeTag() != ATypeTag.NULL) {
+ throw new NotImplementedException("Non-optional UNION type is not supported.");
+ }
+ tag = unionTypes.get(1).getTypeTag();
+ } else {
+ tag = recordType.getFieldTypes()[i].getTypeTag();
+ }
+ if (tag == null) {
+ throw new NotImplementedException("Failed to get the type information for field " + i + ".");
+ }
IValueParserFactory vpf = typeToValueParserFactMap.get(tag);
if (vpf == null) {
throw new NotImplementedException("No value parser factory for delimited fields of type " + tag);
diff --git a/asterix-installer/src/main/resources/conf/asterix-configuration.xml b/asterix-installer/src/main/resources/conf/asterix-configuration.xml
index 6b3133c..084a3f8 100644
--- a/asterix-installer/src/main/resources/conf/asterix-configuration.xml
+++ b/asterix-installer/src/main/resources/conf/asterix-configuration.xml
@@ -68,6 +68,14 @@
</description>
</property>
+ <property>
+ <name>storage.memorycomponent.numcomponents</name>
+ <value>2</value>
+ <description>The number of memory components to be used per lsm index.
+ (Default = 2)
+ </description>
+ </property>
+
<property>
<name>storage.memorycomponent.globalbudget</name>
<value>536870192</value>
@@ -118,21 +126,6 @@
</property>
<property>
- <name>txn.log.disksectorsize</name>
- <value>4096</value>
- <description>The size of a disk sector. (Default = "4096")
- </description>
- </property>
-
- <property>
- <name>txn.log.groupcommitinterval</name>
- <value>40</value>
- <description>The group commit wait time in milliseconds. (Default =
- "40" // 40ms)
- </description>
- </property>
-
- <property>
<name>txn.log.checkpoint.lsnthreshold</name>
<value>67108864</value>
<description>The size of the window that the maximum LSN is allowed to
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 d9fed5b..6509da4 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
@@ -27,7 +27,7 @@
import edu.uci.ics.asterix.common.functions.FunctionSignature;
import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
import edu.uci.ics.asterix.common.transactions.DatasetId;
-import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.IRecoveryManager.ResourceType;
import edu.uci.ics.asterix.common.transactions.ITransactionContext;
import edu.uci.ics.asterix.common.transactions.ITransactionContext.TransactionType;
import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
@@ -91,86 +91,88 @@
import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
public class MetadataNode implements IMetadataNode {
- private static final long serialVersionUID = 1L;
+ private static final long serialVersionUID = 1L;
- private static final DatasetId METADATA_DATASET_ID = new DatasetId(MetadataPrimaryIndexes.METADATA_DATASET_ID);
+ private static final DatasetId METADATA_DATASET_ID = new DatasetId(MetadataPrimaryIndexes.METADATA_DATASET_ID);
- private IIndexLifecycleManager indexLifecycleManager;
- private ITransactionSubsystem transactionSubsystem;
+ private IIndexLifecycleManager indexLifecycleManager;
+ private ITransactionSubsystem transactionSubsystem;
- public static final MetadataNode INSTANCE = new MetadataNode();
+ public static final MetadataNode INSTANCE = new MetadataNode();
- private MetadataNode() {
- super();
- }
+ private MetadataNode() {
+ super();
+ }
- public void initialize(IAsterixAppRuntimeContext runtimeContext) {
- this.transactionSubsystem = runtimeContext.getTransactionSubsystem();
- this.indexLifecycleManager = runtimeContext.getIndexLifecycleManager();
- }
+ public void initialize(IAsterixAppRuntimeContext runtimeContext) {
+ this.transactionSubsystem = runtimeContext.getTransactionSubsystem();
+ this.indexLifecycleManager = runtimeContext.getIndexLifecycleManager();
+ }
- @Override
- public void beginTransaction(JobId transactionId) throws ACIDException, RemoteException {
- transactionSubsystem.getTransactionManager().beginTransaction(transactionId);
- }
+ @Override
+ public void beginTransaction(JobId transactionId) throws ACIDException, RemoteException {
+ ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().beginTransaction(transactionId);
+ txnCtx.setMetadataTransaction(true);
+ }
- @Override
- public void commitTransaction(JobId jobId) throws RemoteException, ACIDException {
- ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
- transactionSubsystem.getTransactionManager().commitTransaction(txnCtx, new DatasetId(-1), -1);
- }
+ @Override
+ public void commitTransaction(JobId jobId) throws RemoteException, ACIDException {
+ ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+ transactionSubsystem.getTransactionManager().commitTransaction(txnCtx, new DatasetId(-1), -1);
+ }
- @Override
- public void abortTransaction(JobId jobId) throws RemoteException, ACIDException {
- try {
- ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
- transactionSubsystem.getTransactionManager().abortTransaction(txnCtx, new DatasetId(-1), -1);
- } catch (ACIDException e) {
- e.printStackTrace();
- throw e;
- }
- }
+ @Override
+ public void abortTransaction(JobId jobId) throws RemoteException, ACIDException {
+ try {
+ ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+ transactionSubsystem.getTransactionManager().abortTransaction(txnCtx, new DatasetId(-1), -1);
+ } catch (ACIDException e) {
+ e.printStackTrace();
+ throw e;
+ }
+ }
- @Override
- public void lock(JobId jobId, byte lockMode) throws ACIDException, RemoteException {
- ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
- transactionSubsystem.getLockManager().lock(METADATA_DATASET_ID, -1, lockMode, txnCtx);
- }
+ @Override
+ public void lock(JobId jobId, byte lockMode) throws ACIDException, RemoteException {
+ ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+ transactionSubsystem.getLockManager().lock(METADATA_DATASET_ID, -1, lockMode, txnCtx);
+ }
- @Override
- public void unlock(JobId jobId) throws ACIDException, RemoteException {
- ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
- transactionSubsystem.getLockManager().unlock(METADATA_DATASET_ID, -1, txnCtx);
- }
+ @Override
+ public void unlock(JobId jobId) throws ACIDException, RemoteException {
+ ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+ transactionSubsystem.getLockManager().unlock(METADATA_DATASET_ID, -1, txnCtx);
+ }
- @Override
- public void addDataverse(JobId jobId, Dataverse dataverse) throws MetadataException, RemoteException {
- try {
- DataverseTupleTranslator tupleReaderWriter = new DataverseTupleTranslator(true);
- ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(dataverse);
- insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, tuple);
- } catch (TreeIndexDuplicateKeyException e) {
- throw new MetadataException("A dataverse with this name " + dataverse.getDataverseName()
- + " already exists.", e);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public void addDataverse(JobId jobId, Dataverse dataverse) throws MetadataException, RemoteException {
+ try {
+ DataverseTupleTranslator tupleReaderWriter = new DataverseTupleTranslator(true);
+ ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(dataverse);
+ insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, tuple);
+ } catch (TreeIndexDuplicateKeyException e) {
+ throw new MetadataException("A dataverse with this name " + dataverse.getDataverseName()
+ + " already exists.", e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public void addDataset(JobId jobId, Dataset dataset) throws MetadataException, RemoteException {
- try {
- // Insert into the 'dataset' dataset.
- DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(true);
- ITupleReference datasetTuple = tupleReaderWriter.getTupleFromMetadataEntity(dataset);
- insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
- if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
- // Add the primary index for the dataset.
- InternalDatasetDetails id = (InternalDatasetDetails) dataset.getDatasetDetails();
- Index primaryIndex = new Index(dataset.getDataverseName(), dataset.getDatasetName(),
- dataset.getDatasetName(), IndexType.BTREE, id.getPrimaryKey(), true, dataset.getPendingOp());
+ @Override
+ public void addDataset(JobId jobId, Dataset dataset) throws MetadataException, RemoteException {
+ try {
+ // Insert into the 'dataset' dataset.
+ DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(true);
+ ITupleReference datasetTuple = tupleReaderWriter.getTupleFromMetadataEntity(dataset);
+ insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
+ if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
+ // Add the primary index for the dataset.
+ InternalDatasetDetails id = (InternalDatasetDetails) dataset.getDatasetDetails();
+ Index primaryIndex = new Index(dataset.getDataverseName(), dataset.getDatasetName(),
+ dataset.getDatasetName(), IndexType.BTREE, id.getPrimaryKey(), true, dataset.getPendingOp());
addIndex(jobId, primaryIndex);
ITupleReference nodeGroupTuple = createTuple(id.getNodeGroupName(), dataset.getDataverseName(),
@@ -223,176 +225,177 @@
}
}
- @Override
- public void addNodeGroup(JobId jobId, NodeGroup nodeGroup) throws MetadataException, RemoteException {
- try {
- NodeGroupTupleTranslator tupleReaderWriter = new NodeGroupTupleTranslator(true);
- ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(nodeGroup);
- insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, tuple);
- } catch (TreeIndexDuplicateKeyException e) {
- throw new MetadataException("A nodegroup with name '" + nodeGroup.getNodeGroupName() + "' already exists.",
- e);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public void addNodeGroup(JobId jobId, NodeGroup nodeGroup) throws MetadataException, RemoteException {
+ try {
+ NodeGroupTupleTranslator tupleReaderWriter = new NodeGroupTupleTranslator(true);
+ ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(nodeGroup);
+ insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, tuple);
+ } catch (TreeIndexDuplicateKeyException e) {
+ throw new MetadataException("A nodegroup with name '" + nodeGroup.getNodeGroupName() + "' already exists.",
+ e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public void addDatatype(JobId jobId, Datatype datatype) throws MetadataException, RemoteException {
- try {
- DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(jobId, this, true);
- ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(datatype);
- insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
- } catch (TreeIndexDuplicateKeyException e) {
- throw new MetadataException("A datatype with name '" + datatype.getDatatypeName() + "' already exists.", e);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public void addDatatype(JobId jobId, Datatype datatype) throws MetadataException, RemoteException {
+ try {
+ DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(jobId, this, true);
+ ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(datatype);
+ insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
+ } catch (TreeIndexDuplicateKeyException e) {
+ throw new MetadataException("A datatype with name '" + datatype.getDatatypeName() + "' already exists.", e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public void addFunction(JobId jobId, Function function) throws MetadataException, RemoteException {
- try {
- // Insert into the 'function' dataset.
- FunctionTupleTranslator tupleReaderWriter = new FunctionTupleTranslator(true);
- ITupleReference functionTuple = tupleReaderWriter.getTupleFromMetadataEntity(function);
- insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, functionTuple);
+ @Override
+ public void addFunction(JobId jobId, Function function) throws MetadataException, RemoteException {
+ try {
+ // Insert into the 'function' dataset.
+ FunctionTupleTranslator tupleReaderWriter = new FunctionTupleTranslator(true);
+ ITupleReference functionTuple = tupleReaderWriter.getTupleFromMetadataEntity(function);
+ insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, functionTuple);
- } catch (TreeIndexDuplicateKeyException e) {
- throw new MetadataException("A function with this name " + function.getName() + " and arity "
- + function.getArity() + " already exists in dataverse '" + function.getDataverseName() + "'.", e);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ } catch (TreeIndexDuplicateKeyException e) {
+ throw new MetadataException("A function with this name " + function.getName() + " and arity "
+ + function.getArity() + " already exists in dataverse '" + function.getDataverseName() + "'.", e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- public void insertIntoDatatypeSecondaryIndex(JobId jobId, String dataverseName, String nestedTypeName,
- String topTypeName) throws Exception {
- ITupleReference tuple = createTuple(dataverseName, nestedTypeName, topTypeName);
- insertTupleIntoIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, tuple);
- }
+ public void insertIntoDatatypeSecondaryIndex(JobId jobId, String dataverseName, String nestedTypeName,
+ String topTypeName) throws Exception {
+ ITupleReference tuple = createTuple(dataverseName, nestedTypeName, topTypeName);
+ insertTupleIntoIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, tuple);
+ }
- private void insertTupleIntoIndex(JobId jobId, IMetadataIndex metadataIndex, ITupleReference tuple)
- throws Exception {
- long resourceID = metadataIndex.getResourceID();
- ILSMIndex lsmIndex = (ILSMIndex) indexLifecycleManager.getIndex(resourceID);
- indexLifecycleManager.open(resourceID);
+ private void insertTupleIntoIndex(JobId jobId, IMetadataIndex metadataIndex, ITupleReference tuple)
+ throws Exception {
+ long resourceID = metadataIndex.getResourceID();
+ ILSMIndex lsmIndex = (ILSMIndex) indexLifecycleManager.getIndex(resourceID);
+ indexLifecycleManager.open(resourceID);
- // prepare a Callback for logging
- IModificationOperationCallback modCallback = createIndexModificationCallback(jobId, resourceID, metadataIndex,
- lsmIndex, IndexOperation.INSERT);
+ // prepare a Callback for logging
+ IModificationOperationCallback modCallback = createIndexModificationCallback(jobId, resourceID, metadataIndex,
+ lsmIndex, IndexOperation.INSERT);
- IIndexAccessor indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
+ ILSMIndexAccessor indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
- ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
- txnCtx.setTransactionType(TransactionType.READ_WRITE);
- txnCtx.registerIndexAndCallback(lsmIndex, (AbstractOperationCallback) modCallback);
+ ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+ txnCtx.setWriteTxn(true);
+ txnCtx.registerIndexAndCallback(resourceID, lsmIndex, (AbstractOperationCallback) modCallback,
+ metadataIndex.isPrimaryIndex());
- // TODO: fix exceptions once new BTree exception model is in hyracks.
- indexAccessor.insert(tuple);
+ // TODO: fix exceptions once new BTree exception model is in hyracks.
+ indexAccessor.forceInsert(tuple);
- indexLifecycleManager.close(resourceID);
- }
+ indexLifecycleManager.close(resourceID);
+ }
- private IModificationOperationCallback createIndexModificationCallback(JobId jobId, long resourceId,
- IMetadataIndex metadataIndex, ILSMIndex lsmIndex, IndexOperation indexOp) throws Exception {
- ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+ private IModificationOperationCallback createIndexModificationCallback(JobId jobId, long resourceId,
+ IMetadataIndex metadataIndex, ILSMIndex lsmIndex, IndexOperation indexOp) throws Exception {
+ ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
- if (metadataIndex.isPrimaryIndex()) {
- return new PrimaryIndexModificationOperationCallback(metadataIndex.getDatasetId().getId(),
- metadataIndex.getPrimaryKeyIndexes(), txnCtx, transactionSubsystem.getLockManager(),
- transactionSubsystem, resourceId, ResourceType.LSM_BTREE, indexOp);
- } else {
- return new SecondaryIndexModificationOperationCallback(metadataIndex.getDatasetId().getId(),
- metadataIndex.getPrimaryKeyIndexes(), txnCtx, transactionSubsystem.getLockManager(),
- transactionSubsystem, resourceId, ResourceType.LSM_BTREE, indexOp);
- }
- }
+ if (metadataIndex.isPrimaryIndex()) {
+ return new PrimaryIndexModificationOperationCallback(metadataIndex.getDatasetId().getId(),
+ metadataIndex.getPrimaryKeyIndexes(), txnCtx, transactionSubsystem.getLockManager(),
+ transactionSubsystem, resourceId, ResourceType.LSM_BTREE, indexOp);
+ } else {
+ return new SecondaryIndexModificationOperationCallback(metadataIndex.getDatasetId().getId(),
+ metadataIndex.getPrimaryKeyIndexes(), txnCtx, transactionSubsystem.getLockManager(),
+ transactionSubsystem, resourceId, ResourceType.LSM_BTREE, indexOp);
+ }
+ }
- @Override
- public void dropDataverse(JobId jobId, String dataverseName) throws MetadataException, RemoteException {
- try {
- List<Dataset> dataverseDatasets;
+ @Override
+ public void dropDataverse(JobId jobId, String dataverseName) throws MetadataException, RemoteException {
+ try {
+ List<Dataset> dataverseDatasets;
- dataverseDatasets = getDataverseDatasets(jobId, dataverseName);
- if (dataverseDatasets != null && dataverseDatasets.size() > 0) {
- // Drop all datasets in this dataverse.
- for (int i = 0; i < dataverseDatasets.size(); i++) {
- dropDataset(jobId, dataverseName, dataverseDatasets.get(i).getDatasetName());
- }
- }
- List<Datatype> dataverseDatatypes;
- // As a side effect, acquires an S lock on the 'datatype' dataset
- // on behalf of txnId.
- dataverseDatatypes = getDataverseDatatypes(jobId, dataverseName);
- if (dataverseDatatypes != null && dataverseDatatypes.size() > 0) {
- // Drop all types in this dataverse.
- for (int i = 0; i < dataverseDatatypes.size(); i++) {
- forceDropDatatype(jobId, dataverseName, dataverseDatatypes.get(i).getDatatypeName());
- }
- }
+ dataverseDatasets = getDataverseDatasets(jobId, dataverseName);
+ if (dataverseDatasets != null && dataverseDatasets.size() > 0) {
+ // Drop all datasets in this dataverse.
+ for (int i = 0; i < dataverseDatasets.size(); i++) {
+ dropDataset(jobId, dataverseName, dataverseDatasets.get(i).getDatasetName());
+ }
+ }
+ List<Datatype> dataverseDatatypes;
+ // As a side effect, acquires an S lock on the 'datatype' dataset
+ // on behalf of txnId.
+ dataverseDatatypes = getDataverseDatatypes(jobId, dataverseName);
+ if (dataverseDatatypes != null && dataverseDatatypes.size() > 0) {
+ // Drop all types in this dataverse.
+ for (int i = 0; i < dataverseDatatypes.size(); i++) {
+ forceDropDatatype(jobId, dataverseName, dataverseDatatypes.get(i).getDatatypeName());
+ }
+ }
- // As a side effect, acquires an S lock on the 'Function' dataset
- // on behalf of txnId.
- List<Function> dataverseFunctions = getDataverseFunctions(jobId, dataverseName);
- if (dataverseFunctions != null && dataverseFunctions.size() > 0) {
- // Drop all functions in this dataverse.
- for (Function function : dataverseFunctions) {
- dropFunction(jobId, new FunctionSignature(dataverseName, function.getName(), function.getArity()));
- }
- }
+ // As a side effect, acquires an S lock on the 'Function' dataset
+ // on behalf of txnId.
+ List<Function> dataverseFunctions = getDataverseFunctions(jobId, dataverseName);
+ if (dataverseFunctions != null && dataverseFunctions.size() > 0) {
+ // Drop all functions in this dataverse.
+ for (Function function : dataverseFunctions) {
+ dropFunction(jobId, new FunctionSignature(dataverseName, function.getName(), function.getArity()));
+ }
+ }
- // As a side effect, acquires an S lock on the 'Adapter' dataset
- // on behalf of txnId.
- List<DatasourceAdapter> dataverseAdapters = getDataverseAdapters(jobId, dataverseName);
- if (dataverseAdapters != null && dataverseAdapters.size() > 0) {
- // Drop all functions in this dataverse.
- for (DatasourceAdapter adapter : dataverseAdapters) {
- dropAdapter(jobId, dataverseName, adapter.getAdapterIdentifier().getAdapterName());
- }
- }
+ // As a side effect, acquires an S lock on the 'Adapter' dataset
+ // on behalf of txnId.
+ List<DatasourceAdapter> dataverseAdapters = getDataverseAdapters(jobId, dataverseName);
+ if (dataverseAdapters != null && dataverseAdapters.size() > 0) {
+ // Drop all functions in this dataverse.
+ for (DatasourceAdapter adapter : dataverseAdapters) {
+ dropAdapter(jobId, dataverseName, adapter.getAdapterIdentifier().getAdapterName());
+ }
+ }
- // Delete the dataverse entry from the 'dataverse' dataset.
- ITupleReference searchKey = createTuple(dataverseName);
- // As a side effect, acquires an S lock on the 'dataverse' dataset
- // on behalf of txnId.
- ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, searchKey);
- deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, tuple);
+ // Delete the dataverse entry from the 'dataverse' dataset.
+ ITupleReference searchKey = createTuple(dataverseName);
+ // As a side effect, acquires an S lock on the 'dataverse' dataset
+ // on behalf of txnId.
+ ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, searchKey);
+ deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, tuple);
- // TODO: Change this to be a BTree specific exception, e.g.,
- // BTreeKeyDoesNotExistException.
- } catch (TreeIndexException e) {
- throw new MetadataException("Cannot drop dataverse '" + dataverseName + "' because it doesn't exist.", e);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ // TODO: Change this to be a BTree specific exception, e.g.,
+ // BTreeKeyDoesNotExistException.
+ } catch (TreeIndexException e) {
+ throw new MetadataException("Cannot drop dataverse '" + dataverseName + "' because it doesn't exist.", e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public void dropDataset(JobId jobId, String dataverseName, String datasetName) throws MetadataException,
- RemoteException {
- Dataset dataset;
- try {
- dataset = getDataset(jobId, dataverseName, datasetName);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- if (dataset == null) {
- throw new MetadataException("Cannot drop dataset '" + datasetName + "' because it doesn't exist.");
- }
- try {
- // Delete entry from the 'datasets' dataset.
- ITupleReference searchKey = createTuple(dataverseName, datasetName);
- // Searches the index for the tuple to be deleted. Acquires an S
- // lock on the 'dataset' dataset.
- try {
- ITupleReference datasetTuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATASET_DATASET,
- searchKey);
- deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
- } catch (TreeIndexException tie) {
- // ignore this exception and continue deleting all relevant
- // artifacts.
- }
+ @Override
+ public void dropDataset(JobId jobId, String dataverseName, String datasetName) throws MetadataException,
+ RemoteException {
+ Dataset dataset;
+ try {
+ dataset = getDataset(jobId, dataverseName, datasetName);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ if (dataset == null) {
+ throw new MetadataException("Cannot drop dataset '" + datasetName + "' because it doesn't exist.");
+ }
+ try {
+ // Delete entry from the 'datasets' dataset.
+ ITupleReference searchKey = createTuple(dataverseName, datasetName);
+ // Searches the index for the tuple to be deleted. Acquires an S
+ // lock on the 'dataset' dataset.
+ try {
+ ITupleReference datasetTuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATASET_DATASET,
+ searchKey);
+ deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
+ } catch (TreeIndexException tie) {
+ // ignore this exception and continue deleting all relevant
+ // artifacts.
+ }
// Delete entry from secondary index 'group'.
if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
@@ -449,729 +452,714 @@
}
}
- @Override
- public void dropIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
- throws MetadataException, RemoteException {
- try {
- ITupleReference searchKey = createTuple(dataverseName, datasetName, indexName);
- // Searches the index for the tuple to be deleted. Acquires an S
- // lock on the 'index' dataset.
- ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey);
- deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, tuple);
- // TODO: Change this to be a BTree specific exception, e.g.,
- // BTreeKeyDoesNotExistException.
- } catch (TreeIndexException e) {
- throw new MetadataException("Cannot drop index '" + datasetName + "." + indexName
- + "' because it doesn't exist.", e);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public void dropIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
+ throws MetadataException, RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataverseName, datasetName, indexName);
+ // Searches the index for the tuple to be deleted. Acquires an S
+ // lock on the 'index' dataset.
+ ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey);
+ deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, tuple);
+ // TODO: Change this to be a BTree specific exception, e.g.,
+ // BTreeKeyDoesNotExistException.
+ } catch (TreeIndexException e) {
+ throw new MetadataException("Cannot drop index '" + datasetName + "." + indexName
+ + "' because it doesn't exist.", e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public void dropNodegroup(JobId jobId, String nodeGroupName) throws MetadataException, RemoteException {
- List<String> datasetNames;
- try {
- datasetNames = getDatasetNamesPartitionedOnThisNodeGroup(jobId, nodeGroupName);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- if (!datasetNames.isEmpty()) {
- StringBuilder sb = new StringBuilder();
- sb.append("Nodegroup '" + nodeGroupName
- + "' cannot be dropped; it was used for partitioning these datasets:");
- for (int i = 0; i < datasetNames.size(); i++)
- sb.append("\n" + (i + 1) + "- " + datasetNames.get(i) + ".");
- throw new MetadataException(sb.toString());
- }
- try {
- ITupleReference searchKey = createTuple(nodeGroupName);
- // Searches the index for the tuple to be deleted. Acquires an S
- // lock on the 'nodegroup' dataset.
- ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, searchKey);
- deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, tuple);
- // TODO: Change this to be a BTree specific exception, e.g.,
- // BTreeKeyDoesNotExistException.
- } catch (TreeIndexException e) {
- throw new MetadataException("Cannot drop nodegroup '" + nodeGroupName + "' because it doesn't exist", e);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public void dropNodegroup(JobId jobId, String nodeGroupName) throws MetadataException, RemoteException {
+ List<String> datasetNames;
+ try {
+ datasetNames = getDatasetNamesPartitionedOnThisNodeGroup(jobId, nodeGroupName);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ if (!datasetNames.isEmpty()) {
+ StringBuilder sb = new StringBuilder();
+ sb.append("Nodegroup '" + nodeGroupName
+ + "' cannot be dropped; it was used for partitioning these datasets:");
+ for (int i = 0; i < datasetNames.size(); i++)
+ sb.append("\n" + (i + 1) + "- " + datasetNames.get(i) + ".");
+ throw new MetadataException(sb.toString());
+ }
+ try {
+ ITupleReference searchKey = createTuple(nodeGroupName);
+ // Searches the index for the tuple to be deleted. Acquires an S
+ // lock on the 'nodegroup' dataset.
+ ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, searchKey);
+ deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, tuple);
+ // TODO: Change this to be a BTree specific exception, e.g.,
+ // BTreeKeyDoesNotExistException.
+ } catch (TreeIndexException e) {
+ throw new MetadataException("Cannot drop nodegroup '" + nodeGroupName + "' because it doesn't exist", e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public void dropDatatype(JobId jobId, String dataverseName, String datatypeName) throws MetadataException,
- RemoteException {
- List<String> datasetNames;
- List<String> usedDatatypes;
- try {
- datasetNames = getDatasetNamesDeclaredByThisDatatype(jobId, dataverseName, datatypeName);
- usedDatatypes = getDatatypeNamesUsingThisDatatype(jobId, dataverseName, datatypeName);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- // Check whether type is being used by datasets.
- if (!datasetNames.isEmpty()) {
- StringBuilder sb = new StringBuilder();
- sb.append("Cannot drop type '" + datatypeName + "'; it was used when creating these datasets:");
- for (int i = 0; i < datasetNames.size(); i++)
- sb.append("\n" + (i + 1) + "- " + datasetNames.get(i) + ".");
- throw new MetadataException(sb.toString());
- }
- // Check whether type is being used by other types.
- if (!usedDatatypes.isEmpty()) {
- StringBuilder sb = new StringBuilder();
- sb.append("Cannot drop type '" + datatypeName + "'; it is used in these datatypes:");
- for (int i = 0; i < usedDatatypes.size(); i++)
- sb.append("\n" + (i + 1) + "- " + usedDatatypes.get(i) + ".");
- throw new MetadataException(sb.toString());
- }
- // Delete the datatype entry, including all it's nested types.
- try {
- ITupleReference searchKey = createTuple(dataverseName, datatypeName);
- // Searches the index for the tuple to be deleted. Acquires an S
- // lock on the 'datatype' dataset.
- ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey);
- // This call uses the secondary index on datatype. Get nested types
- // before deleting entry from secondary index.
- List<String> nestedTypes = getNestedDatatypeNames(jobId, dataverseName, datatypeName);
- deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
- deleteFromDatatypeSecondaryIndex(jobId, dataverseName, datatypeName);
- for (String nestedType : nestedTypes) {
- Datatype dt = getDatatype(jobId, dataverseName, nestedType);
- if (dt != null && dt.getIsAnonymous()) {
- dropDatatype(jobId, dataverseName, dt.getDatatypeName());
- }
- }
- // TODO: Change this to be a BTree specific exception, e.g.,
- // BTreeKeyDoesNotExistException.
- } catch (TreeIndexException e) {
- throw new MetadataException("Cannot drop type '" + datatypeName + "' because it doesn't exist", e);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public void dropDatatype(JobId jobId, String dataverseName, String datatypeName) throws MetadataException,
+ RemoteException {
+ List<String> datasetNames;
+ List<String> usedDatatypes;
+ try {
+ datasetNames = getDatasetNamesDeclaredByThisDatatype(jobId, dataverseName, datatypeName);
+ usedDatatypes = getDatatypeNamesUsingThisDatatype(jobId, dataverseName, datatypeName);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ // Check whether type is being used by datasets.
+ if (!datasetNames.isEmpty()) {
+ StringBuilder sb = new StringBuilder();
+ sb.append("Cannot drop type '" + datatypeName + "'; it was used when creating these datasets:");
+ for (int i = 0; i < datasetNames.size(); i++)
+ sb.append("\n" + (i + 1) + "- " + datasetNames.get(i) + ".");
+ throw new MetadataException(sb.toString());
+ }
+ // Check whether type is being used by other types.
+ if (!usedDatatypes.isEmpty()) {
+ StringBuilder sb = new StringBuilder();
+ sb.append("Cannot drop type '" + datatypeName + "'; it is used in these datatypes:");
+ for (int i = 0; i < usedDatatypes.size(); i++)
+ sb.append("\n" + (i + 1) + "- " + usedDatatypes.get(i) + ".");
+ throw new MetadataException(sb.toString());
+ }
+ // Delete the datatype entry, including all it's nested types.
+ try {
+ ITupleReference searchKey = createTuple(dataverseName, datatypeName);
+ // Searches the index for the tuple to be deleted. Acquires an S
+ // lock on the 'datatype' dataset.
+ ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey);
+ // This call uses the secondary index on datatype. Get nested types
+ // before deleting entry from secondary index.
+ List<String> nestedTypes = getNestedDatatypeNames(jobId, dataverseName, datatypeName);
+ deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
+ deleteFromDatatypeSecondaryIndex(jobId, dataverseName, datatypeName);
+ for (String nestedType : nestedTypes) {
+ Datatype dt = getDatatype(jobId, dataverseName, nestedType);
+ if (dt != null && dt.getIsAnonymous()) {
+ dropDatatype(jobId, dataverseName, dt.getDatatypeName());
+ }
+ }
+ // TODO: Change this to be a BTree specific exception, e.g.,
+ // BTreeKeyDoesNotExistException.
+ } catch (TreeIndexException e) {
+ throw new MetadataException("Cannot drop type '" + datatypeName + "' because it doesn't exist", e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- private void forceDropDatatype(JobId jobId, String dataverseName, String datatypeName) throws AsterixException {
- try {
- ITupleReference searchKey = createTuple(dataverseName, datatypeName);
- // Searches the index for the tuple to be deleted. Acquires an S
- // lock on the 'datatype' dataset.
- ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey);
- deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
- deleteFromDatatypeSecondaryIndex(jobId, dataverseName, datatypeName);
- // TODO: Change this to be a BTree specific exception, e.g.,
- // BTreeKeyDoesNotExistException.
- } catch (TreeIndexException e) {
- throw new AsterixException("Cannot drop type '" + datatypeName + "' because it doesn't exist", e);
- } catch (AsterixException e) {
- throw e;
- } catch (Exception e) {
- throw new AsterixException(e);
- }
- }
+ private void forceDropDatatype(JobId jobId, String dataverseName, String datatypeName) throws AsterixException {
+ try {
+ ITupleReference searchKey = createTuple(dataverseName, datatypeName);
+ // Searches the index for the tuple to be deleted. Acquires an S
+ // lock on the 'datatype' dataset.
+ ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey);
+ deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
+ deleteFromDatatypeSecondaryIndex(jobId, dataverseName, datatypeName);
+ // TODO: Change this to be a BTree specific exception, e.g.,
+ // BTreeKeyDoesNotExistException.
+ } catch (TreeIndexException e) {
+ throw new AsterixException("Cannot drop type '" + datatypeName + "' because it doesn't exist", e);
+ } catch (AsterixException e) {
+ throw e;
+ } catch (Exception e) {
+ throw new AsterixException(e);
+ }
+ }
- private void deleteFromDatatypeSecondaryIndex(JobId jobId, String dataverseName, String datatypeName)
- throws AsterixException {
- try {
- List<String> nestedTypes = getNestedDatatypeNames(jobId, dataverseName, datatypeName);
- for (String nestedType : nestedTypes) {
- ITupleReference searchKey = createTuple(dataverseName, nestedType, datatypeName);
- // Searches the index for the tuple to be deleted. Acquires an S
- // lock on the DATATYPENAME_ON_DATATYPE_INDEX index.
- ITupleReference tuple = getTupleToBeDeleted(jobId,
- MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, searchKey);
- deleteTupleFromIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, tuple);
- }
- // TODO: Change this to be a BTree specific exception, e.g.,
- // BTreeKeyDoesNotExistException.
- } catch (TreeIndexException e) {
- throw new AsterixException("Cannot drop type '" + datatypeName + "' because it doesn't exist", e);
- } catch (AsterixException e) {
- throw e;
- } catch (Exception e) {
- throw new AsterixException(e);
- }
- }
+ private void deleteFromDatatypeSecondaryIndex(JobId jobId, String dataverseName, String datatypeName)
+ throws AsterixException {
+ try {
+ List<String> nestedTypes = getNestedDatatypeNames(jobId, dataverseName, datatypeName);
+ for (String nestedType : nestedTypes) {
+ ITupleReference searchKey = createTuple(dataverseName, nestedType, datatypeName);
+ // Searches the index for the tuple to be deleted. Acquires an S
+ // lock on the DATATYPENAME_ON_DATATYPE_INDEX index.
+ ITupleReference tuple = getTupleToBeDeleted(jobId,
+ MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, searchKey);
+ deleteTupleFromIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, tuple);
+ }
+ // TODO: Change this to be a BTree specific exception, e.g.,
+ // BTreeKeyDoesNotExistException.
+ } catch (TreeIndexException e) {
+ throw new AsterixException("Cannot drop type '" + datatypeName + "' because it doesn't exist", e);
+ } catch (AsterixException e) {
+ throw e;
+ } catch (Exception e) {
+ throw new AsterixException(e);
+ }
+ }
- private void deleteTupleFromIndex(JobId jobId, IMetadataIndex metadataIndex, ITupleReference tuple)
- throws Exception {
- long resourceID = metadataIndex.getResourceID();
- ILSMIndex lsmIndex = (ILSMIndex) indexLifecycleManager.getIndex(resourceID);
- indexLifecycleManager.open(resourceID);
- // prepare a Callback for logging
- IModificationOperationCallback modCallback = createIndexModificationCallback(jobId, resourceID, metadataIndex,
- lsmIndex, IndexOperation.DELETE);
- IIndexAccessor indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
+ private void deleteTupleFromIndex(JobId jobId, IMetadataIndex metadataIndex, ITupleReference tuple)
+ throws Exception {
+ long resourceID = metadataIndex.getResourceID();
+ ILSMIndex lsmIndex = (ILSMIndex) indexLifecycleManager.getIndex(resourceID);
+ indexLifecycleManager.open(resourceID);
+ // prepare a Callback for logging
+ IModificationOperationCallback modCallback = createIndexModificationCallback(jobId, resourceID, metadataIndex,
+ lsmIndex, IndexOperation.DELETE);
+ ILSMIndexAccessor indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
- ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
- txnCtx.setTransactionType(TransactionType.READ_WRITE);
- txnCtx.registerIndexAndCallback(lsmIndex, (AbstractOperationCallback) modCallback);
+ ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+ txnCtx.setWriteTxn(true);
+ txnCtx.registerIndexAndCallback(resourceID, lsmIndex, (AbstractOperationCallback) modCallback,
+ metadataIndex.isPrimaryIndex());
- indexAccessor.delete(tuple);
- indexLifecycleManager.close(resourceID);
- }
+ indexAccessor.forceDelete(tuple);
+ indexLifecycleManager.close(resourceID);
+ }
- @Override
- public List<Dataverse> getDataverses(JobId jobId) throws MetadataException, RemoteException {
- try {
- DataverseTupleTranslator tupleReaderWriter = new DataverseTupleTranslator(false);
- IValueExtractor<Dataverse> valueExtractor = new MetadataEntityValueExtractor<Dataverse>(tupleReaderWriter);
- List<Dataverse> results = new ArrayList<Dataverse>();
- searchIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, null, valueExtractor, results);
- if (results.isEmpty()) {
- return null;
- }
- return results;
- } catch (Exception e) {
- throw new MetadataException(e);
- }
+ @Override
+ public List<Dataverse> getDataverses(JobId jobId) throws MetadataException, RemoteException {
+ try {
+ DataverseTupleTranslator tupleReaderWriter = new DataverseTupleTranslator(false);
+ IValueExtractor<Dataverse> valueExtractor = new MetadataEntityValueExtractor<Dataverse>(tupleReaderWriter);
+ List<Dataverse> results = new ArrayList<Dataverse>();
+ searchIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, null, valueExtractor, results);
+ if (results.isEmpty()) {
+ return null;
+ }
+ return results;
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
- }
+ }
- @Override
- public Dataverse getDataverse(JobId jobId, String dataverseName) throws MetadataException, RemoteException {
+ @Override
+ public Dataverse getDataverse(JobId jobId, String dataverseName) throws MetadataException, RemoteException {
- try {
- ITupleReference searchKey = createTuple(dataverseName);
- DataverseTupleTranslator tupleReaderWriter = new DataverseTupleTranslator(false);
- IValueExtractor<Dataverse> valueExtractor = new MetadataEntityValueExtractor<Dataverse>(tupleReaderWriter);
- List<Dataverse> results = new ArrayList<Dataverse>();
- searchIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, searchKey, valueExtractor, results);
- if (results.isEmpty()) {
- return null;
- }
- return results.get(0);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
+ try {
+ ITupleReference searchKey = createTuple(dataverseName);
+ DataverseTupleTranslator tupleReaderWriter = new DataverseTupleTranslator(false);
+ IValueExtractor<Dataverse> valueExtractor = new MetadataEntityValueExtractor<Dataverse>(tupleReaderWriter);
+ List<Dataverse> results = new ArrayList<Dataverse>();
+ searchIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, searchKey, valueExtractor, results);
+ if (results.isEmpty()) {
+ return null;
+ }
+ return results.get(0);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
- }
+ }
- @Override
- public List<Dataset> getDataverseDatasets(JobId jobId, String dataverseName) throws MetadataException,
- RemoteException {
- try {
- ITupleReference searchKey = createTuple(dataverseName);
- DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(false);
- IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<Dataset>(tupleReaderWriter);
- List<Dataset> results = new ArrayList<Dataset>();
- searchIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey, valueExtractor, results);
- return results;
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public List<Dataset> getDataverseDatasets(JobId jobId, String dataverseName) throws MetadataException,
+ RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataverseName);
+ DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(false);
+ IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<Dataset>(tupleReaderWriter);
+ List<Dataset> results = new ArrayList<Dataset>();
+ searchIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey, valueExtractor, results);
+ return results;
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- private List<Datatype> getDataverseDatatypes(JobId jobId, String dataverseName) throws MetadataException,
- RemoteException {
- try {
- ITupleReference searchKey = createTuple(dataverseName);
- DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(jobId, this, false);
- IValueExtractor<Datatype> valueExtractor = new MetadataEntityValueExtractor<Datatype>(tupleReaderWriter);
- List<Datatype> results = new ArrayList<Datatype>();
- searchIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey, valueExtractor, results);
- return results;
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ private List<Datatype> getDataverseDatatypes(JobId jobId, String dataverseName) throws MetadataException,
+ RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataverseName);
+ DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(jobId, this, false);
+ IValueExtractor<Datatype> valueExtractor = new MetadataEntityValueExtractor<Datatype>(tupleReaderWriter);
+ List<Datatype> results = new ArrayList<Datatype>();
+ searchIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey, valueExtractor, results);
+ return results;
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public Dataset getDataset(JobId jobId, String dataverseName, String datasetName) throws MetadataException,
- RemoteException {
- try {
- ITupleReference searchKey = createTuple(dataverseName, datasetName);
- DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(false);
- List<Dataset> results = new ArrayList<Dataset>();
- IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<Dataset>(tupleReaderWriter);
- searchIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey, valueExtractor, results);
- if (results.isEmpty()) {
- return null;
- }
- return results.get(0);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public Dataset getDataset(JobId jobId, String dataverseName, String datasetName) throws MetadataException,
+ RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataverseName, datasetName);
+ DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(false);
+ List<Dataset> results = new ArrayList<Dataset>();
+ IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<Dataset>(tupleReaderWriter);
+ searchIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey, valueExtractor, results);
+ if (results.isEmpty()) {
+ return null;
+ }
+ return results.get(0);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- private List<String> getDatasetNamesDeclaredByThisDatatype(JobId jobId, String dataverseName, String datatypeName)
- throws MetadataException, RemoteException {
- try {
- ITupleReference searchKey = createTuple(dataverseName, datatypeName);
- List<String> results = new ArrayList<String>();
- IValueExtractor<String> valueExtractor = new DatasetNameValueExtractor();
- searchIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, searchKey, valueExtractor,
- results);
- return results;
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ private List<String> getDatasetNamesDeclaredByThisDatatype(JobId jobId, String dataverseName, String datatypeName)
+ throws MetadataException, RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataverseName, datatypeName);
+ List<String> results = new ArrayList<String>();
+ IValueExtractor<String> valueExtractor = new DatasetNameValueExtractor();
+ searchIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, searchKey, valueExtractor,
+ results);
+ return results;
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- public List<String> getDatatypeNamesUsingThisDatatype(JobId jobId, String dataverseName, String datatypeName)
- throws MetadataException, RemoteException {
- try {
- ITupleReference searchKey = createTuple(dataverseName, datatypeName);
- List<String> results = new ArrayList<String>();
- IValueExtractor<String> valueExtractor = new DatatypeNameValueExtractor(dataverseName, this);
- searchIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, searchKey, valueExtractor,
- results);
- return results;
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ public List<String> getDatatypeNamesUsingThisDatatype(JobId jobId, String dataverseName, String datatypeName)
+ throws MetadataException, RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataverseName, datatypeName);
+ List<String> results = new ArrayList<String>();
+ IValueExtractor<String> valueExtractor = new DatatypeNameValueExtractor(dataverseName, this);
+ searchIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, searchKey, valueExtractor,
+ results);
+ return results;
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- private List<String> getNestedDatatypeNames(JobId jobId, String dataverseName, String datatypeName)
- throws MetadataException, RemoteException {
- try {
- ITupleReference searchKey = createTuple(dataverseName);
- List<String> results = new ArrayList<String>();
- IValueExtractor<String> valueExtractor = new NestedDatatypeNameValueExtractor(datatypeName);
- searchIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, searchKey, valueExtractor,
- results);
- return results;
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ private List<String> getNestedDatatypeNames(JobId jobId, String dataverseName, String datatypeName)
+ throws MetadataException, RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataverseName);
+ List<String> results = new ArrayList<String>();
+ IValueExtractor<String> valueExtractor = new NestedDatatypeNameValueExtractor(datatypeName);
+ searchIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, searchKey, valueExtractor,
+ results);
+ return results;
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- public List<String> getDatasetNamesPartitionedOnThisNodeGroup(JobId jobId, String nodegroup)
- throws MetadataException, RemoteException {
- try {
- ITupleReference searchKey = createTuple(nodegroup);
- List<String> results = new ArrayList<String>();
- IValueExtractor<String> valueExtractor = new DatasetNameValueExtractor();
- searchIndex(jobId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, searchKey, valueExtractor, results);
- return results;
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ public List<String> getDatasetNamesPartitionedOnThisNodeGroup(JobId jobId, String nodegroup)
+ throws MetadataException, RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(nodegroup);
+ List<String> results = new ArrayList<String>();
+ IValueExtractor<String> valueExtractor = new DatasetNameValueExtractor();
+ searchIndex(jobId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, searchKey, valueExtractor, results);
+ return results;
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public Index getIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
- throws MetadataException, RemoteException {
- try {
- ITupleReference searchKey = createTuple(dataverseName, datasetName, indexName);
- IndexTupleTranslator tupleReaderWriter = new IndexTupleTranslator(false);
- IValueExtractor<Index> valueExtractor = new MetadataEntityValueExtractor<Index>(tupleReaderWriter);
- List<Index> results = new ArrayList<Index>();
- searchIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey, valueExtractor, results);
- if (results.isEmpty()) {
- return null;
- }
- return results.get(0);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public Index getIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
+ throws MetadataException, RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataverseName, datasetName, indexName);
+ IndexTupleTranslator tupleReaderWriter = new IndexTupleTranslator(false);
+ IValueExtractor<Index> valueExtractor = new MetadataEntityValueExtractor<Index>(tupleReaderWriter);
+ List<Index> results = new ArrayList<Index>();
+ searchIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey, valueExtractor, results);
+ if (results.isEmpty()) {
+ return null;
+ }
+ return results.get(0);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public List<Index> getDatasetIndexes(JobId jobId, String dataverseName, String datasetName)
- throws MetadataException, RemoteException {
- try {
- ITupleReference searchKey = createTuple(dataverseName, datasetName);
- IndexTupleTranslator tupleReaderWriter = new IndexTupleTranslator(false);
- IValueExtractor<Index> valueExtractor = new MetadataEntityValueExtractor<Index>(tupleReaderWriter);
- List<Index> results = new ArrayList<Index>();
- searchIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey, valueExtractor, results);
- return results;
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public List<Index> getDatasetIndexes(JobId jobId, String dataverseName, String datasetName)
+ throws MetadataException, RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataverseName, datasetName);
+ IndexTupleTranslator tupleReaderWriter = new IndexTupleTranslator(false);
+ IValueExtractor<Index> valueExtractor = new MetadataEntityValueExtractor<Index>(tupleReaderWriter);
+ List<Index> results = new ArrayList<Index>();
+ searchIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey, valueExtractor, results);
+ return results;
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public Datatype getDatatype(JobId jobId, String dataverseName, String datatypeName) throws MetadataException,
- RemoteException {
- try {
- ITupleReference searchKey = createTuple(dataverseName, datatypeName);
- DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(jobId, this, false);
- IValueExtractor<Datatype> valueExtractor = new MetadataEntityValueExtractor<Datatype>(tupleReaderWriter);
- List<Datatype> results = new ArrayList<Datatype>();
- searchIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey, valueExtractor, results);
- if (results.isEmpty()) {
- return null;
- }
- return results.get(0);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public Datatype getDatatype(JobId jobId, String dataverseName, String datatypeName) throws MetadataException,
+ RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataverseName, datatypeName);
+ DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(jobId, this, false);
+ IValueExtractor<Datatype> valueExtractor = new MetadataEntityValueExtractor<Datatype>(tupleReaderWriter);
+ List<Datatype> results = new ArrayList<Datatype>();
+ searchIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey, valueExtractor, results);
+ if (results.isEmpty()) {
+ return null;
+ }
+ return results.get(0);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public NodeGroup getNodeGroup(JobId jobId, String nodeGroupName) throws MetadataException, RemoteException {
- try {
- ITupleReference searchKey = createTuple(nodeGroupName);
- NodeGroupTupleTranslator tupleReaderWriter = new NodeGroupTupleTranslator(false);
- IValueExtractor<NodeGroup> valueExtractor = new MetadataEntityValueExtractor<NodeGroup>(tupleReaderWriter);
- List<NodeGroup> results = new ArrayList<NodeGroup>();
- searchIndex(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, searchKey, valueExtractor, results);
- if (results.isEmpty()) {
- return null;
- }
- return results.get(0);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public NodeGroup getNodeGroup(JobId jobId, String nodeGroupName) throws MetadataException, RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(nodeGroupName);
+ NodeGroupTupleTranslator tupleReaderWriter = new NodeGroupTupleTranslator(false);
+ IValueExtractor<NodeGroup> valueExtractor = new MetadataEntityValueExtractor<NodeGroup>(tupleReaderWriter);
+ List<NodeGroup> results = new ArrayList<NodeGroup>();
+ searchIndex(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, searchKey, valueExtractor, results);
+ if (results.isEmpty()) {
+ return null;
+ }
+ return results.get(0);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public Function getFunction(JobId jobId, FunctionSignature functionSignature) throws MetadataException,
- RemoteException {
- try {
- ITupleReference searchKey = createTuple(functionSignature.getNamespace(), functionSignature.getName(), ""
- + functionSignature.getArity());
- FunctionTupleTranslator tupleReaderWriter = new FunctionTupleTranslator(false);
- List<Function> results = new ArrayList<Function>();
- IValueExtractor<Function> valueExtractor = new MetadataEntityValueExtractor<Function>(tupleReaderWriter);
- searchIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey, valueExtractor, results);
- if (results.isEmpty()) {
- return null;
- }
- return results.get(0);
- } catch (Exception e) {
- e.printStackTrace();
- throw new MetadataException(e);
- }
- }
+ @Override
+ public Function getFunction(JobId jobId, FunctionSignature functionSignature) throws MetadataException,
+ RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(functionSignature.getNamespace(), functionSignature.getName(), ""
+ + functionSignature.getArity());
+ FunctionTupleTranslator tupleReaderWriter = new FunctionTupleTranslator(false);
+ List<Function> results = new ArrayList<Function>();
+ IValueExtractor<Function> valueExtractor = new MetadataEntityValueExtractor<Function>(tupleReaderWriter);
+ searchIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey, valueExtractor, results);
+ if (results.isEmpty()) {
+ return null;
+ }
+ return results.get(0);
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public void dropFunction(JobId jobId, FunctionSignature functionSignature) throws MetadataException,
- RemoteException {
+ @Override
+ public void dropFunction(JobId jobId, FunctionSignature functionSignature) throws MetadataException,
+ RemoteException {
- Function function;
- try {
- function = getFunction(jobId, functionSignature);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- if (function == null) {
- throw new MetadataException("Cannot drop function '" + functionSignature.toString()
- + "' because it doesn't exist.");
- }
- try {
- // Delete entry from the 'function' dataset.
- ITupleReference searchKey = createTuple(functionSignature.getNamespace(), functionSignature.getName(), ""
- + functionSignature.getArity());
- // Searches the index for the tuple to be deleted. Acquires an S
- // lock on the 'function' dataset.
- ITupleReference datasetTuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET,
- searchKey);
- deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, datasetTuple);
+ Function function;
+ try {
+ function = getFunction(jobId, functionSignature);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ if (function == null) {
+ throw new MetadataException("Cannot drop function '" + functionSignature.toString()
+ + "' because it doesn't exist.");
+ }
+ try {
+ // Delete entry from the 'function' dataset.
+ ITupleReference searchKey = createTuple(functionSignature.getNamespace(), functionSignature.getName(), ""
+ + functionSignature.getArity());
+ // Searches the index for the tuple to be deleted. Acquires an S
+ // lock on the 'function' dataset.
+ ITupleReference datasetTuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET,
+ searchKey);
+ deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, datasetTuple);
- // TODO: Change this to be a BTree specific exception, e.g.,
- // BTreeKeyDoesNotExistException.
- } catch (TreeIndexException e) {
- throw new MetadataException("There is no function with the name " + functionSignature.getName()
- + " and arity " + functionSignature.getArity(), e);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ // TODO: Change this to be a BTree specific exception, e.g.,
+ // BTreeKeyDoesNotExistException.
+ } catch (TreeIndexException e) {
+ throw new MetadataException("There is no function with the name " + functionSignature.getName()
+ + " and arity " + functionSignature.getArity(), e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- private ITupleReference getTupleToBeDeleted(JobId jobId, IMetadataIndex metadataIndex, ITupleReference searchKey)
- throws Exception {
- IValueExtractor<ITupleReference> valueExtractor = new TupleCopyValueExtractor(metadataIndex.getTypeTraits());
- List<ITupleReference> results = new ArrayList<ITupleReference>();
- searchIndex(jobId, metadataIndex, searchKey, valueExtractor, results);
- if (results.isEmpty()) {
- // TODO: Temporarily a TreeIndexException to make it get caught by
- // caller in the appropriate catch block.
- throw new TreeIndexException("Could not find entry to be deleted.");
- }
- // There should be exactly one result returned from the search.
- return results.get(0);
- }
+ private ITupleReference getTupleToBeDeleted(JobId jobId, IMetadataIndex metadataIndex, ITupleReference searchKey)
+ throws Exception {
+ IValueExtractor<ITupleReference> valueExtractor = new TupleCopyValueExtractor(metadataIndex.getTypeTraits());
+ List<ITupleReference> results = new ArrayList<ITupleReference>();
+ searchIndex(jobId, metadataIndex, searchKey, valueExtractor, results);
+ if (results.isEmpty()) {
+ // TODO: Temporarily a TreeIndexException to make it get caught by
+ // caller in the appropriate catch block.
+ throw new TreeIndexException("Could not find entry to be deleted.");
+ }
+ // There should be exactly one result returned from the search.
+ return results.get(0);
+ }
- // Debugging Method
- public String printMetadata() {
+ // Debugging Method
+ public String printMetadata() {
- StringBuilder sb = new StringBuilder();
- try {
- IMetadataIndex index = MetadataPrimaryIndexes.DATAVERSE_DATASET;
- long resourceID = index.getResourceID();
- IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
- indexLifecycleManager.open(resourceID);
- IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
- NoOpOperationCallback.INSTANCE);
- ITreeIndexCursor rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor();
+ StringBuilder sb = new StringBuilder();
+ try {
+ IMetadataIndex index = MetadataPrimaryIndexes.DATAVERSE_DATASET;
+ long resourceID = index.getResourceID();
+ IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
+ indexLifecycleManager.open(resourceID);
+ IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
+ NoOpOperationCallback.INSTANCE);
+ ITreeIndexCursor rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor();
- RangePredicate rangePred = null;
- rangePred = new RangePredicate(null, null, true, true, null, null);
- indexAccessor.search(rangeCursor, rangePred);
- try {
- while (rangeCursor.hasNext()) {
- rangeCursor.next();
- sb.append(TupleUtils.printTuple(rangeCursor.getTuple(),
- new ISerializerDeserializer[] { AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ASTRING) }));
- }
- } finally {
- rangeCursor.close();
- }
- indexLifecycleManager.close(resourceID);
+ RangePredicate rangePred = null;
+ rangePred = new RangePredicate(null, null, true, true, null, null);
+ indexAccessor.search(rangeCursor, rangePred);
+ try {
+ while (rangeCursor.hasNext()) {
+ rangeCursor.next();
+ sb.append(TupleUtils.printTuple(rangeCursor.getTuple(),
+ new ISerializerDeserializer[] { AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ASTRING) }));
+ }
+ } finally {
+ rangeCursor.close();
+ }
+ indexLifecycleManager.close(resourceID);
- index = MetadataPrimaryIndexes.DATASET_DATASET;
- resourceID = index.getResourceID();
- indexInstance = indexLifecycleManager.getIndex(resourceID);
- indexLifecycleManager.open(resourceID);
- indexAccessor = indexInstance
- .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
- rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor();
+ index = MetadataPrimaryIndexes.DATASET_DATASET;
+ resourceID = index.getResourceID();
+ indexInstance = indexLifecycleManager.getIndex(resourceID);
+ indexLifecycleManager.open(resourceID);
+ indexAccessor = indexInstance
+ .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+ rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor();
- rangePred = null;
- rangePred = new RangePredicate(null, null, true, true, null, null);
- indexAccessor.search(rangeCursor, rangePred);
- try {
- while (rangeCursor.hasNext()) {
- rangeCursor.next();
- sb.append(TupleUtils.printTuple(rangeCursor.getTuple(), new ISerializerDeserializer[] {
- AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING),
- AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING) }));
- }
- } finally {
- rangeCursor.close();
- }
- indexLifecycleManager.close(resourceID);
+ rangePred = null;
+ rangePred = new RangePredicate(null, null, true, true, null, null);
+ indexAccessor.search(rangeCursor, rangePred);
+ try {
+ while (rangeCursor.hasNext()) {
+ rangeCursor.next();
+ sb.append(TupleUtils.printTuple(rangeCursor.getTuple(), new ISerializerDeserializer[] {
+ AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING),
+ AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING) }));
+ }
+ } finally {
+ rangeCursor.close();
+ }
+ indexLifecycleManager.close(resourceID);
- index = MetadataPrimaryIndexes.INDEX_DATASET;
- resourceID = index.getResourceID();
- indexInstance = indexLifecycleManager.getIndex(resourceID);
- indexLifecycleManager.open(resourceID);
- indexAccessor = indexInstance
- .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
- rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor();
+ index = MetadataPrimaryIndexes.INDEX_DATASET;
+ resourceID = index.getResourceID();
+ indexInstance = indexLifecycleManager.getIndex(resourceID);
+ indexLifecycleManager.open(resourceID);
+ indexAccessor = indexInstance
+ .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+ rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor();
- rangePred = null;
- rangePred = new RangePredicate(null, null, true, true, null, null);
- indexAccessor.search(rangeCursor, rangePred);
- try {
- while (rangeCursor.hasNext()) {
- rangeCursor.next();
- sb.append(TupleUtils.printTuple(rangeCursor.getTuple(), new ISerializerDeserializer[] {
- AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING),
- AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING),
- AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING) }));
- }
- } finally {
- rangeCursor.close();
- }
- indexLifecycleManager.close(resourceID);
- } catch (Exception e) {
- e.printStackTrace();
- }
- return sb.toString();
- }
+ rangePred = null;
+ rangePred = new RangePredicate(null, null, true, true, null, null);
+ indexAccessor.search(rangeCursor, rangePred);
+ try {
+ while (rangeCursor.hasNext()) {
+ rangeCursor.next();
+ sb.append(TupleUtils.printTuple(rangeCursor.getTuple(), new ISerializerDeserializer[] {
+ AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING),
+ AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING),
+ AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING) }));
+ }
+ } finally {
+ rangeCursor.close();
+ }
+ indexLifecycleManager.close(resourceID);
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ return sb.toString();
+ }
- private <ResultType> void searchIndex(JobId jobId, IMetadataIndex index, ITupleReference searchKey,
- IValueExtractor<ResultType> valueExtractor, List<ResultType> results) throws Exception {
- IBinaryComparatorFactory[] comparatorFactories = index.getKeyBinaryComparatorFactory();
- long resourceID = index.getResourceID();
- IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
- indexLifecycleManager.open(resourceID);
- IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
- NoOpOperationCallback.INSTANCE);
- ITreeIndexCursor rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor();
+ private <ResultType> void searchIndex(JobId jobId, IMetadataIndex index, ITupleReference searchKey,
+ IValueExtractor<ResultType> valueExtractor, List<ResultType> results) throws Exception {
+ IBinaryComparatorFactory[] comparatorFactories = index.getKeyBinaryComparatorFactory();
+ long resourceID = index.getResourceID();
+ IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
+ indexLifecycleManager.open(resourceID);
+ IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
+ NoOpOperationCallback.INSTANCE);
+ ITreeIndexCursor rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor();
- IBinaryComparator[] searchCmps = null;
- MultiComparator searchCmp = null;
- RangePredicate rangePred = null;
- if (searchKey != null) {
- searchCmps = new IBinaryComparator[searchKey.getFieldCount()];
- for (int i = 0; i < searchKey.getFieldCount(); i++) {
- searchCmps[i] = comparatorFactories[i].createBinaryComparator();
- }
- searchCmp = new MultiComparator(searchCmps);
- }
- rangePred = new RangePredicate(searchKey, searchKey, true, true, searchCmp, searchCmp);
- indexAccessor.search(rangeCursor, rangePred);
+ IBinaryComparator[] searchCmps = null;
+ MultiComparator searchCmp = null;
+ RangePredicate rangePred = null;
+ if (searchKey != null) {
+ searchCmps = new IBinaryComparator[searchKey.getFieldCount()];
+ for (int i = 0; i < searchKey.getFieldCount(); i++) {
+ searchCmps[i] = comparatorFactories[i].createBinaryComparator();
+ }
+ searchCmp = new MultiComparator(searchCmps);
+ }
+ rangePred = new RangePredicate(searchKey, searchKey, true, true, searchCmp, searchCmp);
+ indexAccessor.search(rangeCursor, rangePred);
- try {
- while (rangeCursor.hasNext()) {
- rangeCursor.next();
- ResultType result = valueExtractor.getValue(jobId, rangeCursor.getTuple());
- if (result != null) {
- results.add(result);
- }
- }
- } finally {
- rangeCursor.close();
- }
- indexLifecycleManager.close(resourceID);
- }
+ try {
+ while (rangeCursor.hasNext()) {
+ rangeCursor.next();
+ ResultType result = valueExtractor.getValue(jobId, rangeCursor.getTuple());
+ if (result != null) {
+ results.add(result);
+ }
+ }
+ } finally {
+ rangeCursor.close();
+ }
+ indexLifecycleManager.close(resourceID);
+ }
- @Override
- public void initializeDatasetIdFactory(JobId jobId) throws MetadataException, RemoteException {
- int mostRecentDatasetId = MetadataPrimaryIndexes.FIRST_AVAILABLE_USER_DATASET_ID;
- long resourceID = MetadataPrimaryIndexes.DATASET_DATASET.getResourceID();
- try {
- IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
- indexLifecycleManager.open(resourceID);
- IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
- NoOpOperationCallback.INSTANCE);
- IIndexCursor rangeCursor = indexAccessor.createSearchCursor();
+ @Override
+ public void initializeDatasetIdFactory(JobId jobId) throws MetadataException, RemoteException {
+ int mostRecentDatasetId = MetadataPrimaryIndexes.FIRST_AVAILABLE_USER_DATASET_ID;
+ long resourceID = MetadataPrimaryIndexes.DATASET_DATASET.getResourceID();
+ try {
+ IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
+ indexLifecycleManager.open(resourceID);
+ IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
+ NoOpOperationCallback.INSTANCE);
+ IIndexCursor rangeCursor = indexAccessor.createSearchCursor();
- DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(false);
- IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<Dataset>(tupleReaderWriter);
- RangePredicate rangePred = new RangePredicate(null, null, true, true, null, null);
+ DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(false);
+ IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<Dataset>(tupleReaderWriter);
+ RangePredicate rangePred = new RangePredicate(null, null, true, true, null, null);
- indexAccessor.search(rangeCursor, rangePred);
- int datasetId;
+ indexAccessor.search(rangeCursor, rangePred);
+ int datasetId;
- try {
- while (rangeCursor.hasNext()) {
- rangeCursor.next();
- ITupleReference ref = rangeCursor.getTuple();
- Dataset ds = valueExtractor.getValue(jobId, rangeCursor.getTuple());
- datasetId = ((Dataset) valueExtractor.getValue(jobId, rangeCursor.getTuple())).getDatasetId();
- if (mostRecentDatasetId < datasetId) {
- mostRecentDatasetId = datasetId;
- }
- }
- } finally {
- rangeCursor.close();
- }
+ try {
+ while (rangeCursor.hasNext()) {
+ rangeCursor.next();
+ ITupleReference ref = rangeCursor.getTuple();
+ Dataset ds = valueExtractor.getValue(jobId, rangeCursor.getTuple());
+ datasetId = ((Dataset) valueExtractor.getValue(jobId, rangeCursor.getTuple())).getDatasetId();
+ if (mostRecentDatasetId < datasetId) {
+ mostRecentDatasetId = datasetId;
+ }
+ }
+ } finally {
+ rangeCursor.close();
+ }
- } catch (Exception e) {
- throw new MetadataException(e);
- }
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
- DatasetIdFactory.initialize(mostRecentDatasetId);
- }
+ DatasetIdFactory.initialize(mostRecentDatasetId);
+ }
- // TODO: Can use Hyrack's TupleUtils for this, once we switch to a newer
- // Hyracks version.
- public ITupleReference createTuple(String... fields) throws HyracksDataException {
- ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ASTRING);
- AMutableString aString = new AMutableString("");
- ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fields.length);
- for (String s : fields) {
- aString.setValue(s);
- stringSerde.serialize(aString, tupleBuilder.getDataOutput());
- tupleBuilder.addFieldEndOffset();
- }
- ArrayTupleReference tuple = new ArrayTupleReference();
- tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
- return tuple;
- }
+ // TODO: Can use Hyrack's TupleUtils for this, once we switch to a newer
+ // Hyracks version.
+ public ITupleReference createTuple(String... fields) throws HyracksDataException {
+ ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ASTRING);
+ AMutableString aString = new AMutableString("");
+ ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fields.length);
+ for (String s : fields) {
+ aString.setValue(s);
+ stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+ tupleBuilder.addFieldEndOffset();
+ }
+ ArrayTupleReference tuple = new ArrayTupleReference();
+ tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+ return tuple;
+ }
- @Override
- public List<Function> getDataverseFunctions(JobId jobId, String dataverseName) throws MetadataException,
- RemoteException {
- try {
- ITupleReference searchKey = createTuple(dataverseName);
- FunctionTupleTranslator tupleReaderWriter = new FunctionTupleTranslator(false);
- IValueExtractor<Function> valueExtractor = new MetadataEntityValueExtractor<Function>(tupleReaderWriter);
- List<Function> results = new ArrayList<Function>();
- searchIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey, valueExtractor, results);
- return results;
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public List<Function> getDataverseFunctions(JobId jobId, String dataverseName) throws MetadataException,
+ RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataverseName);
+ FunctionTupleTranslator tupleReaderWriter = new FunctionTupleTranslator(false);
+ IValueExtractor<Function> valueExtractor = new MetadataEntityValueExtractor<Function>(tupleReaderWriter);
+ List<Function> results = new ArrayList<Function>();
+ searchIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey, valueExtractor, results);
+ return results;
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public void addAdapter(JobId jobId, DatasourceAdapter adapter) throws MetadataException, RemoteException {
- try {
- // Insert into the 'Adapter' dataset.
- DatasourceAdapterTupleTranslator tupleReaderWriter = new DatasourceAdapterTupleTranslator(true);
- ITupleReference adapterTuple = tupleReaderWriter.getTupleFromMetadataEntity(adapter);
- insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, adapterTuple);
+ @Override
+ public void addAdapter(JobId jobId, DatasourceAdapter adapter) throws MetadataException, RemoteException {
+ try {
+ // Insert into the 'Adapter' dataset.
+ DatasourceAdapterTupleTranslator tupleReaderWriter = new DatasourceAdapterTupleTranslator(true);
+ ITupleReference adapterTuple = tupleReaderWriter.getTupleFromMetadataEntity(adapter);
+ insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, adapterTuple);
- } catch (TreeIndexDuplicateKeyException e) {
- throw new MetadataException("A adapter with this name " + adapter.getAdapterIdentifier().getAdapterName()
- + " already exists in dataverse '" + adapter.getAdapterIdentifier().getNamespace() + "'.", e);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
+ } catch (TreeIndexDuplicateKeyException e) {
+ throw new MetadataException("A adapter with this name " + adapter.getAdapterIdentifier().getAdapterName()
+ + " already exists in dataverse '" + adapter.getAdapterIdentifier().getNamespace() + "'.", e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
- }
+ }
- @Override
- public void dropAdapter(JobId jobId, String dataverseName, String adapterName) throws MetadataException,
- RemoteException {
- DatasourceAdapter adapter;
- try {
- adapter = getAdapter(jobId, dataverseName, adapterName);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- if (adapter == null) {
- throw new MetadataException("Cannot drop adapter '" + adapter + "' because it doesn't exist.");
- }
- try {
- // Delete entry from the 'Adapter' dataset.
- ITupleReference searchKey = createTuple(dataverseName, adapterName);
- // Searches the index for the tuple to be deleted. Acquires an S
- // lock on the 'Adapter' dataset.
- ITupleReference datasetTuple = getTupleToBeDeleted(jobId,
- MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey);
- deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, datasetTuple);
+ @Override
+ public void dropAdapter(JobId jobId, String dataverseName, String adapterName) throws MetadataException,
+ RemoteException {
+ DatasourceAdapter adapter;
+ try {
+ adapter = getAdapter(jobId, dataverseName, adapterName);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ if (adapter == null) {
+ throw new MetadataException("Cannot drop adapter '" + adapter + "' because it doesn't exist.");
+ }
+ try {
+ // Delete entry from the 'Adapter' dataset.
+ ITupleReference searchKey = createTuple(dataverseName, adapterName);
+ // Searches the index for the tuple to be deleted. Acquires an S
+ // lock on the 'Adapter' dataset.
+ ITupleReference datasetTuple = getTupleToBeDeleted(jobId,
+ MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey);
+ deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, datasetTuple);
- // TODO: Change this to be a BTree specific exception, e.g.,
- // BTreeKeyDoesNotExistException.
- } catch (TreeIndexException e) {
- throw new MetadataException("Cannot drop adapter '" + adapterName, e);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
+ // TODO: Change this to be a BTree specific exception, e.g.,
+ // BTreeKeyDoesNotExistException.
+ } catch (TreeIndexException e) {
+ throw new MetadataException("Cannot drop adapter '" + adapterName, e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
- }
+ }
- @Override
- public DatasourceAdapter getAdapter(JobId jobId, String dataverseName, String adapterName)
- throws MetadataException, RemoteException {
- try {
- ITupleReference searchKey = createTuple(dataverseName, adapterName);
- DatasourceAdapterTupleTranslator tupleReaderWriter = new DatasourceAdapterTupleTranslator(false);
- List<DatasourceAdapter> results = new ArrayList<DatasourceAdapter>();
- IValueExtractor<DatasourceAdapter> valueExtractor = new MetadataEntityValueExtractor<DatasourceAdapter>(
- tupleReaderWriter);
- searchIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey, valueExtractor, results);
- if (results.isEmpty()) {
- return null;
- }
- return results.get(0);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public DatasourceAdapter getAdapter(JobId jobId, String dataverseName, String adapterName)
+ throws MetadataException, RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataverseName, adapterName);
+ DatasourceAdapterTupleTranslator tupleReaderWriter = new DatasourceAdapterTupleTranslator(false);
+ List<DatasourceAdapter> results = new ArrayList<DatasourceAdapter>();
+ IValueExtractor<DatasourceAdapter> valueExtractor = new MetadataEntityValueExtractor<DatasourceAdapter>(
+ tupleReaderWriter);
+ searchIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey, valueExtractor, results);
+ if (results.isEmpty()) {
+ return null;
+ }
+ return results.get(0);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
- @Override
- public List<DatasourceAdapter> getDataverseAdapters(JobId jobId, String dataverseName) throws MetadataException,
- RemoteException {
- try {
- ITupleReference searchKey = createTuple(dataverseName);
- DatasourceAdapterTupleTranslator tupleReaderWriter = new DatasourceAdapterTupleTranslator(false);
- IValueExtractor<DatasourceAdapter> valueExtractor = new MetadataEntityValueExtractor<DatasourceAdapter>(
- tupleReaderWriter);
- List<DatasourceAdapter> results = new ArrayList<DatasourceAdapter>();
- searchIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey, valueExtractor, results);
- return results;
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
-
- @Override
- public void addExternalDatasetFile(JobId jobId, ExternalFile externalFile)
- throws MetadataException, RemoteException {
- try {
- // Insert into the 'externalFiles' dataset.
- ExternalFileTupleTranslator tupleReaderWriter = new ExternalFileTupleTranslator(true);
- ITupleReference externalFileTuple = tupleReaderWriter.getTupleFromMetadataEntity(externalFile);
- insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, externalFileTuple);
- } catch (TreeIndexDuplicateKeyException e) {
- throw new MetadataException("An externalFile with this number " + externalFile.getFileNumber()
- + " already exists in dataset '" + externalFile.getDatasetName() + "' in dataverse '"+externalFile.getDataverseName()+"'.", e);
- } catch (Exception e) {
- throw new MetadataException(e);
- }
- }
+ @Override
+ public List<DatasourceAdapter> getDataverseAdapters(JobId jobId, String dataverseName) throws MetadataException,
+ RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataverseName);
+ DatasourceAdapterTupleTranslator tupleReaderWriter = new DatasourceAdapterTupleTranslator(false);
+ IValueExtractor<DatasourceAdapter> valueExtractor = new MetadataEntityValueExtractor<DatasourceAdapter>(
+ tupleReaderWriter);
+ List<DatasourceAdapter> results = new ArrayList<DatasourceAdapter>();
+ searchIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey, valueExtractor, results);
+ return results;
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
@Override
public List<ExternalFile> getExternalDatasetFiles(JobId jobId,
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 9d795f5..ed89cfc 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
@@ -33,10 +33,9 @@
import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
import edu.uci.ics.asterix.common.context.BaseOperationTracker;
+import edu.uci.ics.asterix.common.context.DatasetLifecycleManager;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
-import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
-import edu.uci.ics.asterix.common.transactions.TransactionalResourceManagerRepository;
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;
@@ -61,7 +60,6 @@
import edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat;
import edu.uci.ics.asterix.transaction.management.resource.LSMBTreeLocalResourceMetadata;
import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
-import edu.uci.ics.asterix.transaction.management.service.logging.IndexResourceManager;
import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
import edu.uci.ics.hyracks.api.application.INCApplicationContext;
@@ -94,6 +92,7 @@
*/
public class MetadataBootstrap {
private static final Logger LOGGER = Logger.getLogger(MetadataBootstrap.class.getName());
+ public static final boolean IS_DEBUG_MODE = false;//true
private static IAsterixAppRuntimeContext runtimeContext;
@@ -138,15 +137,6 @@
MetadataSecondaryIndexes.init();
initLocalIndexArrays();
- TransactionalResourceManagerRepository resourceRepository = runtimeContext.getTransactionSubsystem()
- .getTransactionalResourceRepository();
- 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()));
-
AsterixMetadataProperties metadataProperties = propertiesProvider.getMetadataProperties();
metadataNodeName = metadataProperties.getMetadataNodeName();
metadataStore = metadataProperties.getMetadataStore();
@@ -201,6 +191,9 @@
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
} catch (Exception e) {
try {
+ if (IS_DEBUG_MODE) {
+ e.printStackTrace();
+ }
MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
} catch (Exception e2) {
e.addSuppressed(e2);
@@ -333,18 +326,19 @@
+ IndexFileNameUtil.prepareFileName(metadataStore + File.separator + index.getFileNameRelativePath(),
runtimeContext.getMetaDataIODeviceId());
FileReference file = new FileReference(new File(filePath));
- IVirtualBufferCache virtualBufferCache = runtimeContext.getVirtualBufferCache(index.getDatasetId().getId());
+ List<IVirtualBufferCache> virtualBufferCaches = runtimeContext.getVirtualBufferCaches(index.getDatasetId()
+ .getId());
ITypeTraits[] typeTraits = index.getTypeTraits();
IBinaryComparatorFactory[] comparatorFactories = index.getKeyBinaryComparatorFactory();
int[] bloomFilterKeyFields = index.getBloomFilterKeyFields();
LSMBTree lsmBtree = null;
long resourceID = -1;
- AsterixRuntimeComponentsProvider rtcProvider = index.isPrimaryIndex() ? AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER
- : AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER;
+ AsterixRuntimeComponentsProvider rtcProvider = AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
ILSMOperationTracker opTracker = index.isPrimaryIndex() ? runtimeContext.getLSMBTreeOperationTracker(index
- .getDatasetId().getId()) : new BaseOperationTracker(LSMBTreeIOOperationCallbackFactory.INSTANCE);
+ .getDatasetId().getId()) : new BaseOperationTracker((DatasetLifecycleManager) indexLifecycleManager,
+ LSMBTreeIOOperationCallbackFactory.INSTANCE, index.getDatasetId().getId());
if (create) {
- lsmBtree = LSMBTreeUtils.createLSMTree(virtualBufferCache, file, bufferCache, fileMapProvider, typeTraits,
+ lsmBtree = LSMBTreeUtils.createLSMTree(virtualBufferCaches, file, bufferCache, fileMapProvider, typeTraits,
comparatorFactories, bloomFilterKeyFields, runtimeContext.getBloomFilterFalsePositiveRate(),
runtimeContext.getLSMMergePolicy(), opTracker, runtimeContext.getLSMIOScheduler(), rtcProvider);
lsmBtree.create();
@@ -361,11 +355,11 @@
resourceID = localResourceRepository.getResourceByName(file.getFile().getPath()).getResourceId();
lsmBtree = (LSMBTree) indexLifecycleManager.getIndex(resourceID);
if (lsmBtree == null) {
- lsmBtree = LSMBTreeUtils.createLSMTree(virtualBufferCache, file, bufferCache, fileMapProvider,
+ lsmBtree = LSMBTreeUtils.createLSMTree(virtualBufferCaches, file, bufferCache, fileMapProvider,
typeTraits, comparatorFactories, bloomFilterKeyFields,
runtimeContext.getBloomFilterFalsePositiveRate(), runtimeContext.getLSMMergePolicy(),
opTracker, runtimeContext.getLSMIOScheduler(),
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER);
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER);
indexLifecycleManager.register(resourceID, lsmBtree);
}
}
@@ -442,6 +436,9 @@
}
} catch (Exception e) {
try {
+ if (IS_DEBUG_MODE) {
+ e.printStackTrace();
+ }
MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
} catch (Exception e2) {
e.addSuppressed(e2);
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 6dd0f16..f0c3a63 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
@@ -36,8 +36,11 @@
import edu.uci.ics.asterix.common.dataflow.AsterixLSMInvertedIndexInsertDeleteOperatorDescriptor;
import edu.uci.ics.asterix.common.dataflow.AsterixLSMTreeInsertDeleteOperatorDescriptor;
import edu.uci.ics.asterix.common.dataflow.IAsterixApplicationContextInfo;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
import edu.uci.ics.asterix.common.parse.IParseFileSplitsDecl;
-import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.IRecoveryManager.ResourceType;
import edu.uci.ics.asterix.common.transactions.JobId;
import edu.uci.ics.asterix.dataflow.data.nontagged.valueproviders.AqlPrimitiveValueProviderFactory;
import edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory;
@@ -89,6 +92,7 @@
import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexOperationTrackerProvider;
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.SecondaryIndexOperationTrackerProvider;
import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexSearchOperationCallbackFactory;
import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
@@ -166,7 +170,7 @@
private IResultSerializerFactoryProvider resultSerializerFactoryProvider;
private static boolean optimizeExternalIndexes = false;
- private final Dataverse defaultDataverse;
+ private final Dataverse defaultDataverse;
private JobId jobId;
private final AsterixStorageProperties storageProperties;
@@ -174,6 +178,22 @@
private static final Map<String, String> adapterFactoryMapping = initializeAdapterFactoryMapping();
private static Scheduler hdfsScheduler;
+ public String getPropertyValue(String propertyName) {
+ return config.get(propertyName);
+ }
+
+ public void setConfig(Map<String, String> config) {
+ this.config = config;
+ }
+
+ public Map<String, String[]> getAllStores() {
+ return stores;
+ }
+
+ public Map<String, String> getConfig() {
+ return config;
+ }
+
public AqlMetadataProvider(Dataverse defaultDataverse) {
this.defaultDataverse = defaultDataverse;
this.stores = AsterixAppContextInfo.getInstance().getMetadataProperties().getStores();
@@ -830,27 +850,27 @@
primaryKeyFields[i] = i;
}
- AqlMetadataImplConfig aqlMetadataImplConfig = (AqlMetadataImplConfig) implConfig;
- ITransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
- if (aqlMetadataImplConfig != null && aqlMetadataImplConfig.isInstantLock()) {
- searchCallbackFactory = new PrimaryIndexInstantSearchOperationCallbackFactory(jobId, datasetId,
- primaryKeyFields, txnSubsystemProvider, ResourceType.LSM_BTREE);
- } else {
- searchCallbackFactory = new PrimaryIndexSearchOperationCallbackFactory(jobId, datasetId,
- primaryKeyFields, txnSubsystemProvider, ResourceType.LSM_BTREE);
- }
- }
- AsterixRuntimeComponentsProvider rtcProvider = isSecondary ? AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER
- : AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER;
- BTreeSearchOperatorDescriptor btreeSearchOp = new BTreeSearchOperatorDescriptor(jobSpec, outputRecDesc,
- appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(), spPc.first,
- typeTraits, comparatorFactories, bloomFilterKeyFields, lowKeyFields, highKeyFields,
- lowKeyInclusive, highKeyInclusive, new LSMBTreeDataflowHelperFactory(
- new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), rtcProvider,
- isSecondary ? AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER
- : new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()), rtcProvider,
- rtcProvider, storageProperties.getBloomFilterFalsePositiveRate()), retainInput,
- searchCallbackFactory);
+ AqlMetadataImplConfig aqlMetadataImplConfig = (AqlMetadataImplConfig) implConfig;
+ ITransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
+ if (aqlMetadataImplConfig != null && aqlMetadataImplConfig.isInstantLock()) {
+ searchCallbackFactory = new PrimaryIndexInstantSearchOperationCallbackFactory(jobId, datasetId,
+ primaryKeyFields, txnSubsystemProvider, ResourceType.LSM_BTREE);
+ } else {
+ searchCallbackFactory = new PrimaryIndexSearchOperationCallbackFactory(jobId, datasetId,
+ primaryKeyFields, txnSubsystemProvider, ResourceType.LSM_BTREE);
+ }
+ }
+ AsterixRuntimeComponentsProvider rtcProvider = AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
+ BTreeSearchOperatorDescriptor btreeSearchOp = new BTreeSearchOperatorDescriptor(jobSpec, outputRecDesc,
+ appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(), spPc.first,
+ typeTraits, comparatorFactories, bloomFilterKeyFields, lowKeyFields, highKeyFields,
+ lowKeyInclusive, highKeyInclusive, new LSMBTreeDataflowHelperFactory(
+ new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), rtcProvider,
+ isSecondary ? new SecondaryIndexOperationTrackerProvider(
+ LSMBTreeIOOperationCallbackFactory.INSTANCE, dataset.getDatasetId())
+ : new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()), rtcProvider,
+ rtcProvider, storageProperties.getBloomFilterFalsePositiveRate()), retainInput,
+ searchCallbackFactory);
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeSearchOp, spPc.second);
@@ -970,6 +990,9 @@
throws AlgebricksException {
AqlDataSource ads = findDataSource(dataSourceId);
Dataset dataset = ads.getDataset();
+ if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+ throw new AlgebricksException("No index for external dataset " + dataSourceId);
+ }
try {
String indexName = (String) indexId;
Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
@@ -1071,10 +1094,10 @@
splitsAndConstraint.first, typeTraits, comparatorFactories, bloomFilterKeyFields, fieldPermutation,
GlobalConfig.DEFAULT_BTREE_FILL_FACTOR, false, numElementsHint, true,
new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
storageProperties.getBloomFilterFalsePositiveRate()), NoOpOperationCallbackFactory.INSTANCE);
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad,
splitsAndConstraint.second);
@@ -1335,10 +1358,10 @@
appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
comparatorFactories, bloomFilterKeyFields, fieldPermutation, indexOp,
new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(datasetId),
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER, storageProperties
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ new SecondaryIndexOperationTrackerProvider(LSMBTreeIOOperationCallbackFactory.INSTANCE,
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, storageProperties
.getBloomFilterFalsePositiveRate()), filterFactory, modificationCallbackFactory,
false);
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad,
@@ -1463,10 +1486,11 @@
appContext.getIndexLifecycleManagerProvider(), tokenTypeTraits, tokenComparatorFactories,
invListsTypeTraits, invListComparatorFactories, tokenizerFactory, fieldPermutation, indexOp,
new LSMInvertedIndexDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(datasetId),
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER, storageProperties
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ new SecondaryIndexOperationTrackerProvider(
+ LSMInvertedIndexIOOperationCallbackFactory.INSTANCE, dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, storageProperties
.getBloomFilterFalsePositiveRate()), filterFactory, modificationCallbackFactory);
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(insertDeleteOp,
splitsAndConstraint.second);
@@ -1557,10 +1581,10 @@
comparatorFactories, null, fieldPermutation, indexOp, new LSMRTreeDataflowHelperFactory(
valueProviderFactories, RTreePolicyType.RTREE, primaryComparatorFactories,
new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, proposeLinearizer(
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ new SecondaryIndexOperationTrackerProvider(LSMRTreeIOOperationCallbackFactory.INSTANCE,
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, proposeLinearizer(
nestedKeyType.getTypeTag(), comparatorFactories.length),
storageProperties.getBloomFilterFalsePositiveRate()), filterFactory,
modificationCallbackFactory, false);
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/ATimeParserFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/ATimeParserFactory.java
index 2200ea3..3a0a80a 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/ATimeParserFactory.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/ATimeParserFactory.java
@@ -181,11 +181,6 @@
}
}
- if (timezoneHour < GregorianCalendarSystem.TIMEZONE_HOUR_MIN
- || timezoneHour > GregorianCalendarSystem.TIMEZONE_HOUR_MAX) {
- throw new HyracksDataException(timeErrorMessage + ": time zone hour " + timezoneHour);
- }
-
int temp_offset = (timeString.charAt(start + 3) == ':') ? 1 : 0;
for (int i = 0; i < 2; i++) {
@@ -197,15 +192,11 @@
}
}
- if (timezoneMinute < GregorianCalendarSystem.TIMEZONE_MIN_MIN
- || timezoneMinute > GregorianCalendarSystem.TIMEZONE_MIN_MAX) {
- throw new HyracksDataException(timeErrorMessage + ": time zone minute " + timezoneMinute);
- }
+ timezone = (int) (timezoneHour * GregorianCalendarSystem.CHRONON_OF_HOUR + timezoneMinute
+ * GregorianCalendarSystem.CHRONON_OF_MINUTE);
- if (timeString.charAt(start) == '-') {
- timezone = (byte) -((timezoneHour * 4) + timezoneMinute / 15);
- } else {
- timezone = (byte) ((timezoneHour * 4) + timezoneMinute / 15);
+ if (timeString.charAt(start) == '+') {
+ timezone *= -1;
}
}
return timezone;
@@ -345,11 +336,6 @@
}
}
- if (timezoneHour < GregorianCalendarSystem.TIMEZONE_HOUR_MIN
- || timezoneHour > GregorianCalendarSystem.TIMEZONE_HOUR_MAX) {
- throw new HyracksDataException(timeErrorMessage + ": time zone hour " + timezoneHour);
- }
-
int temp_offset = (timeString[start + 3] == ':') ? 1 : 0;
for (int i = 0; i < 2; i++) {
@@ -360,15 +346,11 @@
}
}
- if (timezoneMinute < GregorianCalendarSystem.TIMEZONE_MIN_MIN
- || timezoneMinute > GregorianCalendarSystem.TIMEZONE_MIN_MAX) {
- throw new HyracksDataException(timeErrorMessage + ": time zone minute " + timezoneMinute);
- }
+ timezone = (int) (timezoneHour * GregorianCalendarSystem.CHRONON_OF_HOUR + timezoneMinute
+ * GregorianCalendarSystem.CHRONON_OF_MINUTE);
- if (timeString[start] == '-') {
- timezone = (byte) -((timezoneHour * 4) + timezoneMinute / 15);
- } else {
- timezone = (byte) ((timezoneHour * 4) + timezoneMinute / 15);
+ if (timeString[start] == '+') {
+ timezone *= -1;
}
}
return timezone;
@@ -508,11 +490,6 @@
}
}
- if (timezoneHour < GregorianCalendarSystem.TIMEZONE_HOUR_MIN
- || timezoneHour > GregorianCalendarSystem.TIMEZONE_HOUR_MAX) {
- throw new HyracksDataException(timeErrorMessage + ": time zone hour " + timezoneHour);
- }
-
int temp_offset = (timeString[start + 3] == ':') ? 1 : 0;
for (int i = 0; i < 2; i++) {
@@ -523,15 +500,11 @@
}
}
- if (timezoneMinute < GregorianCalendarSystem.TIMEZONE_MIN_MIN
- || timezoneMinute > GregorianCalendarSystem.TIMEZONE_MIN_MAX) {
- throw new HyracksDataException(timeErrorMessage + ": time zone minute " + timezoneMinute);
- }
+ timezone = (int) (timezoneHour * GregorianCalendarSystem.CHRONON_OF_HOUR + timezoneMinute
+ * GregorianCalendarSystem.CHRONON_OF_MINUTE);
- if (timeString[start] == '-') {
- timezone = (byte) -((timezoneHour * 4) + timezoneMinute / 15);
- } else {
- timezone = (byte) ((timezoneHour * 4) + timezoneMinute / 15);
+ if (timeString[start] == '+') {
+ timezone *= -1;
}
}
return timezone;
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/AsterixTemporalTypeParseException.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/AsterixTemporalTypeParseException.java
new file mode 100644
index 0000000..9354d5f
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/AsterixTemporalTypeParseException.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.om.base.temporal;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class AsterixTemporalTypeParseException extends HyracksDataException {
+
+ private static final long serialVersionUID = 1L;
+
+ public AsterixTemporalTypeParseException() {
+ super();
+ }
+
+ public AsterixTemporalTypeParseException(Exception ex) {
+ super(ex);
+ }
+
+ public AsterixTemporalTypeParseException(String msg) {
+ super(msg);
+ }
+
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/DateTimeFormatUtils.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/DateTimeFormatUtils.java
new file mode 100644
index 0000000..f0ee739
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/DateTimeFormatUtils.java
@@ -0,0 +1,965 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.om.base.temporal;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.TimeZone;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * {@link DateTimeFormatUtils} provides the utility methods to parse and print a date/time/datetime
+ * value based on the given format string. The format string may contain the following <b>format characters</b> (note that
+ * format string is <b>case-sensitive</b>):
+ * <p/>
+ * - <b>Y</b>: a digit for the year field. At most 4 year format characters are allowed for a valid format string.<br/>
+ * - <b>M</b>: a digit or character for the month field. At most 3 month format characters are allowed for a valid format string. When three month format characters are used, the shorten month names (like JAN, FEB etc.) are expected in the string to be parsed. Otherwise digits are expected.<br/>
+ * - <b>D</b>: a digit for the day field. At most 2 day format characters are allowed.<br/>
+ * - <b>h</b>: a digit for the hour field. At most 2 hour format characters are allowed.<br/>
+ * - <b>m</b>: a digit for the minute field. At most 2 minute format characters are allowed.<br/>
+ * - <b>s</b>: a digit for the second field. At most 2 second format characters are allowed.<br/>
+ * - <b>n</b>: a digit for the millisecond field. At most 3 millisecond format characters are allowed.<br/>
+ * - <b>a</b>: the AM/PM field. At most 1 am/pm format character is allowed, and it matches with AM and PM case-insensitively. <br/>
+ * - <b>z</b>: the timezone field. At most 1 timezone format characters are allowed. The valid timezone string matching with this format character include:<br/>
+ * -- <b>Z</b>: a single upper-case character representing the UTC timezone;<br/>
+ * -- <b>[UTC|GMT]+xx[:]xx</b>: representing a timezone by providing the actual offset time from the UTC time;<br/>
+ * -- A string representation of a timezone like PST, Asia/Shanghai. The names of the timezones are following the Zoneinfo database provided by the JDK library. See {@link TimeZone} for more details on this.<br/>
+ * - <b>Separators</b>: separators that can be used to separate the different fields. Currently only the following characters can be used as separator: <b>-(hyphen), :(colon), /(solidus), .(period) and ,(comma)</b>.
+ * <p/>
+ * For the matching algorithm, both the format string and the data string are scanned from the beginning to the end, and the algorithm tried to match the format with the characters/digits/separators in the data string. The format string represents the <b>minimum</b> length of the required field (similar to the C-style printf formatting). This means that something like a year <it>1990</it> will match with the format strings <it>Y, YY, YYY and YYYY</it>.
+ * <p/>
+ * If the given string cannot be parsed by the given format string, an {@link AsterixTemporalTypeParseException} will be returned.
+ */
+public class DateTimeFormatUtils {
+
+ private final GregorianCalendarSystem CAL = GregorianCalendarSystem.getInstance();
+
+ // For time
+ private final char HOUR_CHAR = 'h';
+ private final char MINUTE_CHAR = 'm';
+ private final char SECOND_CHAR = 's';
+ private final char MILLISECOND_CHAR = 'n';
+ private final char AMPM_CHAR = 'a';
+ private final char TIMEZONE_CHAR = 'z';
+
+ private final int MAX_HOUR_CHARS = 2;
+ private final int MAX_MINUTE_CHARS = 2;
+ private final int MAX_SECOND_CHARS = 2;
+ private final int MAX_MILLISECOND_CHARS = 3;
+ private final int MAX_AMPM_CHARS = 1;
+ private final int MAX_TIMEZONE_CHARS = 1;
+
+ private enum DateTimeProcessState {
+ INIT,
+ YEAR,
+ MONTH,
+ DAY,
+ WEEKDAY,
+ HOUR,
+ MINUTE,
+ SECOND,
+ MILLISECOND,
+ AMPM,
+ TIMEZONE,
+ SKIPPER,
+ SEPARATOR
+ }
+
+ // For date
+ private final char YEAR_CHAR = 'Y';
+ private final char MONTH_CHAR = 'M';
+ private final char DAY_CHAR = 'D';
+ private final char WEEKDAY_CHAR = 'W';
+
+ private final int MAX_YEAR_CHARS = 4;
+ private final int MAX_MONTH_CHARS = 3;
+ private final int MAX_DAY_CHARS = 2;
+ private final int MAX_WEEKDAY_CHAR = 1;
+
+ private final byte[][] MONTH_NAMES = new byte[][] { "jan".getBytes(), "feb".getBytes(), "mar".getBytes(),
+ "apr".getBytes(), "may".getBytes(), "jun".getBytes(), "jul".getBytes(), "aug".getBytes(), "sep".getBytes(),
+ "oct".getBytes(), "nov".getBytes(), "dec".getBytes() };
+
+ private final byte[][] WEEKDAY_FULL_NAMES = new byte[][] { "monday".getBytes(), "tuesday".getBytes(),
+ "wednesday".getBytes(), "thursday".getBytes(), "friday".getBytes(), "saturday".getBytes(),
+ "sunday".getBytes() };
+
+ private final byte[] UTC_BYTEARRAY = "utc".getBytes();
+ private final byte[] GMT_BYTEARRAY = "gmt".getBytes();
+
+ private final byte[] AM_BYTEARRAY = "am".getBytes();
+ private final byte[] PM_BYTEARRAY = "pm".getBytes();
+
+ // Separators, for both time and date
+ private final char HYPHEN_CHAR = '-';
+ private final char COLON_CHAR = ':';
+ private final char SOLIDUS_CHAR = '/';
+ private final char PERIOD_CHAR = '.';
+ private final char COMMA_CHAR = ',';
+ private final char T_CHAR = 'T';
+
+ // Skipper, representing a field with characters and numbers that to be skipped
+ private final char SKIPPER_CHAR = 'O';
+ private final int MAX_SKIPPER_CHAR = 1;
+
+ private final int MS_PER_MINUTE = 60 * 1000;
+ private final int MS_PER_HOUR = 60 * MS_PER_MINUTE;
+
+ private final byte TO_LOWER_OFFSET = 'A' - 'a';
+
+ private final String[] TZ_IDS = TimeZone.getAvailableIDs();
+ private final byte[][] TIMEZONE_IDS = new byte[TZ_IDS.length][];
+ {
+ Arrays.sort(TZ_IDS);
+ for (int i = 0; i < TIMEZONE_IDS.length; i++) {
+ TIMEZONE_IDS[i] = TZ_IDS[i].getBytes();
+ }
+ }
+
+ private final int[] TIMEZONE_OFFSETS = new int[TIMEZONE_IDS.length];
+ {
+ for (int i = 0; i < TIMEZONE_IDS.length; i++) {
+ TIMEZONE_OFFSETS[i] = TimeZone.getTimeZone(TZ_IDS[i]).getRawOffset();
+ }
+ }
+
+ private static class DateTimeFormatUtilsHolder {
+ private static final DateTimeFormatUtils INSTANCE = new DateTimeFormatUtils();
+ }
+
+ public static DateTimeFormatUtils getInstance() {
+ return DateTimeFormatUtilsHolder.INSTANCE;
+ }
+
+ private DateTimeFormatUtils() {
+ }
+
+ private int parseFormatField(byte[] format, int formatStart, int formatLength, int formatPointer, char formatChar,
+ int maxAllowedFormatCharCopied) {
+
+ int formatCharCopies = 0;
+
+ formatPointer++;
+ formatCharCopies++;
+ while (formatPointer < formatLength && format[formatStart + formatPointer] == formatChar) {
+ formatPointer++;
+ formatCharCopies++;
+ }
+ if (formatCharCopies > maxAllowedFormatCharCopied) {
+ throw new IllegalStateException("The format string for " + formatChar
+ + " is too long: expected no more than " + maxAllowedFormatCharCopied + " but got "
+ + formatCharCopies);
+ }
+
+ return formatCharCopies;
+ }
+
+ public enum DateTimeParseMode {
+ DATE_ONLY,
+ TIME_ONLY,
+ DATETIME
+ }
+
+ private boolean byteArrayEqualToString(byte[] barray, int start, int length, byte[] str) {
+ if (length != str.length) {
+ return false;
+ } else {
+ return byteArrayBeingWithString(barray, start, length, str);
+ }
+ }
+
+ private boolean byteArrayBeingWithString(byte[] barray, int start, int length, byte[] str) {
+ boolean beginWith = true;
+ if (length <= str.length) {
+ for (int i = 0; i < length; i++) {
+ if (toLower(barray[start + i]) != str[i]) {
+ beginWith = false;
+ break;
+ }
+ }
+ } else {
+ beginWith = false;
+ }
+ return beginWith;
+ }
+
+ private Comparator<byte[]> byteArrayComparator = new Comparator<byte[]>() {
+ @Override
+ public int compare(byte[] o1, byte[] o2) {
+ int i = 0;
+ for (; i < o1.length && i < o2.length; i++) {
+ if (o1[i] != o2[i]) {
+ return o1[i] - o2[i];
+ }
+ }
+ if (i < o1.length) {
+ return -1;
+ } else if (i < o2.length) {
+ return 1;
+ }
+ return 0;
+ }
+ };
+
+ private int monthIDSearch(byte[] barray, int start, int length) {
+ for (int i = 0; i < MONTH_NAMES.length; i++) {
+ if (byteArrayEqualToString(barray, start, length, MONTH_NAMES[i])) {
+ return i;
+ }
+ }
+ return -1;
+ }
+
+ private int weekdayIDSearch(byte[] barray, int start, int length) {
+ for (int i = 0; i < WEEKDAY_FULL_NAMES.length; i++) {
+ if (byteArrayBeingWithString(barray, start, length, WEEKDAY_FULL_NAMES[i])) {
+ return i;
+ }
+ }
+ return -1;
+ }
+
+ private int binaryTimezoneIDSearch(byte[] barray, int start, int length) {
+ return Arrays.binarySearch(TIMEZONE_IDS, 0, TIMEZONE_IDS.length,
+ Arrays.copyOfRange(barray, start, start + length), byteArrayComparator);
+ }
+
+ private int indexOf(byte[] barray, int start, int length, char c) {
+ int i = 0;
+ for (; i < length; i++) {
+ if (barray[start + i] == c) {
+ return i;
+ }
+ }
+ return -1;
+ }
+
+ private byte toLower(byte b) {
+ if (b >= 'A' && b <= 'Z') {
+ return (byte) (b - TO_LOWER_OFFSET);
+ }
+ return b;
+ }
+
+ private byte toUpper(byte b) {
+ if (b >= 'a' && b <= 'z') {
+ return (byte) (b + TO_LOWER_OFFSET);
+ }
+ return b;
+ }
+
+ public long parseDateTime(byte[] data, int dataStart, int dataLength, byte[] format, int formatStart,
+ int formatLength, DateTimeParseMode parseMode) throws AsterixTemporalTypeParseException {
+ int year = 0, month = 0, day = 0, hour = 0, min = 0, sec = 0, ms = 0, timezone = 0;
+
+ boolean negativeYear = false;
+ int formatCharCopies = 0;
+
+ int dataStringPointer = 0, formatPointer = 0;
+
+ byte separatorChar = '\0';
+
+ DateTimeProcessState processState = DateTimeProcessState.INIT;
+
+ int pointerMove = 0;
+
+ while (dataStringPointer < dataLength && formatPointer < formatLength) {
+ formatCharCopies = 0;
+ switch (format[formatStart + formatPointer]) {
+ case YEAR_CHAR:
+ processState = DateTimeProcessState.YEAR;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, YEAR_CHAR,
+ MAX_YEAR_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case MONTH_CHAR:
+ processState = DateTimeProcessState.MONTH;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, MONTH_CHAR,
+ MAX_MONTH_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case DAY_CHAR:
+ processState = DateTimeProcessState.DAY;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, DAY_CHAR,
+ MAX_DAY_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case WEEKDAY_CHAR:
+ processState = DateTimeProcessState.WEEKDAY;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, WEEKDAY_CHAR,
+ MAX_WEEKDAY_CHAR);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case HOUR_CHAR:
+ processState = DateTimeProcessState.HOUR;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, HOUR_CHAR,
+ MAX_HOUR_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case MINUTE_CHAR:
+ processState = DateTimeProcessState.MINUTE;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, MINUTE_CHAR,
+ MAX_MINUTE_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case SECOND_CHAR:
+ processState = DateTimeProcessState.SECOND;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, SECOND_CHAR,
+ MAX_SECOND_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case MILLISECOND_CHAR:
+ processState = DateTimeProcessState.MILLISECOND;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, MILLISECOND_CHAR,
+ MAX_MILLISECOND_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+
+ case AMPM_CHAR:
+ processState = DateTimeProcessState.AMPM;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, AMPM_CHAR,
+ MAX_AMPM_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+
+ case TIMEZONE_CHAR:
+ processState = DateTimeProcessState.TIMEZONE;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, TIMEZONE_CHAR,
+ MAX_TIMEZONE_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case SKIPPER_CHAR:
+ processState = DateTimeProcessState.SKIPPER;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, SKIPPER_CHAR,
+ MAX_SKIPPER_CHAR);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case ' ':
+ case HYPHEN_CHAR:
+ case COLON_CHAR:
+ case SOLIDUS_CHAR:
+ case PERIOD_CHAR:
+ case COMMA_CHAR:
+ case T_CHAR:
+ // separator
+ separatorChar = format[formatStart + formatPointer];
+ processState = DateTimeProcessState.SEPARATOR;
+ formatPointer++;
+ formatCharCopies++;
+ while (formatPointer < formatLength && format[formatStart + formatPointer] == separatorChar) {
+ formatPointer++;
+ formatCharCopies++;
+ }
+ break;
+
+ default:
+ throw new AsterixTemporalTypeParseException("Unexpected date format string at "
+ + (formatStart + formatPointer) + ": " + format[formatStart + formatPointer]);
+ }
+
+ // check whether the process state is valid for the parse mode
+
+ switch (processState) {
+ case YEAR:
+ case MONTH:
+ case DAY:
+ if (parseMode == DateTimeParseMode.TIME_ONLY) {
+ throw new AsterixTemporalTypeParseException(
+ "Unexpected date format string when parsing a time value");
+ }
+ break;
+ case HOUR:
+ case MINUTE:
+ case SECOND:
+ case MILLISECOND:
+ case AMPM:
+ case TIMEZONE:
+ if (parseMode == DateTimeParseMode.DATE_ONLY) {
+ throw new AsterixTemporalTypeParseException(
+ "Unexpected time format string when parsing a date value");
+ }
+ break;
+ default:
+ // do nothing
+ }
+
+ switch (processState) {
+ case INIT:
+ break;
+ case YEAR:
+ if (dataStringPointer < dataLength && data[dataStart + dataStringPointer] == HYPHEN_CHAR) {
+ negativeYear = true;
+ dataStringPointer++;
+ }
+ case DAY:
+ int maxAllowedFormatCharCopies = (processState == DateTimeProcessState.YEAR) ? 4 : 2;
+ int parsedValue = 0;
+
+ int processedFieldsCount = 0;
+ for (int i = 0; i < formatCharCopies; i++) {
+ if (data[dataStart + dataStringPointer] < '0' || data[dataStart + dataStringPointer] > '9') {
+ throw new AsterixTemporalTypeParseException("Unexpected char for year field at "
+ + (dataStart + dataStringPointer) + ": " + data[dataStart + dataStringPointer]);
+ }
+ parsedValue = parsedValue * 10 + (data[dataStart + dataStringPointer] - '0');
+ dataStringPointer++;
+ processedFieldsCount++;
+ }
+ // for more digits
+ while (processedFieldsCount < maxAllowedFormatCharCopies && dataStringPointer < dataLength
+ && data[dataStart + dataStringPointer] >= '0' && data[dataStart + dataStringPointer] <= '9') {
+ parsedValue = parsedValue * 10 + (data[dataStart + dataStringPointer] - '0');
+ dataStringPointer++;
+ processedFieldsCount++;
+ }
+ if (processState == DateTimeProcessState.YEAR) {
+ year = parsedValue;
+ if (negativeYear) {
+ year *= -1;
+ }
+ } else {
+ day = parsedValue;
+ }
+ break;
+ case MONTH:
+ if (formatCharCopies == 3) {
+ // the month is in the text format
+ int monthNameMatch = monthIDSearch(data, dataStart + dataStringPointer, 3);
+ if (monthNameMatch >= 0) {
+ month = monthNameMatch + 1;
+ dataStringPointer += 3;
+ } else {
+ throw new AsterixTemporalTypeParseException("Unrecognizable month string "
+ + (char) data[dataStart + dataStringPointer] + " "
+ + (char) data[dataStart + dataStringPointer + 1] + " "
+ + (char) data[dataStart + dataStringPointer + 2]);
+ }
+ } else {
+ int processedMonthFieldsCount = 0;
+ for (int i = 0; i < formatCharCopies; i++) {
+ if (data[dataStart + dataStringPointer] < '0' || data[dataStart + dataStringPointer] > '9') {
+ throw new AsterixTemporalTypeParseException("Unexpected char for month field at "
+ + (dataStart + dataStringPointer) + ": " + data[dataStart + dataStringPointer]);
+ }
+ month = month * 10 + (data[dataStart + dataStringPointer] - '0');
+ dataStringPointer++;
+ if (processedMonthFieldsCount++ > 2) {
+ throw new AsterixTemporalTypeParseException("Unexpected char for month field at "
+ + (dataStart + dataStringPointer) + ": " + data[dataStart + dataStringPointer]);
+ }
+ }
+ // if there are more than 2 digits for the day string
+ while (processedMonthFieldsCount < 2 && dataStringPointer < dataLength
+ && data[dataStart + dataStringPointer] >= '0'
+ && data[dataStart + dataStringPointer] <= '9') {
+ month = month * 10 + (data[dataStart + dataStringPointer] - '0');
+ dataStringPointer++;
+ processedMonthFieldsCount++;
+ }
+ }
+ break;
+ case WEEKDAY:
+ int processedWeekdayFieldsCount = 0;
+ while ((data[dataStart + dataStringPointer + processedWeekdayFieldsCount] >= 'a' && data[dataStart
+ + dataStringPointer + processedWeekdayFieldsCount] <= 'z')
+ || (data[dataStart + dataStringPointer + processedWeekdayFieldsCount] >= 'A' && data[dataStart
+ + dataStringPointer + processedWeekdayFieldsCount] <= 'Z')) {
+ processedWeekdayFieldsCount++;
+ }
+ // match the weekday name
+ if (weekdayIDSearch(data, dataStart + dataStringPointer, processedWeekdayFieldsCount) < 0) {
+ throw new AsterixTemporalTypeParseException("Unexpected string for day-of-week: "
+ + (new String(Arrays.copyOfRange(data, dataStart + dataStringPointer, dataStart
+ + dataStringPointer + processedWeekdayFieldsCount))));
+ }
+ dataStringPointer += processedWeekdayFieldsCount;
+ break;
+ case HOUR:
+ case MINUTE:
+ case SECOND:
+ case MILLISECOND:
+ int processFieldsCount = 0;
+ int expectedMaxCount = (processState == DateTimeProcessState.MILLISECOND) ? 3 : 2;
+ parsedValue = 0;
+ for (int i = 0; i < formatCharCopies; i++) {
+ if (data[dataStart + dataStringPointer] < '0' || data[dataStart + dataStringPointer] > '9') {
+ throw new AsterixTemporalTypeParseException("Unexpected char for " + processState.name()
+ + " field at " + (dataStart + dataStringPointer) + ": "
+ + data[dataStart + dataStringPointer]);
+ }
+ parsedValue = parsedValue * 10 + (data[dataStart + dataStringPointer] - '0');
+ dataStringPointer++;
+ if (processFieldsCount++ > expectedMaxCount) {
+ throw new AsterixTemporalTypeParseException("Unexpected char for " + processState.name()
+ + " field at " + dataStringPointer + ": " + data[dataStart + dataStringPointer]);
+ }
+ }
+ // if there are more than formatCharCopies digits for the hour string
+ while (processFieldsCount < expectedMaxCount && dataStringPointer < dataLength
+ && data[dataStart + dataStringPointer] >= '0' && data[dataStart + dataStringPointer] <= '9') {
+ parsedValue = parsedValue * 10 + (data[dataStart + dataStringPointer] - '0');
+ dataStringPointer++;
+ processFieldsCount++;
+ }
+ if (processState == DateTimeProcessState.HOUR) {
+ hour = parsedValue;
+ } else if (processState == DateTimeProcessState.MINUTE) {
+ min = parsedValue;
+ } else if (processState == DateTimeProcessState.SECOND) {
+ sec = parsedValue;
+ } else if (processState == DateTimeProcessState.MILLISECOND) {
+ ms = parsedValue;
+ for (int i = processFieldsCount; i < 3; i++) {
+ ms *= 10;
+ }
+ }
+ break;
+ case TIMEZONE:
+ if (data[dataStart + dataStringPointer] == 'Z'
+ && ((dataStringPointer + 1 >= dataLength) || (data[dataStart + dataStringPointer + 1] < 'A'
+ && data[dataStart + dataStringPointer + 1] > 'Z'
+ && data[dataStart + dataStringPointer + 1] < 'a' && data[dataStart
+ + dataStringPointer + 1] > 'z'))) {
+ // UTC as Z
+ timezone = 0;
+ dataStringPointer++;
+ } else if ((data[dataStart + dataStringPointer] == '+' || data[dataStart + dataStringPointer] == '-')
+ || (dataStringPointer + 3 < dataLength && (data[dataStart + dataStringPointer + 3] == '+' || data[dataStart
+ + dataStringPointer + 3] == '-'))) {
+ // UTC+ or GMT+ format
+ if (dataStringPointer + 3 < dataLength
+ && (byteArrayEqualToString(data, dataStart + dataStringPointer, 3, UTC_BYTEARRAY) || byteArrayEqualToString(
+ data, dataStart + dataStringPointer, 3, GMT_BYTEARRAY))) {
+ dataStringPointer += 3;
+ }
+ // parse timezone as +zz:zz or +zzzz
+ boolean negativeTimeZone = false;
+ if (data[dataStart + dataStringPointer] == '-') {
+ negativeTimeZone = true;
+ dataStringPointer++;
+ } else if (data[dataStart + dataStringPointer] == '+') {
+ dataStringPointer++;
+ } else {
+ throw new AsterixTemporalTypeParseException(
+ "Incorrect timezone hour field: expecting sign + or - but got: "
+ + data[dataStart + dataStringPointer]);
+ }
+ // timezone hours
+ for (int i = 0; i < 2; i++) {
+ if (data[dataStart + dataStringPointer + i] >= '0'
+ && data[dataStart + dataStringPointer + i] <= '9') {
+ timezone += (data[dataStart + dataStringPointer + i] - '0') * MS_PER_HOUR;
+ } else {
+ throw new AsterixTemporalTypeParseException(
+ "Unexpected character for timezone hour field at "
+ + (dataStart + dataStringPointer) + ": "
+ + data[dataStart + dataStringPointer]);
+ }
+ }
+ dataStringPointer += 2;
+ // skip the ":" separator
+ if (data[dataStart + dataStringPointer] == ':') {
+ dataStringPointer++;
+ }
+ // timezone minutes
+ for (int i = 0; i < 2; i++) {
+ if (data[dataStart + dataStringPointer + i] >= '0'
+ && data[dataStart + dataStringPointer + i] <= '9') {
+ timezone += (data[dataStart + dataStringPointer + i] - '0') * MS_PER_MINUTE;
+ } else {
+ throw new AsterixTemporalTypeParseException(
+ "Unexpected character for timezone minute field at "
+ + (dataStart + dataStringPointer) + ": "
+ + data[dataStart + dataStringPointer]);
+ }
+ }
+ dataStringPointer += 2;
+ if (!negativeTimeZone) {
+ timezone *= -1;
+ }
+ } else {
+ // do lookup from the zoneinfor database
+ int timezoneEndField = dataStringPointer;
+ while (timezoneEndField < dataLength
+ && ((data[dataStart + timezoneEndField] >= '0' && data[dataStart + timezoneEndField] <= '9')
+ || (data[dataStart + timezoneEndField] >= 'a' && data[dataStart
+ + timezoneEndField] <= 'z')
+ || (data[dataStart + timezoneEndField] >= 'A' && data[dataStart
+ + timezoneEndField] <= 'Z')
+ || data[dataStart + timezoneEndField] == '/' || data[dataStart
+ + timezoneEndField] == '_')) {
+ timezoneEndField++;
+ }
+ int searchIdx = binaryTimezoneIDSearch(data, dataStart + dataStringPointer, timezoneEndField
+ - dataStringPointer);
+ if (searchIdx >= 0) {
+ timezone = TIMEZONE_OFFSETS[searchIdx];
+ } else {
+ throw new AsterixTemporalTypeParseException("Unexpected timezone string: "
+ + new String(Arrays.copyOfRange(data, dataStart + dataStringPointer, dataStart
+ + dataStringPointer)));
+ }
+ dataStringPointer = timezoneEndField;
+ }
+ break;
+ case AMPM:
+ if (dataStringPointer + 1 < dataLength) {
+ if (hour > 12 || hour <= 0) {
+ throw new IllegalStateException("Hour " + hour + " cannot be a time for AM.");
+ }
+ if (byteArrayEqualToString(data, dataStart + dataStringPointer, 2, AM_BYTEARRAY)) {
+ // do nothing
+ } else if (byteArrayEqualToString(data, dataStart + dataStringPointer, 2, PM_BYTEARRAY)) {
+ hour += 12;
+ if (hour == 24) {
+ hour = 0;
+ }
+ } else {
+ throw new AsterixTemporalTypeParseException("Unexpected string for AM/PM marker "
+ + new String(Arrays.copyOfRange(data, dataStart + dataStringPointer, dataStart
+ + dataStringPointer + 2)));
+ }
+ dataStringPointer += 2;
+ } else {
+ throw new AsterixTemporalTypeParseException("Cannot find valid AM/PM marker.");
+ }
+ break;
+ case SKIPPER:
+ // just skip all continuous character and numbers
+ while ((data[dataStart + dataStringPointer] >= 'a' && data[dataStart + dataStringPointer] <= 'z')
+ || (data[dataStart + dataStringPointer] >= 'A' && data[dataStart + dataStringPointer] <= 'Z')
+ || (data[dataStart + dataStringPointer] >= '0' && data[dataStart + dataStringPointer] <= '9')) {
+ dataStringPointer++;
+ }
+ break;
+ case SEPARATOR:
+ if (separatorChar == '\0') {
+ throw new AsterixTemporalTypeParseException("Incorrect separator char in date string as "
+ + data[dataStart + dataStringPointer]);
+ }
+ for (int i = 0; i < formatCharCopies; i++) {
+ if (data[dataStart + dataStringPointer] != separatorChar) {
+ throw new AsterixTemporalTypeParseException("Expecting separator " + separatorChar
+ + " but got " + data[dataStart + dataStringPointer]);
+ }
+ dataStringPointer++;
+ }
+ break;
+ default:
+ throw new AsterixTemporalTypeParseException(
+ "Unexpected time format information when parsing a date value");
+ }
+ }
+
+ if (dataStringPointer < dataLength) {
+ throw new AsterixTemporalTypeParseException(
+ "The given data string is not fully parsed by the given format string");
+ }
+
+ if (formatPointer < formatLength) {
+ throw new AsterixTemporalTypeParseException(
+ "The given format string is not fully used for the given format string");
+ }
+
+ if (parseMode == DateTimeParseMode.TIME_ONLY) {
+ return CAL.getChronon(hour, min, sec, ms, timezone);
+ }
+ return CAL.getChronon(year, month, day, hour, min, sec, ms, timezone);
+ }
+
+ public void printDateTime(long chronon, int timezone, byte[] format, int formatStart, int formatLength,
+ Appendable appender, DateTimeParseMode parseMode) throws HyracksDataException {
+ int year = CAL.getYear(chronon);
+ int month = CAL.getMonthOfYear(chronon, year);
+ int day = CAL.getDayOfMonthYear(chronon, year, month);
+ int hour = CAL.getHourOfDay(chronon);
+ int min = CAL.getMinOfHour(chronon);
+ int sec = CAL.getSecOfMin(chronon);
+ int ms = CAL.getMillisOfSec(chronon);
+
+ int formatCharCopies = 0;
+
+ int formatPointer = 0;
+
+ byte separatorChar = '\0';
+
+ DateTimeProcessState processState = DateTimeProcessState.INIT;
+
+ int pointerMove = 0;
+
+ boolean usePM = false;
+ if (indexOf(format, formatStart, formatLength, 'a') >= 0) {
+ if (hour >= 12) {
+ usePM = true;
+ hour -= 12;
+ }
+ if (hour == 0) {
+ hour = 12;
+ }
+ }
+
+ while (formatPointer < formatLength) {
+
+ formatCharCopies = 0;
+
+ switch (format[formatStart + formatPointer]) {
+ case YEAR_CHAR:
+ processState = DateTimeProcessState.YEAR;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, YEAR_CHAR,
+ MAX_YEAR_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case MONTH_CHAR:
+ processState = DateTimeProcessState.MONTH;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, MONTH_CHAR,
+ MAX_MONTH_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case DAY_CHAR:
+ processState = DateTimeProcessState.DAY;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, DAY_CHAR,
+ MAX_DAY_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case HOUR_CHAR:
+ processState = DateTimeProcessState.HOUR;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, HOUR_CHAR,
+ MAX_HOUR_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case MINUTE_CHAR:
+ processState = DateTimeProcessState.MINUTE;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, MINUTE_CHAR,
+ MAX_MINUTE_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case SECOND_CHAR:
+ processState = DateTimeProcessState.SECOND;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, SECOND_CHAR,
+ MAX_SECOND_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case MILLISECOND_CHAR:
+ processState = DateTimeProcessState.MILLISECOND;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, MILLISECOND_CHAR,
+ MAX_MILLISECOND_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+
+ case AMPM_CHAR:
+ processState = DateTimeProcessState.AMPM;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, AMPM_CHAR,
+ MAX_AMPM_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+
+ case TIMEZONE_CHAR:
+ processState = DateTimeProcessState.TIMEZONE;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, TIMEZONE_CHAR,
+ MAX_TIMEZONE_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case ' ':
+ case HYPHEN_CHAR:
+ case COLON_CHAR:
+ case SOLIDUS_CHAR:
+ case PERIOD_CHAR:
+ case COMMA_CHAR:
+ case T_CHAR:
+ // separator
+ separatorChar = format[formatStart + formatPointer];
+ processState = DateTimeProcessState.SEPARATOR;
+ formatPointer++;
+ formatCharCopies++;
+ while (formatPointer < formatLength && format[formatStart + formatPointer] == separatorChar) {
+ formatPointer++;
+ formatCharCopies++;
+ }
+ break;
+
+ default:
+ throw new HyracksDataException("Unexpected format string at " + (formatStart + formatPointer)
+ + ": " + format[formatStart + formatPointer]);
+ }
+
+ // check whether the process state is valid for the parse mode
+
+ switch (processState) {
+ case YEAR:
+ case MONTH:
+ case DAY:
+ if (parseMode == DateTimeParseMode.TIME_ONLY) {
+ throw new HyracksDataException("Unexpected date format string when parsing a time value");
+ }
+ break;
+ case HOUR:
+ case MINUTE:
+ case SECOND:
+ case MILLISECOND:
+ case AMPM:
+ case TIMEZONE:
+ if (parseMode == DateTimeParseMode.DATE_ONLY) {
+ throw new HyracksDataException("Unexpected time format string when parsing a date value");
+ }
+ break;
+ default:
+ // do nothing
+ }
+
+ try {
+ switch (processState) {
+ case INIT:
+ break;
+ case YEAR:
+ if (year < 0) {
+ appender.append('-');
+ year *= -1;
+ }
+ case MONTH:
+ if (processState == DateTimeProcessState.MONTH && formatCharCopies == 3) {
+ for (byte b : MONTH_NAMES[month - 1]) {
+ appender.append((char) toUpper(b));
+ }
+ break;
+ }
+ case DAY:
+ int val = 0;
+ if (processState == DateTimeProcessState.YEAR) {
+ val = year;
+ } else if (processState == DateTimeProcessState.MONTH) {
+ val = month;
+ } else {
+ val = day;
+ }
+ int valFieldCount = (int) Math.ceil(Math.log10(val));
+ if (val == 1 || val == 0) {
+ valFieldCount = 1;
+ }
+ for (int i = 0; i < formatCharCopies - valFieldCount; i++) {
+ appender.append('0');
+ }
+ appender.append(String.valueOf(val));
+ break;
+ case HOUR:
+ case MINUTE:
+ case SECOND:
+ val = 0;
+ if (processState == DateTimeProcessState.HOUR) {
+ val = hour;
+ } else if (processState == DateTimeProcessState.MINUTE) {
+ val = min;
+ } else if (processState == DateTimeProcessState.SECOND) {
+ val = sec;
+ }
+
+ if (val < 10) {
+ for (int i = 0; i < formatCharCopies - 1; i++) {
+ appender.append('0');
+ }
+ }
+ appender.append(String.valueOf(val));
+ break;
+ case MILLISECOND:
+ int msFieldCount = (int) Math.ceil(Math.log10(ms));
+ for (int i = 0; i < 3 - msFieldCount; i++) {
+ appender.append('0');
+ }
+ if (formatCharCopies < 3) {
+
+ if (formatCharCopies == 1) {
+ if (ms % 100 == 0) {
+ // the tailing two zeros can be removed
+ ms = ms / 100;
+ } else if (ms % 10 == 0) {
+ // the tailing one zero can be removed
+ ms = ms / 10;
+ }
+ } else {
+ if (ms % 10 == 0) {
+ // the tailing one zero can be removed
+ ms = ms / 10;
+ }
+ }
+
+ }
+ appender.append(String.valueOf(ms));
+ break;
+ case TIMEZONE:
+ if (timezone == 0) {
+ appender.append('Z');
+ break;
+ }
+ if (timezone < 0) {
+ appender.append('-');
+ timezone *= -1;
+ }
+ int timezoneField = timezone / MS_PER_HOUR;
+ if (timezoneField < 10) {
+ appender.append('0');
+ }
+ appender.append(String.valueOf(timezoneField));
+ timezoneField = timezone % MS_PER_HOUR / MS_PER_MINUTE;
+ if (timezoneField < 10) {
+ appender.append('0');
+ }
+ appender.append(String.valueOf(timezoneField));
+ break;
+ case AMPM:
+ if (usePM) {
+ appender.append("PM");
+ } else {
+ appender.append("AM");
+ }
+ break;
+ case SEPARATOR:
+ if (separatorChar == '\0') {
+ throw new HyracksDataException(
+ "Incorrect separator: separator char is not initialized properly!");
+ }
+ for (int i = 0; i < formatCharCopies; i++) {
+ appender.append((char) separatorChar);
+ }
+ break;
+ default:
+ throw new HyracksDataException("Unexpected time state when printing a date value");
+ }
+ } catch (IOException ex) {
+ throw new HyracksDataException(ex);
+ }
+ }
+ }
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/GregorianCalendarSystem.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/GregorianCalendarSystem.java
index 5d27fb0..3be0858 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/GregorianCalendarSystem.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/GregorianCalendarSystem.java
@@ -164,15 +164,8 @@
* @return
*/
public boolean validateTimeZone(int timezone) {
- short tzMin = (short) ((timezone % 4) * 15);
- if (tzMin < -60 || tzMin >= 60) {
- return false;
- }
-
- short tzHr = (short) (timezone / 4);
-
- if (tzHr < -12 && tzHr > 14) {
+ if (timezone < -12 * CHRONON_OF_DAY || timezone > 14 * CHRONON_OF_DAY) {
return false;
}
@@ -211,8 +204,8 @@
*/
public long getChronon(int year, int month, int day, int hour, int min, int sec, int millis, int timezone) {
// Added milliseconds for all fields but month and day
- long chrononTime = chrononizeBeginningOfYear(year) + (hour - timezone / 4) * CHRONON_OF_HOUR
- + (min - (timezone % 4) * 15) * CHRONON_OF_MINUTE + sec * CHRONON_OF_SECOND + millis;
+ long chrononTime = chrononizeBeginningOfYear(year) + hour * CHRONON_OF_HOUR + min * CHRONON_OF_MINUTE + sec
+ * CHRONON_OF_SECOND + millis + timezone;
// Added milliseconds for days of the month.
chrononTime += (day - 1 + DAYS_SINCE_MONTH_BEGIN_ORDI[month - 1]) * CHRONON_OF_DAY;
@@ -237,13 +230,13 @@
*/
public int getChronon(int hour, int min, int sec, int millis, int timezone) {
// Added milliseconds for all fields but month and day
- long chrononTime = (hour - timezone / 4) * CHRONON_OF_HOUR + (min - (timezone % 4) * 15) * CHRONON_OF_MINUTE
- + sec * CHRONON_OF_SECOND + millis;
- return (int)chrononTime;
+ long chrononTime = hour * CHRONON_OF_HOUR + min * CHRONON_OF_MINUTE + sec * CHRONON_OF_SECOND + millis
+ + timezone;
+ return (int) chrononTime;
}
public long adjustChrononByTimezone(long chronon, int timezone) {
- return chronon + timezone / 4 * CHRONON_OF_HOUR + (timezone % 4) * 15 * CHRONON_OF_MINUTE;
+ return chronon - timezone;
}
public static int getChrononInDays(long chronon) {
@@ -330,12 +323,12 @@
if (timezone == 0) {
sbder.append("Z");
} else {
- short tzMin = (short) ((timezone % 4) * 15);
+ int tzMin = (int) (timezone % CHRONON_OF_HOUR / CHRONON_OF_MINUTE);
if (tzMin < 0) {
tzMin = (short) (-1 * tzMin);
}
- short tzHr = (short) (timezone / 4);
- sbder.append((tzHr >= 0 ? "+" : "-")).append(String.format("%02d", (tzHr < 0 ? -tzHr : tzHr)))
+ int tzHr = (int) (timezone / CHRONON_OF_HOUR);
+ sbder.append((tzHr >= 0 ? "-" : "+")).append(String.format("%02d", (tzHr < 0 ? -tzHr : tzHr)))
.append(":").append(String.format("%02d", tzMin));
}
}
@@ -391,12 +384,12 @@
if (timezone == 0) {
sbder.append("Z");
} else {
- short tzMin = (short) ((timezone % 4) * 15);
+ int tzMin = (int) (timezone % CHRONON_OF_HOUR / CHRONON_OF_MINUTE);
if (tzMin < 0) {
tzMin = (short) (-1 * tzMin);
}
- short tzHr = (short) (timezone / 4);
- sbder.append((tzHr >= 0 ? "+" : "-")).append(String.format("%02d", (tzHr < 0 ? -tzHr : tzHr)))
+ int tzHr = (int) (timezone / CHRONON_OF_HOUR);
+ sbder.append((tzHr >= 0 ? "-" : "+")).append(String.format("%02d", (tzHr < 0 ? -tzHr : tzHr)))
.append(String.format("%02d", tzMin));
}
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java
index 2aa765c..b6bdb4b 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java
@@ -544,6 +544,18 @@
FunctionConstants.ASTERIX_NS, "adjust-datetime-for-timezone", 2);
public final static FunctionIdentifier DAY_OF_WEEK = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
"day-of-week");
+ public final static FunctionIdentifier PARSE_DATE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+ "parse-date", 2);
+ public final static FunctionIdentifier PARSE_TIME = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+ "parse-time", 2);
+ public final static FunctionIdentifier PARSE_DATETIME = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+ "parse-datetime", 2);
+ public final static FunctionIdentifier PRINT_DATE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+ "print-date", 2);
+ public final static FunctionIdentifier PRINT_TIME = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+ "print-time", 2);
+ public final static FunctionIdentifier PRINT_DATETIME = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+ "print-datetime", 2);
public final static FunctionIdentifier GET_POINT_X_COORDINATE_ACCESSOR = new FunctionIdentifier(
FunctionConstants.ASTERIX_NS, "get-x", 1);
@@ -868,6 +880,12 @@
addFunction(GET_YEAR_MONTH_DURATION, OptionalAYearMonthDurationTypeComputer.INSTANCE);
addFunction(INTERVAL_BIN, OptionalAIntervalTypeComputer.INSTANCE);
addFunction(DAY_OF_WEEK, OptionalAInt32TypeComputer.INSTANCE);
+ addFunction(PARSE_DATE, OptionalADateTypeComputer.INSTANCE);
+ addFunction(PARSE_TIME, OptionalATimeTypeComputer.INSTANCE);
+ addFunction(PARSE_DATETIME, OptionalADateTimeTypeComputer.INSTANCE);
+ addFunction(PRINT_DATE, OptionalAStringTypeComputer.INSTANCE);
+ addFunction(PRINT_TIME, OptionalAStringTypeComputer.INSTANCE);
+ addFunction(PRINT_DATETIME, OptionalAStringTypeComputer.INSTANCE);
// interval constructors
addFunction(INTERVAL_CONSTRUCTOR_DATE, OptionalAIntervalTypeComputer.INSTANCE);
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixAppContextInfo.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixAppContextInfo.java
index 6cdaea0..0f836af 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixAppContextInfo.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixAppContextInfo.java
@@ -100,11 +100,11 @@
@Override
public IIndexLifecycleManagerProvider getIndexLifecycleManagerProvider() {
- return AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER;
+ return AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
}
@Override
public IStorageManagerInterface getStorageManagerInterface() {
- return AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER;
+ return AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
}
}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
index 03c4140..5070210 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
@@ -83,6 +83,13 @@
int stringLength = (serString[1] & 0xff << 8) + (serString[2] & 0xff << 0);
+ // the string to be parsed should be at least 8 characters: YYYYMMDD
+ if (stringLength < 8) {
+ throw new AlgebricksException(errorMessage
+ + ": the string length should be at least 8 (YYYYMMDD) but it is "
+ + stringLength);
+ }
+
int startOffset = 3;
while (serString[startOffset] == ' ') {
startOffset++;
@@ -111,8 +118,6 @@
}
} catch (IOException e1) {
throw new AlgebricksException(errorMessage);
- } catch (Exception e2) {
- throw new AlgebricksException(e2);
}
}
};
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
index 986b158..880de44 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
@@ -82,7 +82,12 @@
if (serString[0] == SER_STRING_TYPE_TAG) {
int stringLength = (serString[1] & 0xff << 8) + (serString[2] & 0xff << 0);
-
+ // the string to be parsed should be at least 14 characters: YYYYMMDDhhmmss
+ if (stringLength < 14) {
+ throw new AlgebricksException(errorMessage
+ + ": the string length should be at least 14 (YYYYMMDDhhmmss) but it is "
+ + stringLength);
+ }
// +1 if it is negative (-)
short timeOffset = (short) ((serString[3] == '-') ? 1 : 0);
@@ -109,8 +114,6 @@
}
} catch (IOException e1) {
throw new AlgebricksException(errorMessage);
- } catch (Exception e2) {
- throw new AlgebricksException(e2);
}
}
};
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
index 1b3e7f6..d344d1c 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
@@ -83,6 +83,13 @@
int stringLength = (serString[1] & 0xff << 8) + (serString[2] & 0xff << 0);
+ // the string to be parsed should be at least 6 characters: hhmmss
+ if (stringLength < 6) {
+ throw new AlgebricksException(errorMessage
+ + ": the string length should be at least 6 (hhmmss) but it is "
+ + stringLength);
+ }
+
int chrononTimeInMs = ATimeParserFactory.parseTimePart(serString, 3, stringLength);
if (chrononTimeInMs < 0) {
@@ -99,8 +106,6 @@
}
} catch (IOException e1) {
throw new AlgebricksException(errorMessage);
- } catch (Exception e2) {
- throw new AlgebricksException(e2);
}
}
};
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/ParseDateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/ParseDateDescriptor.java
new file mode 100644
index 0000000..782d424
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/ParseDateDescriptor.java
@@ -0,0 +1,168 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.runtime.evaluators.functions.temporal;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.ADate;
+import edu.uci.ics.asterix.om.base.AMutableDate;
+import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.base.temporal.AsterixTemporalTypeParseException;
+import edu.uci.ics.asterix.om.base.temporal.DateTimeFormatUtils;
+import edu.uci.ics.asterix.om.base.temporal.GregorianCalendarSystem;
+import edu.uci.ics.asterix.om.base.temporal.DateTimeFormatUtils.DateTimeParseMode;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+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;
+
+/**
+ * <b>|(bar)</b> is a special separator used to separate different formatting options.
+ * Multiple format strings can be used by separating them using <b>|(bar)</b>, and the parsing will be successful only when the format string has the <b>exact</b> match with the given data string. This means that a time string like <it>08:23:12 AM</it> will not be valid for the format string <it>h:m:s</it> as there is no AM/PM format character in the format string.
+ * <p/>
+ */
+public class ParseDateDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public final static FunctionIdentifier FID = AsterixBuiltinFunctions.PARSE_DATE;
+
+ private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
+ private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
+ private final static DateTimeFormatUtils DT_UTILS = DateTimeFormatUtils.getInstance();
+
+ public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new ParseDateDescriptor();
+ }
+ };
+
+ @Override
+ public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
+ return new ICopyEvaluatorFactory() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+ return new ICopyEvaluator() {
+
+ private DataOutput out = output.getDataOutput();
+ private ArrayBackedValueStorage argOut0 = new ArrayBackedValueStorage();
+ private ArrayBackedValueStorage argOut1 = new ArrayBackedValueStorage();
+ private ICopyEvaluator eval0 = args[0].createEvaluator(argOut0);
+ private ICopyEvaluator eval1 = args[1].createEvaluator(argOut1);
+
+ @SuppressWarnings("unchecked")
+ private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ANULL);
+ @SuppressWarnings("unchecked")
+ private ISerializerDeserializer<ADate> dateSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ADATE);
+
+ private AMutableDate aDate = new AMutableDate(0);
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+ argOut0.reset();
+ eval0.evaluate(tuple);
+ argOut1.reset();
+ eval1.evaluate(tuple);
+
+ try {
+ if (argOut0.getByteArray()[0] == SER_NULL_TYPE_TAG
+ || argOut1.getByteArray()[0] == SER_NULL_TYPE_TAG) {
+ nullSerde.serialize(ANull.NULL, out);
+ return;
+ }
+
+ if (argOut0.getByteArray()[0] != SER_STRING_TYPE_TAG
+ || argOut1.getByteArray()[0] != SER_STRING_TYPE_TAG) {
+ throw new AlgebricksException(getIdentifier().getName()
+ + ": expects two strings but got ("
+ + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut0.getByteArray()[0])
+ + ", "
+ + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut1.getByteArray()[0])
+ + ")");
+ }
+ int length0 = (argOut0.getByteArray()[1] & 0xff << 8)
+ + (argOut0.getByteArray()[2] & 0xff << 0);
+ int length1 = (argOut1.getByteArray()[1] & 0xff << 8)
+ + (argOut1.getByteArray()[2] & 0xff << 0);
+ long chronon = 0;
+
+ int formatStart = 3;
+ int formatLength = 0;
+ boolean processSuccessfully = false;
+ while (!processSuccessfully && formatStart < 3 + length1) {
+ // search for "|"
+ formatLength = 0;
+ for (; formatStart + formatLength < 3 + length1; formatLength++) {
+ if (argOut1.getByteArray()[formatStart + formatLength] == '|') {
+ break;
+ }
+ }
+ try {
+ chronon = DT_UTILS.parseDateTime(argOut0.getByteArray(), 3, length0,
+ argOut1.getByteArray(), formatStart, formatLength,
+ DateTimeParseMode.DATE_ONLY);
+ } catch (AsterixTemporalTypeParseException ex) {
+ formatStart += formatLength + 1;
+ continue;
+ }
+ processSuccessfully = true;
+ }
+
+ if (!processSuccessfully) {
+ throw new HyracksDataException(
+ "parse-date: Failed to match with any given format string!");
+ }
+
+ aDate.setValue((int) (chronon / GregorianCalendarSystem.CHRONON_OF_DAY));
+ dateSerde.serialize(aDate, out);
+
+ } catch (HyracksDataException ex) {
+ throw new AlgebricksException(ex);
+ }
+ }
+ };
+ }
+
+ };
+ }
+
+ /* (non-Javadoc)
+ * @see edu.uci.ics.asterix.om.functions.AbstractFunctionDescriptor#getIdentifier()
+ */
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return FID;
+ }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/ParseDateTimeDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/ParseDateTimeDescriptor.java
new file mode 100644
index 0000000..89cfbe5
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/ParseDateTimeDescriptor.java
@@ -0,0 +1,162 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.runtime.evaluators.functions.temporal;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.ADateTime;
+import edu.uci.ics.asterix.om.base.AMutableDateTime;
+import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.base.temporal.AsterixTemporalTypeParseException;
+import edu.uci.ics.asterix.om.base.temporal.DateTimeFormatUtils;
+import edu.uci.ics.asterix.om.base.temporal.DateTimeFormatUtils.DateTimeParseMode;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+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;
+
+public class ParseDateTimeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public final static FunctionIdentifier FID = AsterixBuiltinFunctions.PARSE_DATETIME;
+
+ private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
+ private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
+ private final static DateTimeFormatUtils DT_UTILS = DateTimeFormatUtils.getInstance();
+
+ public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new ParseDateTimeDescriptor();
+ }
+ };
+
+ @Override
+ public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
+ return new ICopyEvaluatorFactory() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+ return new ICopyEvaluator() {
+
+ private DataOutput out = output.getDataOutput();
+ private ArrayBackedValueStorage argOut0 = new ArrayBackedValueStorage();
+ private ArrayBackedValueStorage argOut1 = new ArrayBackedValueStorage();
+ private ICopyEvaluator eval0 = args[0].createEvaluator(argOut0);
+ private ICopyEvaluator eval1 = args[1].createEvaluator(argOut1);
+
+ @SuppressWarnings("unchecked")
+ private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ANULL);
+ @SuppressWarnings("unchecked")
+ private ISerializerDeserializer<ADateTime> datetimeSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ADATETIME);
+
+ private AMutableDateTime aDateTime = new AMutableDateTime(0);
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+ argOut0.reset();
+ eval0.evaluate(tuple);
+ argOut1.reset();
+ eval1.evaluate(tuple);
+
+ try {
+ if (argOut0.getByteArray()[0] == SER_NULL_TYPE_TAG
+ || argOut1.getByteArray()[0] == SER_NULL_TYPE_TAG) {
+ nullSerde.serialize(ANull.NULL, out);
+ return;
+ }
+
+ if (argOut0.getByteArray()[0] != SER_STRING_TYPE_TAG
+ || argOut1.getByteArray()[0] != SER_STRING_TYPE_TAG) {
+ throw new AlgebricksException(getIdentifier().getName()
+ + ": expects two strings but got ("
+ + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut0.getByteArray()[0])
+ + ", "
+ + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut1.getByteArray()[0])
+ + ")");
+ }
+ int length0 = (argOut0.getByteArray()[1] & 0xff << 8)
+ + (argOut0.getByteArray()[2] & 0xff << 0);
+ int length1 = (argOut1.getByteArray()[1] & 0xff << 8)
+ + (argOut1.getByteArray()[2] & 0xff << 0);
+ long chronon = 0;
+
+ int formatStart = 3;
+ int formatLength = 0;
+ boolean processSuccessfully = false;
+ while (!processSuccessfully && formatStart < 3 + length1) {
+ // search for "|"
+ formatLength = 0;
+ for (; formatStart + formatLength < 3 + length1; formatLength++) {
+ if (argOut1.getByteArray()[formatStart + formatLength] == '|') {
+ break;
+ }
+ }
+ try {
+ chronon = DT_UTILS.parseDateTime(argOut0.getByteArray(), 3, length0,
+ argOut1.getByteArray(), formatStart, formatLength,
+ DateTimeParseMode.DATETIME);
+ } catch (AsterixTemporalTypeParseException ex) {
+ formatStart += formatLength + 1;
+ continue;
+ }
+ processSuccessfully = true;
+ }
+
+ if (!processSuccessfully) {
+ throw new HyracksDataException(
+ "parse-date: Failed to match with any given format string!");
+ }
+
+ aDateTime.setValue(chronon);
+ datetimeSerde.serialize(aDateTime, out);
+
+ } catch (HyracksDataException ex) {
+ throw new AlgebricksException(ex);
+ }
+ }
+ };
+ }
+
+ };
+ }
+
+ /* (non-Javadoc)
+ * @see edu.uci.ics.asterix.om.functions.AbstractFunctionDescriptor#getIdentifier()
+ */
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return FID;
+ }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/ParseTimeDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/ParseTimeDescriptor.java
new file mode 100644
index 0000000..8ee76e7
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/ParseTimeDescriptor.java
@@ -0,0 +1,162 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.runtime.evaluators.functions.temporal;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.AMutableTime;
+import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.base.ATime;
+import edu.uci.ics.asterix.om.base.temporal.AsterixTemporalTypeParseException;
+import edu.uci.ics.asterix.om.base.temporal.DateTimeFormatUtils;
+import edu.uci.ics.asterix.om.base.temporal.DateTimeFormatUtils.DateTimeParseMode;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+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;
+
+public class ParseTimeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public final static FunctionIdentifier FID = AsterixBuiltinFunctions.PARSE_TIME;
+
+ private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
+ private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
+ private final static DateTimeFormatUtils DT_UTILS = DateTimeFormatUtils.getInstance();
+
+ public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new ParseTimeDescriptor();
+ }
+ };
+
+ @Override
+ public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
+ return new ICopyEvaluatorFactory() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+ return new ICopyEvaluator() {
+
+ private DataOutput out = output.getDataOutput();
+ private ArrayBackedValueStorage argOut0 = new ArrayBackedValueStorage();
+ private ArrayBackedValueStorage argOut1 = new ArrayBackedValueStorage();
+ private ICopyEvaluator eval0 = args[0].createEvaluator(argOut0);
+ private ICopyEvaluator eval1 = args[1].createEvaluator(argOut1);
+
+ @SuppressWarnings("unchecked")
+ private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ANULL);
+ @SuppressWarnings("unchecked")
+ private ISerializerDeserializer<ATime> timeSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ATIME);
+
+ private AMutableTime aTime = new AMutableTime(0);
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+ argOut0.reset();
+ eval0.evaluate(tuple);
+ argOut1.reset();
+ eval1.evaluate(tuple);
+
+ try {
+ if (argOut0.getByteArray()[0] == SER_NULL_TYPE_TAG
+ || argOut1.getByteArray()[0] == SER_NULL_TYPE_TAG) {
+ nullSerde.serialize(ANull.NULL, out);
+ return;
+ }
+
+ if (argOut0.getByteArray()[0] != SER_STRING_TYPE_TAG
+ || argOut1.getByteArray()[0] != SER_STRING_TYPE_TAG) {
+ throw new AlgebricksException(getIdentifier().getName()
+ + ": expects two strings but got ("
+ + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut0.getByteArray()[0])
+ + ", "
+ + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut1.getByteArray()[0])
+ + ")");
+ }
+ int length0 = (argOut0.getByteArray()[1] & 0xff << 8)
+ + (argOut0.getByteArray()[2] & 0xff << 0);
+ int length1 = (argOut1.getByteArray()[1] & 0xff << 8)
+ + (argOut1.getByteArray()[2] & 0xff << 0);
+ long chronon = 0;
+
+ int formatStart = 3;
+ int formatLength = 0;
+ boolean processSuccessfully = false;
+ while (!processSuccessfully && formatStart < 3 + length1) {
+ // search for "|"
+ formatLength = 0;
+ for (; formatStart + formatLength < 3 + length1; formatLength++) {
+ if (argOut1.getByteArray()[formatStart + formatLength] == '|') {
+ break;
+ }
+ }
+ try {
+ chronon = DT_UTILS.parseDateTime(argOut0.getByteArray(), 3, length0,
+ argOut1.getByteArray(), formatStart, formatLength,
+ DateTimeParseMode.TIME_ONLY);
+ } catch (AsterixTemporalTypeParseException ex) {
+ formatStart += formatLength + 1;
+ continue;
+ }
+ processSuccessfully = true;
+ }
+
+ if (!processSuccessfully) {
+ throw new HyracksDataException(
+ "parse-date: Failed to match with any given format string!");
+ }
+
+ aTime.setValue((int) chronon);
+ timeSerde.serialize(aTime, out);
+
+ } catch (HyracksDataException ex) {
+ throw new AlgebricksException(ex);
+ }
+ }
+ };
+ }
+
+ };
+ }
+
+ /* (non-Javadoc)
+ * @see edu.uci.ics.asterix.om.functions.AbstractFunctionDescriptor#getIdentifier()
+ */
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return FID;
+ }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/PrintDateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/PrintDateDescriptor.java
new file mode 100644
index 0000000..a48e44c
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/PrintDateDescriptor.java
@@ -0,0 +1,135 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.runtime.evaluators.functions.temporal;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADateSerializerDeserializer;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.base.temporal.DateTimeFormatUtils;
+import edu.uci.ics.asterix.om.base.temporal.GregorianCalendarSystem;
+import edu.uci.ics.asterix.om.base.temporal.DateTimeFormatUtils.DateTimeParseMode;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+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;
+
+public class PrintDateDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public final static FunctionIdentifier FID = AsterixBuiltinFunctions.PRINT_DATE;
+
+ private final static byte SER_DATE_TYPE_TAG = ATypeTag.DATE.serialize();
+ private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
+ private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
+ private final static DateTimeFormatUtils DT_UTILS = DateTimeFormatUtils.getInstance();
+
+ public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new PrintDateDescriptor();
+ }
+ };
+
+ @Override
+ public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
+ return new ICopyEvaluatorFactory() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+ return new ICopyEvaluator() {
+
+ private DataOutput out = output.getDataOutput();
+ private ArrayBackedValueStorage argOut0 = new ArrayBackedValueStorage();
+ private ArrayBackedValueStorage argOut1 = new ArrayBackedValueStorage();
+ private ICopyEvaluator eval0 = args[0].createEvaluator(argOut0);
+ private ICopyEvaluator eval1 = args[1].createEvaluator(argOut1);
+
+ @SuppressWarnings("unchecked")
+ private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ANULL);
+
+ private StringBuilder sbder = new StringBuilder();
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+ argOut0.reset();
+ eval0.evaluate(tuple);
+ argOut1.reset();
+ eval1.evaluate(tuple);
+
+ try {
+ if (argOut0.getByteArray()[0] == SER_NULL_TYPE_TAG
+ || argOut1.getByteArray()[0] == SER_NULL_TYPE_TAG) {
+ nullSerde.serialize(ANull.NULL, out);
+ return;
+ }
+
+ if (argOut0.getByteArray()[0] != SER_DATE_TYPE_TAG
+ || argOut1.getByteArray()[0] != SER_STRING_TYPE_TAG) {
+ throw new AlgebricksException(getIdentifier().getName()
+ + ": expects (DATE, STRING) but got ("
+ + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut0.getByteArray()[0])
+ + ", "
+ + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut1.getByteArray()[0])
+ + ")");
+ }
+
+ long chronon = ADateSerializerDeserializer.getChronon(argOut0.getByteArray(), 1)
+ * GregorianCalendarSystem.CHRONON_OF_DAY;
+ int formatLength = (argOut1.getByteArray()[1] & 0xff << 8)
+ + (argOut1.getByteArray()[2] & 0xff << 0);
+ sbder.delete(0, sbder.length());
+ DT_UTILS.printDateTime(chronon, 0, argOut1.getByteArray(), 3, formatLength, sbder,
+ DateTimeParseMode.DATE_ONLY);
+
+ out.writeByte(ATypeTag.STRING.serialize());
+ out.writeUTF(sbder.toString());
+
+ } catch (IOException ex) {
+ throw new AlgebricksException(ex);
+ }
+ }
+ };
+ }
+
+ };
+ }
+
+ /* (non-Javadoc)
+ * @see edu.uci.ics.asterix.om.functions.AbstractFunctionDescriptor#getIdentifier()
+ */
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return FID;
+ }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/PrintDateTimeDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/PrintDateTimeDescriptor.java
new file mode 100644
index 0000000..9c507e3
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/PrintDateTimeDescriptor.java
@@ -0,0 +1,132 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.runtime.evaluators.functions.temporal;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.base.temporal.DateTimeFormatUtils;
+import edu.uci.ics.asterix.om.base.temporal.DateTimeFormatUtils.DateTimeParseMode;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+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;
+
+public class PrintDateTimeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public final static FunctionIdentifier FID = AsterixBuiltinFunctions.PRINT_DATETIME;
+
+ private final static byte SER_DATETIME_TYPE_TAG = ATypeTag.DATETIME.serialize();
+ private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
+ private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
+ private final static DateTimeFormatUtils DT_UTILS = DateTimeFormatUtils.getInstance();
+
+ public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new PrintDateTimeDescriptor();
+ }
+ };
+
+ @Override
+ public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
+ return new ICopyEvaluatorFactory() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+ return new ICopyEvaluator() {
+
+ private DataOutput out = output.getDataOutput();
+ private ArrayBackedValueStorage argOut0 = new ArrayBackedValueStorage();
+ private ArrayBackedValueStorage argOut1 = new ArrayBackedValueStorage();
+ private ICopyEvaluator eval0 = args[0].createEvaluator(argOut0);
+ private ICopyEvaluator eval1 = args[1].createEvaluator(argOut1);
+
+ @SuppressWarnings("unchecked")
+ private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ANULL);
+
+ private StringBuilder sbder = new StringBuilder();
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+ argOut0.reset();
+ eval0.evaluate(tuple);
+ argOut1.reset();
+ eval1.evaluate(tuple);
+
+ try {
+ if (argOut0.getByteArray()[0] == SER_NULL_TYPE_TAG
+ || argOut1.getByteArray()[0] == SER_NULL_TYPE_TAG) {
+ nullSerde.serialize(ANull.NULL, out);
+ return;
+ }
+
+ if (argOut0.getByteArray()[0] != SER_DATETIME_TYPE_TAG
+ || argOut1.getByteArray()[0] != SER_STRING_TYPE_TAG) {
+ throw new AlgebricksException(getIdentifier().getName()
+ + ": expects (DATETIME, STRING) but got ("
+ + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut0.getByteArray()[0])
+ + ", "
+ + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut1.getByteArray()[0])
+ + ")");
+ }
+ long chronon = ADateTimeSerializerDeserializer.getChronon(argOut0.getByteArray(), 1);
+ int formatLength = (argOut1.getByteArray()[1] & 0xff << 8)
+ + (argOut1.getByteArray()[2] & 0xff << 0);
+ sbder.delete(0, sbder.length());
+ DT_UTILS.printDateTime(chronon, 0, argOut1.getByteArray(), 3, formatLength, sbder,
+ DateTimeParseMode.DATETIME);
+
+ out.writeByte(ATypeTag.STRING.serialize());
+ out.writeUTF(sbder.toString());
+
+ } catch (IOException ex) {
+ throw new AlgebricksException(ex);
+ }
+ }
+ };
+ }
+
+ };
+ }
+
+ /* (non-Javadoc)
+ * @see edu.uci.ics.asterix.om.functions.AbstractFunctionDescriptor#getIdentifier()
+ */
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return FID;
+ }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/PrintTimeDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/PrintTimeDescriptor.java
new file mode 100644
index 0000000..7d589a4
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/PrintTimeDescriptor.java
@@ -0,0 +1,133 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.runtime.evaluators.functions.temporal;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ATimeSerializerDeserializer;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.base.temporal.DateTimeFormatUtils;
+import edu.uci.ics.asterix.om.base.temporal.DateTimeFormatUtils.DateTimeParseMode;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+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;
+
+public class PrintTimeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public final static FunctionIdentifier FID = AsterixBuiltinFunctions.PRINT_TIME;
+
+ private final static byte SER_TIME_TYPE_TAG = ATypeTag.TIME.serialize();
+ private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
+ private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
+ private final static DateTimeFormatUtils DT_UTILS = DateTimeFormatUtils.getInstance();
+
+ public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new PrintTimeDescriptor();
+ }
+ };
+
+ @Override
+ public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
+ return new ICopyEvaluatorFactory() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+ return new ICopyEvaluator() {
+
+ private DataOutput out = output.getDataOutput();
+ private ArrayBackedValueStorage argOut0 = new ArrayBackedValueStorage();
+ private ArrayBackedValueStorage argOut1 = new ArrayBackedValueStorage();
+ private ICopyEvaluator eval0 = args[0].createEvaluator(argOut0);
+ private ICopyEvaluator eval1 = args[1].createEvaluator(argOut1);
+
+ @SuppressWarnings("unchecked")
+ private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ANULL);
+
+ private StringBuilder sbder = new StringBuilder();
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+ argOut0.reset();
+ eval0.evaluate(tuple);
+ argOut1.reset();
+ eval1.evaluate(tuple);
+
+ try {
+ if (argOut0.getByteArray()[0] == SER_NULL_TYPE_TAG
+ || argOut1.getByteArray()[0] == SER_NULL_TYPE_TAG) {
+ nullSerde.serialize(ANull.NULL, out);
+ return;
+ }
+
+ if (argOut0.getByteArray()[0] != SER_TIME_TYPE_TAG
+ || argOut1.getByteArray()[0] != SER_STRING_TYPE_TAG) {
+ throw new AlgebricksException(getIdentifier().getName()
+ + ": expects (TIME, STRING) but got ("
+ + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut0.getByteArray()[0])
+ + ", "
+ + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut1.getByteArray()[0])
+ + ")");
+ }
+
+ long chronon = ATimeSerializerDeserializer.getChronon(argOut0.getByteArray(), 1);
+ int formatLength = (argOut1.getByteArray()[1] & 0xff << 8)
+ + (argOut1.getByteArray()[2] & 0xff << 0);
+ sbder.delete(0, sbder.length());
+ DT_UTILS.printDateTime(chronon, 0, argOut1.getByteArray(), 3, formatLength, sbder,
+ DateTimeParseMode.TIME_ONLY);
+
+ out.writeByte(ATypeTag.STRING.serialize());
+ out.writeUTF(sbder.toString());
+
+ } catch (IOException ex) {
+ throw new AlgebricksException(ex);
+ }
+ }
+ };
+ }
+
+ };
+ }
+
+ /* (non-Javadoc)
+ * @see edu.uci.ics.asterix.om.functions.AbstractFunctionDescriptor#getIdentifier()
+ */
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return FID;
+ }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java
index a13eed3..566aa98 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java
@@ -245,6 +245,12 @@
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalOverlapsDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalStartedByDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalStartsDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.ParseDateDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.ParseDateTimeDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.ParseTimeDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.PrintDateDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.PrintDateTimeDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.PrintTimeDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.TimeFromDatetimeDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.TimeFromUnixTimeInMsDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.YearMonthDurationComparatorDecriptor;
@@ -541,6 +547,12 @@
temp.add(GetDayTimeDurationDescriptor.FACTORY);
temp.add(IntervalBinDescriptor.FACTORY);
temp.add(DayOfWeekDescriptor.FACTORY);
+ temp.add(ParseDateDescriptor.FACTORY);
+ temp.add(ParseTimeDescriptor.FACTORY);
+ temp.add(ParseDateTimeDescriptor.FACTORY);
+ temp.add(PrintDateDescriptor.FACTORY);
+ temp.add(PrintTimeDescriptor.FACTORY);
+ temp.add(PrintDateTimeDescriptor.FACTORY);
// Interval constructor
temp.add(AIntervalFromDateConstructorDescriptor.FACTORY);
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 8092ecd..b44755c 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
@@ -18,7 +18,6 @@
import edu.uci.ics.asterix.common.exceptions.ACIDException;
import edu.uci.ics.asterix.common.transactions.DatasetId;
import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.ITransactionContext.TransactionType;
import edu.uci.ics.asterix.common.transactions.ITransactionManager;
import edu.uci.ics.asterix.common.transactions.JobId;
import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
@@ -51,8 +50,7 @@
ITransactionManager txnManager = ((IAsterixAppRuntimeContext) jobletContext.getApplicationContext()
.getApplicationObject()).getTransactionSubsystem().getTransactionManager();
ITransactionContext txnContext = txnManager.getTransactionContext(jobId);
- txnContext.setTransactionType(transactionalWrite ? TransactionType.READ_WRITE
- : TransactionType.READ);
+ txnContext.setWriteTxn(transactionalWrite);
txnManager.completedTransaction(txnContext, new DatasetId(-1), -1,
!(jobStatus == JobStatus.FAILURE));
} catch (ACIDException e) {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/DelimitedDataParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/DelimitedDataParser.java
index 59eb908..22c43ec 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/DelimitedDataParser.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/DelimitedDataParser.java
@@ -24,7 +24,9 @@
import edu.uci.ics.asterix.builders.IARecordBuilder;
import edu.uci.ics.asterix.builders.RecordBuilder;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ANullSerializerDeserializer;
import edu.uci.ics.asterix.om.base.AMutableString;
+import edu.uci.ics.asterix.om.base.ANull;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.ATypeTag;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -47,6 +49,8 @@
private int[] fldIds;
private ArrayBackedValueStorage[] nameBuffers;
+ private boolean areAllNullFields;
+
public DelimitedDataParser(ARecordType recordType, IValueParserFactory[] valueParserFactories, char fieldDelimter) {
this.recordType = recordType;
this.valueParserFactories = valueParserFactories;
@@ -108,29 +112,44 @@
@Override
public boolean parse(DataOutput out) throws AsterixException, IOException {
-
- if (cursor.nextRecord()) {
+ while (cursor.nextRecord()) {
recBuilder.reset(recordType);
recBuilder.init();
+ areAllNullFields = true;
for (int i = 0; i < valueParsers.length; ++i) {
if (!cursor.nextField()) {
break;
}
fieldValueBuffer.reset();
- fieldValueBufferOutput.writeByte(fieldTypeTags[i]);
- valueParsers[i]
- .parse(cursor.buffer, cursor.fStart, cursor.fEnd - cursor.fStart, fieldValueBufferOutput);
+
+ if (cursor.fStart == cursor.fEnd && recordType.getFieldTypes()[i].getTypeTag() != ATypeTag.STRING
+ && recordType.getFieldTypes()[i].getTypeTag() != ATypeTag.NULL) {
+ // if the field is empty and the type is optional, insert NULL
+ // note that string type can also process empty field as an empty string
+ if (recordType.getFieldTypes()[i].getTypeTag() != ATypeTag.UNION) {
+ throw new AsterixException("Field " + i + " cannot be NULL. ");
+ }
+ fieldValueBufferOutput.writeByte(ATypeTag.NULL.serialize());
+ ANullSerializerDeserializer.INSTANCE.serialize(ANull.NULL, out);
+ } else {
+ fieldValueBufferOutput.writeByte(fieldTypeTags[i]);
+ valueParsers[i].parse(cursor.buffer, cursor.fStart, cursor.fEnd - cursor.fStart,
+ fieldValueBufferOutput);
+ areAllNullFields = false;
+ }
if (fldIds[i] < 0) {
recBuilder.addField(nameBuffers[i], fieldValueBuffer);
} else {
recBuilder.addField(fldIds[i], fieldValueBuffer);
}
+
}
- recBuilder.write(out, true);
- return true;
- } else {
- return false;
+ if (!areAllNullFields) {
+ recBuilder.write(out, true);
+ return true;
+ }
}
+ return false;
}
protected void fieldNameToBytes(String fieldName, AMutableString str, ArrayBackedValueStorage buffer)
@@ -278,11 +297,13 @@
if (p >= end) {
int s = start;
eof = !readMore();
+ p -= (s - start);
if (eof) {
state = State.EOF;
+ fStart = start;
+ fEnd = p;
return true;
}
- p -= (s - start);
}
char ch = buffer[p];
if (ch == fieldDelimiter) {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/AbstractIndexModificationOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/AbstractIndexModificationOperationCallback.java
new file mode 100644
index 0000000..031a26e
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/AbstractIndexModificationOperationCallback.java
@@ -0,0 +1,82 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.transaction.management.opcallbacks;
+
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
+import edu.uci.ics.asterix.common.transactions.ILockManager;
+import edu.uci.ics.asterix.common.transactions.ILogRecord;
+import edu.uci.ics.asterix.common.transactions.IRecoveryManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogRecord;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.common.tuples.SimpleTupleWriter;
+
+public abstract class AbstractIndexModificationOperationCallback extends AbstractOperationCallback {
+
+ protected final long resourceId;
+ protected final byte resourceType;
+ protected final IndexOperation indexOp;
+ protected final ITransactionSubsystem txnSubsystem;
+ protected final SimpleTupleWriter tupleWriter;
+ protected final ILogRecord logRecord;
+
+ protected AbstractIndexModificationOperationCallback(int datasetId, int[] primaryKeyFields,
+ ITransactionContext txnCtx, ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId,
+ byte resourceType, IndexOperation indexOp) {
+ super(datasetId, primaryKeyFields, txnCtx, lockManager);
+ this.resourceId = resourceId;
+ this.resourceType = resourceType;
+ this.indexOp = indexOp;
+ this.txnSubsystem = txnSubsystem;
+ tupleWriter = new SimpleTupleWriter();
+ logRecord = new LogRecord();
+ logRecord.setTxnCtx(txnCtx);
+ logRecord.setLogType(LogType.UPDATE);
+ logRecord.setJobId(txnCtx.getJobId().getId());
+ logRecord.setDatasetId(datasetId);
+ logRecord.setResourceId(resourceId);
+ logRecord.setResourceType(resourceType);
+ logRecord.setNewOp((byte) (indexOp.ordinal()));
+ }
+
+ protected void log(int PKHash, ITupleReference newValue, IndexOperation oldOp, ITupleReference oldValue)
+ throws ACIDException {
+ logRecord.setPKHashValue(PKHash);
+ logRecord.setPKFields(primaryKeyFields);
+ logRecord.setPKValue(newValue);
+ logRecord.computeAndSetPKValueSize();
+ if (newValue != null) {
+ logRecord.setNewValueSize(tupleWriter.bytesRequired(newValue));
+ logRecord.setNewValue(newValue);
+ } else {
+ logRecord.setNewValueSize(0);
+ }
+ if (resourceType == ResourceType.LSM_BTREE) {
+ logRecord.setOldOp((byte) (oldOp.ordinal()));
+ if (oldValue != null) {
+ logRecord.setOldValueSize(tupleWriter.bytesRequired(oldValue));
+ logRecord.setOldValue(oldValue);
+ } else {
+ logRecord.setOldValueSize(0);
+ }
+ }
+ logRecord.computeAndSetLogSize();
+ txnSubsystem.getLogManager().log(logRecord);
+ }
+}
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
index 0024717..44dcad8 100644
--- 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
@@ -16,12 +16,9 @@
package edu.uci.ics.asterix.transaction.management.opcallbacks;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
import edu.uci.ics.asterix.common.transactions.ILockManager;
-import edu.uci.ics.asterix.common.transactions.ILogger;
import edu.uci.ics.asterix.common.transactions.ITransactionContext;
import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
-import edu.uci.ics.asterix.transaction.management.service.logging.IndexLogger;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -33,22 +30,13 @@
* Assumes LSM-BTrees as primary indexes.
* Performs locking on primary keys, and also logs before/after images.
*/
-public class PrimaryIndexModificationOperationCallback extends AbstractOperationCallback implements
+public class PrimaryIndexModificationOperationCallback extends AbstractIndexModificationOperationCallback implements
IModificationOperationCallback {
- protected final long resourceId;
- protected final byte resourceType;
- protected final IndexOperation indexOp;
- protected final ITransactionSubsystem txnSubsystem;
-
public PrimaryIndexModificationOperationCallback(int datasetId, int[] primaryKeyFields, ITransactionContext txnCtx,
ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId, byte resourceType,
IndexOperation indexOp) {
- super(datasetId, primaryKeyFields, txnCtx, lockManager);
- this.resourceId = resourceId;
- this.resourceType = resourceType;
- this.indexOp = indexOp;
- this.txnSubsystem = txnSubsystem;
+ super(datasetId, primaryKeyFields, txnCtx, lockManager, txnSubsystem, resourceId, resourceType, indexOp);
}
@Override
@@ -64,7 +52,6 @@
@Override
public void found(ITupleReference before, ITupleReference after) throws HyracksDataException {
try {
- ILogger logger = txnSubsystem.getTreeLoggerRepository().getIndexLogger(resourceId, resourceType);
int pkHash = computePrimaryKeyHashValue(after, primaryKeyFields);
LSMBTreeTupleReference lsmBTreeTuple = (LSMBTreeTupleReference) before;
IndexOperation oldOp = IndexOperation.INSERT;
@@ -74,8 +61,7 @@
if (lsmBTreeTuple != null && lsmBTreeTuple.isAntimatter()) {
oldOp = IndexOperation.DELETE;
}
- ((IndexLogger) logger).generateLogRecord(txnSubsystem, txnCtx, datasetId.getId(), pkHash, resourceId,
- indexOp, after, oldOp, before);
+ log(pkHash, 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
index 428a04e..abeec62 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
@@ -62,7 +62,7 @@
IModificationOperationCallback modCallback = new PrimaryIndexModificationOperationCallback(datasetId,
primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId, resourceType,
indexOp);
- txnCtx.registerIndexAndCallback(index, (AbstractOperationCallback) modCallback);
+ txnCtx.registerIndexAndCallback(resourceId, index, (AbstractOperationCallback) modCallback, true);
return modCallback;
} 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
index e92f82c..ef5ad21 100644
--- 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
@@ -16,13 +16,10 @@
package edu.uci.ics.asterix.transaction.management.opcallbacks;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
import edu.uci.ics.asterix.common.transactions.ILockManager;
-import edu.uci.ics.asterix.common.transactions.ILogger;
-import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.IRecoveryManager.ResourceType;
import edu.uci.ics.asterix.common.transactions.ITransactionContext;
import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
-import edu.uci.ics.asterix.transaction.management.service.logging.IndexLogger;
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;
@@ -34,24 +31,16 @@
* 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
+public class SecondaryIndexModificationOperationCallback extends AbstractIndexModificationOperationCallback implements
IModificationOperationCallback {
- protected final long resourceId;
- protected final byte resourceType;
- protected final IndexOperation indexOp;
protected final IndexOperation oldOp;
- protected final ITransactionSubsystem txnSubsystem;
public SecondaryIndexModificationOperationCallback(int datasetId, int[] primaryKeyFields,
ITransactionContext txnCtx, ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId,
byte resourceType, IndexOperation indexOp) {
- super(datasetId, primaryKeyFields, txnCtx, lockManager);
- this.resourceId = resourceId;
- this.resourceType = resourceType;
- this.indexOp = indexOp;
+ super(datasetId, primaryKeyFields, txnCtx, lockManager, txnSubsystem, resourceId, resourceType, indexOp);
oldOp = (indexOp == IndexOperation.DELETE) ? IndexOperation.INSERT : IndexOperation.DELETE;
- this.txnSubsystem = txnSubsystem;
}
@Override
@@ -61,9 +50,7 @@
@Override
public void found(ITupleReference before, ITupleReference after) throws HyracksDataException {
-
try {
- ILogger logger = txnSubsystem.getTreeLoggerRepository().getIndexLogger(resourceId, resourceType);
int pkHash = computePrimaryKeyHashValue(after, primaryKeyFields);
IndexOperation effectiveOldOp;
if (resourceType == ResourceType.LSM_BTREE) {
@@ -78,8 +65,7 @@
} else {
effectiveOldOp = oldOp;
}
- ((IndexLogger) logger).generateLogRecord(txnSubsystem, txnCtx, datasetId.getId(), pkHash, resourceId,
- indexOp, after, effectiveOldOp, before);
+ this.log(pkHash, after, effectiveOldOp, 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
index 55ca399..d5bc877 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
@@ -17,6 +17,7 @@
import edu.uci.ics.asterix.common.context.ITransactionSubsystemProvider;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
import edu.uci.ics.asterix.common.transactions.AbstractOperationCallbackFactory;
import edu.uci.ics.asterix.common.transactions.ITransactionContext;
import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
@@ -55,8 +56,10 @@
try {
ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
- return new SecondaryIndexModificationOperationCallback(datasetId, primaryKeyFields, txnCtx,
+ IModificationOperationCallback modCallback = new SecondaryIndexModificationOperationCallback(datasetId, primaryKeyFields, txnCtx,
txnSubsystem.getLockManager(), txnSubsystem, resourceId, resourceType, indexOp);
+ txnCtx.registerIndexAndCallback(resourceId, index, (AbstractOperationCallback) modCallback, false);
+ return modCallback;
} catch (ACIDException e) {
throw new HyracksDataException(e);
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerProvider.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerProvider.java
index 3661537..aec378b 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerProvider.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerProvider.java
@@ -14,7 +14,9 @@
*/
package edu.uci.ics.asterix.transaction.management.opcallbacks;
+import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
import edu.uci.ics.asterix.common.context.BaseOperationTracker;
+import edu.uci.ics.asterix.common.context.DatasetLifecycleManager;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
@@ -24,15 +26,19 @@
private static final long serialVersionUID = 1L;
+ private final int datasetID;
private final ILSMIOOperationCallbackFactory ioOpCallbackFactory;
- public SecondaryIndexOperationTrackerProvider(ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
+ public SecondaryIndexOperationTrackerProvider(ILSMIOOperationCallbackFactory ioOpCallbackFactory, int datasetID) {
this.ioOpCallbackFactory = ioOpCallbackFactory;
+ this.datasetID = datasetID;
}
@Override
public ILSMOperationTracker getOperationTracker(IHyracksTaskContext ctx) {
- return new BaseOperationTracker(ioOpCallbackFactory);
+ DatasetLifecycleManager dslcManager = (DatasetLifecycleManager) ((IAsterixAppRuntimeContext) ctx
+ .getJobletContext().getApplicationContext().getApplicationObject()).getIndexLifecycleManager();
+ return new BaseOperationTracker(dslcManager, ioOpCallbackFactory, datasetID);
}
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java
index 8ce0174..d243dd2 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java
@@ -15,8 +15,10 @@
package edu.uci.ics.asterix.transaction.management.resource;
import java.io.File;
+import java.util.List;
import edu.uci.ics.asterix.common.context.BaseOperationTracker;
+import edu.uci.ics.asterix.common.context.DatasetLifecycleManager;
import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -49,12 +51,13 @@
public ILSMIndex createIndexInstance(IAsterixAppRuntimeContextProvider runtimeContextProvider, String filePath,
int partition) {
FileReference file = new FileReference(new File(filePath));
- IVirtualBufferCache virtualBufferCache = runtimeContextProvider.getVirtualBufferCache(datasetID);
- LSMBTree lsmBTree = LSMBTreeUtils.createLSMTree(virtualBufferCache, file, runtimeContextProvider
+ List<IVirtualBufferCache> virtualBufferCaches = runtimeContextProvider.getVirtualBufferCaches(datasetID);
+ LSMBTree lsmBTree = LSMBTreeUtils.createLSMTree(virtualBufferCaches, file, runtimeContextProvider
.getBufferCache(), runtimeContextProvider.getFileMapManager(), typeTraits, cmpFactories,
bloomFilterKeyFields, runtimeContextProvider.getBloomFilterFalsePositiveRate(), runtimeContextProvider
.getLSMMergePolicy(), isPrimary ? runtimeContextProvider.getLSMBTreeOperationTracker(datasetID)
- : new BaseOperationTracker(LSMBTreeIOOperationCallbackFactory.INSTANCE), runtimeContextProvider
+ : new BaseOperationTracker((DatasetLifecycleManager) runtimeContextProvider.getIndexLifecycleManager(),
+ LSMBTreeIOOperationCallbackFactory.INSTANCE, datasetID), runtimeContextProvider
.getLSMIOScheduler(), runtimeContextProvider.getLSMBTreeIOOperationCallbackProvider(isPrimary));
return lsmBTree;
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java
index b3da3ee..8482172 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java
@@ -14,7 +14,10 @@
*/
package edu.uci.ics.asterix.transaction.management.resource;
+import java.util.List;
+
import edu.uci.ics.asterix.common.context.BaseOperationTracker;
+import edu.uci.ics.asterix.common.context.DatasetLifecycleManager;
import edu.uci.ics.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -53,24 +56,26 @@
@Override
public ILSMIndex createIndexInstance(IAsterixAppRuntimeContextProvider runtimeContextProvider, String filePath,
int partition) throws HyracksDataException {
- IVirtualBufferCache virtualBufferCache = runtimeContextProvider.getVirtualBufferCache(datasetID);
+ List<IVirtualBufferCache> virtualBufferCaches = runtimeContextProvider.getVirtualBufferCaches(datasetID);
try {
if (isPartitioned) {
- return InvertedIndexUtils.createPartitionedLSMInvertedIndex(virtualBufferCache, runtimeContextProvider
+ return InvertedIndexUtils.createPartitionedLSMInvertedIndex(virtualBufferCaches, runtimeContextProvider
.getFileMapManager(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
tokenCmpFactories, tokenizerFactory, runtimeContextProvider.getBufferCache(), filePath,
runtimeContextProvider.getBloomFilterFalsePositiveRate(), runtimeContextProvider
.getLSMMergePolicy(), new BaseOperationTracker(
- LSMInvertedIndexIOOperationCallbackFactory.INSTANCE), runtimeContextProvider
+ (DatasetLifecycleManager) runtimeContextProvider.getIndexLifecycleManager(),
+ LSMInvertedIndexIOOperationCallbackFactory.INSTANCE, datasetID), runtimeContextProvider
.getLSMIOScheduler(), runtimeContextProvider
.getLSMInvertedIndexIOOperationCallbackProvider());
} else {
- return InvertedIndexUtils.createLSMInvertedIndex(virtualBufferCache, runtimeContextProvider
+ return InvertedIndexUtils.createLSMInvertedIndex(virtualBufferCaches, runtimeContextProvider
.getFileMapManager(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
tokenCmpFactories, tokenizerFactory, runtimeContextProvider.getBufferCache(), filePath,
runtimeContextProvider.getBloomFilterFalsePositiveRate(), runtimeContextProvider
.getLSMMergePolicy(), new BaseOperationTracker(
- LSMInvertedIndexIOOperationCallbackFactory.INSTANCE), runtimeContextProvider
+ (DatasetLifecycleManager) runtimeContextProvider.getIndexLifecycleManager(),
+ LSMInvertedIndexIOOperationCallbackFactory.INSTANCE, datasetID), runtimeContextProvider
.getLSMIOScheduler(), runtimeContextProvider
.getLSMInvertedIndexIOOperationCallbackProvider());
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java
index 9b9faef..bc1e889 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java
@@ -15,8 +15,10 @@
package edu.uci.ics.asterix.transaction.management.resource;
import java.io.File;
+import java.util.List;
import edu.uci.ics.asterix.common.context.BaseOperationTracker;
+import edu.uci.ics.asterix.common.context.DatasetLifecycleManager;
import edu.uci.ics.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -58,14 +60,16 @@
public ILSMIndex createIndexInstance(IAsterixAppRuntimeContextProvider runtimeContextProvider, String filePath,
int partition) throws HyracksDataException {
FileReference file = new FileReference(new File(filePath));
- IVirtualBufferCache virtualBufferCache = runtimeContextProvider.getVirtualBufferCache(datasetID);
+ List<IVirtualBufferCache> virtualBufferCaches = runtimeContextProvider.getVirtualBufferCaches(datasetID);
try {
- return LSMRTreeUtils.createLSMTree(virtualBufferCache, file, runtimeContextProvider.getBufferCache(),
+ return LSMRTreeUtils.createLSMTree(virtualBufferCaches, file, runtimeContextProvider.getBufferCache(),
runtimeContextProvider.getFileMapManager(), typeTraits, rtreeCmpFactories, btreeCmpFactories,
valueProviderFactories, rtreePolicyType, runtimeContextProvider.getBloomFilterFalsePositiveRate(),
runtimeContextProvider.getLSMMergePolicy(), new BaseOperationTracker(
- LSMRTreeIOOperationCallbackFactory.INSTANCE), runtimeContextProvider.getLSMIOScheduler(),
- runtimeContextProvider.getLSMRTreeIOOperationCallbackProvider(), linearizeCmpFactory);
+ (DatasetLifecycleManager) runtimeContextProvider.getIndexLifecycleManager(),
+ LSMRTreeIOOperationCallbackFactory.INSTANCE, datasetID), runtimeContextProvider
+ .getLSMIOScheduler(), runtimeContextProvider.getLSMRTreeIOOperationCallbackProvider(),
+ linearizeCmpFactory);
} catch (TreeIndexException e) {
throw new HyracksDataException(e);
}
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 b09cd52..6d86f70 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
@@ -30,10 +30,11 @@
import edu.uci.ics.asterix.common.transactions.DatasetId;
import edu.uci.ics.asterix.common.transactions.ILockManager;
import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.ITransactionManager.TransactionState;
+import edu.uci.ics.asterix.common.transactions.ITransactionManager;
import edu.uci.ics.asterix.common.transactions.JobId;
-import edu.uci.ics.asterix.common.transactions.LogUtil;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogPage;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogPageReader;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogRecord;
import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
@@ -78,38 +79,34 @@
private DeadlockDetector deadlockDetector;
private TimeOutDetector toutDetector;
private DatasetId tempDatasetIdObj; //temporary object to avoid object creation
+ private JobId tempJobIdObj;
private int tryLockDatasetGranuleRevertOperation;
private LockRequestTracker lockRequestTracker; //for debugging
private ConsecutiveWakeupContext consecutiveWakeupContext;
- //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(txnSubsystem.getTransactionProperties().getLockManagerShrinkTimer());
+ this.entityInfoManager = new EntityInfoManager(txnSubsystem.getTransactionProperties()
+ .getLockManagerShrinkTimer());
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.tempJobIdObj = new JobId(0);
this.consecutiveWakeupContext = new ConsecutiveWakeupContext();
-
- this.logicalLogLocator = LogUtil.getDummyLogicalLogLocator(txnSubsystem.getLogManager());
-
if (IS_DEBUG_MODE) {
this.lockRequestTracker = new LockRequestTracker();
}
}
-
+
public AsterixTransactionProperties getTransactionProperties() {
return this.txnSubsystem.getTransactionProperties();
}
@@ -200,7 +197,8 @@
if (doEscalate) {
throw new IllegalStateException(
"ESCALATE_TRHESHOLD_ENTITY_TO_DATASET should not be set to "
- + txnSubsystem.getTransactionProperties().getEntityToDatasetLockEscalationThreshold());
+ + txnSubsystem.getTransactionProperties()
+ .getEntityToDatasetLockEscalationThreshold());
}
}
}
@@ -304,9 +302,9 @@
}
private void validateJob(ITransactionContext txnContext) throws ACIDException {
- if (txnContext.getTxnState() == TransactionState.ABORTED) {
+ if (txnContext.getTxnState() == ITransactionManager.ABORTED) {
throw new ACIDException("" + txnContext.getJobId() + " is in ABORTED state.");
- } else if (txnContext.getStatus() == ITransactionContext.TIMED_OUT_STATUS) {
+ } else if (txnContext.isTimeout()) {
requestAbort(txnContext);
}
}
@@ -640,24 +638,17 @@
}
@Override
- public boolean unlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext)
- throws ACIDException {
- return internalUnlock(datasetId, entityHashValue, txnContext, false, false);
- }
-
- @Override
- public boolean unlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext, boolean commitFlag)
- throws ACIDException {
- return internalUnlock(datasetId, entityHashValue, txnContext, false, commitFlag);
+ public void unlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext) throws ACIDException {
+ internalUnlock(datasetId, entityHashValue, txnContext, false);
}
private void instantUnlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext)
throws ACIDException {
- internalUnlock(datasetId, entityHashValue, txnContext, true, false);
+ internalUnlock(datasetId, entityHashValue, txnContext, true);
}
- private boolean internalUnlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext,
- boolean isInstant, boolean commitFlag) throws ACIDException {
+ private void internalUnlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext,
+ boolean isInstant) throws ACIDException {
JobId jobId = txnContext.getJobId();
int eLockInfo = -1;
DatasetLockInfo dLockInfo = null;
@@ -665,7 +656,6 @@
int entityInfo = -1;
byte datasetLockMode;
- boolean lockCountIsZero = false;
if (IS_DEBUG_MODE) {
if (entityHashValue == -1) {
throw new UnsupportedOperationException(
@@ -713,28 +703,11 @@
if (entityInfoManager.getEntityLockCount(entityInfo) == 0
&& entityInfoManager.getDatasetLockCount(entityInfo) == 0) {
- lockCountIsZero = true;
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(ITransactionContext.TransactionType.READ_WRITE)) {
- try {
- txnSubsystem.getLogManager().log(LogType.ENTITY_COMMIT, txnContext, datasetId.getId(),
- entityHashValue, -1, (byte) 0, 0, null, null, logicalLogLocator);
- } catch (ACIDException e) {
- requestAbort(txnContext);
- }
- }
-
- txnContext.updateLastLSNForIndexes(logicalLogLocator.getLsn());
- }
-
//1) wake up waiters and remove holder
//wake up waiters of dataset-granule lock
wakeUpDatasetLockWaiters(dLockInfo);
@@ -760,11 +733,6 @@
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);
}
}
@@ -782,7 +750,8 @@
if (ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
if (!isInstant && datasetLockMode == LockMode.IS) {
- jobInfo.decreaseDatasetISLockCount(datasetId.getId(), txnSubsystem.getTransactionProperties().getEntityToDatasetLockEscalationThreshold());
+ jobInfo.decreaseDatasetISLockCount(datasetId.getId(), txnSubsystem.getTransactionProperties()
+ .getEntityToDatasetLockEscalationThreshold());
}
}
@@ -793,7 +762,6 @@
} finally {
unlatchLockTable();
}
- return lockCountIsZero;
}
@Override
@@ -974,8 +942,8 @@
jobHT.remove(jobId);
if (existWaiter) {
- txnContext.setStatus(ITransactionContext.TIMED_OUT_STATUS);
- txnContext.setTxnState(TransactionState.ABORTED);
+ txnContext.setTimeout(true);
+ txnContext.setTxnState(ITransactionManager.ABORTED);
}
if (IS_DEBUG_MODE) {
@@ -1297,7 +1265,8 @@
//We don't want to allow the lock escalation when there is a first lock request on a dataset.
throw new IllegalStateException(
"ESCALATE_TRHESHOLD_ENTITY_TO_DATASET should not be set to "
- + txnSubsystem.getTransactionProperties().getEntityToDatasetLockEscalationThreshold());
+ + txnSubsystem.getTransactionProperties()
+ .getEntityToDatasetLockEscalationThreshold());
}
}
}
@@ -1824,7 +1793,7 @@
//waiter woke up -> remove/deallocate waiter object and abort if timeout
latchLockTable();
- if (txnContext.getStatus() == ITransactionContext.TIMED_OUT_STATUS || waiter.isVictim()) {
+ if (txnContext.isTimeout() || waiter.isVictim()) {
requestAbort(txnContext);
}
@@ -1881,8 +1850,7 @@
}
private void requestAbort(ITransactionContext txnContext) throws ACIDException {
- txnContext.setStatus(ITransactionContext.TIMED_OUT_STATUS);
- txnContext.setStartWaitTime(ITransactionContext.INVALID_TIME);
+ txnContext.setTimeout(true);
throw new ACIDException("Transaction " + txnContext.getJobId()
+ " should abort (requested by the Lock Manager)");
}
@@ -2097,7 +2065,8 @@
try {
StringBuilder sb = new StringBuilder();
sb.append("\n>>dump_begin\t>>----- [ConfVars] -----");
- sb.append("\nESCALATE_TRHESHOLD_ENTITY_TO_DATASET: " + txnSubsystem.getTransactionProperties().getEntityToDatasetLockEscalationThreshold());
+ sb.append("\nESCALATE_TRHESHOLD_ENTITY_TO_DATASET: "
+ + txnSubsystem.getTransactionProperties().getEntityToDatasetLockEscalationThreshold());
sb.append("\nSHRINK_TIMER_THRESHOLD (entityLockInfoManager): "
+ entityLockInfoManager.getShrinkTimerThreshold());
sb.append("\nSHRINK_TIMER_THRESHOLD (entityInfoManager): " + entityInfoManager.getShrinkTimerThreshold());
@@ -2233,6 +2202,31 @@
}
}
}
+
+ public void batchUnlock(LogPage logPage, LogPageReader logPageReader) throws ACIDException {
+ latchLockTable();
+ try {
+ ITransactionContext txnCtx = null;
+ LogRecord logRecord = logPageReader.next();
+ while (logRecord != null) {
+ if (logRecord.getLogType() == LogType.ENTITY_COMMIT) {
+ tempDatasetIdObj.setId(logRecord.getDatasetId());
+ tempJobIdObj.setId(logRecord.getJobId());
+ txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(tempJobIdObj);
+ unlock(tempDatasetIdObj, logRecord.getPKHashValue(), txnCtx);
+ txnCtx.notifyOptracker(false);
+ } else if (logRecord.getLogType() == LogType.JOB_COMMIT) {
+ tempJobIdObj.setId(logRecord.getJobId());
+ txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(tempJobIdObj);
+ txnCtx.notifyOptracker(true);
+ ((LogPage) logPage).notifyJobCommitter();
+ }
+ logRecord = logPageReader.next();
+ }
+ } finally {
+ unlatchLockTable();
+ }
+ }
}
class ConsecutiveWakeupContext {
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
index e97c177..e61cb55 100644
--- 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
@@ -14,6 +14,7 @@
*/
package edu.uci.ics.asterix.transaction.management.service.locking;
+import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.util.ArrayList;
@@ -21,17 +22,15 @@
import java.util.NoSuchElementException;
import java.util.Scanner;
-import edu.uci.ics.asterix.common.config.AsterixCompilerProperties;
-import edu.uci.ics.asterix.common.config.AsterixExternalProperties;
-import edu.uci.ics.asterix.common.config.AsterixMetadataProperties;
+import org.apache.commons.io.FileUtils;
+
import edu.uci.ics.asterix.common.config.AsterixPropertiesAccessor;
-import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
import edu.uci.ics.asterix.common.config.AsterixTransactionProperties;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.common.transactions.DatasetId;
import edu.uci.ics.asterix.common.transactions.ILockManager;
-import edu.uci.ics.asterix.common.transactions.ITransactionManager.TransactionState;
+import edu.uci.ics.asterix.common.transactions.ITransactionManager;
import edu.uci.ics.asterix.common.transactions.JobId;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
@@ -40,6 +39,13 @@
public class LockManagerDeterministicUnitTest {
public static void main(String args[]) throws ACIDException, IOException, AsterixException {
+ //prepare configuration file
+ File cwd = new File(System.getProperty("user.dir"));
+ File asterixdbDir = cwd.getParentFile();
+ File srcFile = new File(asterixdbDir.getAbsoluteFile(), "asterix-app/src/main/resources/asterix-build-configuration.xml");
+ File destFile = new File(cwd, "target/classes/asterix-configuration.xml");
+ FileUtils.copyFile(srcFile, destFile);
+
//initialize controller thread
String requestFileName = new String(
"src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockRequestFile");
@@ -61,8 +67,8 @@
long defaultWaitTime;
public LockRequestController(String requestFileName) throws ACIDException, AsterixException {
- this.txnProvider = new TransactionSubsystem("LockManagerPredefinedUnitTest", null,
- new AsterixTransactionProperties(new AsterixPropertiesAccessor()));
+ this.txnProvider = new TransactionSubsystem("nc1", null, new AsterixTransactionProperties(
+ new AsterixPropertiesAccessor()));
this.workerReadyQueue = new WorkerReadyQueue();
this.requestList = new ArrayList<LockRequest>();
this.expectedResultList = new ArrayList<ArrayList<Integer>>();
@@ -428,9 +434,9 @@
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);
+ if (lockRequest.txnContext.isTimeout()) {
+ if (lockRequest.txnContext.getTxnState() != ITransactionManager.ABORTED) {
+ lockRequest.txnContext.setTxnState(ITransactionManager.ABORTED);
log("*** " + lockRequest.txnContext.getJobId() + " lock request causing deadlock ***");
log("Abort --> Releasing all locks acquired by " + lockRequest.txnContext.getJobId());
try {
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
index f73460d..e6f2798 100644
--- 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
@@ -24,7 +24,7 @@
import edu.uci.ics.asterix.common.transactions.DatasetId;
import edu.uci.ics.asterix.common.transactions.ILockManager;
import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.ITransactionManager.TransactionState;
+import edu.uci.ics.asterix.common.transactions.ITransactionManager;
import edu.uci.ics.asterix.common.transactions.JobId;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
@@ -287,8 +287,8 @@
} else {
try {
synchronized (txnContext) {
- if (txnContext.getTxnState() != TransactionState.ABORTED) {
- txnContext.setTxnState(TransactionState.ABORTED);
+ if (txnContext.getTxnState() != ITransactionManager.ABORTED) {
+ txnContext.setTxnState(ITransactionManager.ABORTED);
mayRelease = true;
}
}
@@ -446,9 +446,9 @@
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);
+ if (request.txnContext.isTimeout()) {
+ if (request.txnContext.getTxnState() != ITransactionManager.ABORTED) {
+ request.txnContext.setTxnState(ITransactionManager.ABORTED);
log("*** " + request.txnContext.getJobId() + " lock request causing deadlock ***");
log("Abort --> Releasing all locks acquired by " + request.txnContext.getJobId());
try {
@@ -469,9 +469,9 @@
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);
+ if (request.txnContext.isTimeout()) {
+ if (request.txnContext.getTxnState() != ITransactionManager.ABORTED) {
+ request.txnContext.setTxnState(ITransactionManager.ABORTED);
log("*** " + request.txnContext.getJobId() + " lock request causing deadlock ***");
log("Abort --> Releasing all locks acquired by " + request.txnContext.getJobId());
try {
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
index fc2a883..c755895 100644
--- 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
@@ -1,20 +1,11 @@
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
+T3 L J3 D1 E1 S
+T2 L J2 D1 E1 X
+T0 CSQ 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
+T3 UL J3 D1 E1 S
+T0 CST 1 2 3 -1
+T2 RL J2 D1 E1 X
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
+T3 END
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/Buffer.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/Buffer.java
deleted file mode 100644
index 272ef5e..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/Buffer.java
+++ /dev/null
@@ -1,145 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.service.logging;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-
-import edu.uci.ics.asterix.common.transactions.IBuffer;
-
-/**
- * Represents a wrapper over @see ByteBuffer supporting some custom APIs for
- * transaction support. This class is not "thread-safe". For performance
- * concerns, it is required for multiple writers to be able to write to the
- * buffer concurrently and that a writer is never blocked by another writer. The
- * users of this class must ensure that two concurrent writers get to write in
- * exclusive areas in the buffer. A reader and writer may or may not conflict
- * with each other. For example, reading of logs during roll back of a
- * transaction t1 does not conflict with writing of logs by another transaction
- * t2 as they are concerned with exclusive areas of the buffer. On the contrary,
- * a flushing the buffer to disk conflicts with a reader reading the buffer.
- * Appropriate locks are taken on the Buffer in the application logic and not
- * directly imposed by synchronized methods.
- */
-
-public class Buffer implements IBuffer {
-
- ByteBuffer buffer;
-
- public Buffer(ByteBuffer buffer) {
- this.buffer = buffer;
- }
-
- protected Buffer() {
- }
-
- public void setBuffer(ByteBuffer buffer) {
- this.buffer = buffer;
- }
-
- @Override
- public byte getByte(int offset) {
- return buffer.get(offset);
- }
-
- @Override
- public byte getByte() {
- return buffer.get();
- }
-
- @Override
- public void getBytes(byte[] bytes, int offset, int size) {
- System.arraycopy(buffer.array(), offset, bytes, 0, size);
- }
-
- @Override
- public int getSize() {
- return buffer.capacity();
- }
-
- @Override
- public int readInt() {
- return buffer.getInt();
- }
-
- @Override
- public int readInt(int offset) {
- return buffer.getInt(offset);
- }
-
- @Override
- public long readLong(int offset) {
- return buffer.getLong(offset);
- }
-
- @Override
- public void put(byte b) {
- buffer.put(b);
- }
-
- @Override
- public void put(int offset, byte b) {
- buffer.put(offset, b);
- }
-
- @Override
- public void put(byte[] bytes, int start, int length) {
- buffer.put(bytes, start, length);
-
- }
-
- @Override
- public void put(byte[] bytes) {
- buffer.put(bytes);
- }
-
- @Override
- public void writeInt(int value) {
- buffer.putInt(value);
- }
-
- @Override
- public void writeInt(int offset, int value) {
- buffer.putInt(offset, value);
-
- }
-
- @Override
- public void writeLong(long value) {
- buffer.putLong(value);
- }
-
- @Override
- public void writeLong(int offset, long value) {
- buffer.putLong(offset, value);
-
- }
-
- @Override
- public byte[] getArray() {
- return buffer.array();
- }
-
- @Override
- public void erase() {
- Arrays.fill(buffer.array(), (byte) 0);
- }
-
- @Override
- public ByteBuffer getByteBuffer() {
- return buffer;
- }
-
-}
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
deleted file mode 100644
index 79c95c4..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/DataUtil.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.service.logging;
-
-import java.util.zip.CRC32;
-
-import edu.uci.ics.asterix.common.transactions.IBuffer;
-
-/**
- * A utility class for doing bit level operations such as forming checksum or
- * converting between Integer and byte array. Used extensively during writing
- * and reading of logs.
- */
-public class DataUtil {
-
- public static long getChecksum(IBuffer buffer, int offset, int length) {
- CRC32 checksumEngine = new CRC32();
- byte[] bytes = new byte[1];
- for (int i = 0; i < length; i++) {
- bytes[0] = buffer.getByte(offset + i);
- checksumEngine.update(bytes, 0, 1);
- }
- return checksumEngine.getValue();
- }
-
- public static int byteArrayToInt(byte[] bytes, int offset) {
- return ((bytes[offset] & 0xff) << 24) + ((bytes[offset + 1] & 0xff) << 16) + ((bytes[offset + 2] & 0xff) << 8)
- + ((bytes[offset + 3] & 0xff) << 0);
- }
-
- public static byte[] intToByteArray(int value) {
- byte[] bytes = new byte[4];
- bytes[0] = (byte) ((value >>> 24) & 0xFF);
- bytes[1] = (byte) ((value >>> 16) & 0xFF);
- bytes[2] = (byte) ((value >>> 8) & 0xFF);
- bytes[3] = (byte) ((value >>> 0) & 0xFF);
- 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/GenericFilter.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/GenericFilter.java
deleted file mode 100644
index 0de43cc..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/GenericFilter.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.service.logging;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import edu.uci.ics.asterix.common.transactions.IBuffer;
-import edu.uci.ics.asterix.common.transactions.ILogFilter;
-
-/*
- * A generic filter that encompasses one or more filters (see @ILogFiler) that may be applied when selectively retrieving logs.
- * The contained filters are assumed to form a conjunction.
- */
-public class GenericFilter implements ILogFilter {
-
- private final List<ILogFilter> logFilters;
-
- public GenericFilter() {
- logFilters = new ArrayList<ILogFilter>();
- }
-
- public GenericFilter(List<ILogFilter> logFilters) {
- this.logFilters = logFilters;
- }
-
- public boolean accept(IBuffer fileBuffer, long offset, int length) {
- boolean satisfies = true;
- for (ILogFilter logFilter : logFilters) {
- satisfies = satisfies && logFilter.accept(fileBuffer, offset, length);
- if (!satisfies) {
- break;
- }
- }
- return satisfies;
- }
-
- public void addFilter(ILogFilter logFilter) {
- logFilters.add(logFilter);
- }
-
- public boolean removeFilter(ILogFilter logFilter) {
- return logFilters.remove(logFilter);
- }
-}
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
deleted file mode 100644
index 755e85b..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLogger.java
+++ /dev/null
@@ -1,214 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.service.logging;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.ICloseable;
-import edu.uci.ics.asterix.common.transactions.ILogger;
-import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
-import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
-import edu.uci.ics.asterix.common.transactions.LogUtil;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.common.transactions.ReusableLogContentObject;
-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(ITransactionContext context) {
- ReusableLogContentObjectRepository txnThreadStateRepository = (ReusableLogContentObjectRepository) jobId2ReusableLogContentObjectRepositoryMap
- .get(context.getJobId());
- txnThreadStateRepository.remove(Thread.currentThread().getId());
- jobId2ReusableLogContentObjectRepositoryMap.remove(context.getJobId());
- }
-
- public void generateLogRecord(ITransactionSubsystem txnSubsystem, ITransactionContext 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(ITransactionContext 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.getOldValue(), logicalLogLocator.getBuffer()
- .getArray(), logicalLogLocator.getMemoryOffset() + offset);
- }
- }
- }
- }
-
- @Override
- public void postLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject)
- throws ACIDException {
- }
-
- @Override
- public void preLog(ITransactionContext 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);
- }
- }
-
-}
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
deleted file mode 100644
index b95b943..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLoggerRepository.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.service.logging;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.ILogger;
-import edu.uci.ics.asterix.common.transactions.ILoggerRepository;
-import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
-import edu.uci.ics.asterix.common.transactions.MutableResourceId;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
-
-public class IndexLoggerRepository implements ILoggerRepository {
-
- private final Map<MutableResourceId, ILogger> loggers = new HashMap<MutableResourceId, ILogger>();
- private final ITransactionSubsystem txnSubsystem;
- private MutableResourceId mutableResourceId;
-
- public IndexLoggerRepository(ITransactionSubsystem provider) {
- this.txnSubsystem = provider;
- mutableResourceId = new MutableResourceId(0);
- }
-
- @Override
- public synchronized ILogger getIndexLogger(long resourceId, byte resourceType) throws ACIDException {
- mutableResourceId.setId(resourceId);
- ILogger logger = loggers.get(mutableResourceId);
- if (logger == null) {
- MutableResourceId newMutableResourceId = new MutableResourceId(resourceId);
- IIndex index;
- try {
- index = (IIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
- .getIndex(resourceId);
- } catch (HyracksDataException e) {
- throw new ACIDException(e);
- }
- 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
deleted file mode 100644
index 16ffa69..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexResourceManager.java
+++ /dev/null
@@ -1,162 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.service.logging;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
-import edu.uci.ics.asterix.common.transactions.IResourceManager;
-import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-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 ITransactionSubsystem txnSubsystem;
-
- public IndexResourceManager(byte resourceType, ITransactionSubsystem provider) {
- this.resourceType = resourceType;
- this.txnSubsystem = 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);
-
- IIndex index;
- try {
- index = (IIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
- .getIndex(resourceId);
- } catch (HyracksDataException e1) {
- throw new ACIDException("Cannot undo: unable to find index");
- }
-
- /* field count */
- int fieldCount = logLocator.getBuffer().readInt(offset);
- offset += 4;
-
- /* new operation */
- byte newOperation = logLocator.getBuffer().getByte(offset);
- offset += 1;
-
- /* new value size */
- int newValueSize = logLocator.getBuffer().readInt(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;
-
- try {
- ILSMIndexAccessor indexAccessor = (ILSMIndexAccessor) index.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
-
- if (resourceType == ResourceType.LSM_BTREE) {
-
- /* old operation */
- byte oldOperation = logLocator.getBuffer().getByte(offset);
- offset += 1;
-
- if (oldOperation != (byte) IndexOperation.NOOP.ordinal()) {
- /* old value size */
- int oldValueSize = logLocator.getBuffer().readInt(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.forceDelete(oldTuple);
- } else {
- indexAccessor.forceInsert(oldTuple);
- }
- } else {
- indexAccessor.forcePhysicalDelete(newTuple);
- }
- } else {
- if (newOperation == (byte) IndexOperation.DELETE.ordinal()) {
- indexAccessor.forceInsert(newTuple);
- } else {
- indexAccessor.forceDelete(newTuple);
- }
- }
- } catch (Exception e) {
- throw new ACIDException("Undo failed", e);
- }
- }
-
- public void redo(ILogRecordHelper logRecordHelper, LogicalLogLocator logLocator) throws ACIDException {
- long resourceId = logRecordHelper.getResourceId(logLocator);
- int offset = logRecordHelper.getLogContentBeginPos(logLocator);
-
- IIndex index;
- try {
- index = (IIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
- .getIndex(resourceId);
- } catch (HyracksDataException e1) {
- throw new ACIDException("Cannot redo: unable to find index");
- }
-
- /* field count */
- int fieldCount = logLocator.getBuffer().readInt(offset);
- offset += 4;
-
- /* new operation */
- byte newOperation = logLocator.getBuffer().getByte(offset);
- offset += 1;
-
- /* new value size */
- int newValueSize = logLocator.getBuffer().readInt(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;
-
- try {
- ILSMIndexAccessor indexAccessor = (ILSMIndexAccessor) index.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
-
- if (newOperation == IndexOperation.INSERT.ordinal()) {
- indexAccessor.insert(newTuple);
- } else if (newOperation == IndexOperation.DELETE.ordinal()) {
- indexAccessor.delete(newTuple);
- } else {
- new ACIDException("Unsupported operation type for undo operation : " + newOperation);
- }
- } catch (Exception e) {
- throw new ACIDException("Redo failed", e);
- }
- }
-
-}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogActionType.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogActionType.java
deleted file mode 100644
index 5f558f3..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogActionType.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.service.logging;
-
-public class LogActionType {
-
- public static final byte REDO = 0; // used for a log record that contains
- // just redo information.
- public static final byte REDO_UNDO = 1; // used for a log record that
- // contains both redo and undo
- // information.
- public static final byte UNDO = 2; // used for a log record that contains
- // just undo information.
- public static final byte NO_OP = 3; // used for a log record that does not
- // require any action.
-
-}
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
deleted file mode 100644
index d3e1ad1..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogCursor.java
+++ /dev/null
@@ -1,268 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.service.logging;
-
-import java.io.File;
-import java.io.IOException;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.FileUtil;
-import edu.uci.ics.asterix.common.transactions.IBuffer;
-import edu.uci.ics.asterix.common.transactions.IFileBasedBuffer;
-import edu.uci.ics.asterix.common.transactions.ILogCursor;
-import edu.uci.ics.asterix.common.transactions.ILogFilter;
-import edu.uci.ics.asterix.common.transactions.LogManagerProperties;
-import edu.uci.ics.asterix.common.transactions.LogUtil;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
-
-public class LogCursor implements ILogCursor {
-
- private final LogManager logManager;
- private final ILogFilter logFilter;
- private final int logPageSize;
- private IBuffer readOnlyBuffer;
- private LogicalLogLocator logicalLogLocator = null;
- private boolean needReloadBuffer = true;
-
- public LogCursor(final LogManager logManager, PhysicalLogLocator startingPhysicalLogLocator, ILogFilter logFilter,
- int logPageSize) throws IOException, ACIDException {
- this.logFilter = logFilter;
- this.logManager = logManager;
- this.logPageSize = logPageSize;
- initialize(startingPhysicalLogLocator);
- }
-
- 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 {
- int fileId = (int) (lsn / logManager.getLogManagerProperties().getLogPartitionSize());
- String filePath = LogUtil.getLogFilePath(logManager.getLogManagerProperties(), fileId);
- File file = new File(filePath);
- if (file.exists()) {
- return FileUtil.getFileBasedBuffer(filePath, lsn
- % logManager.getLogManagerProperties().getLogPartitionSize(), size, logManager
- .getLogManagerProperties().getDiskSectorSize());
- } else {
- return null;
- }
- }
-
- /**
- * Moves the cursor to the next log record that satisfies the configured
- * filter. The parameter nextLogLocator is set to the point to the next log
- * record.
- *
- * @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 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 or equal to the most recent lsn, then return false
- if (logicalLogLocator.getLsn() >= logManager.getCurrentLsn().get()) {
- return false;
- }
-
- if (logManager.isMemoryRead(logicalLogLocator.getLsn())) {
- 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) {
- //log page size doesn't exceed integer boundary
- int offset = (int) (logicalLogLocator.getLsn() % logPageSize);
- long adjustedLSN = logicalLogLocator.getLsn() - offset;
- readOnlyBuffer = getReadOnlyBuffer(adjustedLSN, logPageSize);
- logicalLogLocator.setBuffer(readOnlyBuffer);
- logicalLogLocator.setMemoryOffset(offset);
- 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.getLogRecordHelper().getLogHeaderSize(LogType.COMMIT)) {
- integerRead = readOnlyBuffer.readInt(logicalLogLocator.getMemoryOffset());
- if (integerRead == LogManagerProperties.LOG_MAGIC_NUMBER) {
- 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 (logManager.isMemoryRead(logicalLogLocator.getLsn())) {
- return next(currentLogLocator); //should read from memory if there is any further log
- }
- }
-
- if (!logRecordBeginPosFound) {
- // need to reload the buffer
- // TODO
- // reduce IO by reading more pages(equal to logBufferSize) at a time.
- long lsnpos = ((logicalLogLocator.getLsn() / logPageSize) + 1) * logPageSize;
-
- readOnlyBuffer = getReadOnlyBuffer(lsnpos, logPageSize);
- if (readOnlyBuffer != null) {
- logicalLogLocator.setBuffer(readOnlyBuffer);
- logicalLogLocator.setLsn(lsnpos);
- logicalLogLocator.setMemoryOffset(0);
- return next(currentLogLocator);
- } else {
- return false;
- }
- }
-
- 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);
- }
- 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, currentLogLocator.getMemoryOffset(), logLength);
- }
-
- /**
- * Returns the filter associated with the cursor.
- *
- * @return ILogFilter
- */
- @Override
- public ILogFilter getLogFilter() {
- 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);
- 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 (logManager.isMemoryRead(lsn)) {
-
- //find the magic number to identify the start of the log record
- //----------------------------------------------------------------
- int readNumber = -1;
- int logMagicNumber = LogManagerProperties.LOG_MAGIC_NUMBER;
- 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);
- lsn = ((logicalLogLocator.getLsn() / logPageSize) + 1) * logPageSize;
- logicalLogLocator.setLsn(lsn);
- 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 5b10144..4f0bb59 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
@@ -15,746 +15,181 @@
package edu.uci.ics.asterix.transaction.management.service.logging;
import java.io.File;
+import java.io.FilenameFilter;
import java.io.IOException;
import java.io.OutputStream;
import java.io.RandomAccessFile;
-import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
import java.util.ArrayList;
-import java.util.HashMap;
+import java.util.Collections;
+import java.util.Comparator;
import java.util.List;
-import java.util.Map;
-import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicBoolean;
import java.util.logging.Level;
import java.util.logging.Logger;
import edu.uci.ics.asterix.common.api.AsterixThreadExecutor;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.FileBasedBuffer;
-import edu.uci.ics.asterix.common.transactions.FileUtil;
-import edu.uci.ics.asterix.common.transactions.IFileBasedBuffer;
-import edu.uci.ics.asterix.common.transactions.ILogCursor;
-import edu.uci.ics.asterix.common.transactions.ILogFilter;
import edu.uci.ics.asterix.common.transactions.ILogManager;
-import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
-import edu.uci.ics.asterix.common.transactions.ILogger;
+import edu.uci.ics.asterix.common.transactions.ILogReader;
+import edu.uci.ics.asterix.common.transactions.ILogRecord;
import edu.uci.ics.asterix.common.transactions.ITransactionContext;
import edu.uci.ics.asterix.common.transactions.LogManagerProperties;
-import edu.uci.ics.asterix.common.transactions.LogUtil;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
-import edu.uci.ics.asterix.common.transactions.ReusableLogContentObject;
+import edu.uci.ics.asterix.common.transactions.MutableLong;
+import edu.uci.ics.asterix.transaction.management.service.locking.LockManager;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.lifecycle.ILifeCycleComponent;
public class LogManager implements ILogManager, ILifeCycleComponent {
- public static final boolean IS_DEBUG_MODE = false;//true
+ public static final boolean IS_DEBUG_MODE = false;// true
private static final Logger LOGGER = Logger.getLogger(LogManager.class.getName());
- private final TransactionSubsystem provider;
- private LogManagerProperties logManagerProperties;
- private LogPageFlushThread logPageFlusher;
+ private final TransactionSubsystem txnSubsystem;
+ private final LogManagerProperties logManagerProperties;
+ private final long logFileSize;
private final int logPageSize;
- private long statLogSize;
- private long statLogCount;
+ private final int numLogPages;
+ private final String logDir;
+ private final String logFilePrefix;
+ private final MutableLong flushLSN;
+ private LinkedBlockingQueue<LogPage> emptyQ;
+ private LinkedBlockingQueue<LogPage> flushQ;
+ private long appendLSN;
+ private FileChannel appendChannel;
+ private LogPage appendPage;
+ private LogFlusher logFlusher;
+ private Future<Object> futureLogFlusher;
- /*
- * the array of log pages. The number of log pages is configurable. Pages
- * taken together form an in-memory log buffer.
- */
- private IFileBasedBuffer[] logPages;
-
- private ILogRecordHelper logRecordHelper;
-
- /*
- * Number of log pages that constitute the in-memory log buffer.
- */
- private int numLogPages;
-
- private AtomicLong lastFlushedLSN = new AtomicLong(-1);
-
- /*
- * When the transaction eco-system comes to life, the log manager positions
- * itself to the end of the last written log. the startingLsn represent the
- * lsn value of the next log record to be written after a system (re)start.
- * The value is zero when the system is starting for the first time.
- */
- private long startingLSN = 0;
-
- /*
- * lsn represents the monotonically increasing long value that can be broken
- * down into a file id and an offset within a log file.
- */
- private AtomicLong lsn = new AtomicLong(0);
-
- private List<HashMap<ITransactionContext, Integer>> activeTxnCountMaps;
-
- public void addFlushRequest(int pageIndex, long lsn, boolean isSynchronous) {
- logPageFlusher.requestFlush(pageIndex, lsn, isSynchronous);
- }
-
- public AtomicLong getLastFlushedLsn() {
- return lastFlushedLSN;
- }
-
- public AtomicLong getCurrentLsn() {
- return lsn;
- }
-
- public long incrementLastFlushedLsn(long delta) {
- return lastFlushedLSN.addAndGet(delta);
- }
-
- public LogManager(TransactionSubsystem provider) throws ACIDException {
- this.provider = provider;
- logManagerProperties = new LogManagerProperties(this.provider.getTransactionProperties(), this.provider.getId());
+ public LogManager(TransactionSubsystem txnSubsystem) throws ACIDException {
+ this.txnSubsystem = txnSubsystem;
+ logManagerProperties = new LogManagerProperties(this.txnSubsystem.getTransactionProperties(),
+ this.txnSubsystem.getId());
+ logFileSize = logManagerProperties.getLogPartitionSize();
logPageSize = logManagerProperties.getLogPageSize();
- initLogManager();
- statLogSize = 0;
- statLogCount = 0;
- }
-
- public LogManager(TransactionSubsystem provider, String nodeId) throws ACIDException {
- this.provider = provider;
- logManagerProperties = new LogManagerProperties(provider.getTransactionProperties(), nodeId);
- logPageSize = logManagerProperties.getLogPageSize();
- initLogManager();
- statLogSize = 0;
- statLogCount = 0;
- }
-
- private void initLogManager() throws ACIDException {
- logRecordHelper = new LogRecordHelper(this);
numLogPages = logManagerProperties.getNumLogPages();
- activeTxnCountMaps = new ArrayList<HashMap<ITransactionContext, Integer>>(numLogPages);
+ logDir = logManagerProperties.getLogDir();
+ logFilePrefix = logManagerProperties.getLogFilePrefix();
+ flushLSN = new MutableLong();
+ initializeLogManager();
+ }
+ private void initializeLogManager() {
+ emptyQ = new LinkedBlockingQueue<LogPage>(numLogPages);
+ flushQ = new LinkedBlockingQueue<LogPage>(numLogPages);
for (int i = 0; i < numLogPages; i++) {
- activeTxnCountMaps.add(new HashMap<ITransactionContext, Integer>());
+ emptyQ.offer(new LogPage((LockManager) txnSubsystem.getLockManager(), logPageSize, flushLSN));
}
-
- logPages = new FileBasedBuffer[numLogPages];
-
- /*
- * place the log anchor at the end of the last log record written.
- */
- initLSN();
-
- /*
- * initialize the log pages.
- */
- initializeLogPages(startingLSN);
-
- /*
- * Instantiate and begin the LogFlusher thread. The Log Flusher thread
- * is responsible for putting log pages to disk. It is configured as a
- * daemon thread so that it does not stop the JVM from exiting when all
- * other threads are done with their work.
- */
- logPageFlusher = new LogPageFlushThread(this);
- logPageFlusher.setDaemon(true);
- AsterixThreadExecutor.INSTANCE.execute(logPageFlusher);
- }
-
- public int getLogPageIndex(long lsnValue) {
- return (int) (((lsnValue - startingLSN) / logPageSize) % numLogPages);
- }
-
- /*
- * given a lsn, get the file id where the corresponding log record is
- * located.
- */
- public int getLogFileId(long lsnValue) {
- return (int) ((lsnValue) / logManagerProperties.getLogPartitionSize());
- }
-
- /*
- * given a lsn, get the offset within a log page where the corresponding log
- * record is (to be) placed.
- */
- public int getLogPageOffset(long lsnValue) {
- return (int) (lsnValue % logPageSize);
- }
-
- /*
- * The method that reserves the space for a transaction to write log record
- * in the log buffer. Note that the method is not synchronized for
- * performance reasons as we do not want transactions to be blocked by each
- * other when writing log records.
- *
- * @param entrySize: the requested space.
- *
- * @param logType: the type of log record.
- */
- private long getLsn(int entrySize, byte logType) throws ACIDException {
-
- while (true) {
- boolean forwardPage = false;
- long old = lsn.get();
-
- // get the log page corresponding to the current lsn value
- int pageIndex = getLogPageIndex(old);
- long retVal = old;
-
- // the lsn value for the next request if the current request is
- // served.
- long next = old + entrySize;
- int prevPage = -1;
-
- // check if the log record will cross page boundaries, a case that
- // is not allowed.
- if ((next - 1) / logPageSize != old / logPageSize || (next % logPageSize == 0)) {
-
- if ((old != 0 && old % logPageSize == 0)) {
- // On second thought, this shall never be the case as it
- // means that the lsn is
- // currently at the beginning of a page and we still need to
- // forward the page which
- // means that the entrySize exceeds a log page size. If this
- // is the case, an
- // exception is thrown before calling this API. would remove
- // this case.
- retVal = old;
-
- } else {
- // set the lsn to point to the beginning of the next page.
- retVal = ((old / logPageSize) + 1) * logPageSize;
- }
-
- next = retVal;
-
- // as the log record shall cross log page boundary, we must
- // re-assign the lsn so
- // that the log record begins on a different location.
- forwardPage = true;
-
- prevPage = pageIndex;
- pageIndex = getNextPageInSequence(pageIndex);
- }
-
- if (!lsn.compareAndSet(old, next)) {
- // Atomic call -> returns true only when the value represented
- // by lsn is same as
- // "old". The value is updated to "next".
- continue;
- }
-
- if (forwardPage) {
- logPages[prevPage].acquireReadLatch();
- // increment the counter as the transaction thread now holds a
- // space in the log page and hence is an owner.
- logPages[prevPage].incRefCnt();
- logPages[prevPage].releaseReadLatch();
-
- // forward the nextWriteOffset in the log page
- logPages[prevPage].setBufferNextWriteOffset(logPageSize);
-
- logPages[prevPage].decRefCnt();
-
- addFlushRequest(prevPage, old, false);
-
- // The transaction thread that discovers the need to forward a
- // page is made to re-acquire a lsn.
- continue;
-
- } else {
- logPages[pageIndex].acquireReadLatch();
- // increment the counter as the transaction thread now holds a
- // space in the log page and hence is an owner.
- logPages[pageIndex].incRefCnt();
- logPages[pageIndex].releaseReadLatch();
-
- // Before the count is incremented, if the flusher flushed the
- // allocated page,
- // then retry to get new LSN. Otherwise, the log with allocated
- // lsn will be lost.
- if (lastFlushedLSN.get() >= retVal) {
- logPages[pageIndex].decRefCnt();
- continue;
- }
- }
-
- return retVal;
- }
- }
-
- @Override
- public void log(byte logType, ITransactionContext txnCtx, int datasetId, int PKHashValue, long resourceId,
- byte resourceMgrId, int logContentSize, ReusableLogContentObject reusableLogContentObject, ILogger logger,
- LogicalLogLocator logicalLogLocator) throws ACIDException {
-
- HashMap<ITransactionContext, Integer> map = null;
- int activeTxnCount;
-
- // 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 (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 = logRecordHelper.getLogRecordSize(logType, logContentSize);
-
- // check for the total space requirement to be less than a log page.
- if (totalLogSize > logPageSize) {
- throw new ACIDException(
- " Maximum Log Content Size is "
- + (logPageSize - logRecordHelper.getLogHeaderSize(LogType.UPDATE) - logRecordHelper
- .getLogChecksumSize()));
- }
-
- // all constraints checked and we are good to go and acquire a lsn.
- long previousLSN = -1;
-
- // the will be set to the location (a long value) where the log record
- // needs to be placed.
- long currentLSN;
-
- // The logs written by a transaction need to be linked to each other for
- // a successful rollback/recovery. However there could be multiple
- // threads operating concurrently that are part of a common transaction.
- // These threads need to synchronize and record the lsn corresponding to
- // the last log record written by (any thread of) the transaction.
- synchronized (txnCtx) {
- previousLSN = txnCtx.getLastLogLocator().getLsn();
- currentLSN = getLsn(totalLogSize, logType);
- txnCtx.setLastLSN(currentLSN);
- if (IS_DEBUG_MODE) {
- System.out.println("--------------> LSN(" + currentLSN + ") is allocated");
- }
- logicalLogLocator.setLsn(currentLSN);
- }
-
- /*
- * At this point, the transaction thread has obtained reserved space for
- * writing the log record. In doing so, it has acquired (shared)
- * ownership of the log page. All subsequent actions are under a try
- * catch block so that if any exception is encountered, a clean can be
- * performed correctly that is ownership is released.
- */
-
- // indicates if the transaction thread has release ownership of the
- // page.
- boolean decremented = false;
-
- int pageIndex = (int) getLogPageIndex(currentLSN);
-
- // the lsn has been obtained for the log record. need to set the
- // LogLocator instance accordingly.
- try {
- logicalLogLocator.setBuffer(logPages[pageIndex]);
- int pageOffset = getLogPageOffset(currentLSN);
- logicalLogLocator.setMemoryOffset(pageOffset);
-
- // write the log header.
- logRecordHelper.writeLogHeader(logicalLogLocator, logType, txnCtx, 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.
- 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 (logContentSize != 0) {
- logger.preLog(txnCtx, reusableLogContentObject);
- }
-
- if (logContentSize != 0) {
- // call the logger implementation and ask to fill in the log
- // record content at the allocated space.
- logger.log(txnCtx, logicalLogLocator, logContentSize, reusableLogContentObject);
- logger.postLog(txnCtx, 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 = logicalLogLocator.getMemoryOffset() - logRecordHelper.getLogHeaderSize(logType);
- int length = totalLogSize - logRecordHelper.getLogChecksumSize();
- long checksum = DataUtil.getChecksum(logPages[pageIndex], startPosChecksum, length);
- logPages[pageIndex].writeLong(pageOffset + logRecordHelper.getLogHeaderSize(logType) + logContentSize,
- checksum);
-
- // forward the nextWriteOffset in the log page
- int bufferNextWriteOffset = (int) ((currentLSN + totalLogSize) % logPageSize);
- if (bufferNextWriteOffset == 0) {
- bufferNextWriteOffset = logPageSize;
- }
- logPages[pageIndex].setBufferNextWriteOffset(bufferNextWriteOffset);
-
- if (logType != LogType.ENTITY_COMMIT) {
- if (logType == LogType.COMMIT) {
- txnCtx.setExclusiveJobLevelCommit();
- map = activeTxnCountMaps.get(pageIndex);
- map.put(txnCtx, 1);
- }
- // release the ownership as the log record has been placed in
- // created space.
- logPages[pageIndex].decRefCnt();
-
- // indicating that the transaction thread has released ownership
- decremented = true;
- }
-
- if (logType == LogType.ENTITY_COMMIT) {
- map = activeTxnCountMaps.get(pageIndex);
- if (map.containsKey(txnCtx)) {
- activeTxnCount = (Integer) map.get(txnCtx);
- activeTxnCount++;
- map.put(txnCtx, activeTxnCount);
- } else {
- map.put(txnCtx, 1);
- }
- //------------------------------------------------------------------------------
- // [Notice]
- // reference count should be decremented
- // after activeTxnCount is incremented, but before addFlushRequest() is called.
- //------------------------------------------------------------------------------
- // release the ownership as the log record has been placed in
- // created space.
- logPages[pageIndex].decRefCnt();
-
- // indicating that the transaction thread has released ownership
- decremented = true;
-
- addFlushRequest(pageIndex, currentLSN, false);
- } else if (logType == LogType.COMMIT) {
-
- addFlushRequest(pageIndex, currentLSN, true);
- if (IS_DEBUG_MODE) {
- System.out.println("Running sum of log size: " + statLogSize + ", log count: " + statLogCount);
- }
- }
-
- if (IS_DEBUG_MODE) {
- System.out.println("--------------> LSN(" + currentLSN + ") is written");
- }
-
- //collect statistics
- statLogSize += totalLogSize;
- statLogCount++;
-
- } catch (Exception e) {
- e.printStackTrace();
- throw new ACIDException(txnCtx, "Thread: " + Thread.currentThread().getName()
- + " logger encountered exception", e);
- } finally {
- if (!decremented) {
- logPages[pageIndex].decRefCnt();
- }
- }
- }
-
- /*
- * This method resets the log page and is called by the log flusher thread
- * after a page has been flushed to disk.
- */
- public void resetLogPage(long lsn, long nextWritePosition, int pageIndex) throws IOException {
-
- String filePath = LogUtil.getLogFilePath(logManagerProperties, getLogFileId(lsn));
-
- logPages[pageIndex].reset(filePath, LogUtil.getFileOffset(this, nextWritePosition), logPageSize);
- }
-
- @Override
- public ILogCursor readLog(PhysicalLogLocator physicalLogLocator, ILogFilter logFilter) throws IOException,
- ACIDException {
- LogCursor cursor = new LogCursor(this, physicalLogLocator, logFilter, logPageSize);
- return cursor;
- }
-
- /*
- * Read a log that is residing on the disk.
- */
- 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(logPageSize);
- RandomAccessFile raf = null;
- FileChannel fileChannel = null;
- try {
- raf = new RandomAccessFile(filePath, "r");
- fileChannel = raf.getChannel();
- fileChannel.position(fileOffset);
- fileChannel.read(buffer);
- buffer.position(0);
-
- 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());
- 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) {
- fnfe.printStackTrace();
- throw new ACIDException(" unable to retrieve log record with lsn " + lsnValue + " from the file system",
- fnfe);
- } finally {
- try {
- if (fileChannel != null) {
- fileChannel.close();
- } else if (raf != null) {
- raf.close();
- }
- } catch (IOException ioe) {
- ioe.printStackTrace();
- throw new ACIDException(" exception in closing a file: " + filePath, ioe);
- }
- }
- }
-
- @Override
- public void readLog(long lsnValue, LogicalLogLocator logicalLogLocator) throws ACIDException {
- byte[] logRecord = null;
-
- if (lsnValue >= lsn.get()) {
- throw new ACIDException(" invalid lsn " + lsnValue);
- }
-
- /* check if the log record in the log buffer or has reached the disk. */
- if (isMemoryRead(lsnValue)) {
- 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[logPageSize];
-
- // take a lock on the log page so that the page is not flushed to
- // disk interim
- synchronized (logPages[pageIndex]) {
-
- // need to check again (this thread may have got de-scheduled
- // and must refresh!)
-
- if (isMemoryRead(lsnValue)) {
- // get the log record length
- logPages[pageIndex].getBytes(pageContent, 0, pageContent.length);
- 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, logRecordSize);
- MemBasedBuffer memBuffer = new MemBasedBuffer(logRecord);
- 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(logicalLogLocator)) {
- throw new ACIDException(" invalid log record at lsn " + lsnValue);
- }
- } catch (Exception e) {
- throw new ACIDException("exception encoutered in validating log record at lsn " + lsnValue, e);
- }
- return;
- }
- }
- }
-
- // the log record is residing on the disk, read it from there.
- readDiskLog(lsnValue, logicalLogLocator);
- }
-
- public boolean isMemoryRead(long readLSN) {
- long flushLSN = lastFlushedLSN.get();
- if ((flushLSN + 1) == readLSN) {
- return false;
- }
- long logPageBeginOffset = flushLSN - (flushLSN % logPageSize);
- long logPageEndOffset = logPageBeginOffset + logPageSize;
- if (readLSN > flushLSN || (readLSN >= logPageBeginOffset && readLSN < logPageEndOffset)) {
- return true;
- } else {
- return false;
- }
- }
-
- public void renewLogFiles() throws ACIDException {
- List<String> logFileNames = LogUtil.getLogFiles(logManagerProperties);
- for (String name : logFileNames) {
- File file = new File(LogUtil.getLogFilePath(logManagerProperties, Long.parseLong(name)));
- if (!file.delete()) {
- throw new ACIDException("Failed to delete a file: " + name);
- }
- }
- closeLogPages();
- initLSN();
- openLogPages();
- logPageFlusher.renew();
- }
-
- private void initLSN() throws ACIDException {
- PhysicalLogLocator nextPhysicalLsn = LogUtil.initializeLogAnchor(this);
- startingLSN = nextPhysicalLsn.getLsn();
- lastFlushedLSN.set(startingLSN - 1);
+ appendLSN = initializeLogAnchor();
+ flushLSN.set(appendLSN);
if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(" Starting lsn is : " + startingLSN);
+ LOGGER.info("LogManager starts logging in LSN: " + appendLSN);
}
- lsn.set(startingLSN);
- }
-
- private void closeLogPages() throws ACIDException {
- for (int i = 0; i < numLogPages; i++) {
- try {
- logPages[i].close();
- } catch (IOException e) {
- throw new ACIDException(e);
- }
- }
- }
-
- private void openLogPages() throws ACIDException {
- try {
- String filePath = LogUtil.getLogFilePath(logManagerProperties, LogUtil.getFileId(this, startingLSN));
- for (int i = 0; i < numLogPages; i++) {
- logPages[i].open(filePath, LogUtil.getFileOffset(this, startingLSN) + i * logPageSize, logPageSize);
- }
- } catch (Exception e) {
- throw new ACIDException(Thread.currentThread().getName() + " unable to create log buffer", e);
- }
+ appendChannel = getFileChannel(appendLSN, false);
+ getAndInitNewPage();
+ logFlusher = new LogFlusher(this, emptyQ, flushQ);
+ futureLogFlusher = AsterixThreadExecutor.INSTANCE.submit(logFlusher);
}
@Override
- public ILogRecordHelper getLogRecordHelper() {
- return logRecordHelper;
- }
-
- /*
- * Map each log page to cover a physical byte range over a log file. When a
- * page is flushed, the page contents are put to disk in the corresponding
- * byte range.
- */
- private void initializeLogPages(long beginLsn) throws ACIDException {
- try {
- String filePath = LogUtil.getLogFilePath(logManagerProperties, LogUtil.getFileId(this, beginLsn));
- long nextDiskWriteOffset = LogUtil.getFileOffset(this, beginLsn);
- long nextBufferWriteOffset = nextDiskWriteOffset % logPageSize;
- long bufferBeginOffset = nextDiskWriteOffset - nextBufferWriteOffset;
-
- for (int i = 0; i < numLogPages; i++) {
- logPages[i] = FileUtil.getFileBasedBuffer(filePath, bufferBeginOffset + i * logPageSize, logPageSize,
- logManagerProperties.getDiskSectorSize());
- if (i == 0) {
- logPages[i].setBufferLastFlushOffset((int) nextBufferWriteOffset);
- logPages[i].setBufferNextWriteOffset((int) nextBufferWriteOffset);
- logPages[i].setDiskNextWriteOffset(nextDiskWriteOffset);
+ public void log(ILogRecord logRecord) {
+ if (logRecord.getLogSize() > logPageSize) {
+ throw new IllegalStateException();
+ }
+ syncLog(logRecord);
+ if (logRecord.getLogType() == LogType.JOB_COMMIT && !logRecord.isFlushed()) {
+ synchronized (logRecord) {
+ while (!logRecord.isFlushed()) {
+ try {
+ logRecord.wait();
+ } catch (InterruptedException e) {
+ //ignore
+ }
}
}
- } catch (Exception e) {
- e.printStackTrace();
- throw new ACIDException(Thread.currentThread().getName() + " unable to create log buffer", e);
}
}
- /*
- * Pages are sequenced starting with 0 going upto numLogPages-1.
- */
- public int getNextPageInSequence(int pageNo) {
- return (pageNo + 1) % numLogPages;
+ private synchronized void syncLog(ILogRecord logRecord) {
+ ITransactionContext txnCtx = logRecord.getTxnCtx();
+ if (getLogFileOffset(appendLSN) + logRecord.getLogSize() > logFileSize) {
+ prepareNextLogFile();
+ appendPage.isFull(true);
+ getAndInitNewPage();
+ } else if (!appendPage.hasSpace(logRecord.getLogSize(), getLogFileOffset(appendLSN))) {
+ appendPage.isFull(true);
+ getAndInitNewPage();
+ }
+ if (logRecord.getLogType() == LogType.UPDATE) {
+ logRecord.setPrevLSN(txnCtx.getLastLSN());
+ }
+ appendPage.append(logRecord, appendLSN);
+ appendLSN += logRecord.getLogSize();
}
- public int getPreviousPageInSequence(int pageNo) {
- return pageNo == 0 ? numLogPages - 1 : pageNo - 1;
+ private void getAndInitNewPage() {
+ appendPage = null;
+ while (appendPage == null) {
+ try {
+ appendPage = emptyQ.take();
+ } catch (InterruptedException e) {
+ //ignore
+ }
+ }
+ appendPage.reset();
+ appendPage.setFileChannel(appendChannel);
+ appendPage.setInitialFlushOffset(getLogFileOffset(appendLSN));
+ flushQ.offer(appendPage);
+ }
+
+ private void prepareNextLogFile() {
+ appendLSN += logFileSize - getLogFileOffset(appendLSN);
+ appendChannel = getFileChannel(appendLSN, true);
+ appendPage.isLastPage(true);
+ //[Notice]
+ //the current log file channel is closed if
+ //LogPage.flush() completely flush the last page of the file.
}
@Override
+ public ILogReader getLogReader(boolean isRecoveryMode) {
+ return new LogReader(this, logFileSize, logPageSize, flushLSN, isRecoveryMode);
+ }
+
public LogManagerProperties getLogManagerProperties() {
return logManagerProperties;
}
- public IFileBasedBuffer getLogPage(int pageIndex) {
- return logPages[pageIndex];
- }
-
- public IFileBasedBuffer[] getLogPages() {
- return logPages;
- }
-
- @Override
public TransactionSubsystem getTransactionSubsystem() {
- return provider;
+ return txnSubsystem;
}
- static AtomicInteger t = new AtomicInteger();
-
- public void decrementActiveTxnCountOnIndexes(int pageIndex) throws HyracksDataException {
- ITransactionContext ctx = null;
- int count = 0;
- int i = 0;
-
- HashMap<ITransactionContext, Integer> map = activeTxnCountMaps.get(pageIndex);
- Set<Map.Entry<ITransactionContext, Integer>> entrySet = map.entrySet();
- if (entrySet != null) {
- for (Map.Entry<ITransactionContext, Integer> entry : entrySet) {
- if (entry != null) {
- if (entry.getValue() != null) {
- count = entry.getValue();
- }
- if (count > 0) {
- ctx = entry.getKey();
- for (i = 0; i < count; i++) {
- ctx.decreaseActiveTransactionCountOnIndexes();
- }
- }
- }
- }
- }
-
- map.clear();
+ public long getAppendLSN() {
+ return appendLSN;
}
@Override
public void start() {
- //no op
+ // no op
}
@Override
public void stop(boolean dumpState, OutputStream os) {
+ terminateLogFlusher();
if (dumpState) {
- //#. dump Configurable Variables
+ // #. dump Configurable Variables
dumpConfVars(os);
- //#. dump LSNInfo
+ // #. dump LSNInfo
dumpLSNInfo(os);
try {
os.flush();
} catch (IOException e) {
- //ignore
+ // ignore
}
}
}
@@ -767,7 +202,7 @@
sb.append("\n>>dump_end\t>>----- [ConfVars] -----\n");
os.write(sb.toString().getBytes());
} catch (Exception e) {
- //ignore exception and continue dumping as much as possible.
+ // ignore exception and continue dumping as much as possible.
if (IS_DEBUG_MODE) {
e.printStackTrace();
}
@@ -778,191 +213,243 @@
try {
StringBuilder sb = new StringBuilder();
sb.append("\n>>dump_begin\t>>----- [LSNInfo] -----");
- sb.append("\nstartingLSN: " + startingLSN);
- sb.append("\ncurrentLSN: " + lsn.get());
- sb.append("\nlastFlushedLSN: " + lastFlushedLSN.get());
+ sb.append("\nappendLsn: " + appendLSN);
+ sb.append("\nflushLsn: " + flushLSN.get());
sb.append("\n>>dump_end\t>>----- [LSNInfo] -----\n");
os.write(sb.toString().getBytes());
} catch (Exception e) {
- //ignore exception and continue dumping as much as possible.
+ // ignore exception and continue dumping as much as possible.
if (IS_DEBUG_MODE) {
e.printStackTrace();
}
}
}
+
+ public MutableLong getFlushLSN() {
+ return flushLSN;
+ }
+
+ private long initializeLogAnchor() {
+ long fileId = 0;
+ long offset = 0;
+ File fileLogDir = new File(logDir);
+ try {
+ if (fileLogDir.exists()) {
+ List<Long> logFileIds = getLogFileIds();
+ if (logFileIds == null) {
+ createFileIfNotExists(getLogFilePath(0));
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("created a log file: " + getLogFilePath(0));
+ }
+ } else {
+ fileId = logFileIds.get(logFileIds.size() - 1);
+ File logFile = new File(getLogFilePath(fileId));
+ offset = logFile.length();
+ }
+ } else {
+ createNewDirectory(logDir);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("created the log directory: " + logManagerProperties.getLogDir());
+ }
+ createFileIfNotExists(getLogFilePath(0));
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("created a log file: " + getLogFilePath(0));
+ }
+ }
+ } catch (IOException ioe) {
+ throw new IllegalStateException("Failed to initialize the log anchor", ioe);
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("log file Id: " + fileId + ", offset: " + offset);
+ }
+ return logFileSize * fileId + offset;
+ }
+
+ public void renewLogFiles() {
+ terminateLogFlusher();
+ deleteAllLogFiles();
+ initializeLogManager();
+ }
+
+ private void terminateLogFlusher() {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Terminating LogFlusher thread ...");
+ }
+ logFlusher.terminate();
+ try {
+ futureLogFlusher.get();
+ } catch (ExecutionException | InterruptedException e) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("---------- warning(begin): LogFlusher thread is terminated abnormally --------");
+ e.printStackTrace();
+ LOGGER.info("---------- warning(end) : LogFlusher thread is terminated abnormally --------");
+ }
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("LogFlusher thread is terminated.");
+ }
+ }
+
+ private void deleteAllLogFiles() {
+ if (appendChannel != null) {
+ try {
+ appendChannel.close();
+ } catch (IOException e) {
+ throw new IllegalStateException("Failed to close a fileChannel of a log file");
+ }
+ }
+ List<Long> logFileIds = getLogFileIds();
+ for (Long id : logFileIds) {
+ File file = new File(getLogFilePath(id));
+ if (!file.delete()) {
+ throw new IllegalStateException("Failed to delete a file: " + file.getAbsolutePath());
+ }
+ }
+ }
+
+ private List<Long> getLogFileIds() {
+ File fileLogDir = new File(logDir);
+ String[] logFileNames = null;
+ List<Long> logFileIds = null;
+ if (fileLogDir.exists()) {
+ logFileNames = fileLogDir.list(new FilenameFilter() {
+ public boolean accept(File dir, String name) {
+ if (name.startsWith(logFilePrefix)) {
+ return true;
+ }
+ return false;
+ }
+ });
+ if (logFileNames != null && logFileNames.length != 0) {
+ logFileIds = new ArrayList<Long>();
+ for (String fileName : logFileNames) {
+ logFileIds.add(Long.parseLong(fileName.substring(logFilePrefix.length() + 1)));
+ }
+ Collections.sort(logFileIds, new Comparator<Long>() {
+ @Override
+ public int compare(Long arg0, Long arg1) {
+ return arg0.compareTo(arg1);
+ }
+ });
+ }
+ }
+ return logFileIds;
+ }
+
+ public String getLogFilePath(long fileId) {
+ return logDir + File.separator + logFilePrefix + "_" + fileId;
+ }
+
+ public long getLogFileOffset(long lsn) {
+ return lsn % logFileSize;
+ }
+
+ public long getLogFileId(long lsn) {
+ return lsn / logFileSize;
+ }
+
+ private boolean createFileIfNotExists(String path) throws IOException {
+ File file = new File(path);
+ File parentFile = file.getParentFile();
+ if (parentFile != null) {
+ parentFile.mkdirs();
+ }
+ return file.createNewFile();
+ }
+
+ private boolean createNewDirectory(String path) throws IOException {
+ return (new File(path)).mkdir();
+ }
+
+ public FileChannel getFileChannel(long lsn, boolean create) {
+ FileChannel newFileChannel = null;
+ try {
+ long fileId = getLogFileId(lsn);
+ String logFilePath = getLogFilePath(fileId);
+ File file = new File(logFilePath);
+ if (create) {
+ if (!file.createNewFile()) {
+ throw new IllegalStateException();
+ }
+ } else {
+ if (!file.exists()) {
+ throw new IllegalStateException();
+ }
+ }
+ RandomAccessFile raf = new RandomAccessFile(new File(logFilePath), "rw");
+ newFileChannel = raf.getChannel();
+ newFileChannel.position(getLogFileOffset(lsn));
+ } catch (IOException e) {
+ throw new IllegalStateException(e);
+ }
+ return newFileChannel;
+ }
}
-/*
- * The thread responsible for putting log pages to disk in an ordered manner.
- * The Log Flusher updates the bookkeeping data internal to the log manager and
- * acquires appropriate locks. It also acquires finer level locks on the log
- * page when it is in process of flushing the content to disk.
- */
-class LogPageFlushThread extends Thread {
+class LogFlusher implements Callable<Boolean> {
+ private final static LogPage POISON_PILL = new LogPage(null, ILogRecord.JOB_COMMIT_LOG_SIZE, null);
+ private final LogManager logMgr;//for debugging
+ private final LinkedBlockingQueue<LogPage> emptyQ;
+ private final LinkedBlockingQueue<LogPage> flushQ;
+ private LogPage flushPage;
+ private final AtomicBoolean isStarted;
+ private final AtomicBoolean terminateFlag;
- private LogManager logManager;
- /*
- * pendingFlushRequests is a map with key as Integer denoting the page
- * index. When a (transaction) thread discovers the need to flush a page, it
- * puts its Thread object into the corresponding value that is a
- * LinkedBlockingQueue. The LogManager has a LogFlusher thread that scans
- * this map in order of page index (and circling around). The flusher thread
- * needs to flush pages in order and waits for a thread to deposit an object
- * in the blocking queue corresponding to the next page in order. A request
- * to flush a page is conveyed to the flush thread by simply depositing an
- * object in to corresponding blocking queue. It is blocking in the sense
- * that the flusher thread will continue to wait for an object to arrive in
- * the queue. The object itself is ignored by the fliusher and just acts as
- * a signal/event that a page needs to be flushed.
- */
- private final LinkedBlockingQueue<Object>[] flushRequestQueue;
- private final Object[] flushRequests;
- private int flushPageIndex;
- private final long groupCommitWaitPeriod;
- private boolean isRenewRequest;
-
- public LogPageFlushThread(LogManager logManager) {
- this.logManager = logManager;
- setName("Flusher");
- int numLogPages = logManager.getLogManagerProperties().getNumLogPages();
- this.flushRequestQueue = new LinkedBlockingQueue[numLogPages];
- this.flushRequests = new Object[numLogPages];
- for (int i = 0; i < numLogPages; i++) {
- flushRequestQueue[i] = new LinkedBlockingQueue<Object>(1);
- flushRequests[i] = new Object();
- }
- this.flushPageIndex = 0;
- groupCommitWaitPeriod = logManager.getLogManagerProperties().getGroupCommitWaitPeriod();
- isRenewRequest = false;
+ public LogFlusher(LogManager logMgr, LinkedBlockingQueue<LogPage> emptyQ, LinkedBlockingQueue<LogPage> flushQ) {
+ this.logMgr = logMgr;
+ this.emptyQ = emptyQ;
+ this.flushQ = flushQ;
+ flushPage = null;
+ isStarted = new AtomicBoolean(false);
+ terminateFlag = new AtomicBoolean(false);
+
}
- public void renew() {
- isRenewRequest = true;
- flushPageIndex = 0;
- this.interrupt();
- isRenewRequest = false;
- }
-
- public void requestFlush(int pageIndex, long lsn, boolean isSynchronous) {
- synchronized (logManager.getLogPage(pageIndex)) {
- // return if flushedLSN >= lsn
- if (logManager.getLastFlushedLsn().get() >= lsn) {
- return;
- }
-
- // put a new request to the queue only if the request on the page is
- // not in the queue.
- flushRequestQueue[pageIndex].offer(flushRequests[pageIndex]);
-
- // return if the request is asynchronous
- if (!isSynchronous) {
- return;
- }
-
- // wait until there is flush.
- boolean isNotified = false;
- while (!isNotified) {
+ public void terminate() {
+ //make sure the LogFlusher thread started before terminating it.
+ synchronized (isStarted) {
+ while(!isStarted.get()) {
try {
- logManager.getLogPage(pageIndex).wait();
- isNotified = true;
+ isStarted.wait();
} catch (InterruptedException e) {
- e.printStackTrace();
+ //ignore
}
}
}
+
+ terminateFlag.set(true);
+ if (flushPage != null) {
+ synchronized (flushPage) {
+ flushPage.isStop(true);
+ flushPage.notify();
+ }
+ }
+ //[Notice]
+ //The return value doesn't need to be checked
+ //since terminateFlag will trigger termination if the flushQ is full.
+ flushQ.offer(POISON_PILL);
}
@Override
- public void run() {
- int logPageSize = logManager.getLogManagerProperties().getLogPageSize();
- int logBufferSize = logManager.getLogManagerProperties().getLogBufferSize();
- int beforeFlushOffset = 0;
- int afterFlushOffset = 0;
- boolean resetFlushPageIndex = false;
-
+ public Boolean call() {
+ synchronized(isStarted) {
+ isStarted.set(true);
+ isStarted.notify();
+ }
while (true) {
+ flushPage = null;
try {
- // A wait call on the linkedBLockingQueue. The flusher thread is
- // notified when an object is added to the queue. Please note
- // that each page has an associated blocking queue.
- try {
- flushRequestQueue[flushPageIndex].take();
- } catch (InterruptedException ie) {
- while (isRenewRequest) {
- sleep(1);
- }
+ flushPage = flushQ.take();
+ if (flushPage == POISON_PILL || terminateFlag.get()) {
+ return true;
+ }
+ } catch (InterruptedException e) {
+ if (flushPage == null) {
continue;
}
-
- //if the log page is already full, don't wait.
- if (logManager.getLogPage(flushPageIndex).getBufferNextWriteOffset() < logPageSize
- - logManager.getLogRecordHelper().getCommitLogSize()) {
- // #. sleep for the groupCommitWaitTime
- sleep(groupCommitWaitPeriod);
- }
-
- synchronized (logManager.getLogPage(flushPageIndex)) {
- logManager.getLogPage(flushPageIndex).acquireWriteLatch();
- try {
-
- // #. need to wait until the reference count reaches 0
- while (logManager.getLogPage(flushPageIndex).getRefCnt() != 0) {
- sleep(0);
- }
-
- beforeFlushOffset = logManager.getLogPage(flushPageIndex).getBufferLastFlushOffset();
-
- // put the content to disk (the thread still has a lock on the log page)
- logManager.getLogPage(flushPageIndex).flush();
-
- afterFlushOffset = logManager.getLogPage(flushPageIndex).getBufferLastFlushOffset();
-
- // increment the last flushed lsn
- logManager.incrementLastFlushedLsn(afterFlushOffset - beforeFlushOffset);
-
- // increment currentLSN if currentLSN is less than flushLSN.
- if (logManager.getLastFlushedLsn().get() + 1 > logManager.getCurrentLsn().get()) {
- logManager.getCurrentLsn().set(logManager.getLastFlushedLsn().get() + 1);
- }
-
- // Map the log page to a new region in the log file if the flushOffset reached the logPageSize
- if (afterFlushOffset == logPageSize) {
- long diskNextWriteOffset = logManager.getLogPages()[flushPageIndex]
- .getDiskNextWriteOffset() + logBufferSize;
- logManager.resetLogPage(logManager.getLastFlushedLsn().get() + 1 + logBufferSize,
- diskNextWriteOffset, flushPageIndex);
- resetFlushPageIndex = true;
- }
-
- // decrement activeTxnCountOnIndexes
- logManager.decrementActiveTxnCountOnIndexes(flushPageIndex);
-
- } finally {
- logManager.getLogPage(flushPageIndex).releaseWriteLatch();
- }
-
- // #. checks the queue whether there is another flush
- // request on the same log buffer
- // If there is another request, then simply remove it.
- if (flushRequestQueue[flushPageIndex].peek() != null) {
- flushRequestQueue[flushPageIndex].take();
- }
-
- // notify all waiting (transaction) threads.
- logManager.getLogPage(flushPageIndex).notifyAll();
-
- if (resetFlushPageIndex) {
- flushPageIndex = logManager.getNextPageInSequence(flushPageIndex);
- resetFlushPageIndex = false;
- }
- }
- } catch (IOException ioe) {
- ioe.printStackTrace();
- throw new Error(" exception in flushing log page", ioe);
- } catch (InterruptedException e) {
- e.printStackTrace();
- break;
}
+ flushPage.flush();
+ emptyQ.offer(flushPage);
}
}
}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPage.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPage.java
new file mode 100644
index 0000000..edfec69
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPage.java
@@ -0,0 +1,217 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.transaction.management.service.logging;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.ILogPage;
+import edu.uci.ics.asterix.common.transactions.ILogRecord;
+import edu.uci.ics.asterix.common.transactions.MutableLong;
+import edu.uci.ics.asterix.transaction.management.service.locking.LockManager;
+
+public class LogPage implements ILogPage {
+
+ public static final boolean IS_DEBUG_MODE = false;//true
+ private static final Logger LOGGER = Logger.getLogger(LogPage.class.getName());
+ private final LockManager lockMgr;
+ private final LogPageReader logPageReader;
+ private final int logPageSize;
+ private final MutableLong flushLSN;
+ private final AtomicBoolean full;
+ private int appendOffset;
+ private int flushOffset;
+ private final ByteBuffer appendBuffer;
+ private final ByteBuffer flushBuffer;
+ private final ByteBuffer unlockBuffer;
+ private boolean isLastPage;
+ private final LinkedBlockingQueue<ILogRecord> syncCommitQ;
+ private FileChannel fileChannel;
+ private boolean stop;
+
+ public LogPage(LockManager lockMgr, int logPageSize, MutableLong flushLSN) {
+ this.lockMgr = lockMgr;
+ this.logPageSize = logPageSize;
+ this.flushLSN = flushLSN;
+ appendBuffer = ByteBuffer.allocate(logPageSize);
+ flushBuffer = appendBuffer.duplicate();
+ unlockBuffer = appendBuffer.duplicate();
+ logPageReader = getLogPageReader();
+ full = new AtomicBoolean(false);
+ appendOffset = 0;
+ flushOffset = 0;
+ isLastPage = false;
+ syncCommitQ = new LinkedBlockingQueue<ILogRecord>(logPageSize / ILogRecord.JOB_COMMIT_LOG_SIZE);
+ }
+
+ ////////////////////////////////////
+ // LogAppender Methods
+ ////////////////////////////////////
+
+ @Override
+ public void append(ILogRecord logRecord, long appendLSN) {
+ logRecord.writeLogRecord(appendBuffer);
+ if (logRecord.getLogType() == LogType.UPDATE) {
+ logRecord.getTxnCtx().setLastLSN(logRecord.getResourceId(), appendLSN);
+ }
+ synchronized (this) {
+ appendOffset += logRecord.getLogSize();
+ if (IS_DEBUG_MODE) {
+ LOGGER.info("append()| appendOffset: " + appendOffset);
+ }
+ if (logRecord.getLogType() == LogType.JOB_COMMIT) {
+ logRecord.isFlushed(false);
+ syncCommitQ.offer(logRecord);
+ }
+ this.notify();
+ }
+ }
+
+ public void setFileChannel(FileChannel fileChannel) {
+ this.fileChannel = fileChannel;
+ }
+
+ public void setInitialFlushOffset(long offset) {
+ try {
+ fileChannel.position(offset);
+ } catch (IOException e) {
+ throw new IllegalStateException(e);
+ }
+ }
+
+ public synchronized void isFull(boolean full) {
+ this.full.set(full);
+ this.notify();
+ }
+
+ public void isLastPage(boolean isLastPage) {
+ this.isLastPage = isLastPage;
+ }
+
+ public boolean hasSpace(int logSize, long logFileOffset) {
+ return appendOffset + logSize <= logPageSize;
+ }
+
+ public void reset() {
+ appendBuffer.position(0);
+ appendBuffer.limit(logPageSize);
+ flushBuffer.position(0);
+ flushBuffer.limit(logPageSize);
+ unlockBuffer.position(0);
+ unlockBuffer.limit(logPageSize);
+ full.set(false);
+ appendOffset = 0;
+ flushOffset = 0;
+ isLastPage = false;
+ }
+
+ ////////////////////////////////////
+ // LogFlusher Methods
+ ////////////////////////////////////
+
+ @Override
+ public void flush() {
+ try {
+ int endOffset;
+ while (!full.get()) {
+ synchronized (this) {
+ if (appendOffset - flushOffset == 0 && !full.get()) {
+ try {
+ if (IS_DEBUG_MODE) {
+ LOGGER.info("flush()| appendOffset: " + appendOffset + ", flushOffset: " + flushOffset
+ + ", full: " + full.get());
+ }
+ if (stop) {
+ fileChannel.close();
+ break;
+ }
+ this.wait();
+ } catch (InterruptedException e) {
+ continue;
+ }
+ }
+ endOffset = appendOffset;
+ }
+ internalFlush(flushOffset, endOffset);
+ }
+ internalFlush(flushOffset, appendOffset);
+ if (isLastPage) {
+ fileChannel.close();
+ }
+ } catch (IOException e) {
+ throw new IllegalStateException(e);
+ }
+ }
+
+ private void internalFlush(int beginOffset, int endOffset) {
+ try {
+ if (endOffset > beginOffset) {
+ flushBuffer.limit(endOffset);
+ fileChannel.write(flushBuffer);
+ fileChannel.force(false);
+ flushOffset = endOffset;
+ synchronized (flushLSN) {
+ flushLSN.set(flushLSN.get() + (endOffset - beginOffset));
+ flushLSN.notifyAll(); //notify to LogReaders if any
+ }
+ if (IS_DEBUG_MODE) {
+ LOGGER.info("internalFlush()| flushOffset: " + flushOffset + ", flushLSN: " + flushLSN.get());
+ }
+ batchUnlock(beginOffset, endOffset);
+ }
+ } catch (Exception e) {
+ throw new IllegalStateException(e);
+ }
+ }
+
+ private LogPageReader getLogPageReader() {
+ return new LogPageReader(unlockBuffer);
+ }
+
+ private void batchUnlock(int beginOffset, int endOffset) throws ACIDException {
+ if (endOffset > beginOffset) {
+ logPageReader.initializeScan(beginOffset, endOffset);
+ lockMgr.batchUnlock(this, logPageReader);
+ }
+ }
+
+ public void notifyJobCommitter() {
+ ILogRecord logRecord = null;
+ while (logRecord == null) {
+ try {
+ logRecord = syncCommitQ.take();
+ } catch (InterruptedException e) {
+ //ignore
+ }
+ }
+ synchronized (logRecord) {
+ logRecord.isFlushed(true);
+ logRecord.notifyAll();
+ }
+ }
+
+ public boolean isStop() {
+ return stop;
+ }
+
+ public void isStop(boolean stop) {
+ this.stop = stop;
+ }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPageReader.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPageReader.java
new file mode 100644
index 0000000..9e54abc
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPageReader.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.transaction.management.service.logging;
+
+import java.nio.ByteBuffer;
+
+public class LogPageReader {
+
+ private final ByteBuffer buffer;
+ private final LogRecord logRecord;
+ private int endOffset;
+
+ public LogPageReader(ByteBuffer buffer) {
+ this.buffer = buffer;
+ logRecord = new LogRecord();
+ }
+
+ public void initializeScan(int beginOffset, int endOffset) {
+ this.endOffset = endOffset;
+ buffer.position(beginOffset);
+ }
+
+ public LogRecord next() {
+ if (buffer.position() == endOffset) {
+ return null;
+ }
+ if (!logRecord.readLogRecord(buffer)) {
+ throw new IllegalStateException();
+ }
+ return logRecord;
+ }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogReader.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogReader.java
new file mode 100644
index 0000000..9dc966c
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogReader.java
@@ -0,0 +1,188 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.transaction.management.service.logging;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.ILogReader;
+import edu.uci.ics.asterix.common.transactions.ILogRecord;
+import edu.uci.ics.asterix.common.transactions.MutableLong;
+
+public class LogReader implements ILogReader {
+
+ public static final boolean IS_DEBUG_MODE = false;//true
+ private static final Logger LOGGER = Logger.getLogger(LogReader.class.getName());
+ private final LogManager logMgr;
+ private final long logFileSize;
+ private final int logPageSize;
+ private final MutableLong flushLSN;
+ private final boolean isRecoveryMode;
+ private final ByteBuffer readBuffer;
+ private final ILogRecord logRecord;
+ private long readLSN;
+ private long bufferBeginLSN;
+ private long fileBeginLSN;
+ private FileChannel fileChannel;
+
+ private enum ReturnState {
+ FLUSH,
+ EOF
+ };
+
+ public LogReader(LogManager logMgr, long logFileSize, int logPageSize, MutableLong flushLSN, boolean isRecoveryMode) {
+ this.logMgr = logMgr;
+ this.logFileSize = logFileSize;
+ this.logPageSize = logPageSize;
+ this.flushLSN = flushLSN;
+ this.isRecoveryMode = isRecoveryMode;
+ this.readBuffer = ByteBuffer.allocate(logPageSize);
+ this.logRecord = new LogRecord();
+ }
+
+ @Override
+ public void initializeScan(long beginLSN) throws ACIDException {
+ readLSN = beginLSN;
+ if (waitForFlushOrReturnIfEOF() == ReturnState.EOF) {
+ return;
+ }
+ getFileChannel();
+ readPage();
+ }
+
+ //for scanning
+ @Override
+ public ILogRecord next() throws ACIDException {
+ if (waitForFlushOrReturnIfEOF() == ReturnState.EOF) {
+ return null;
+ }
+ if (readBuffer.position() == readBuffer.limit() || !logRecord.readLogRecord(readBuffer)) {
+ readNextPage();
+ if (!logRecord.readLogRecord(readBuffer)) {
+ throw new IllegalStateException();
+ }
+ }
+ logRecord.setLSN(readLSN);
+ readLSN += logRecord.getLogSize();
+ return logRecord;
+ }
+
+ private ReturnState waitForFlushOrReturnIfEOF() {
+ synchronized (flushLSN) {
+ while (readLSN >= flushLSN.get()) {
+ if (isRecoveryMode) {
+ return ReturnState.EOF;
+ }
+ try {
+ if (IS_DEBUG_MODE) {
+ LOGGER.info("waitForFlushOrReturnIfEOF()| flushLSN: " + flushLSN.get() + ", readLSN: "
+ + readLSN);
+ }
+ flushLSN.wait();
+ } catch (InterruptedException e) {
+ //ignore
+ }
+ }
+ return ReturnState.FLUSH;
+ }
+ }
+
+ private void readNextPage() throws ACIDException {
+ try {
+ if (readLSN % logFileSize == fileChannel.size()) {
+ fileChannel.close();
+ readLSN += logFileSize - (readLSN % logFileSize);
+ getFileChannel();
+ }
+ readPage();
+ } catch (IOException e) {
+ throw new ACIDException(e);
+ }
+ }
+
+ private void readPage() throws ACIDException {
+ int size;
+ readBuffer.position(0);
+ readBuffer.limit(logPageSize);
+ try {
+ fileChannel.position(readLSN % logFileSize);
+ size = fileChannel.read(readBuffer);
+ } catch (IOException e) {
+ throw new ACIDException(e);
+ }
+ readBuffer.position(0);
+ readBuffer.limit(size);
+ bufferBeginLSN = readLSN;
+ }
+
+ //for random reading
+ @Override
+ public ILogRecord read(long LSN) throws ACIDException {
+ readLSN = LSN;
+ synchronized (flushLSN) {
+ while (readLSN >= flushLSN.get()) {
+ try {
+ flushLSN.wait();
+ } catch (InterruptedException e) {
+ //ignore
+ }
+ }
+ }
+ try {
+ if (fileChannel == null) {
+ getFileChannel();
+ readPage();
+ } else if (readLSN < fileBeginLSN || readLSN >= fileBeginLSN + fileChannel.size()) {
+ fileChannel.close();
+ getFileChannel();
+ readPage();
+ } else if (readLSN < bufferBeginLSN || readLSN >= bufferBeginLSN + readBuffer.limit()) {
+ readPage();
+ } else {
+ readBuffer.position((int) (readLSN - bufferBeginLSN));
+ }
+ } catch (IOException e) {
+ throw new ACIDException(e);
+ }
+ if (!logRecord.readLogRecord(readBuffer)) {
+ readNextPage();
+ if (!logRecord.readLogRecord(readBuffer)) {
+ throw new IllegalStateException();
+ }
+ }
+ logRecord.setLSN(readLSN);
+ readLSN += logRecord.getLogSize();
+ return logRecord;
+ }
+
+ private void getFileChannel() throws ACIDException {
+ fileChannel = logMgr.getFileChannel(readLSN, false);
+ fileBeginLSN = readLSN;
+ }
+
+ @Override
+ public void close() throws ACIDException {
+ try {
+ if (fileChannel != null) {
+ fileChannel.close();
+ }
+ } catch (IOException e) {
+ throw new ACIDException(e);
+ }
+ }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecord.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecord.java
new file mode 100644
index 0000000..4b0e1f2
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecord.java
@@ -0,0 +1,520 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.transaction.management.service.logging;
+
+import java.nio.BufferUnderflowException;
+import java.nio.ByteBuffer;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.zip.CRC32;
+
+import edu.uci.ics.asterix.common.transactions.ILogRecord;
+import edu.uci.ics.asterix.common.transactions.IRecoveryManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+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;
+
+/*
+ * == LogRecordFormat ==
+ * ---------------------------
+ * [Header1] (5 bytes) : for all log types
+ * LogType(1)
+ * JobId(4)
+ * ---------------------------
+ * [Header2] (16 bytes + PKValueSize) : for entity_commit and update log types
+ * DatasetId(4) //stored in dataset_dataset in Metadata Node
+ * PKHashValue(4)
+ * PKFieldCnt(4)
+ * PKValueSize(4)
+ * PKValue(PKValueSize)
+ * ---------------------------
+ * [Header3] (21 bytes) : only for update log type
+ * PrevLSN(8)
+ * ResourceId(8) //stored in .metadata of the corresponding index in NC node
+ * ResourceType(1)
+ * LogRecordSize(4)
+ * ---------------------------
+ * [Body] (Variable size) : only for update log type
+ * FieldCnt(4)
+ * NewOp(1)
+ * NewValueSize(4)
+ * NewValue(NewValueSize)
+ * OldOp(1)
+ * OldValueSize(4)
+ * OldValue(OldValueSize)
+ * ---------------------------
+ * [Tail] (8 bytes) : for all log types
+ * Checksum(8)
+ * ---------------------------
+ * = LogSize =
+ * 1) JOB_COMMIT_LOG_SIZE: 13 bytes (5 + 8)
+ * 2) ENTITY_COMMIT: 29 + PKSize (5 + 16 + PKSize + 8)
+ * --> ENTITY_COMMIT_LOG_BASE_SIZE = 29
+ * 3) UPDATE: 64 + PKSize + New/OldValueSize (5 + 16 + PKSize + 21 + 14 + New/OldValueSize + 8)
+ * --> UPDATE_LOG_BASE_SIZE = 64
+ */
+public class LogRecord implements ILogRecord {
+
+ //------------- fields in a log record (begin) ------------//
+ private byte logType;
+ private int jobId;
+ private int datasetId;
+ private int PKHashValue;
+ private int PKFieldCnt;
+ private int PKValueSize;
+ private ITupleReference PKValue;
+ private long prevLSN;
+ private long resourceId;
+ private byte resourceType;
+ private int logSize;
+ private int fieldCnt;
+ private byte newOp;
+ private int newValueSize;
+ private ITupleReference newValue;
+ private byte oldOp;
+ private int oldValueSize;
+ private ITupleReference oldValue;
+ private long checksum;
+ //------------- fields in a log record (end) --------------//
+
+ private static final int CHECKSUM_SIZE = 8;
+ private ITransactionContext txnCtx;
+ private long LSN;
+ private final AtomicBoolean isFlushed;
+ private final SimpleTupleWriter tupleWriter;
+ private final SimpleTupleReference readPKValue;
+ private final SimpleTupleReference readNewValue;
+ private final SimpleTupleReference readOldValue;
+ private final CRC32 checksumGen;
+ private int[] PKFields;
+
+ public LogRecord() {
+ isFlushed = new AtomicBoolean(false);
+ tupleWriter = new SimpleTupleWriter();
+ readPKValue = (SimpleTupleReference) tupleWriter.createTupleReference();
+ readNewValue = (SimpleTupleReference) tupleWriter.createTupleReference();
+ readOldValue = (SimpleTupleReference) tupleWriter.createTupleReference();
+ checksumGen = new CRC32();
+ }
+
+ @Override
+ public void writeLogRecord(ByteBuffer buffer) {
+ int beginOffset = buffer.position();
+ buffer.put(logType);
+ buffer.putInt(jobId);
+ if (logType != LogType.JOB_COMMIT) {
+ buffer.putInt(datasetId);
+ buffer.putInt(PKHashValue);
+ buffer.putInt(PKFieldCnt);
+ if (PKValueSize <= 0) {
+ throw new IllegalStateException("Primary Key Size is less than or equal to 0");
+ }
+ buffer.putInt(PKValueSize);
+ writePKValue(buffer);
+ }
+ if (logType == LogType.UPDATE) {
+ buffer.putLong(prevLSN);
+ buffer.putLong(resourceId);
+ buffer.put(resourceType);
+ buffer.putInt(logSize);
+ buffer.putInt(fieldCnt);
+ buffer.put(newOp);
+ buffer.putInt(newValueSize);
+ writeTuple(buffer, newValue, newValueSize);
+ if (resourceType == ResourceType.LSM_BTREE) {
+ buffer.put(oldOp);
+ if (oldOp != (byte) (IndexOperation.NOOP.ordinal())) {
+ buffer.putInt(oldValueSize);
+ if (oldValueSize > 0) {
+ writeTuple(buffer, oldValue, oldValueSize);
+ }
+ }
+ }
+ }
+ checksum = generateChecksum(buffer, beginOffset, logSize - CHECKSUM_SIZE);
+ buffer.putLong(checksum);
+ }
+
+ private void writePKValue(ByteBuffer buffer) {
+ int i;
+ for (i = 0; i < PKFieldCnt; i++) {
+ buffer.put(PKValue.getFieldData(0), PKValue.getFieldStart(PKFields[i]), PKValue.getFieldLength(PKFields[i]));
+ }
+ }
+
+ private void writeTuple(ByteBuffer buffer, ITupleReference tuple, int size) {
+ tupleWriter.writeTuple(tuple, buffer.array(), buffer.position());
+ //writeTuple() doesn't change the position of the buffer.
+ buffer.position(buffer.position() + size);
+ }
+
+ private long generateChecksum(ByteBuffer buffer, int offset, int len) {
+ checksumGen.reset();
+ checksumGen.update(buffer.array(), offset, len);
+ return checksumGen.getValue();
+ }
+
+ @Override
+ public boolean readLogRecord(ByteBuffer buffer) {
+ int beginOffset = buffer.position();
+ try {
+ logType = buffer.get();
+ jobId = buffer.getInt();
+ if (logType == LogType.JOB_COMMIT) {
+ datasetId = -1;
+ PKHashValue = -1;
+ } else {
+ datasetId = buffer.getInt();
+ PKHashValue = buffer.getInt();
+ PKFieldCnt = buffer.getInt();
+ PKValueSize = buffer.getInt();
+ if (PKValueSize <= 0) {
+ throw new IllegalStateException("Primary Key Size is less than or equal to 0");
+ }
+ PKValue = readPKValue(buffer);
+ }
+ if (logType == LogType.UPDATE) {
+ prevLSN = buffer.getLong();
+ resourceId = buffer.getLong();
+ resourceType = buffer.get();
+ logSize = buffer.getInt();
+ fieldCnt = buffer.getInt();
+ newOp = buffer.get();
+ newValueSize = buffer.getInt();
+ newValue = readTuple(buffer, readNewValue, fieldCnt, newValueSize);
+ if (resourceType == ResourceType.LSM_BTREE) {
+ oldOp = buffer.get();
+ if (oldOp != (byte) (IndexOperation.NOOP.ordinal())) {
+ oldValueSize = buffer.getInt();
+ if (oldValueSize > 0) {
+ oldValue = readTuple(buffer, readOldValue, fieldCnt, oldValueSize);
+ }
+ }
+ }
+ } else {
+ computeAndSetLogSize();
+ }
+ checksum = buffer.getLong();
+ if (checksum != generateChecksum(buffer, beginOffset, logSize - CHECKSUM_SIZE)) {
+ throw new IllegalStateException();
+ }
+ } catch (BufferUnderflowException e) {
+ buffer.position(beginOffset);
+ return false;
+ }
+ return true;
+ }
+
+ private ITupleReference readPKValue(ByteBuffer buffer) {
+ return readTuple(buffer, readPKValue, PKFieldCnt, PKValueSize);
+ }
+
+ private ITupleReference readTuple(ByteBuffer srcBuffer, SimpleTupleReference destTuple, int fieldCnt, int size) {
+ destTuple.setFieldCount(fieldCnt);
+ destTuple.resetByTupleOffset(srcBuffer, srcBuffer.position());
+ srcBuffer.position(srcBuffer.position() + size);
+ return destTuple;
+ }
+
+ @Override
+ public void formJobCommitLogRecord(ITransactionContext txnCtx) {
+ this.txnCtx = txnCtx;
+ this.logType = LogType.JOB_COMMIT;
+ this.jobId = txnCtx.getJobId().getId();
+ this.datasetId = -1;
+ this.PKHashValue = -1;
+ computeAndSetLogSize();
+ }
+
+ @Override
+ public void formEntityCommitLogRecord(ITransactionContext txnCtx, int datasetId, int PKHashValue,
+ ITupleReference PKValue, int[] PKFields) {
+ this.txnCtx = txnCtx;
+ this.logType = LogType.ENTITY_COMMIT;
+ this.jobId = txnCtx.getJobId().getId();
+ this.datasetId = datasetId;
+ this.PKHashValue = PKHashValue;
+ this.PKFieldCnt = PKFields.length;
+ this.PKValue = PKValue;
+ this.PKFields = PKFields;
+ computeAndSetPKValueSize();
+ computeAndSetLogSize();
+ }
+
+ @Override
+ public void computeAndSetPKValueSize() {
+ int i;
+ PKValueSize = 0;
+ for (i = 0; i < PKFieldCnt; i++) {
+ PKValueSize += PKValue.getFieldLength(PKFields[i]);
+ }
+ }
+
+ private void setUpdateLogSize() {
+ logSize = UPDATE_LOG_BASE_SIZE + PKValueSize + newValueSize + oldValueSize;
+ if (resourceType != ResourceType.LSM_BTREE) {
+ logSize -= 5; //oldOp(byte: 1) + oldValueLength(int: 4)
+ } else {
+ if (oldOp == (byte) (IndexOperation.NOOP.ordinal())) {
+ logSize -= 4; //oldValueLength(int: 4)
+ }
+ }
+ }
+
+ @Override
+ public void computeAndSetLogSize() {
+ switch (logType) {
+ case LogType.UPDATE:
+ setUpdateLogSize();
+ break;
+ case LogType.JOB_COMMIT:
+ logSize = JOB_COMMIT_LOG_SIZE;
+ break;
+ case LogType.ENTITY_COMMIT:
+ logSize = ENTITY_COMMIT_LOG_BASE_SIZE + PKValueSize;
+ break;
+ default:
+ throw new IllegalStateException("Unsupported Log Type");
+ }
+ }
+
+ @Override
+ public String getLogRecordForDisplay() {
+ StringBuilder builder = new StringBuilder();
+ builder.append(" LSN : ").append(LSN);
+ builder.append(" LogType : ").append(LogType.toString(logType));
+ builder.append(" LogSize : ").append(logSize);
+ builder.append(" JobId : ").append(jobId);
+ if (logType != LogType.JOB_COMMIT) {
+ builder.append(" DatasetId : ").append(datasetId);
+ builder.append(" PKHashValue : ").append(PKHashValue);
+ builder.append(" PKFieldCnt : ").append(PKFieldCnt);
+ builder.append(" PKSize: ").append(PKValueSize);
+ }
+ if (logType == LogType.UPDATE) {
+ builder.append(" PrevLSN : ").append(prevLSN);
+ builder.append(" ResourceId : ").append(resourceId);
+ builder.append(" ResourceType : ").append(resourceType);
+ }
+ return builder.toString();
+ }
+
+ ////////////////////////////////////////////
+ // getter and setter methods
+ ////////////////////////////////////////////
+
+ @Override
+ public ITransactionContext getTxnCtx() {
+ return txnCtx;
+ }
+
+ @Override
+ public void setTxnCtx(ITransactionContext txnCtx) {
+ this.txnCtx = txnCtx;
+ }
+
+ @Override
+ public boolean isFlushed() {
+ return isFlushed.get();
+ }
+
+ @Override
+ public void isFlushed(boolean isFlushed) {
+ this.isFlushed.set(isFlushed);
+ }
+
+ @Override
+ public byte getLogType() {
+ return logType;
+ }
+
+ @Override
+ public void setLogType(byte logType) {
+ this.logType = logType;
+ }
+
+ @Override
+ public int getJobId() {
+ return jobId;
+ }
+
+ @Override
+ public void setJobId(int jobId) {
+ this.jobId = jobId;
+ }
+
+ @Override
+ public int getDatasetId() {
+ return datasetId;
+ }
+
+ @Override
+ public void setDatasetId(int datasetId) {
+ this.datasetId = datasetId;
+ }
+
+ @Override
+ public int getPKHashValue() {
+ return PKHashValue;
+ }
+
+ @Override
+ public void setPKHashValue(int PKHashValue) {
+ this.PKHashValue = PKHashValue;
+ }
+
+ @Override
+ public long getPrevLSN() {
+ return prevLSN;
+ }
+
+ @Override
+ public void setPrevLSN(long prevLSN) {
+ this.prevLSN = prevLSN;
+ }
+
+ @Override
+ public long getResourceId() {
+ return resourceId;
+ }
+
+ @Override
+ public void setResourceId(long resourceId) {
+ this.resourceId = resourceId;
+ }
+
+ @Override
+ public byte getResourceType() {
+ return resourceType;
+ }
+
+ @Override
+ public void setResourceType(byte resourceType) {
+ this.resourceType = resourceType;
+ }
+
+ @Override
+ public int getLogSize() {
+ return logSize;
+ }
+
+ @Override
+ public void setLogSize(int logSize) {
+ this.logSize = logSize;
+ }
+
+ @Override
+ public byte getNewOp() {
+ return newOp;
+ }
+
+ @Override
+ public void setNewOp(byte newOp) {
+ this.newOp = newOp;
+ }
+
+ @Override
+ public int getNewValueSize() {
+ return newValueSize;
+ }
+
+ @Override
+ public void setNewValueSize(int newValueSize) {
+ this.newValueSize = newValueSize;
+ }
+
+ @Override
+ public ITupleReference getNewValue() {
+ return newValue;
+ }
+
+ @Override
+ public void setNewValue(ITupleReference newValue) {
+ this.newValue = newValue;
+ this.fieldCnt = newValue.getFieldCount();
+ }
+
+ @Override
+ public byte getOldOp() {
+ return oldOp;
+ }
+
+ @Override
+ public void setOldOp(byte oldOp) {
+ this.oldOp = oldOp;
+ }
+
+ @Override
+ public int getOldValueSize() {
+ return oldValueSize;
+ }
+
+ @Override
+ public void setOldValueSize(int oldValueSize) {
+ this.oldValueSize = oldValueSize;
+ }
+
+ @Override
+ public ITupleReference getOldValue() {
+ return oldValue;
+ }
+
+ @Override
+ public void setOldValue(ITupleReference oldValue) {
+ this.oldValue = oldValue;
+ }
+
+ @Override
+ public long getChecksum() {
+ return checksum;
+ }
+
+ @Override
+ public void setChecksum(long checksum) {
+ this.checksum = checksum;
+ }
+
+ @Override
+ public long getLSN() {
+ return LSN;
+ }
+
+ @Override
+ public void setLSN(long LSN) {
+ this.LSN = LSN;
+ }
+
+ @Override
+ public int getPKValueSize() {
+ return PKValueSize;
+ }
+
+ @Override
+ public ITupleReference getPKValue() {
+ return PKValue;
+ }
+
+ @Override
+ public void setPKFields(int[] primaryKeyFields) {
+ PKFields = primaryKeyFields;
+ PKFieldCnt = PKFields.length;
+ }
+
+ @Override
+ public void setPKValue(ITupleReference PKValue) {
+ this.PKValue = PKValue;
+ }
+}
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
deleted file mode 100644
index d46ade8..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecordHelper.java
+++ /dev/null
@@ -1,268 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.service.logging;
-
-import edu.uci.ics.asterix.common.transactions.ILogManager;
-import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
-import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.LogManagerProperties;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
-
-/**
- * An implementation of the @see ILogRecordHelper interface that provides API
- * 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 LOG_CHECKSUM_SIZE = 8;
- private final int LOG_HEADER_PART1_SIZE = 17;
- private final int LOG_HEADER_PART2_SIZE = 21;
- private final int COMMIT_LOG_SIZE = LOG_HEADER_PART1_SIZE + LOG_CHECKSUM_SIZE;
-
- 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;
-
- public LogRecordHelper(ILogManager logManager) {
- this.logManager = logManager;
- }
-
- @Override
- public byte getLogType(LogicalLogLocator logicalLogLocator) {
- return logicalLogLocator.getBuffer().getByte(logicalLogLocator.getMemoryOffset() + LOG_TYPE_POS);
- }
-
- @Override
- public int getJobId(LogicalLogLocator logicalLogLocator) {
- return logicalLogLocator.getBuffer().readInt(logicalLogLocator.getMemoryOffset() + JOB_ID_POS);
- }
-
- @Override
- public int getDatasetId(LogicalLogLocator logicalLogLocator) {
- return logicalLogLocator.getBuffer().readInt(logicalLogLocator.getMemoryOffset() + DATASET_ID_POS);
- }
-
- @Override
- public int getPKHashValue(LogicalLogLocator logicalLogLocator) {
- return logicalLogLocator.getBuffer().readInt(logicalLogLocator.getMemoryOffset() + PK_HASH_VALUE_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;
- }
-
- @Override
- public boolean getPrevLSN(PhysicalLogLocator physicalLogLocator, LogicalLogLocator logicalLogLocator) {
- long prevLsnValue = (logicalLogLocator.getBuffer())
- .readLong(logicalLogLocator.getMemoryOffset() + PREV_LSN_POS);
- if (prevLsnValue == -1) {
- return false;
- }
- physicalLogLocator.setLsn(prevLsnValue);
- 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) {
- if (getLogType(logicalLogLocater) == LogType.COMMIT || getLogType(logicalLogLocater) == LogType.ENTITY_COMMIT) {
- return 0;
- } else {
- 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));
- String logTypeDisplay = null;
- switch (logType) {
- case LogType.COMMIT:
- logTypeDisplay = "COMMIT";
- break;
- case LogType.UPDATE:
- logTypeDisplay = "UPDATE";
- break;
- case LogType.ENTITY_COMMIT:
- logTypeDisplay = "ENTITY_COMMIT";
- break;
- }
- 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 void writeLogHeader(LogicalLogLocator logicalLogLocator, byte logType, ITransactionContext context,
- int datasetId, int PKHashValue, long prevLogicalLogLocator, long resourceId, byte resourceMgrId,
- int logRecordSize) {
-
- /* magic no */
- (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + MAGIC_NO_POS,
- LogManagerProperties.LOG_MAGIC_NUMBER);
-
- /* 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 - 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 LOG_HEADER_PART1_SIZE + LOG_HEADER_PART2_SIZE + LOG_CHECKSUM_SIZE + logBodySize;
- } else {
- return COMMIT_LOG_SIZE;
- }
- }
-
- @Override
- public int getLogHeaderSize(byte logType) {
- if (logType == LogType.UPDATE) {
- return LOG_HEADER_PART1_SIZE + LOG_HEADER_PART2_SIZE;
- } else {
- return LOG_HEADER_PART1_SIZE;
- }
- }
-
- @Override
- public int getLogChecksumSize() {
- return LOG_CHECKSUM_SIZE;
- }
-
- public int getCommitLogSize() {
- return COMMIT_LOG_SIZE;
- }
-}
\ No newline at end of file
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 99c0fef..823c8d3 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
@@ -17,7 +17,24 @@
public class LogType {
public static final byte UPDATE = 0;
- public static final byte COMMIT = 1;
+ public static final byte JOB_COMMIT = 1;
public static final byte ENTITY_COMMIT = 2;
+ private static final String STRING_UPDATE = "UPDATE";
+ private static final String STRING_JOB_COMMIT = "JOB_COMMIT";
+ private static final String STRING_ENTITY_COMMIT = "ENTITY_COMMIT";
+ private static final String STRING_INVALID_LOG_TYPE = "INVALID_LOG_TYPE";
+
+ public static String toString(byte logType) {
+ switch (logType) {
+ case LogType.UPDATE:
+ return STRING_UPDATE;
+ case LogType.JOB_COMMIT:
+ return STRING_JOB_COMMIT;
+ case LogType.ENTITY_COMMIT:
+ return STRING_ENTITY_COMMIT;
+ default:
+ return STRING_INVALID_LOG_TYPE;
+ }
+ }
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/MemBasedBuffer.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/MemBasedBuffer.java
deleted file mode 100644
index d63ce1c..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/MemBasedBuffer.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.service.logging;
-
-import java.nio.ByteBuffer;
-
-/**
- * Represents a memory-only buffer that is not associated with and physical
- * file.
- */
-public class MemBasedBuffer extends Buffer {
-
- public MemBasedBuffer(byte[] content) {
- super(ByteBuffer.wrap(content));
- }
-
- public MemBasedBuffer(ByteBuffer buffer) {
- super(buffer);
- }
-
-}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointObject.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointObject.java
index 110ddee..f2cb22a 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointObject.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointObject.java
@@ -20,24 +20,24 @@
private static final long serialVersionUID = 1L;
- private final long checkpointLSN;
- private final long minMCTFirstLSN;
+ private final long checkpointLsn;
+ private final long minMCTFirstLsn;
private final int maxJobId;
private final long timeStamp;
- public CheckpointObject(long checkpointLSN, long minMCTFirstLSN, int maxJobId, long timeStamp) {
- this.checkpointLSN = checkpointLSN;
- this.minMCTFirstLSN = minMCTFirstLSN;
+ public CheckpointObject(long checkpointLsn, long minMCTFirstLsn, int maxJobId, long timeStamp) {
+ this.checkpointLsn = checkpointLsn;
+ this.minMCTFirstLsn = minMCTFirstLsn;
this.maxJobId = maxJobId;
this.timeStamp = timeStamp;
}
- public long getCheckpointLSN() {
- return checkpointLSN;
+ public long getCheckpointLsn() {
+ return checkpointLsn;
}
- public long getMinMCTFirstLSN() {
- return minMCTFirstLSN;
+ public long getMinMCTFirstLsn() {
+ return minMCTFirstLsn;
}
public int getMaxJobId() {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointThread.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointThread.java
index a558969..6f6da4a 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointThread.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointThread.java
@@ -19,6 +19,7 @@
import edu.uci.ics.asterix.common.context.BaseOperationTracker;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
import edu.uci.ics.asterix.common.transactions.IRecoveryManager;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
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.lsm.common.api.ILSMIndex;
@@ -56,7 +57,7 @@
try {
recoveryMgr.checkpoint(false);
lastMinMCTFirstLSN = currentMinMCTFirstLSN;
- } catch (ACIDException e) {
+ } catch (ACIDException | HyracksDataException e) {
throw new Error("failed to checkpoint", e);
}
}
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 ad1db1f..2ad3055 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
@@ -27,6 +27,7 @@
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
@@ -39,20 +40,12 @@
import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
import edu.uci.ics.asterix.common.context.BaseOperationTracker;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
-import edu.uci.ics.asterix.common.transactions.IBuffer;
-import edu.uci.ics.asterix.common.transactions.ILogCursor;
-import edu.uci.ics.asterix.common.transactions.ILogFilter;
-import edu.uci.ics.asterix.common.transactions.ILogManager;
-import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
+import edu.uci.ics.asterix.common.transactions.ILogReader;
+import edu.uci.ics.asterix.common.transactions.ILogRecord;
import edu.uci.ics.asterix.common.transactions.IRecoveryManager;
-import edu.uci.ics.asterix.common.transactions.IResourceManager;
-import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.LogUtil;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
-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.LogType;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants;
@@ -60,23 +53,14 @@
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.lifecycle.ILifeCycleComponent;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+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.api.IIndexLifecycleManager;
-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.common.impls.NoOpOperationCallback;
-import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTreeImmutableComponent;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
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.impls.BlockingIOOperationCallbackWrapper;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexImmutableComponent;
-import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.LSMRTreeImmutableComponent;
-import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
-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;
import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
import edu.uci.ics.hyracks.storage.common.file.LocalResource;
@@ -91,6 +75,7 @@
public static final boolean IS_DEBUG_MODE = false;//true
private static final Logger LOGGER = Logger.getLogger(RecoveryManager.class.getName());
private final TransactionSubsystem txnSubsystem;
+ private final LogManager logMgr;
private final int checkpointHistory;
/**
@@ -100,8 +85,9 @@
private static final String CHECKPOINT_FILENAME_PREFIX = "checkpoint_";
private SystemState state;
- public RecoveryManager(TransactionSubsystem TransactionProvider) throws ACIDException {
- this.txnSubsystem = TransactionProvider;
+ public RecoveryManager(TransactionSubsystem txnSubsystem) throws ACIDException {
+ this.txnSubsystem = txnSubsystem;
+ this.logMgr = (LogManager) txnSubsystem.getLogManager();
this.checkpointHistory = this.txnSubsystem.getTransactionProperties().getCheckpointHistory();
}
@@ -132,12 +118,15 @@
//#. if minMCTFirstLSN is equal to -1 &&
// checkpointLSN in the checkpoint file is equal to the lastLSN in the log file,
// then return healthy state. Otherwise, return corrupted.
- LogManager logMgr = (LogManager) txnSubsystem.getLogManager();
- if (checkpointObject.getMinMCTFirstLSN() == -1
- && checkpointObject.getCheckpointLSN() == logMgr.getCurrentLsn().get()) {
+ if ((checkpointObject.getMinMCTFirstLsn() == -2 && logMgr.getAppendLSN() == 0)
+ || (checkpointObject.getMinMCTFirstLsn() == -1 && checkpointObject.getCheckpointLsn() == logMgr
+ .getAppendLSN())) {
state = SystemState.HEALTHY;
return state;
} else {
+ if (logMgr.getAppendLSN() == 0) {
+ throw new IllegalStateException("Transaction log files are lost.");
+ }
state = SystemState.CORRUPTED;
return state;
}
@@ -146,151 +135,133 @@
public void startRecovery(boolean synchronous) throws IOException, ACIDException {
int updateLogCount = 0;
- int commitLogCount = 0;
+ int entityCommitLogCount = 0;
+ int jobCommitLogCount = 0;
int redoCount = 0;
+ int jobId = -1;
state = SystemState.RECOVERING;
- ILogManager logManager = txnSubsystem.getLogManager();
- ILogRecordHelper logRecordHelper = logManager.getLogRecordHelper();
-
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("[RecoveryMgr] starting recovery ...");
}
- //winnerTxnTable is used to add pairs, <committed TxnId, the most recent commit LSN of the TxnId>
- Map<TxnId, Long> winnerTxnTable = new HashMap<TxnId, Long>();
- TxnId tempKeyTxnId = new TxnId(-1, -1, -1);
- byte logType;
+ Set<Integer> winnerJobSet = new HashSet<Integer>();
+ Map<Integer, Set<TxnId>> jobId2WinnerEntitiesMap = new HashMap<Integer, Set<TxnId>>();
+ //winnerEntity is used to add pairs, <committed TxnId, the most recent commit Lsn of the TxnId>
+ Set<TxnId> winnerEntitySet = null;
+ TxnId tempKeyTxnId = new TxnId(-1, -1, -1, null, -1, false);
+ TxnId winnerEntity = null;
//#. read checkpoint file and set lowWaterMark where anaylsis and redo start
CheckpointObject checkpointObject = readCheckpoint();
- long lowWaterMarkLSN = checkpointObject.getMinMCTFirstLSN();
- if (lowWaterMarkLSN == -1) {
- lowWaterMarkLSN = 0;
+ long lowWaterMarkLsn = checkpointObject.getMinMCTFirstLsn();
+ if (lowWaterMarkLsn == -1 || lowWaterMarkLsn == -2) {
+ lowWaterMarkLsn = 0;
}
int maxJobId = checkpointObject.getMaxJobId();
- int currentJobId;
//-------------------------------------------------------------------------
// [ analysis phase ]
- // - collect all committed LSN
- // - if there are duplicate commits for the same TxnId,
- // keep only the mostRecentCommitLSN among the duplicates.
+ // - collect all committed Lsn
//-------------------------------------------------------------------------
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("[RecoveryMgr] in analysis phase");
}
- //#. set log cursor to the lowWaterMarkLSN
- ILogCursor logCursor = logManager.readLog(new PhysicalLogLocator(lowWaterMarkLSN, logManager),
- new ILogFilter() {
- public boolean accept(IBuffer logs, long startOffset, int endOffset) {
- return true;
- }
- });
- LogicalLogLocator currentLogLocator = LogUtil.getDummyLogicalLogLocator(logManager);
-
- //#. collect all committed txn's pairs,<TxnId, LSN>
- while (logCursor.next(currentLogLocator)) {
-
- if (LogManager.IS_DEBUG_MODE) {
- System.out.println(logManager.getLogRecordHelper().getLogRecordForDisplay(currentLogLocator));
+ //#. set log reader to the lowWaterMarkLsn
+ ILogReader logReader = logMgr.getLogReader(true);
+ logReader.initializeScan(lowWaterMarkLsn);
+ ILogRecord logRecord = logReader.next();
+ while (logRecord != null) {
+ if (IS_DEBUG_MODE) {
+ System.out.println(logRecord.getLogRecordForDisplay());
}
-
- logType = logRecordHelper.getLogType(currentLogLocator);
-
//update max jobId
- currentJobId = logRecordHelper.getJobId(currentLogLocator);
- if (currentJobId > maxJobId) {
- maxJobId = currentJobId;
+ if (logRecord.getJobId() > maxJobId) {
+ maxJobId = logRecord.getJobId();
}
-
- TxnId commitTxnId = null;
- switch (logType) {
+ switch (logRecord.getLogType()) {
case LogType.UPDATE:
if (IS_DEBUG_MODE) {
updateLogCount++;
}
break;
-
- case LogType.COMMIT:
- case LogType.ENTITY_COMMIT:
- commitTxnId = new TxnId(logRecordHelper.getJobId(currentLogLocator),
- logRecordHelper.getDatasetId(currentLogLocator),
- logRecordHelper.getPKHashValue(currentLogLocator));
- winnerTxnTable.put(commitTxnId, currentLogLocator.getLsn());
+ case LogType.JOB_COMMIT:
+ winnerJobSet.add(Integer.valueOf(logRecord.getJobId()));
+ jobId2WinnerEntitiesMap.remove(Integer.valueOf(logRecord.getJobId()));
if (IS_DEBUG_MODE) {
- commitLogCount++;
+ jobCommitLogCount++;
}
break;
-
+ case LogType.ENTITY_COMMIT:
+ jobId = logRecord.getJobId();
+ winnerEntity = new TxnId(jobId, logRecord.getDatasetId(), logRecord.getPKHashValue(),
+ logRecord.getPKValue(), logRecord.getPKValueSize(), true);
+ if (!jobId2WinnerEntitiesMap.containsKey(Integer.valueOf(jobId))) {
+ winnerEntitySet = new HashSet<TxnId>();
+ jobId2WinnerEntitiesMap.put(Integer.valueOf(jobId), winnerEntitySet);
+ } else {
+ winnerEntitySet = jobId2WinnerEntitiesMap.get(Integer.valueOf(jobId));
+ }
+ winnerEntitySet.add(winnerEntity);
+ if (IS_DEBUG_MODE) {
+ entityCommitLogCount++;
+ }
+ break;
default:
- throw new ACIDException("Unsupported LogType: " + logType);
+ throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
}
+ logRecord = logReader.next();
}
//-------------------------------------------------------------------------
// [ redo phase ]
// - redo if
- // 1) The TxnId is committed --> gurantee durability
- // &&
- // 2) the currentLSN > maxDiskLastLSN of the index --> guarantee idempotance
+ // 1) The TxnId is committed && --> guarantee durability
+ // 2) lsn > maxDiskLastLsn of the index --> guarantee idempotence
//-------------------------------------------------------------------------
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("[RecoveryMgr] in redo phase");
}
- //#. set log cursor to the lowWaterMarkLSN again.
- logCursor = logManager.readLog(new PhysicalLogLocator(lowWaterMarkLSN, logManager), new ILogFilter() {
- public boolean accept(IBuffer logs, long startOffset, int endOffset) {
- return true;
- }
- });
- currentLogLocator = LogUtil.getDummyLogicalLogLocator(logManager);
-
long resourceId;
- byte resourceMgrId;
- long maxDiskLastLSN;
- long currentLSN = -1;
- int resourceType;
+ long maxDiskLastLsn;
+ long LSN = -1;
ILSMIndex index = null;
LocalResource localResource = null;
ILocalResourceMetadata localResourceMetadata = null;
Map<Long, Long> resourceId2MaxLSNMap = new HashMap<Long, Long>();
- List<ILSMComponent> immutableDiskIndexList = null;
- TxnId jobLevelTxnId = new TxnId(-1, -1, -1);
- boolean foundWinnerTxn;
+ boolean foundWinner = false;
//#. get indexLifeCycleManager
IAsterixAppRuntimeContextProvider appRuntimeContext = txnSubsystem.getAsterixAppRuntimeContextProvider();
IIndexLifecycleManager indexLifecycleManager = appRuntimeContext.getIndexLifecycleManager();
ILocalResourceRepository localResourceRepository = appRuntimeContext.getLocalResourceRepository();
- //#. redo
- while (logCursor.next(currentLogLocator)) {
- foundWinnerTxn = false;
+ //#. set log reader to the lowWaterMarkLsn again.
+ logReader.initializeScan(lowWaterMarkLsn);
+ logRecord = logReader.next();
+ while (logRecord != null) {
if (LogManager.IS_DEBUG_MODE) {
- System.out.println(logManager.getLogRecordHelper().getLogRecordForDisplay(currentLogLocator));
+ System.out.println(logRecord.getLogRecordForDisplay());
}
-
- logType = logRecordHelper.getLogType(currentLogLocator);
-
- switch (logType) {
+ LSN = logRecord.getLSN();
+ jobId = logRecord.getJobId();
+ foundWinner = false;
+ switch (logRecord.getLogType()) {
case LogType.UPDATE:
- tempKeyTxnId.setTxnId(logRecordHelper.getJobId(currentLogLocator),
- logRecordHelper.getDatasetId(currentLogLocator),
- logRecordHelper.getPKHashValue(currentLogLocator));
- jobLevelTxnId.setTxnId(logRecordHelper.getJobId(currentLogLocator), -1, -1);
- if (winnerTxnTable.containsKey(tempKeyTxnId)) {
- currentLSN = winnerTxnTable.get(tempKeyTxnId);
- foundWinnerTxn = true;
- } else if (winnerTxnTable.containsKey(jobLevelTxnId)) {
- currentLSN = winnerTxnTable.get(jobLevelTxnId);
- foundWinnerTxn = true;
+ if (winnerJobSet.contains(Integer.valueOf(jobId))) {
+ foundWinner = true;
+ } else if (jobId2WinnerEntitiesMap.containsKey(Integer.valueOf(jobId))) {
+ winnerEntitySet = jobId2WinnerEntitiesMap.get(Integer.valueOf(jobId));
+ tempKeyTxnId.setTxnId(jobId, logRecord.getDatasetId(), logRecord.getPKHashValue(),
+ logRecord.getPKValue(), logRecord.getPKValueSize());
+ if (winnerEntitySet.contains(tempKeyTxnId)) {
+ foundWinner = true;
+ }
}
-
- if (foundWinnerTxn) {
- resourceId = logRecordHelper.getResourceId(currentLogLocator);
+ if (foundWinner) {
+ resourceId = logRecord.getResourceId();
localResource = localResourceRepository.getResourceById(resourceId);
//get index instance from IndexLifeCycleManager
@@ -325,59 +296,20 @@
indexLifecycleManager.open(resourceId);
//#. get maxDiskLastLSN
- resourceType = localResource.getResourceType();
- immutableDiskIndexList = index.getImmutableComponents();
-
- maxDiskLastLSN = -1;
- switch (resourceType) {
-
- case ResourceType.LSM_BTREE:
- for (ILSMComponent c : immutableDiskIndexList) {
- BTree btree = ((LSMBTreeImmutableComponent) c).getBTree();
- maxDiskLastLSN = Math.max(getTreeIndexLSN(btree), maxDiskLastLSN);
- }
- break;
-
- case ResourceType.LSM_RTREE:
- for (ILSMComponent c : immutableDiskIndexList) {
- RTree rtree = ((LSMRTreeImmutableComponent) c).getRTree();
- maxDiskLastLSN = Math.max(getTreeIndexLSN(rtree), maxDiskLastLSN);
- }
- break;
-
- case ResourceType.LSM_INVERTED_INDEX:
- for (ILSMComponent c : immutableDiskIndexList) {
- BTree delKeyBtree = ((LSMInvertedIndexImmutableComponent) c)
- .getDeletedKeysBTree();
- maxDiskLastLSN = Math.max(getTreeIndexLSN(delKeyBtree), maxDiskLastLSN);
- }
- break;
-
- default:
- throw new ACIDException("Unsupported resouce type");
- }
+ ILSMIndex lsmIndex = (ILSMIndex) index;
+ BaseOperationTracker indexOpTracker = (BaseOperationTracker) lsmIndex.getOperationTracker();
+ AbstractLSMIOOperationCallback abstractLSMIOCallback = (AbstractLSMIOOperationCallback) indexOpTracker
+ .getIOOperationCallback();
+ maxDiskLastLsn = abstractLSMIOCallback.getComponentLSN(index.getImmutableComponents());
//#. set resourceId and maxDiskLastLSN to the map
- resourceId2MaxLSNMap.put(resourceId, maxDiskLastLSN);
+ resourceId2MaxLSNMap.put(Long.valueOf(resourceId), Long.valueOf(maxDiskLastLsn));
} else {
- maxDiskLastLSN = resourceId2MaxLSNMap.get(resourceId);
+ maxDiskLastLsn = resourceId2MaxLSNMap.get(Long.valueOf(resourceId));
}
- if (currentLSN > maxDiskLastLSN) {
- resourceMgrId = logRecordHelper.getResourceMgrId(currentLogLocator);
-
- // look up the repository to get the resource manager
- // register resourceMgr if it is not registered.
- IResourceManager resourceMgr = txnSubsystem.getTransactionalResourceRepository()
- .getTransactionalResourceMgr(resourceMgrId);
- if (resourceMgr == null) {
- resourceMgr = new IndexResourceManager(resourceMgrId, txnSubsystem);
- txnSubsystem.getTransactionalResourceRepository().registerTransactionalResourceManager(
- resourceMgrId, resourceMgr);
- }
-
- //redo finally.
- resourceMgr.redo(logRecordHelper, currentLogLocator);
+ if (LSN > maxDiskLastLsn) {
+ redo(logRecord);
if (IS_DEBUG_MODE) {
redoCount++;
}
@@ -385,14 +317,15 @@
}
break;
- case LogType.COMMIT:
+ case LogType.JOB_COMMIT:
case LogType.ENTITY_COMMIT:
//do nothing
break;
default:
- throw new ACIDException("Unsupported LogType: " + logType);
+ throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
}
+ logRecord = logReader.next();
}
//close all indexes
@@ -401,42 +334,25 @@
indexLifecycleManager.close(r);
}
+ logReader.close();
+
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("[RecoveryMgr] recovery is completed.");
}
if (IS_DEBUG_MODE) {
- System.out.println("[RecoveryMgr] Count: Update/Commit/Redo = " + updateLogCount + "/" + commitLogCount
- + "/" + redoCount);
- }
- }
-
- //TODO
- //This function came from the AbstractLSMIOOperationCallback class.
- //We'd better factor out this function into a component of reading/writing the local metadata of indexes.
- private 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);
+ System.out.println("[RecoveryMgr] Count: Update/EntityCommit/JobCommit/Redo = " + updateLogCount + "/"
+ + entityCommitLogCount + "/" + jobCommitLogCount + "/" + redoCount);
}
}
@Override
- public synchronized void checkpoint(boolean isSharpCheckpoint) throws ACIDException {
+ public synchronized void checkpoint(boolean isSharpCheckpoint) throws ACIDException, HyracksDataException {
+ long minMCTFirstLSN;
if (isSharpCheckpoint && LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Starting sharp checkpoint ... ");
}
- LogManager logMgr = (LogManager) txnSubsystem.getLogManager();
TransactionManager txnMgr = (TransactionManager) txnSubsystem.getTransactionManager();
String logDir = logMgr.getLogManagerProperties().getLogDir();
@@ -447,10 +363,14 @@
IIndexLifecycleManager indexLifecycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
.getIndexLifecycleManager();
List<IIndex> openIndexList = indexLifecycleManager.getOpenIndexes();
- List<BlockingIOOperationCallbackWrapper> callbackList = new LinkedList<BlockingIOOperationCallbackWrapper>();
//#. flush all in-memory components if it is the sharp checkpoint
if (isSharpCheckpoint) {
+ ///////////////////////////////////////////////
+ //TODO : change the code inside the if statement into indexLifeCycleManager.flushAllDatasets()
+ //indexLifeCycleManager.flushAllDatasets();
+ ///////////////////////////////////////////////
+ List<BlockingIOOperationCallbackWrapper> callbackList = new LinkedList<BlockingIOOperationCallbackWrapper>();
for (IIndex index : openIndexList) {
ILSMIndex lsmIndex = (ILSMIndex) index;
ILSMIndexAccessor indexAccessor = lsmIndex.createAccessor(NoOpOperationCallback.INSTANCE,
@@ -473,21 +393,20 @@
throw new ACIDException(e);
}
}
- }
-
- //#. create and store the checkpointObject into the new checkpoint file
- long minMCTFirstLSN = Long.MAX_VALUE;
- long firstLSN;
- if (openIndexList.size() > 0) {
- for (IIndex index : openIndexList) {
- firstLSN = ((BaseOperationTracker) ((ILSMIndex) index).getOperationTracker()).getFirstLSN();
- minMCTFirstLSN = Math.min(minMCTFirstLSN, firstLSN);
- }
+ minMCTFirstLSN = -2;
} else {
- minMCTFirstLSN = -1;
+ long firstLSN;
+ minMCTFirstLSN = Long.MAX_VALUE;
+ if (openIndexList.size() > 0) {
+ for (IIndex index : openIndexList) {
+ firstLSN = ((BaseOperationTracker) ((ILSMIndex) index).getOperationTracker()).getFirstLSN();
+ minMCTFirstLSN = Math.min(minMCTFirstLSN, firstLSN);
+ }
+ } else {
+ minMCTFirstLSN = -1;
+ }
}
-
- CheckpointObject checkpointObject = new CheckpointObject(logMgr.getCurrentLsn().get(), minMCTFirstLSN,
+ CheckpointObject checkpointObject = new CheckpointObject(logMgr.getAppendLSN(), minMCTFirstLSN,
txnMgr.getMaxJobId(), System.currentTimeMillis());
FileOutputStream fos = null;
@@ -585,7 +504,7 @@
}
private File[] getPreviousCheckpointFiles() {
- String logDir = txnSubsystem.getLogManager().getLogManagerProperties().getLogDir();
+ String logDir = ((LogManager) txnSubsystem.getLogManager()).getLogManagerProperties().getLogDir();
File parentDir = new File(logDir);
@@ -621,24 +540,25 @@
*/
@Override
public void rollbackTransaction(ITransactionContext txnContext) throws ACIDException {
- 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);
+ TxnId tempKeyTxnId = new TxnId(-1, -1, -1, null, -1, false);
int updateLogCount = 0;
- int commitLogCount = 0;
+ int entityCommitLogCount = 0;
+ int jobId = -1;
+ int abortedJobId = txnContext.getJobId().getId();
+ long currentLSN = -1;
+ TxnId loserEntity = null;
- // Obtain the first log record written by the Job
- PhysicalLogLocator firstLSNLogLocator = txnContext.getFirstLogLocator();
- PhysicalLogLocator lastLSNLogLocator = txnContext.getLastLogLocator();
+ // Obtain the first/last log record LSNs written by the Job
+ long firstLSN = txnContext.getFirstLSN();
+ long lastLSN = txnContext.getLastLSN();
if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(" rollbacking transaction log records from " + firstLSNLogLocator.getLsn() + " to "
- + lastLSNLogLocator.getLsn());
+ LOGGER.info(" rollbacking transaction log records from " + firstLSN + " to " + lastLSN);
}
// check if the transaction actually wrote some logs.
- if (firstLSNLogLocator.getLsn() == TransactionManagementConstants.LogManagerConstants.TERMINAL_LSN) {
+ if (firstLSN == 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.getJobId());
@@ -646,147 +566,107 @@
return;
}
- // 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);
- }
-
- LogicalLogLocator currentLogLocator = LogUtil.getDummyLogicalLogLocator(logManager);
- boolean valid;
- byte logType;
- List<Long> undoLSNSet = null;
-
+ // While reading log records from firstLsn to lastLsn, collect uncommitted txn's Lsns
if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(" collecting loser transaction's LSNs from " + firstLSNLogLocator.getLsn() + " to "
- + +lastLSNLogLocator.getLsn());
+ LOGGER.info(" collecting loser transaction's LSNs from " + firstLSN + " to " + lastLSN);
}
-
- while (currentLogLocator.getLsn() != lastLSNLogLocator.getLsn()) {
- try {
- 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("LastLSN mismatch: " + lastLSNLogLocator.getLsn() + " vs "
- + currentLogLocator.getLsn() + " during Rollback a transaction( " + txnContext.getJobId()
- + ")");
- } else {
- break;//End of Log File
+ List<Long> undoLSNSet = null;
+ ILogReader logReader = logMgr.getLogReader(false);
+ logReader.initializeScan(firstLSN);
+ ILogRecord logRecord = null;
+ while (currentLSN < lastLSN) {
+ logRecord = logReader.next();
+ if (logRecord == null) {
+ break;
+ } else {
+ if (IS_DEBUG_MODE) {
+ System.out.println(logRecord.getLogRecordForDisplay());
}
+ currentLSN = logRecord.getLSN();
}
-
- if (IS_DEBUG_MODE) {
- System.out.println(logManager.getLogRecordHelper().getLogRecordForDisplay(currentLogLocator));
+ jobId = logRecord.getJobId();
+ if (jobId != abortedJobId) {
+ continue;
}
-
- tempKeyTxnId.setTxnId(logRecordHelper.getJobId(currentLogLocator),
- logRecordHelper.getDatasetId(currentLogLocator), logRecordHelper.getPKHashValue(currentLogLocator));
- logType = logRecordHelper.getLogType(currentLogLocator);
-
- switch (logType) {
+ tempKeyTxnId.setTxnId(jobId, logRecord.getDatasetId(), logRecord.getPKHashValue(), logRecord.getPKValue(),
+ logRecord.getPKValueSize());
+ switch (logRecord.getLogType()) {
case LogType.UPDATE:
undoLSNSet = loserTxnTable.get(tempKeyTxnId);
if (undoLSNSet == null) {
- TxnId txnId = new TxnId(logRecordHelper.getJobId(currentLogLocator),
- logRecordHelper.getDatasetId(currentLogLocator),
- logRecordHelper.getPKHashValue(currentLogLocator));
+ loserEntity = new TxnId(jobId, logRecord.getDatasetId(), logRecord.getPKHashValue(),
+ logRecord.getPKValue(), logRecord.getPKValueSize(), true);
undoLSNSet = new LinkedList<Long>();
- loserTxnTable.put(txnId, undoLSNSet);
+ loserTxnTable.put(loserEntity, undoLSNSet);
}
- undoLSNSet.add(currentLogLocator.getLsn());
+ undoLSNSet.add(Long.valueOf(currentLSN));
if (IS_DEBUG_MODE) {
updateLogCount++;
- System.out.println("" + Thread.currentThread().getId() + "======> update["
- + currentLogLocator.getLsn() + "]:" + tempKeyTxnId);
+ System.out.println("" + Thread.currentThread().getId() + "======> update[" + currentLSN + "]:"
+ + tempKeyTxnId);
}
break;
- case LogType.COMMIT:
+ case LogType.JOB_COMMIT:
+ throw new ACIDException("Unexpected LogType(" + logRecord.getLogType() + ") during abort.");
+
case LogType.ENTITY_COMMIT:
- undoLSNSet = loserTxnTable.get(tempKeyTxnId);
- if (undoLSNSet != null) {
- loserTxnTable.remove(tempKeyTxnId);
- }
+ loserTxnTable.remove(tempKeyTxnId);
if (IS_DEBUG_MODE) {
- commitLogCount++;
- System.out.println("" + Thread.currentThread().getId() + "======> commit["
- + currentLogLocator.getLsn() + "]" + tempKeyTxnId);
+ entityCommitLogCount++;
+ System.out.println("" + Thread.currentThread().getId() + "======> entity_commit[" + currentLSN + "]"
+ + tempKeyTxnId);
}
break;
default:
- throw new ACIDException("Unsupported LogType: " + logType);
+ throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
}
}
+ if (currentLSN != lastLSN) {
+ throw new ACIDException("LastLSN mismatch: lastLSN(" + lastLSN + ") vs currentLSN(" + currentLSN
+ + ") during abort( " + txnContext.getJobId() + ")");
+ }
//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()) {
//TODO
//Sort the lsns in order to undo in one pass.
Map.Entry<TxnId, List<Long>> loserTxn = (Map.Entry<TxnId, List<Long>>) iter.next();
- txnId = loserTxn.getKey();
-
undoLSNSet = loserTxn.getValue();
for (long undoLSN : undoLSNSet) {
- // here, all the log records are UPDATE type. So, we don't need to check the type again.
-
+ //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);
-
+ logRecord = logReader.read(undoLSN);
+ if (logRecord == null) {
+ throw new ACIDException("IllegalState exception during abort( " + txnContext.getJobId() + ")");
+ }
if (IS_DEBUG_MODE) {
- System.out.println(logManager.getLogRecordHelper().getLogRecordForDisplay(currentLogLocator));
+ System.out.println(logRecord.getLogRecordForDisplay());
}
-
- // 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);
-
+ undo(logRecord);
if (IS_DEBUG_MODE) {
undoCount++;
}
}
}
+ logReader.close();
+
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info(" undone loser transaction's effect");
}
if (IS_DEBUG_MODE) {
- System.out.println("UpdateLogCount/CommitLogCount/UndoCount:" + updateLogCount + "/" + commitLogCount + "/"
- + undoCount);
+ System.out.println("UpdateLogCount/CommitLogCount/UndoCount:" + updateLogCount + "/" + entityCommitLogCount
+ + "/" + undoCount);
}
}
@@ -799,39 +679,103 @@
public void stop(boolean dumpState, OutputStream os) {
//no op
}
+
+ private void undo(ILogRecord logRecord) {
+ try {
+ ILSMIndex index = (ILSMIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
+ .getIndex(logRecord.getResourceId());
+ ILSMIndexAccessor indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE,
+ NoOpOperationCallback.INSTANCE);
+ if (logRecord.getResourceType() == ResourceType.LSM_BTREE) {
+ if (logRecord.getOldOp() != IndexOperation.NOOP.ordinal()) {
+ if (logRecord.getOldOp() == IndexOperation.DELETE.ordinal()) {
+ indexAccessor.forceDelete(logRecord.getOldValue());
+ } else {
+ indexAccessor.forceInsert(logRecord.getOldValue());
+ }
+ } else {
+ indexAccessor.forcePhysicalDelete(logRecord.getNewValue());
+ }
+ } else {
+ if (logRecord.getNewOp() == IndexOperation.DELETE.ordinal()) {
+ indexAccessor.forceInsert(logRecord.getNewValue());
+ } else {
+ indexAccessor.forceDelete(logRecord.getNewValue());
+ }
+ }
+ } catch (Exception e) {
+ throw new IllegalStateException("Failed to undo", e);
+ }
+ }
+
+ private void redo(ILogRecord logRecord) {
+ try {
+ ILSMIndex index = (ILSMIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
+ .getIndex(logRecord.getResourceId());
+ ILSMIndexAccessor indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE,
+ NoOpOperationCallback.INSTANCE);
+ if (logRecord.getNewOp() == IndexOperation.INSERT.ordinal()) {
+ indexAccessor.forceInsert(logRecord.getNewValue());
+ } else if (logRecord.getNewOp() == IndexOperation.DELETE.ordinal()) {
+ indexAccessor.forceDelete(logRecord.getNewValue());
+ } else {
+ throw new IllegalStateException("Unsupported OperationType: " + logRecord.getNewOp());
+ }
+ ((BaseOperationTracker) index.getOperationTracker()).updateLastLSN(logRecord.getLSN());
+ } catch (Exception e) {
+ throw new IllegalStateException("Failed to redo", e);
+ }
+ }
}
class TxnId {
+ public boolean isByteArrayPKValue;
public int jobId;
public int datasetId;
- public int pkHashVal;
+ public int pkHashValue;
+ public int pkSize;
+ public byte[] byteArrayPKValue;
+ public ITupleReference tupleReferencePKValue;
- public TxnId(int jobId, int datasetId, int pkHashVal) {
+ public TxnId(int jobId, int datasetId, int pkHashValue, ITupleReference pkValue, int pkSize,
+ boolean isByteArrayPKValue) {
this.jobId = jobId;
this.datasetId = datasetId;
- this.pkHashVal = pkHashVal;
+ this.pkHashValue = pkHashValue;
+ this.pkSize = pkSize;
+ this.isByteArrayPKValue = isByteArrayPKValue;
+ if (isByteArrayPKValue) {
+ this.byteArrayPKValue = new byte[pkSize];
+ readPKValueIntoByteArray(pkValue, pkSize, byteArrayPKValue);
+ } else {
+ this.tupleReferencePKValue = pkValue;
+ }
}
- public void setTxnId(int jobId, int datasetId, int pkHashVal) {
+ private void readPKValueIntoByteArray(ITupleReference pkValue, int pkSize, byte[] byteArrayPKValue) {
+ int readOffset = pkValue.getFieldStart(0);
+ byte[] readBuffer = pkValue.getFieldData(0);
+ for (int i = 0; i < pkSize; i++) {
+ byteArrayPKValue[i] = readBuffer[readOffset + i];
+ }
+ }
+
+ public void setTxnId(int jobId, int datasetId, int pkHashValue, ITupleReference pkValue, int pkSize) {
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;
+ this.pkHashValue = pkHashValue;
+ this.tupleReferencePKValue = pkValue;
+ isByteArrayPKValue = false;
}
@Override
public String toString() {
- return "[" + jobId + "," + datasetId + "," + pkHashVal + "]";
+ return "[" + jobId + "," + datasetId + "," + pkHashValue + "," + pkSize + "]";
}
@Override
public int hashCode() {
- return pkHashVal;
+ return pkHashValue;
}
@Override
@@ -843,7 +787,52 @@
return false;
}
TxnId txnId = (TxnId) o;
+ return (txnId.pkHashValue == pkHashValue && txnId.datasetId == datasetId && txnId.jobId == jobId
+ && pkSize == txnId.pkSize && isEqualTo(txnId));
+ }
- return (txnId.pkHashVal == pkHashVal && txnId.datasetId == datasetId && txnId.jobId == jobId);
+ private boolean isEqualTo(TxnId txnId) {
+ if (isByteArrayPKValue && txnId.isByteArrayPKValue) {
+ return isEqual(byteArrayPKValue, txnId.byteArrayPKValue, pkSize);
+ } else if (isByteArrayPKValue && (!txnId.isByteArrayPKValue)) {
+ return isEqual(byteArrayPKValue, txnId.tupleReferencePKValue, pkSize);
+ } else if ((!isByteArrayPKValue) && txnId.isByteArrayPKValue) {
+ return isEqual(txnId.byteArrayPKValue, tupleReferencePKValue, pkSize);
+ } else {
+ return isEqual(tupleReferencePKValue, txnId.tupleReferencePKValue, pkSize);
+ }
+ }
+
+ private boolean isEqual(byte[] a, byte[] b, int size) {
+ for (int i = 0; i < size; i++) {
+ if (a[i] != b[i]) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ private boolean isEqual(byte[] a, ITupleReference b, int size) {
+ int readOffset = b.getFieldStart(0);
+ byte[] readBuffer = b.getFieldData(0);
+ for (int i = 0; i < size; i++) {
+ if (a[i] != readBuffer[readOffset + i]) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ private boolean isEqual(ITupleReference a, ITupleReference b, int size) {
+ int aOffset = a.getFieldStart(0);
+ byte[] aBuffer = a.getFieldData(0);
+ int bOffset = b.getFieldStart(0);
+ byte[] bBuffer = b.getFieldData(0);
+ for (int i = 0; i < size; i++) {
+ if (aBuffer[aOffset + i] != bBuffer[bOffset + i]) {
+ return false;
+ }
+ }
+ return true;
}
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/TransactionTableEntry.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/TransactionTableEntry.java
deleted file mode 100644
index e64ec01..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/TransactionTableEntry.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.service.recovery;
-
-import edu.uci.ics.asterix.common.transactions.ITransactionManager;
-import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
-
-/**
- * Represents a bookkeeping data-structure that is populated duing the analysis
- * phase of recovery. It contains for each transaction, the transaction state,
- * the LSN corresponding to the last log record written by the transaction and
- * the log record written by the transaction that needs to be undone.
- */
-public class TransactionTableEntry {
-
- private long transactionId;
- private ITransactionManager.TransactionState transactionState;
- private PhysicalLogLocator lastLSN;
- private PhysicalLogLocator undoNextLSN;
-
- public TransactionTableEntry(long transactionId, ITransactionManager.TransactionState transactionState,
- PhysicalLogLocator lastLSN, PhysicalLogLocator undoNextLSN) {
- this.transactionId = transactionId;
- this.transactionState = transactionState;
- this.lastLSN = lastLSN;
- this.undoNextLSN = undoNextLSN;
- }
-
- public long getTransactionId() {
- return transactionId;
- }
-
- public void setTransactionId(long transactionId) {
- this.transactionId = transactionId;
- }
-
- public ITransactionManager.TransactionState getTransactionState() {
- return transactionState;
- }
-
- public void setTransactionState(ITransactionManager.TransactionState transactionState) {
- this.transactionState = transactionState;
- }
-
- public PhysicalLogLocator getLastLSN() {
- return lastLSN;
- }
-
- public void setLastLSN(PhysicalLogLocator lastLSN) {
- this.lastLSN = lastLSN;
- }
-
- public PhysicalLogLocator getUndoNextLSN() {
- return undoNextLSN;
- }
-
- public void setUndoNextLSN(PhysicalLogLocator undoNextLSN) {
- this.undoNextLSN = undoNextLSN;
- }
-
-}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/AsterixRuntimeComponentsProvider.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/AsterixRuntimeComponentsProvider.java
index 97f2477..59a8363 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/AsterixRuntimeComponentsProvider.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/AsterixRuntimeComponentsProvider.java
@@ -16,22 +16,16 @@
import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
import edu.uci.ics.asterix.common.context.BaseOperationTracker;
-import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
-import edu.uci.ics.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
-import edu.uci.ics.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
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.ILSMIOOperationCallback;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
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.ILSMOperationTrackerProvider;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
@@ -39,33 +33,13 @@
import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactory;
public class AsterixRuntimeComponentsProvider implements IIndexLifecycleManagerProvider, IStorageManagerInterface,
- ILSMIOOperationSchedulerProvider, ILSMMergePolicyProvider, ILSMOperationTrackerProvider,
+ ILSMIOOperationSchedulerProvider, ILSMMergePolicyProvider,
ILSMIOOperationCallbackProvider {
private static final long serialVersionUID = 1L;
- private final ILSMIOOperationCallbackFactory ioOpCallbackFactory;
- private final boolean isSecondary;
-
- public static final AsterixRuntimeComponentsProvider LSMBTREE_PRIMARY_PROVIDER = new AsterixRuntimeComponentsProvider(
- LSMBTreeIOOperationCallbackFactory.INSTANCE, false);
- public static final AsterixRuntimeComponentsProvider LSMBTREE_SECONDARY_PROVIDER = new AsterixRuntimeComponentsProvider(
- LSMBTreeIOOperationCallbackFactory.INSTANCE, true);
- public static final AsterixRuntimeComponentsProvider LSMRTREE_PROVIDER = new AsterixRuntimeComponentsProvider(
- LSMRTreeIOOperationCallbackFactory.INSTANCE, true);
- public static final AsterixRuntimeComponentsProvider LSMINVERTEDINDEX_PROVIDER = new AsterixRuntimeComponentsProvider(
- LSMInvertedIndexIOOperationCallbackFactory.INSTANCE, true);
- public static final AsterixRuntimeComponentsProvider NOINDEX_PROVIDER = new AsterixRuntimeComponentsProvider(null,
- false);
-
- private AsterixRuntimeComponentsProvider(ILSMIOOperationCallbackFactory ioOpCallbackFactory, boolean isSecondary) {
- this.ioOpCallbackFactory = ioOpCallbackFactory;
- this.isSecondary = isSecondary;
- }
-
- @Override
- public ILSMOperationTracker getOperationTracker(IHyracksTaskContext ctx) {
- assert isSecondary;
- return new BaseOperationTracker(ioOpCallbackFactory);
+ public static final AsterixRuntimeComponentsProvider RUNTIME_PROVIDER = new AsterixRuntimeComponentsProvider();
+
+ private AsterixRuntimeComponentsProvider() {
}
@Override
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 33522e3..678956b 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,176 +15,176 @@
package edu.uci.ics.asterix.transaction.management.service.transaction;
import java.io.Serializable;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Set;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
import edu.uci.ics.asterix.common.context.BaseOperationTracker;
+import edu.uci.ics.asterix.common.context.PrimaryIndexOperationTracker;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
-import edu.uci.ics.asterix.common.transactions.ICloseable;
import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.ITransactionManager.TransactionState;
+import edu.uci.ics.asterix.common.transactions.ITransactionManager;
import edu.uci.ics.asterix.common.transactions.JobId;
-import edu.uci.ics.asterix.common.transactions.LogUtil;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
+import edu.uci.ics.asterix.common.transactions.MutableLong;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexModificationOperationCallback;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogRecord;
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;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMOperationType;
-/**
- * Represents a holder object that contains all information related to a
- * transaction. A TransactionContext instance can be used as a token and
- * provided to Transaction sub-systems (Log/Lock/Recovery/Transaction)Manager to
- * initiate an operation on the behalf of the transaction associated with the
- * context.
+/*
+ * An object of TransactionContext is created and accessed(read/written) by multiple threads which work for
+ * a single job identified by a jobId. Thus, the member variables in the object can be read/written
+ * concurrently. Please see each variable declaration to know which one is accessed concurrently and
+ * which one is not.
*/
public class TransactionContext implements ITransactionContext, Serializable {
private static final long serialVersionUID = -6105616785783310111L;
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;
- private boolean exlusiveJobLevelCommit;
- // List of indexes on which operations were performed on behalf of this transaction.
- private final Set<ILSMIndex> indexes = new HashSet<ILSMIndex>();
+ //jobId is set once and read concurrently.
+ private final JobId jobId;
- // 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 Set<AbstractOperationCallback> callbacks = new HashSet<AbstractOperationCallback>();
+ //There are no concurrent writers on both firstLSN and lastLSN
+ //since both values are updated by serialized log appenders.
+ //But readers and writers can be different threads,
+ //so both LSNs are atomic variables in order to be read and written atomically.
+ private AtomicLong firstLSN;
+ private AtomicLong lastLSN;
+ //txnState is read and written concurrently.
+ private AtomicInteger txnState;
+
+ //isTimeout is read and written under the lockMgr's tableLatch
+ //Thus, no other synchronization is required separately.
+ private boolean isTimeout;
+
+ //isWriteTxn can be set concurrently by multiple threads.
+ private AtomicBoolean isWriteTxn;
+
+ //isMetadataTxn is accessed by a single thread since the metadata is not partitioned
+ private boolean isMetadataTxn;
+
+ //indexMap is concurrently accessed by multiple threads,
+ //so those threads are synchronized on indexMap object itself
+ private Map<MutableLong, BaseOperationTracker> indexMap;
+
+ //TODO: fix ComponentLSNs' issues.
+ //primaryIndex, primaryIndexCallback, and primaryIndexOptracker will be modified accordingly
+ //when the issues of componentLSNs are fixed.
+ private ILSMIndex primaryIndex;
+ private PrimaryIndexModificationOperationCallback primaryIndexCallback;
+ private PrimaryIndexOperationTracker primaryIndexOpTracker;
+
+ //The following three variables are used as temporary variables in order to avoid object creations.
+ //Those are used in synchronized methods.
+ private MutableLong tempResourceIdForRegister;
+ private MutableLong tempResourceIdForSetLSN;
+ private LogRecord logRecord;
+
+ //TODO: implement transactionContext pool in order to avoid object creations.
+ // also, the pool can throttle the number of concurrent active jobs at every moment.
public TransactionContext(JobId jobId, TransactionSubsystem transactionSubsystem) throws ACIDException {
this.jobId = jobId;
this.transactionSubsystem = transactionSubsystem;
- init();
+ firstLSN = new AtomicLong(-1);
+ lastLSN = new AtomicLong(-1);
+ txnState = new AtomicInteger(ITransactionManager.ACTIVE);
+ isTimeout = false;
+ isWriteTxn = new AtomicBoolean(false);
+ isMetadataTxn = false;
+ indexMap = new HashMap<MutableLong, BaseOperationTracker>();
+ primaryIndex = null;
+ tempResourceIdForRegister = new MutableLong();
+ tempResourceIdForSetLSN = new MutableLong();
+ logRecord = new LogRecord();
}
- 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 updateLastLSNForIndexes(long lastLSN) {
- synchronized (indexes) {
- for (ILSMIndex index : indexes) {
- ((BaseOperationTracker) index.getOperationTracker()).updateLastLSN(lastLSN);
+ public void registerIndexAndCallback(long resourceId, ILSMIndex index, AbstractOperationCallback callback,
+ boolean isPrimaryIndex) {
+ synchronized (indexMap) {
+ if (isPrimaryIndex && primaryIndex == null) {
+ primaryIndex = index;
+ primaryIndexCallback = (PrimaryIndexModificationOperationCallback) callback;
+ primaryIndexOpTracker = (PrimaryIndexOperationTracker) index.getOperationTracker();
+ }
+ tempResourceIdForRegister.set(resourceId);
+ if (!indexMap.containsKey(tempResourceIdForRegister)) {
+ indexMap.put(new MutableLong(resourceId), ((BaseOperationTracker) index.getOperationTracker()));
}
}
}
- public void decreaseActiveTransactionCountOnIndexes() throws HyracksDataException {
- synchronized (indexes) {
- Set<BaseOperationTracker> opTrackers = new HashSet<BaseOperationTracker>();
- Iterator<ILSMIndex> indexIt = indexes.iterator();
- Iterator<AbstractOperationCallback> cbIt = callbacks.iterator();
- while (indexIt.hasNext()) {
- ILSMIndex index = indexIt.next();
- opTrackers.add((BaseOperationTracker) index.getOperationTracker());
- assert cbIt.hasNext();
- }
- Iterator<BaseOperationTracker> trackerIt = opTrackers.iterator();
- while (trackerIt.hasNext()) {
- IModificationOperationCallback modificationCallback = (IModificationOperationCallback) cbIt.next();
- BaseOperationTracker opTracker = (BaseOperationTracker) trackerIt.next();
- if (exlusiveJobLevelCommit) {
- opTracker.exclusiveJobCommitted();
- } else {
- opTracker.completeOperation(null, LSMOperationType.MODIFICATION, null, modificationCallback);
- }
- }
+ //[Notice]
+ //This method is called sequentially by the LogAppender threads.
+ //However, the indexMap is concurrently read and modified through this method and registerIndexAndCallback()
+ //TODO: fix issues - 591, 609, 612, and 614.
+ @Override
+ public void setLastLSN(long resourceId, long LSN) {
+ synchronized (indexMap) {
+ firstLSN.compareAndSet(-1, LSN);
+ lastLSN.set(Math.max(lastLSN.get(), LSN));
+ tempResourceIdForSetLSN.set(resourceId);
+ //TODO; create version number tracker and keep LSNs there.
+ BaseOperationTracker opTracker = indexMap.get(tempResourceIdForSetLSN);
+ opTracker.updateLastLSN(LSN);
}
}
@Override
- public int getActiveOperationCountOnIndexes() throws HyracksDataException {
- synchronized (indexes) {
- int count = 0;
- Iterator<AbstractOperationCallback> cbIt = callbacks.iterator();
- while (cbIt.hasNext()) {
- IModificationOperationCallback modificationCallback = (IModificationOperationCallback) cbIt.next();
- count += ((AbstractOperationCallback) modificationCallback).getLocalNumActiveOperations();
+ public void notifyOptracker(boolean isJobLevelCommit) {
+ try {
+ if (isJobLevelCommit && isMetadataTxn) {
+ primaryIndexOpTracker.exclusiveJobCommitted();
+ } else if (!isJobLevelCommit) {
+ primaryIndexOpTracker
+ .completeOperation(null, LSMOperationType.MODIFICATION, null, primaryIndexCallback);
}
- return count;
+ } catch (HyracksDataException e) {
+ throw new IllegalStateException(e);
}
}
- public void setTransactionType(TransactionType transactionType) {
- this.transactionType = transactionType;
+ public void setWriteTxn(boolean isWriteTxn) {
+ this.isWriteTxn.set(isWriteTxn);
}
- public TransactionType getTransactionType() {
- return transactionType;
+ public boolean isWriteTxn() {
+ return isWriteTxn.get();
}
- public void addCloseableResource(ICloseable resource) {
- resources.add(resource);
+ @Override
+ public long getFirstLSN() {
+ return firstLSN.get();
}
- public LogicalLogLocator getFirstLogLocator() {
- return firstLogLocator;
- }
-
- public LogicalLogLocator getLastLogLocator() {
- return lastLogLocator;
- }
-
- public void setLastLSN(long lsn) {
- if (firstLogLocator.getLsn() == -1) {
- firstLogLocator.setLsn(lsn);
- }
- lastLogLocator.setLsn(lsn);
+ @Override
+ public long getLastLSN() {
+ return lastLSN.get();
}
public JobId getJobId() {
return jobId;
}
- public void setStartWaitTime(long time) {
- this.startWaitTime = time;
+ public void setTimeout(boolean isTimeout) {
+ this.isTimeout = isTimeout;
}
- public long getStartWaitTime() {
- return startWaitTime;
+ public boolean isTimeout() {
+ return isTimeout;
}
- public void setStatus(int status) {
- this.status = status;
+ public void setTxnState(int txnState) {
+ this.txnState.set(txnState);
}
- public int getStatus() {
- return status;
- }
-
- public void setTxnState(TransactionState txnState) {
- this.txnState = txnState;
- }
-
- public TransactionState getTxnState() {
- return txnState;
- }
-
- public void releaseResources() throws ACIDException {
- for (ICloseable closeable : resources) {
- closeable.close(this);
- }
+ public int getTxnState() {
+ return txnState.get();
}
@Override
@@ -198,19 +198,27 @@
}
@Override
- public void setExclusiveJobLevelCommit() {
- exlusiveJobLevelCommit = true;
+ public void setMetadataTransaction(boolean isMetadataTxn) {
+ this.isMetadataTxn = isMetadataTxn;
+ }
+
+ @Override
+ public boolean isMetadataTransaction() {
+ return isMetadataTxn;
}
public String prettyPrint() {
StringBuilder sb = new StringBuilder();
sb.append("\n" + jobId + "\n");
- sb.append("transactionType: " + transactionType);
- sb.append("firstLogLocator: " + firstLogLocator.getLsn() + "\n");
- sb.append("lastLogLocator: " + lastLogLocator.getLsn() + "\n");
+ sb.append("isWriteTxn: " + isWriteTxn + "\n");
+ sb.append("firstLSN: " + firstLSN.get() + "\n");
+ sb.append("lastLSN: " + lastLSN.get() + "\n");
sb.append("TransactionState: " + txnState + "\n");
- sb.append("startWaitTime: " + startWaitTime + "\n");
- sb.append("status: " + status + "\n");
+ sb.append("isTimeout: " + isTimeout + "\n");
return sb.toString();
}
+
+ public LogRecord getLogRecord() {
+ return logRecord;
+ }
}
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 54ec8a1..01b38c2 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
@@ -16,9 +16,9 @@
import java.io.IOException;
import java.io.OutputStream;
-import java.util.HashMap;
import java.util.Map;
import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.logging.Level;
import java.util.logging.Logger;
@@ -28,8 +28,7 @@
import edu.uci.ics.asterix.common.transactions.ITransactionContext;
import edu.uci.ics.asterix.common.transactions.ITransactionManager;
import edu.uci.ics.asterix.common.transactions.JobId;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogRecord;
import edu.uci.ics.hyracks.api.lifecycle.ILifeCycleComponent;
/**
@@ -40,115 +39,73 @@
public static final boolean IS_DEBUG_MODE = false;//true
private static final Logger LOGGER = Logger.getLogger(TransactionManager.class.getName());
- private final TransactionSubsystem transactionProvider;
- private Map<JobId, ITransactionContext> transactionContextRepository = new HashMap<JobId, ITransactionContext>();
+ private final TransactionSubsystem txnSubsystem;
+ private Map<JobId, ITransactionContext> transactionContextRepository = new ConcurrentHashMap<JobId, ITransactionContext>();
private AtomicInteger maxJobId = new AtomicInteger(0);
public TransactionManager(TransactionSubsystem provider) {
- this.transactionProvider = provider;
+ this.txnSubsystem = provider;
}
@Override
- public void abortTransaction(ITransactionContext txnContext, DatasetId datasetId, int PKHashVal)
- throws ACIDException {
- synchronized (txnContext) {
- if (txnContext.getTxnState().equals(TransactionState.ABORTED)) {
- return;
+ public void abortTransaction(ITransactionContext txnCtx, DatasetId datasetId, int PKHashVal) throws ACIDException {
+ if (txnCtx.getTxnState() != ITransactionManager.ABORTED) {
+ txnCtx.setTxnState(ITransactionManager.ABORTED);
+ }
+ try {
+ txnSubsystem.getRecoveryManager().rollbackTransaction(txnCtx);
+ } catch (Exception ae) {
+ String msg = "Could not complete rollback! System is in an inconsistent state";
+ if (LOGGER.isLoggable(Level.SEVERE)) {
+ LOGGER.severe(msg);
}
-
- try {
- transactionProvider.getRecoveryManager().rollbackTransaction(txnContext);
- } catch (Exception ae) {
- String msg = "Could not complete rollback! System is in an inconsistent state";
- if (LOGGER.isLoggable(Level.SEVERE)) {
- LOGGER.severe(msg);
- }
- ae.printStackTrace();
- throw new Error(msg);
- } finally {
- txnContext.releaseResources();
- transactionProvider.getLockManager().releaseLocks(txnContext);
- transactionContextRepository.remove(txnContext.getJobId());
- txnContext.setTxnState(TransactionState.ABORTED);
- }
+ ae.printStackTrace();
+ throw new ACIDException(msg, ae);
+ } finally {
+ txnSubsystem.getLockManager().releaseLocks(txnCtx);
+ transactionContextRepository.remove(txnCtx.getJobId());
}
}
@Override
public ITransactionContext beginTransaction(JobId jobId) throws ACIDException {
- setMaxJobId(jobId.getId());
- ITransactionContext txnContext = new TransactionContext(jobId, transactionProvider);
- synchronized (this) {
- transactionContextRepository.put(jobId, txnContext);
- }
- return txnContext;
+ return getTransactionContext(jobId);
}
@Override
public ITransactionContext getTransactionContext(JobId jobId) throws ACIDException {
setMaxJobId(jobId.getId());
- synchronized (transactionContextRepository) {
-
- ITransactionContext context = transactionContextRepository.get(jobId);
- if (context == null) {
- context = transactionContextRepository.get(jobId);
- context = new TransactionContext(jobId, transactionProvider);
- transactionContextRepository.put(jobId, context);
+ ITransactionContext txnCtx = transactionContextRepository.get(jobId);
+ if (txnCtx == null) {
+ synchronized (this) {
+ txnCtx = transactionContextRepository.get(jobId);
+ if (txnCtx == null) {
+ txnCtx = new TransactionContext(jobId, txnSubsystem);
+ transactionContextRepository.put(jobId, txnCtx);
+ }
}
- return context;
}
+ return txnCtx;
}
@Override
- public void commitTransaction(ITransactionContext txnContext, DatasetId datasetId, int PKHashVal)
- throws ACIDException {
- synchronized (txnContext) {
- if ((txnContext.getTxnState().equals(TransactionState.COMMITTED))) {
- return;
+ public void commitTransaction(ITransactionContext txnCtx, DatasetId datasetId, int PKHashVal) throws ACIDException {
+ //Only job-level commits call this method.
+ try {
+ if (txnCtx.isWriteTxn()) {
+ LogRecord logRecord = ((TransactionContext) txnCtx).getLogRecord();
+ logRecord.formJobCommitLogRecord(txnCtx);
+ txnSubsystem.getLogManager().log(logRecord);
}
-
- //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) {
- boolean countIsZero = transactionProvider.getLockManager().unlock(datasetId, PKHashVal, txnContext,
- true);
- if (!countIsZero) {
- // Lock count != 0 for a particular entity implies that the entity has been locked
- // more than once (probably due to a hash collision in our current model).
- // It is safe to decrease the active transaction count on indexes since,
- // by virtue of the counter not being zero, there is another transaction
- // that has increased the transaction count. Thus, decreasing it will not
- // allow the data to be flushed (yet). The flush will occur when the log page
- // flush thread decides to decrease the count for the last time.
- try {
- //decrease the transaction reference count on index
- txnContext.decreaseActiveTransactionCountOnIndexes();
- } catch (HyracksDataException e) {
- throw new ACIDException("failed to complete index operation", e);
- }
- }
- return;
+ } catch (Exception ae) {
+ if (LOGGER.isLoggable(Level.SEVERE)) {
+ LOGGER.severe(" caused exception in commit !" + txnCtx.getJobId());
}
-
- //for job-level commit
- try {
- if (txnContext.getTransactionType().equals(ITransactionContext.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.getJobId());
- }
- throw ae;
- } finally {
- txnContext.releaseResources();
- transactionProvider.getLockManager().releaseLocks(txnContext); // release
- transactionContextRepository.remove(txnContext.getJobId());
- txnContext.setTxnState(TransactionState.COMMITTED);
- }
+ throw ae;
+ } finally {
+ txnSubsystem.getLockManager().releaseLocks(txnCtx); // release
+ transactionContextRepository.remove(txnCtx.getJobId());
+ txnCtx.setTxnState(ITransactionManager.COMMITTED);
}
}
@@ -164,11 +121,14 @@
@Override
public TransactionSubsystem getTransactionProvider() {
- return transactionProvider;
+ return txnSubsystem;
}
public void setMaxJobId(int jobId) {
- maxJobId.set(Math.max(maxJobId.get(), jobId));
+ int maxId = maxJobId.get();
+ if (jobId > maxId) {
+ maxJobId.compareAndSet(maxId, jobId);
+ }
}
public int getMaxJobId() {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionSubsystem.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionSubsystem.java
index b76cf11..aceeb82 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionSubsystem.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionSubsystem.java
@@ -22,9 +22,7 @@
import edu.uci.ics.asterix.common.transactions.IRecoveryManager;
import edu.uci.ics.asterix.common.transactions.ITransactionManager;
import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
-import edu.uci.ics.asterix.common.transactions.TransactionalResourceManagerRepository;
import edu.uci.ics.asterix.transaction.management.service.locking.LockManager;
-import edu.uci.ics.asterix.transaction.management.service.logging.IndexLoggerRepository;
import edu.uci.ics.asterix.transaction.management.service.logging.LogManager;
import edu.uci.ics.asterix.transaction.management.service.recovery.CheckpointThread;
import edu.uci.ics.asterix.transaction.management.service.recovery.RecoveryManager;
@@ -39,8 +37,6 @@
private final ILockManager lockManager;
private final ITransactionManager transactionManager;
private final IRecoveryManager recoveryManager;
- private final TransactionalResourceManagerRepository resourceRepository;
- private final IndexLoggerRepository loggerRepository;
private final IAsterixAppRuntimeContextProvider asterixAppRuntimeContextProvider;
private final CheckpointThread checkpointThread;
private final AsterixTransactionProperties txnProperties;
@@ -50,11 +46,9 @@
this.id = id;
this.txnProperties = txnProperties;
this.transactionManager = new TransactionManager(this);
- this.logManager = new LogManager(this);
this.lockManager = new LockManager(this);
+ this.logManager = new LogManager(this);
this.recoveryManager = new RecoveryManager(this);
- this.loggerRepository = new IndexLoggerRepository(this);
- this.resourceRepository = new TransactionalResourceManagerRepository();
this.asterixAppRuntimeContextProvider = asterixAppRuntimeContextProvider;
if (asterixAppRuntimeContextProvider != null) {
this.checkpointThread = new CheckpointThread(recoveryManager,
@@ -81,14 +75,6 @@
return recoveryManager;
}
- public TransactionalResourceManagerRepository getTransactionalResourceRepository() {
- return resourceRepository;
- }
-
- public IndexLoggerRepository getTreeLoggerRepository() {
- return loggerRepository;
- }
-
public IAsterixAppRuntimeContextProvider getAsterixAppRuntimeContextProvider() {
return asterixAppRuntimeContextProvider;
}
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
deleted file mode 100644
index 06379b4..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/BasicLogger.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.logging;
-
-import java.util.Random;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.IBuffer;
-import edu.uci.ics.asterix.common.transactions.ILogger;
-import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.common.transactions.ReusableLogContentObject;
-
-public class BasicLogger implements ILogger {
-
- private static long averageContentCreationTime = 0;
- private static long count = 0;
-
- public void log(ITransactionContext context, LogicalLogLocator wMemLSN, int length,
- ReusableLogContentObject reusableLogContentObject) throws ACIDException {
-
- byte[] logContent = getRandomBytes(length);
- try {
- long startTime2 = System.nanoTime();
-
- IBuffer buffer = (IBuffer) (wMemLSN.getBuffer());
-
- /*
- * synchronized(buffer){ buffer.position(wMemLSN.getMemoryOffset());
- * buffer.put(logContent); }
- */
-
- byte[] logPageStorage = buffer.getArray();
- System.arraycopy(logContent, 0, logPageStorage, wMemLSN.getMemoryOffset(), logContent.length);
-
- /*
- * for(int i=0;i<logContent.length;i++){
- * ((IFileBasedBuffer)(wMemLSN.
- * getBuffer())).put(wMemLSN.getMemoryOffset() + i, logContent[i]);
- * }
- */
- long endTime2 = System.nanoTime();
- averageContentCreationTime = ((averageContentCreationTime * count) + (endTime2 - startTime2)) / (++count);
- } catch (Exception e) {
- throw new ACIDException("", e);
- }
- }
-
- public static long getAverageContentCreationTime() {
- return averageContentCreationTime;
- }
-
- public static long getNumLogs() {
- return averageContentCreationTime;
- }
-
- public void postLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException {
- // TODO Auto-generated method stub
-
- }
-
- public void preLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException {
- // TODO Auto-generated method stub
-
- }
-
- private static byte[] getRandomBytes(int size) {
- byte[] b = new byte[size];
- Random random = new Random();
- int num = random.nextInt(30);
- Integer number = (new Integer(num + 65));
- byte numByte = number.byteValue();
- for (int i = 0; i < size; i++) {
- b[i] = numByte;
- }
- return b;
- }
-}
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/IResource.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/IResource.java
deleted file mode 100644
index 8b9c970..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/IResource.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.logging;
-
-import edu.uci.ics.asterix.common.transactions.ILogger;
-
-public interface IResource {
-
- public byte[] getId();
-
- public ILogger getLogger();
-}
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
deleted file mode 100644
index e4a8e45..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/RecoverySimulator.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.logging.test;
-
-import java.io.IOException;
-
-import edu.uci.ics.asterix.common.config.AsterixPropertiesAccessor;
-import edu.uci.ics.asterix.common.config.AsterixTransactionProperties;
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.common.transactions.IRecoveryManager;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
-
-public class RecoverySimulator {
-
- private static IRecoveryManager recoveryManager;
-
- public static void startRecovery() throws IOException, ACIDException {
- recoveryManager.startRecovery(true);
- }
-
- public static void main(String args[]) throws IOException, ACIDException, AsterixException {
- String id = "nc1";
- try {
- TransactionSubsystem factory = new TransactionSubsystem(id, null, new AsterixTransactionProperties(
- new AsterixPropertiesAccessor()));
- IRecoveryManager recoveryManager = factory.getRecoveryManager();
- recoveryManager.startRecovery(true);
- } catch (ACIDException acide) {
- acide.printStackTrace();
- throw 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
deleted file mode 100644
index a74b079..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/TransactionWorkloadSimulator.java
+++ /dev/null
@@ -1,247 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.logging.test;
-
-import java.util.Random;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-
-import edu.uci.ics.asterix.common.config.AsterixPropertiesAccessor;
-import edu.uci.ics.asterix.common.config.AsterixTransactionProperties;
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.common.transactions.DatasetId;
-import edu.uci.ics.asterix.common.transactions.ILockManager;
-import edu.uci.ics.asterix.common.transactions.ILogManager;
-import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
-import edu.uci.ics.asterix.common.transactions.IResourceManager;
-import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
-import edu.uci.ics.asterix.common.transactions.LogUtil;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.transaction.management.logging.BasicLogger;
-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.transaction.JobIdFactory;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
-
-public class TransactionWorkloadSimulator {
-
- public static ILogManager logManager;
- public static ILockManager lockManager;
- TransactionSubsystem provider;
-
- public static WorkloadProperties workload;
- Transaction[] transactions;
-
- public TransactionWorkloadSimulator(WorkloadProperties workload) {
- this.workload = workload;
- transactions = new Transaction[workload.numActiveThreads];
- }
-
- public void beginWorkload() throws ACIDException, AsterixException {
- provider = new TransactionSubsystem("nc1", null, new AsterixTransactionProperties(
- new AsterixPropertiesAccessor()));
- logManager = provider.getLogManager();
- lockManager = provider.getLockManager();
- provider.getTransactionalResourceRepository().registerTransactionalResourceManager(DummyResourceMgr.id,
- new DummyResourceMgr());
- Transaction[] transactions = new Transaction[workload.numActiveThreads];
- long startTime = System.nanoTime();
- for (int i = 0; i < workload.numActiveThreads; i++) {
- transactions[i] = new Transaction(provider, "Transaction " + (i + 1), workload.singleTransaction);
- transactions[i].start();
- }
- for (int i = 0; i < workload.numActiveThreads; i++) {
- try {
- transactions[i].join();
- } catch (InterruptedException ignore) {
- }
- }
-
- for (int i = 0; i < workload.numActiveThreads; i++) {
- provider.getTransactionManager().commitTransaction(transactions[i].getContext(), new DatasetId(-1), -1);
- }
-
- long endTime = System.nanoTime();
- int totalLogs = Transaction.logCount.get();
- System.out.println(" Total logs :" + totalLogs);
- long timeTaken = ((endTime - startTime) / 1000000);
- System.out.println(" total time :" + timeTaken);
- System.out.println(" throughput :" + totalLogs * 1000 / timeTaken + " logs/sec");
- long totalBytesWritten = Transaction.logByteCount.get();
- System.out.println(" bytes written :" + totalBytesWritten);
- System.out.println(" IO throughput " + totalBytesWritten * 1000 / timeTaken + " bytes/sec");
- System.out.println(" Avg Content Creation time :" + BasicLogger.getAverageContentCreationTime());
- }
-
- public static void main(String args[]) throws AsterixException {
- WorkloadProperties workload = new WorkloadProperties();
- TransactionWorkloadSimulator simulator = new TransactionWorkloadSimulator(workload);
- try {
- simulator.beginWorkload();
- } catch (ACIDException acide) {
- acide.printStackTrace();
- }
-
- }
-}
-
-class SingleTransactionContextFactory {
- private static TransactionContext context;
-
- public static TransactionContext getContext(TransactionSubsystem provider) throws ACIDException {
- if (context == null) {
- context = new TransactionContext(JobIdFactory.generateJobId(), provider);
- }
- return context;
- }
-}
-
-class MultipleTransactionContextFactory {
-
- public static TransactionContext getContext(TransactionSubsystem provider) throws ACIDException {
- return new TransactionContext(JobIdFactory.generateJobId(), provider);
- }
-}
-
-class Transaction extends Thread {
-
- public static AtomicInteger logCount = new AtomicInteger(0);
- public static AtomicLong logByteCount = new AtomicLong(0);
- Random random = new Random();
- BasicLogger logger = new BasicLogger();
- LogicalLogLocator memLSN;
- String name;
- TransactionContext context;
- //private byte[] resourceID = new byte[1];
- private int resourceID;
- private int myLogCount = 0;
- private TransactionSubsystem transactionProvider;
- private ILogManager logManager;
- private DatasetId tempDatasetId = new DatasetId(-1);
-
- public Transaction(TransactionSubsystem provider, String name, boolean singleTransaction) throws ACIDException {
- this.name = name;
- this.transactionProvider = provider;
- if (singleTransaction) {
- context = SingleTransactionContextFactory.getContext(transactionProvider);
- } else {
- context = MultipleTransactionContextFactory.getContext(transactionProvider);
- }
- memLSN = LogUtil.getDummyLogicalLogLocator(transactionProvider.getLogManager());
- logManager = transactionProvider.getLogManager();
- }
-
- public TransactionContext getContext() {
- return context;
- }
-
- @Override
- public void run() {
- if (TransactionWorkloadSimulator.workload.minLogsPerTransactionThread == TransactionWorkloadSimulator.workload.maxLogsPerTransactionThread) {
- TransactionWorkloadSimulator.workload.maxLogsPerTransactionThread++;
- }
- int numLogs = TransactionWorkloadSimulator.workload.minLogsPerTransactionThread
- + random.nextInt(TransactionWorkloadSimulator.workload.maxLogsPerTransactionThread
- - TransactionWorkloadSimulator.workload.minLogsPerTransactionThread);
- int total = 0;
- LogicalLogLocator memLSN = LogUtil.getDummyLogicalLogLocator(logManager);
- if (TransactionWorkloadSimulator.workload.maxLogSize == TransactionWorkloadSimulator.workload.minLogSize) {
- TransactionWorkloadSimulator.workload.maxLogSize++;
- }
- if (TransactionWorkloadSimulator.workload.singleResource) {
- int choice = random.nextInt(2);
- resourceID = (byte) (choice % 2);
- } else {
- random.nextInt(resourceID);
- }
- boolean retry = false;
- byte lockMode = -1;
- try {
- for (int i = 0; i < numLogs - 1; i++) {
- int logSize = TransactionWorkloadSimulator.workload.minLogSize
- + random.nextInt(TransactionWorkloadSimulator.workload.maxLogSize
- - TransactionWorkloadSimulator.workload.minLogSize);
- total += logSize;
-
- byte logType = LogType.UPDATE;
- byte logActionType = LogActionType.REDO_UNDO;
- long pageId = 0;
- if (!retry) {
- lockMode = (byte) (random.nextInt(2));
- }
- 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.getLogRecordHelper().getLogHeaderSize(logType)
- + TransactionWorkloadSimulator.logManager.getLogRecordHelper().getLogChecksumSize());
- myLogCount++;
- }
- } catch (ACIDException acide) {
- acide.printStackTrace();
- } catch (Exception ie) {
- ie.printStackTrace();
- }
- }
-
-}
-
-class WorkloadProperties {
- public int numActiveThreads = 200;
- public long thinkTime = 0; // (in mesecs)
- public int minLogsPerTransactionThread = 5;
- public int maxLogsPerTransactionThread = 5;
- public int minLogSize = 1024 - 51;
- public int maxLogSize = 1024 - 51;
- public float commitFraction = 0.5f;
- public float rollbackFraction = 0.1f;
- public boolean singleTransaction = false;
- public boolean singleResource = true;
-}
-
-class ResourceMgrInfo {
- public static final byte BTreeResourceMgrId = 1;
- public static final byte MetadataResourceMgrId = 2;
-}
-
-class DummyResourceMgr implements IResourceManager {
-
- public static final byte id = 1;
-
- @Override
- public void redo(ILogRecordHelper logParser, LogicalLogLocator memLSN) throws ACIDException {
- // TODO Auto-generated method stub
-
- }
-
- @Override
- public void undo(ILogRecordHelper logParser, LogicalLogLocator memLSN) throws ACIDException {
- // TODO Auto-generated method stub
-
- }
-
- @Override
- public byte getResourceManagerId() {
- // TODO Auto-generated method stub
- return 1;
- }
-
-}
\ No newline at end of file
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
deleted file mode 100644
index 2eea43e..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileLogger.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.test;
-
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.ILogger;
-import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.common.transactions.ReusableLogContentObject;
-import edu.uci.ics.asterix.transaction.management.logging.IResource;
-
-class FileLogger implements ILogger {
-
- IResource resource;
- String logRecordContent;
-
- public FileLogger(IResource resource) {
- this.resource = resource;
- }
-
- public int generateLogRecordContent(int currentValue, int finalValue) {
- StringBuilder builder = new StringBuilder();
- builder.append("" + currentValue + " " + finalValue);
- logRecordContent = new String(builder);
- return resource.getId().length + logRecordContent.length();
- }
-
- @Override
- public void preLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException {
- // TODO Auto-generated method stub
-
- }
-
- @Override
- public void log(ITransactionContext 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++) {
- buffer[memLSN.getMemoryOffset() + i] = resource.getId()[i];
- }
- for (int i = 0; i < content.length; i++) {
- buffer[memLSN.getMemoryOffset() + resource.getId().length + i] = content[i];
- }
- }
-
- @Override
- public void postLog(ITransactionContext 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/FileResource.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileResource.java
deleted file mode 100644
index 072738f..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileResource.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.test;
-
-import java.io.BufferedReader;
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileReader;
-import java.io.FileWriter;
-import java.io.IOException;
-
-import edu.uci.ics.asterix.common.transactions.FileUtil;
-import edu.uci.ics.asterix.common.transactions.ILogger;
-import edu.uci.ics.asterix.transaction.management.logging.IResource;
-
-class FileResource implements IResource {
-
- private byte[] resourceId = new byte[] { 1 };
- private ILogger logger;
-
- private File file;
- private int memCounter = 0;
- private int diskCounter = 0;
-
- public int getMemoryCounter() {
- return memCounter;
- }
-
- public int getDiskCounter() {
- return diskCounter;
- }
-
- public static enum CounterOperation {
- INCREMENT,
- DECREMENT,
- };
-
- public FileResource(String fileDir, String fileName) throws IOException {
- File dirFile = new File(fileDir);
- if (!dirFile.exists()) {
- FileUtil.createNewDirectory(fileDir);
- }
- file = new File(fileDir + "/" + fileName);
- if (!file.exists()) {
- FileUtil.createFileIfNotExists(file.getAbsolutePath());
- BufferedWriter writer = new BufferedWriter(new FileWriter(file));
- writer.write("0");
- writer.flush();
- } else {
- FileReader fileReader = new FileReader(file);
- BufferedReader bufferedReader = new BufferedReader(fileReader);
- String content = bufferedReader.readLine();
- diskCounter = Integer.parseInt(content);
- }
- logger = new FileLogger(this);
- }
-
- public synchronized void increment() {
- memCounter++;
- }
-
- public synchronized void decrement() {
- memCounter--;
- }
-
- public synchronized void setValue(int value) {
- memCounter = value;
- }
-
- public synchronized void sync() throws IOException {
- BufferedWriter writer = new BufferedWriter(new FileWriter(file));
- writer.write("" + memCounter);
- writer.flush();
- }
-
- public synchronized boolean checkIfValueInSync(int expectedValue) throws IOException {
- FileReader fileReader = new FileReader(file);
- BufferedReader bufferedReader = new BufferedReader(fileReader);
- String content = bufferedReader.readLine();
- return content.equals("" + expectedValue);
- }
-
- @Override
- public byte[] getId() {
- return resourceId;
- }
-
- @Override
- public ILogger getLogger() {
- return logger;
- }
-}
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileResourceManager.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileResourceManager.java
deleted file mode 100644
index 5a6e408..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileResourceManager.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.test;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
-import edu.uci.ics.asterix.common.transactions.IResourceManager;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.transaction.management.logging.IResource;
-
-class FileResourceManager implements IResourceManager {
-
- public static final byte id = 100;
- private Map<String, FileResource> transactionalResources = new HashMap<String, FileResource>();
-
- public void registerTransactionalResource(IResource resource) throws ACIDException {
- if (resource instanceof FileResource) {
- if (transactionalResources.get(new String(resource.getId())) == null) {
- transactionalResources.put(new String(resource.getId()), (FileResource) resource);
- }
- } else {
- throw new ACIDException(" invalid resource type :" + resource);
- }
- }
-
- @Override
- public byte getResourceManagerId() {
- return id;
- }
-
- @Override
- public void undo(ILogRecordHelper logRecordHelper, LogicalLogLocator memLSN) throws ACIDException {
- LogRecordInfo logRecordInfo = new LogRecordInfo(logRecordHelper, memLSN);
- FileResource fileManager = transactionalResources.get(logRecordInfo.getResourceId());
- if (fileManager == null) {
- throw new ACIDException(" Un-registered transactional resource :" + logRecordInfo.getResourceId());
- }
- fileManager.setValue(logRecordInfo.getBeforeValue());
- }
-
- @Override
- public void redo(ILogRecordHelper logRecordHelper, LogicalLogLocator memLSN) throws ACIDException {
- LogRecordInfo logRecordInfo = new LogRecordInfo(logRecordHelper, memLSN);
- FileResource fileManager = transactionalResources.get(new String(logRecordInfo.getResourceId()));
- if (fileManager == null) {
- throw new ACIDException(" Un-registered transactional resource :" + logRecordInfo.getResourceId());
- }
- fileManager.setValue(logRecordInfo.getAfterValue());
- }
-
-}
-
-class LogRecordInfo {
-
- byte[] resourceId;
- int beforeValue;
- int afterValue;
-
- public LogRecordInfo(ILogRecordHelper logParser, LogicalLogLocator memLSN) throws ACIDException {
- int logContentBeginPos = logParser.getLogContentBeginPos(memLSN);
- int logContentEndPos = logParser.getLogContentEndPos(memLSN);
- byte[] bufferContent = memLSN.getBuffer().getArray();
- resourceId = new byte[] { bufferContent[logContentBeginPos] };
- String content = new String(bufferContent, logContentBeginPos + resourceId.length, logContentEndPos
- - (logContentBeginPos + resourceId.length));
- beforeValue = Integer.parseInt(content.split(" ")[0]);
- afterValue = Integer.parseInt(content.split(" ")[1]);
- }
-
- public byte[] getResourceId() {
- return resourceId;
- }
-
- public void setResourceId(byte[] resourceId) {
- this.resourceId = resourceId;
- }
-
- public int getAfterValue() {
- return afterValue;
- }
-
- public void setFinalValue(int afterValue) {
- this.afterValue = afterValue;
- }
-
- public int getBeforeValue() {
- return beforeValue;
- }
-
- public void setBeforeValue(int beforeValue) {
- this.beforeValue = beforeValue;
- }
-
-}
\ No newline at end of file
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
deleted file mode 100644
index 8e13356..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/LogRecordReader.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.test;
-
-import java.io.IOException;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.IBuffer;
-import edu.uci.ics.asterix.common.transactions.ILogCursor;
-import edu.uci.ics.asterix.common.transactions.ILogFilter;
-import edu.uci.ics.asterix.common.transactions.ILogManager;
-import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
-import edu.uci.ics.asterix.common.transactions.LogUtil;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogManager;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
-
-public class LogRecordReader {
-
- ILogManager logManager;
-
- public LogRecordReader(TransactionSubsystem factory) throws ACIDException {
- logManager = factory.getLogManager();
- }
-
- public LogRecordReader(ILogManager logManager) {
- this.logManager = logManager;
- }
-
- public void readLogs(long startingLsn) throws IOException, ACIDException {
- ILogRecordHelper parser = logManager.getLogRecordHelper();
- PhysicalLogLocator lsn = new PhysicalLogLocator(startingLsn, logManager);
- ILogCursor logCursor = logManager.readLog(lsn, new ILogFilter() {
- @Override
- public boolean accept(IBuffer buffer, long startOffset, int length) {
- return true;
- }
- });
- LogicalLogLocator currentLogLocator = LogUtil.getDummyLogicalLogLocator(logManager);
- int logCount = 0;
- while (true) {
- boolean logValidity = logCursor.next(currentLogLocator);
- if (logValidity) {
- System.out.println(++logCount + parser.getLogRecordForDisplay(currentLogLocator));
- } else {
- break;
- }
- }
- }
-
- public void readLogRecord(long lsnValue) throws IOException, ACIDException {
- LogicalLogLocator memLSN = null;
- logManager.readLog(lsnValue, memLSN);
- System.out.println(logManager.getLogRecordHelper().getLogRecordForDisplay(memLSN));
- }
-
- /**
- * @param args
- */
- public static void main(String[] args) throws ACIDException, Exception {
- LogManager logManager = new LogManager(null, "nc1");
- LogRecordReader logReader = new LogRecordReader(logManager);
- logReader.readLogs(0);
- }
-
-}
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionRecoverySimulator.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionRecoverySimulator.java
deleted file mode 100644
index 5195658..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionRecoverySimulator.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.test;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-
-public class TransactionRecoverySimulator {
-
- public boolean recoverSystem() throws ACIDException {
- return true;
- }
-
- public static void main(String args[]) {
- TransactionRecoverySimulator recoverySimulator = new TransactionRecoverySimulator();
- }
-}
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
deleted file mode 100644
index 4bebbc4..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionSimulator.java
+++ /dev/null
@@ -1,201 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.test;
-
-import java.io.IOException;
-import java.util.Random;
-
-import edu.uci.ics.asterix.common.config.AsterixPropertiesAccessor;
-import edu.uci.ics.asterix.common.config.AsterixTransactionProperties;
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.common.transactions.DatasetId;
-import edu.uci.ics.asterix.common.transactions.ILockManager;
-import edu.uci.ics.asterix.common.transactions.ILogManager;
-import edu.uci.ics.asterix.common.transactions.ILogger;
-import edu.uci.ics.asterix.common.transactions.IRecoveryManager;
-import edu.uci.ics.asterix.common.transactions.IResourceManager;
-import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
-import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.ITransactionManager;
-import edu.uci.ics.asterix.common.transactions.JobId;
-import edu.uci.ics.asterix.common.transactions.LogUtil;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.transaction.management.logging.IResource;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobIdFactory;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
-
-public class TransactionSimulator {
-
- private ITransactionManager transactionManager;
- private ILogManager logManager;
- private ILockManager lockManager;
- private IRecoveryManager recoveryManager;
- private IResourceManager resourceMgr;
- private ILogger logger;
- private IResource resource;
- private LogicalLogLocator memLSN;
- private TransactionSubsystem transactionProvider;
-
- public TransactionSimulator(IResource resource, IResourceManager resourceMgr) throws ACIDException, AsterixException {
- String id = "nc1";
- transactionProvider = new TransactionSubsystem(id, null, new AsterixTransactionProperties(
- new AsterixPropertiesAccessor()));
- transactionManager = transactionProvider.getTransactionManager();
- logManager = transactionProvider.getLogManager();
- lockManager = transactionProvider.getLockManager();
- recoveryManager = transactionProvider.getRecoveryManager();
- transactionProvider.getTransactionalResourceRepository().registerTransactionalResourceManager(
- resourceMgr.getResourceManagerId(), resourceMgr);
- this.resourceMgr = resourceMgr;
- this.logger = resource.getLogger();
- this.resource = resource;
- memLSN = LogUtil.getDummyLogicalLogLocator(transactionProvider.getLogManager());
- }
-
- public ITransactionContext beginTransaction() throws ACIDException {
- JobId jobId = JobIdFactory.generateJobId();
- return transactionManager.beginTransaction(jobId);
- }
-
- public void executeTransactionOperation(ITransactionContext txnContext, FileResource.CounterOperation operation)
- throws ACIDException {
- // lockManager.lock(txnContext, resourceId, 0);
- ILogManager logManager = transactionProvider.getLogManager();
- int currentValue = ((FileResource) resource).getMemoryCounter();
- int finalValue;
- switch (operation) {
- case INCREMENT:
- finalValue = currentValue + 1;
- int logRecordLength = ((FileLogger) logger).generateLogRecordContent(currentValue, finalValue);
- 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(LogType.UPDATE, txnContext, 1, -1, 1, ResourceType.LSM_BTREE, 0, null, logger, memLSN);
- ((FileResource) resource).decrement();
- break;
- }
-
- }
-
- public void commitTransaction(ITransactionContext context) throws ACIDException {
- transactionManager.commitTransaction(context, new DatasetId(-1), -1);
- }
-
- public void recover() throws ACIDException, IOException {
- recoveryManager.startRecovery(true);
- ((FileResource) resource).sync();
- }
-
- /**
- * @param args
- */
- public static void main(String[] args) throws IOException, ACIDException, AsterixException {
- String fileDir = "testdata";
- String fileName = "counterFile";
- IResource resource = new FileResource(fileDir, fileName);
- FileResourceManager resourceMgr = new FileResourceManager();
- resourceMgr.registerTransactionalResource(resource);
- int existingValue = ((FileResource) resource).getDiskCounter();
-
- TransactionSimulator txnSimulator = new TransactionSimulator(((FileResource) resource), resourceMgr);
- int numTransactions = 2;
- Schedule schedule = new Schedule(numTransactions);
-
- for (int i = 0; i < numTransactions; i++) {
- ITransactionContext context = txnSimulator.beginTransaction();
- txnSimulator.executeTransactionOperation(context, schedule.getOperations()[i]);
- if (schedule.getWillCommit()[i]) {
- txnSimulator.commitTransaction(context);
- }
- }
-
- int finalExpectedValue = existingValue + schedule.getDeltaChange();
- txnSimulator.recover();
- boolean isCorrect = ((FileResource) resource).checkIfValueInSync(finalExpectedValue);
- System.out.println(" Did recovery happen correctly " + isCorrect);
- }
-
-}
-
-class ResourceMgrIds {
-
- public static final byte FileResourceMgrId = 1;
-
-}
-
-class Schedule {
-
- private int numCommittedIncrements;
- private int numCommittedDecrements;
-
- private FileResource.CounterOperation[] operations;
- private Boolean[] willCommit;
-
- public Boolean[] getWillCommit() {
- return willCommit;
- }
-
- private Random random = new Random();
-
- public int getDeltaChange() {
- return numCommittedIncrements - numCommittedDecrements;
- }
-
- public Schedule(int numTransactions) {
- operations = new FileResource.CounterOperation[numTransactions];
- willCommit = new Boolean[numTransactions];
- for (int i = 0; i < numTransactions; i++) {
- willCommit[i] = random.nextBoolean();
- int nextOp = random.nextInt(2);
- FileResource.CounterOperation op = nextOp == 0 ? FileResource.CounterOperation.INCREMENT
- : FileResource.CounterOperation.DECREMENT;
- operations[i] = op;
- if (willCommit[i]) {
- if (op.equals(FileResource.CounterOperation.INCREMENT)) {
- numCommittedIncrements++;
- } else {
- numCommittedDecrements++;
- }
- }
- }
-
- }
-
- public String toString() {
- StringBuilder builder = new StringBuilder();
- for (int i = 0; i < operations.length; i++) {
- builder.append(" operation " + operations[i]);
- if (willCommit[i]) {
- builder.append(" commit ");
- } else {
- builder.append(" abort ");
- }
- }
-
- builder.append(" number of committed increments " + numCommittedIncrements);
- builder.append(" number of committed decrements " + numCommittedDecrements);
- return new String(builder);
- }
-
- public FileResource.CounterOperation[] getOperations() {
- return operations;
- }
-
-}