Deadlock-free locking protocol is enabled
- Added EntityCommitProfiler class in TransactionSubsystem.java file:
This profiler takes a report interval (in seconds) parameter and
reports entity level commit count every report interval (in seconds)
only if IS_PROFILE_MODE is set to true. The profiler runs in a separate
thread. However, the profiler thread doesn't start reporting the count
until the entityCommitCount > 0. The profiler can be used to measure
1) IPS (Inserts Per Second) and
2) IIPS (instantaneous IPS) for the every report interval.
Change-Id: Ie58ae2f519baa53599e99b51bd61ea5f8366dafd
Reviewed-on: https://asterix-gerrit.ics.uci.edu/825
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Murtadha Hubail <hubailmor@gmail.com>
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
index a5fa9a6..814c570 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
@@ -42,7 +42,6 @@
import org.apache.asterix.optimizer.rules.IntroduceDynamicTypeCastForExternalFunctionRule;
import org.apache.asterix.optimizer.rules.IntroduceDynamicTypeCastRule;
import org.apache.asterix.optimizer.rules.IntroduceEnforcedListTypeRule;
-import org.apache.asterix.optimizer.rules.IntroduceInstantLockSearchCallbackRule;
import org.apache.asterix.optimizer.rules.IntroduceMaterializationForInsertWithSelfScanRule;
import org.apache.asterix.optimizer.rules.IntroduceRandomPartitioningFeedComputationRule;
import org.apache.asterix.optimizer.rules.IntroduceRapidFrameFlushProjectAssignRule;
@@ -291,7 +290,6 @@
physicalRewritesAllLevels.add(new ReplaceSinkOpWithCommitOpRule());
physicalRewritesAllLevels.add(new SetAlgebricksPhysicalOperatorsRule());
physicalRewritesAllLevels.add(new SetAsterixPhysicalOperatorsRule());
- physicalRewritesAllLevels.add(new IntroduceInstantLockSearchCallbackRule());
physicalRewritesAllLevels.add(new AddEquivalenceClassForRecordConstructorRule());
physicalRewritesAllLevels.add(new EnforceStructuralPropertiesRule());
physicalRewritesAllLevels.add(new RemoveSortInFeedIngestionRule());
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceInstantLockSearchCallbackRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceInstantLockSearchCallbackRule.java
deleted file mode 100644
index a3d0640..0000000
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceInstantLockSearchCallbackRule.java
+++ /dev/null
@@ -1,156 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you 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 at
- *
- * 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 org.apache.asterix.optimizer.rules;
-
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.asterix.algebra.operators.CommitOperator;
-import org.apache.asterix.algebra.operators.physical.BTreeSearchPOperator;
-import org.apache.asterix.metadata.declared.AqlDataSource;
-import org.apache.asterix.metadata.declared.AqlMetadataImplConfig;
-import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
-import org.apache.asterix.optimizer.rules.am.AccessMethodJobGenParams;
-import org.apache.commons.lang3.mutable.Mutable;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.utils.Triple;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
-import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
-import org.apache.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
-import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.physical.DataSourceScanPOperator;
-import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
-
-public class IntroduceInstantLockSearchCallbackRule implements IAlgebraicRewriteRule {
-
- @Override
- public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
- throws AlgebricksException {
- return false;
- }
-
- private void extractDataSourcesInfo(AbstractLogicalOperator op,
- Map<String, Triple<Integer, LogicalOperatorTag, IPhysicalOperator>> dataSourcesMap) {
-
- for (int i = 0; i < op.getInputs().size(); ++i) {
- AbstractLogicalOperator descendantOp = (AbstractLogicalOperator) op.getInputs().get(i).getValue();
-
- if (descendantOp.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
- UnnestMapOperator unnestMapOp = (UnnestMapOperator) descendantOp;
- ILogicalExpression unnestExpr = unnestMapOp.getExpressionRef().getValue();
- if (unnestExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
- AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) unnestExpr;
- FunctionIdentifier fid = f.getFunctionIdentifier();
- if (fid.equals(AsterixBuiltinFunctions.EXTERNAL_LOOKUP)) {
- return;
- }
- if (!fid.equals(AsterixBuiltinFunctions.INDEX_SEARCH)) {
- throw new IllegalStateException();
- }
- AccessMethodJobGenParams jobGenParams = new AccessMethodJobGenParams();
- jobGenParams.readFromFuncArgs(f.getArguments());
- boolean isPrimaryIndex = jobGenParams.isPrimaryIndex();
- String indexName = jobGenParams.getIndexName();
- if (isPrimaryIndex) {
- if (dataSourcesMap.containsKey(indexName)) {
- ++(dataSourcesMap.get(indexName).first);
- } else {
- dataSourcesMap.put(indexName, new Triple<Integer, LogicalOperatorTag, IPhysicalOperator>(1,
- LogicalOperatorTag.UNNEST_MAP, unnestMapOp.getPhysicalOperator()));
- }
- }
- }
- } else if (descendantOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
- DataSourceScanOperator dataSourceScanOp = (DataSourceScanOperator) descendantOp;
- String datasourceName = ((AqlDataSource) dataSourceScanOp.getDataSource()).getId().getDatasourceName();
- if (dataSourcesMap.containsKey(datasourceName)) {
- ++(dataSourcesMap.get(datasourceName).first);
- } else {
- dataSourcesMap.put(datasourceName, new Triple<Integer, LogicalOperatorTag, IPhysicalOperator>(1,
- LogicalOperatorTag.DATASOURCESCAN, dataSourceScanOp.getPhysicalOperator()));
- }
- }
- extractDataSourcesInfo(descendantOp, dataSourcesMap);
- }
-
- }
-
- private boolean checkIfRuleIsApplicable(AbstractLogicalOperator op) {
- if (op.getPhysicalOperator() == null) {
- return false;
- }
- if (op.getOperatorTag() == LogicalOperatorTag.EXTENSION_OPERATOR) {
- ExtensionOperator extensionOp = (ExtensionOperator) op;
- if (extensionOp.getDelegate() instanceof CommitOperator) {
- return true;
- }
- }
- if (op.getOperatorTag() == LogicalOperatorTag.DISTRIBUTE_RESULT
- || op.getOperatorTag() == LogicalOperatorTag.WRITE_RESULT) {
- return true;
- }
- return false;
- }
-
- @Override
- public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
- throws AlgebricksException {
-
- AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
-
- if (!checkIfRuleIsApplicable(op)) {
- return false;
- }
- Map<String, Triple<Integer, LogicalOperatorTag, IPhysicalOperator>> dataSourcesMap = new HashMap<String, Triple<Integer, LogicalOperatorTag, IPhysicalOperator>>();
- extractDataSourcesInfo(op, dataSourcesMap);
-
- boolean introducedInstantLock = false;
-
- Iterator<Map.Entry<String, Triple<Integer, LogicalOperatorTag, IPhysicalOperator>>> it = dataSourcesMap
- .entrySet().iterator();
- while (it.hasNext()) {
- Entry<String, Triple<Integer, LogicalOperatorTag, IPhysicalOperator>> entry = it.next();
- Triple<Integer, LogicalOperatorTag, IPhysicalOperator> triple = entry.getValue();
- if (triple.first == 1) {
- AqlMetadataImplConfig aqlMetadataImplConfig = new AqlMetadataImplConfig(true);
- if (triple.second == LogicalOperatorTag.UNNEST_MAP) {
- BTreeSearchPOperator pOperator = (BTreeSearchPOperator) triple.third;
- pOperator.setImplConfig(aqlMetadataImplConfig);
- introducedInstantLock = true;
- } else {
- DataSourceScanPOperator pOperator = (DataSourceScanPOperator) triple.third;
- pOperator.setImplConfig(aqlMetadataImplConfig);
- introducedInstantLock = true;
- }
- }
-
- }
- return introducedInstantLock;
- }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixTransactionProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixTransactionProperties.java
index 27b19e2..abc03e2 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixTransactionProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixTransactionProperties.java
@@ -52,6 +52,9 @@
private static final String TXN_LOCK_TIMEOUT_SWEEPTHRESHOLD_KEY = "txn.lock.timeout.sweepthreshold";
private static final int TXN_LOCK_TIMEOUT_SWEEPTHRESHOLD_DEFAULT = 10000; // 10s
+ private static final String TXN_COMMIT_PROFILER_REPORT_INTERVAL_KEY = "txn.commitprofiler.reportinterval";
+ private static final int TXN_COMMIT_PROFILER_REPORT_INTERVAL_DEFAULT = 5; // 5 seconds
+
public AsterixTransactionProperties(AsterixPropertiesAccessor accessor) {
super(accessor);
}
@@ -114,4 +117,9 @@
PropertyInterpreters.getIntegerPropertyInterpreter());
}
+ public int getCommitProfilerReportInterval() {
+ return accessor.getProperty(TXN_COMMIT_PROFILER_REPORT_INTERVAL_KEY,
+ TXN_COMMIT_PROFILER_REPORT_INTERVAL_DEFAULT, PropertyInterpreters.getIntegerPropertyInterpreter());
+ }
+
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
index d53236b..5445b11 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
@@ -28,6 +28,7 @@
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
@@ -44,6 +45,20 @@
private AbstractLSMIndex lsmIndex;
private int i = 0;
+ /**
+ * The following three variables are used to keep track of the information regarding flushing partial frame such as
+ * 1. whether there was a partial frame flush for the current frame,
+ * ==> captured in flushedPartialTuples variabl
+ * 2. the last flushed tuple index in the frame if there was a partial frame flush,
+ * ==> captured in lastFlushedTupleIdx variable
+ * 3. the current tuple index the frame, where this operator is working on the current tuple.
+ * ==> captured in currentTupleIdx variable
+ * These variables are reset for each frame, i.e., whenever nextFrame() is called, these variables are reset.
+ */
+ private boolean flushedPartialTuples;
+ private int currentTupleIdx;
+ private int lastFlushedTupleIdx;
+
public boolean isPrimary() {
return isPrimary;
}
@@ -60,13 +75,14 @@
RecordDescriptor inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
accessor = new FrameTupleAccessor(inputRecDesc);
writeBuffer = new VSizeFrame(ctx);
+ appender = new FrameTupleAppender(writeBuffer);
indexHelper.open();
lsmIndex = (AbstractLSMIndex) indexHelper.getIndexInstance();
try {
writer.open();
modCallback = opDesc.getModificationOpCallbackFactory().createModificationOperationCallback(
indexHelper.getResourcePath(), indexHelper.getResourceID(), indexHelper.getResourcePartition(),
- lsmIndex, ctx);
+ lsmIndex, ctx, this);
indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
ITupleFilterFactory tupleFilterFactory = opDesc.getTupleFilterFactory();
if (tupleFilterFactory != null) {
@@ -83,11 +99,15 @@
@Override
public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ currentTupleIdx = 0;
+ lastFlushedTupleIdx = 0;
+ flushedPartialTuples = false;
+
accessor.reset(buffer);
ILSMIndexAccessor lsmAccessor = (ILSMIndexAccessor) indexAccessor;
int tupleCount = accessor.getTupleCount();
try {
- for (; i < tupleCount; i++) {
+ for (; i < tupleCount; i++, currentTupleIdx++) {
if (tupleFilter != null) {
frameTuple.reset(accessor, i);
if (!tupleFilter.accept(frameTuple)) {
@@ -120,12 +140,34 @@
FrameDataException fde = new FrameDataException(i, th);
throw fde;
}
+
writeBuffer.ensureFrameSize(buffer.capacity());
- FrameUtils.copyAndFlip(buffer, writeBuffer.getBuffer());
- FrameUtils.flushFrame(writeBuffer.getBuffer(), writer);
+ if (flushedPartialTuples) {
+ flushPartialFrame();
+ } else {
+ FrameUtils.copyAndFlip(buffer, writeBuffer.getBuffer());
+ FrameUtils.flushFrame(writeBuffer.getBuffer(), writer);
+ }
i = 0;
}
+ /**
+ * flushes tuples in a frame from lastFlushedTupleIdx(inclusive) to currentTupleIdx(exclusive)
+ */
+ @Override
+ public void flushPartialFrame() throws HyracksDataException {
+ if (lastFlushedTupleIdx == currentTupleIdx) {
+ //nothing to flush
+ return;
+ }
+ for (int i = lastFlushedTupleIdx; i < currentTupleIdx; i++) {
+ FrameUtils.appendToWriter(writer, appender, accessor, i);
+ }
+ appender.write(writer, true);
+ lastFlushedTupleIdx = currentTupleIdx;
+ flushedPartialTuples = true;
+ }
+
@Override
public void close() throws HyracksDataException {
if (lsmIndex != null) {
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/AbstractOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/AbstractOperationCallbackFactory.java
index 6f54918..d1d869e 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/AbstractOperationCallbackFactory.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/AbstractOperationCallbackFactory.java
@@ -22,7 +22,6 @@
import java.io.Serializable;
import org.apache.asterix.common.context.ITransactionSubsystemProvider;
-import org.apache.asterix.common.transactions.JobId;
public abstract class AbstractOperationCallbackFactory implements Serializable {
private static final long serialVersionUID = 1L;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILockManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILockManager.java
index 7909622..495af9a 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILockManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILockManager.java
@@ -68,7 +68,6 @@
* @param lockMode
* @param txnContext
* @throws ACIDException
- * TODO
* @return
*/
public void unlock(DatasetId datasetId, int entityHashValue, byte lockMode, ITransactionContext txnContext)
@@ -80,7 +79,6 @@
* @param datasetId
* @param entityHashValue
* @param lockMode
- * TODO
* @param context
* @return
* @throws ACIDException
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogRecord.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogRecord.java
index 7e27c54..b86aebe 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogRecord.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogRecord.java
@@ -35,6 +35,7 @@
public static final int ENTITY_COMMIT_LOG_BASE_SIZE = 30;
public static final int UPDATE_LOG_BASE_SIZE = 59;
public static final int FLUSH_LOG_SIZE = 18;
+ public static final int WAIT_LOG_SIZE = 14;
public LogRecord.RECORD_STATUS readLogRecord(ByteBuffer buffer);
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java
index 003d4c6..2eb8244 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java
@@ -65,6 +65,9 @@
* 3) UPDATE: (Header1(6) + Header2(16) + + Header3(20) + Body(9) + Tail(8)) + PKValueSize + NewValueSize
* --> UPDATE_LOG_BASE_SIZE = 59
* 4) FLUSH: 18 bytes (Header1(6) + DatasetId(4) + Tail(8))
+ * 5) WAIT_LOG_SIZE: 14 bytes (Header1(6) + Tail(8))
+ * --> WAIT_LOG only requires LogType Field, but in order to conform the log reader protocol
+ * it also includes LogSource and JobId fields.
*/
public class LogRecord implements ILogRecord {
@@ -265,7 +268,16 @@
logType = buffer.get();
jobId = buffer.getInt();
- if (logType != LogType.FLUSH) {
+ if (logType == LogType.FLUSH) {
+ if (buffer.remaining() < DatasetId.BYTES) {
+ return RECORD_STATUS.TRUNCATED;
+ }
+ datasetId = buffer.getInt();
+ resourceId = 0l;
+ computeAndSetLogSize();
+ } else if (logType == LogType.WAIT) {
+ computeAndSetLogSize();
+ } else {
if (logType == LogType.JOB_COMMIT || logType == LogType.ABORT) {
datasetId = -1;
PKHashValue = -1;
@@ -306,15 +318,8 @@
} else {
computeAndSetLogSize();
}
- } else {
- computeAndSetLogSize();
- if (buffer.remaining() < DatasetId.BYTES) {
- return RECORD_STATUS.TRUNCATED;
- }
- datasetId = buffer.getInt();
- resourceId = 0l;
- computeAndSetLogSize();
}
+
return RECORD_STATUS.OK;
}
@@ -412,6 +417,9 @@
case LogType.FLUSH:
logSize = FLUSH_LOG_SIZE;
break;
+ case LogType.WAIT:
+ logSize = WAIT_LOG_SIZE;
+ break;
default:
throw new IllegalStateException("Unsupported Log Type");
}
@@ -425,7 +433,7 @@
builder.append(" LogType : ").append(LogType.toString(logType));
builder.append(" LogSize : ").append(logSize);
builder.append(" JobId : ").append(jobId);
- if (logType == LogType.ENTITY_COMMIT || logType == LogType.UPDATE) {
+ if (logType == LogType.ENTITY_COMMIT || logType == LogType.UPSERT_ENTITY_COMMIT || logType == LogType.UPDATE) {
builder.append(" DatasetId : ").append(datasetId);
builder.append(" ResourcePartition : ").append(resourcePartition);
builder.append(" PKHashValue : ").append(PKHashValue);
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogType.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogType.java
index d6d2657..714b8f7 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogType.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogType.java
@@ -26,6 +26,7 @@
public static final byte ABORT = 3;
public static final byte FLUSH = 4;
public static final byte UPSERT_ENTITY_COMMIT = 5;
+ public static final byte WAIT = 6;
private static final String STRING_UPDATE = "UPDATE";
private static final String STRING_JOB_COMMIT = "JOB_COMMIT";
@@ -33,6 +34,7 @@
private static final String STRING_ABORT = "ABORT";
private static final String STRING_FLUSH = "FLUSH";
private static final String STRING_UPSERT_ENTITY_COMMIT = "UPSERT_ENTITY_COMMIT";
+ private static final String STRING_WAIT = "WAIT";
private static final String STRING_INVALID_LOG_TYPE = "INVALID_LOG_TYPE";
@@ -50,6 +52,8 @@
return STRING_FLUSH;
case LogType.UPSERT_ENTITY_COMMIT:
return STRING_UPSERT_ENTITY_COMMIT;
+ case LogType.WAIT:
+ return STRING_WAIT;
default:
return STRING_INVALID_LOG_TYPE;
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFileIndexAccessor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFileIndexAccessor.java
index 5d98961..667bae7 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFileIndexAccessor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFileIndexAccessor.java
@@ -92,7 +92,7 @@
// create the accessor and the cursor using the passed version
ISearchOperationCallback searchCallback = opDesc.getSearchOpCallbackFactory()
- .createSearchOperationCallback(indexDataflowHelper.getResourceID(), ctx);
+ .createSearchOperationCallback(indexDataflowHelper.getResourceID(), ctx, null);
fileIndexAccessor = index.createAccessor(searchCallback, indexDataflowHelper.getVersion());
fileIndexSearchCursor = fileIndexAccessor.createSearchCursor(false);
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorNodePushable.java
index 0513f9c..9435387 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorNodePushable.java
@@ -73,7 +73,7 @@
dos = tb.getDataOutput();
appender = new FrameTupleAppender(new VSizeFrame(ctx));
ISearchOperationCallback searchCallback = opDesc.getSearchOpCallbackFactory()
- .createSearchOperationCallback(indexHelper.getResourceID(), ctx);
+ .createSearchOperationCallback(indexHelper.getResourceID(), ctx, null);
// The next line is the reason we override this method
indexAccessor = externalIndex.createAccessor(searchCallback, dataFlowHelper.getTargetVersion());
cursor = createCursor();
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorNodePushable.java
index 75cc1bf..81e6b17 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorNodePushable.java
@@ -72,7 +72,7 @@
dos = tb.getDataOutput();
appender = new FrameTupleAppender(new VSizeFrame(ctx));
ISearchOperationCallback searchCallback = opDesc.getSearchOpCallbackFactory()
- .createSearchOperationCallback(indexHelper.getResourceID(), ctx);
+ .createSearchOperationCallback(indexHelper.getResourceID(), ctx, null);
// The next line is the reason we override this method
indexAccessor = rTreeIndex.createAccessor(searchCallback, rTreeDataflowHelper.getTargetVersion());
cursor = createCursor();
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
index 542c12d..54ec084 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
@@ -79,7 +79,6 @@
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexModificationOperationCallback;
import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexModificationOperationCallback;
import org.apache.asterix.transaction.management.service.transaction.DatasetIdFactory;
import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
@@ -306,15 +305,13 @@
IMetadataIndex metadataIndex, ILSMIndex lsmIndex, IndexOperation indexOp) throws ACIDException {
ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
- if (metadataIndex.isPrimaryIndex()) {
- return new PrimaryIndexModificationOperationCallback(metadataIndex.getDatasetId().getId(),
- metadataIndex.getPrimaryKeyIndexes(), txnCtx, transactionSubsystem.getLockManager(),
- transactionSubsystem, resourceId, metadataStoragePartition, ResourceType.LSM_BTREE, indexOp);
- } else {
- return new SecondaryIndexModificationOperationCallback(metadataIndex.getDatasetId().getId(),
- metadataIndex.getPrimaryKeyIndexes(), txnCtx, transactionSubsystem.getLockManager(),
- transactionSubsystem, resourceId, metadataStoragePartition, ResourceType.LSM_BTREE, indexOp);
- }
+ // Regardless of the index type (primary or secondary index), secondary index modification callback is given
+ // This is still correct since metadata index operation doesn't require any lock from ConcurrentLockMgr and
+ // The difference between primaryIndexModCallback and secondaryIndexModCallback is that primary index requires
+ // locks and secondary index doesn't.
+ return new SecondaryIndexModificationOperationCallback(metadataIndex.getDatasetId().getId(),
+ metadataIndex.getPrimaryKeyIndexes(), txnCtx, transactionSubsystem.getLockManager(),
+ transactionSubsystem, resourceId, metadataStoragePartition, ResourceType.LSM_BTREE, indexOp);
}
@Override
@@ -980,10 +977,8 @@
try {
while (rangeCursor.hasNext()) {
rangeCursor.next();
- sb.append(TupleUtils.printTuple(rangeCursor.getTuple(),
- new ISerializerDeserializer[] {
- AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(
- BuiltinType.ASTRING),
+ sb.append(TupleUtils.printTuple(rangeCursor.getTuple(), new ISerializerDeserializer[] {
+ AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING),
AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING),
AqlSerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(BuiltinType.ASTRING) }));
@@ -1000,7 +995,7 @@
private <ResultType> void searchIndex(JobId jobId, IMetadataIndex index, ITupleReference searchKey,
IValueExtractor<ResultType> valueExtractor, List<ResultType> results)
- throws MetadataException, IndexException, IOException {
+ throws MetadataException, IndexException, IOException {
IBinaryComparatorFactory[] comparatorFactories = index.getKeyBinaryComparatorFactory();
String resourceName = index.getFile().toString();
IIndex indexInstance = datasetLifecycleManager.getIndex(resourceName);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
index 7070a88..ec2e692 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
@@ -148,8 +148,8 @@
* @throws MetadataException
* For example, if the dataverse does not exist. RemoteException
*/
- public List<Dataset> getDataverseDatasets(JobId jobId, String dataverseName) throws MetadataException,
- RemoteException;
+ public List<Dataset> getDataverseDatasets(JobId jobId, String dataverseName)
+ throws MetadataException, RemoteException;
/**
* Deletes the dataverse with given name, and all it's associated datasets,
@@ -194,8 +194,8 @@
* For example, if the dataset does not exist.
* @throws RemoteException
*/
- public Dataset getDataset(JobId jobId, String dataverseName, String datasetName) throws MetadataException,
- RemoteException;
+ public Dataset getDataset(JobId jobId, String dataverseName, String datasetName)
+ throws MetadataException, RemoteException;
/**
* Retrieves all indexes of a dataset, acquiring local locks on behalf of
@@ -229,8 +229,8 @@
* For example, if the dataset and/or dataverse does not exist.
* @throws RemoteException
*/
- public void dropDataset(JobId jobId, String dataverseName, String datasetName) throws MetadataException,
- RemoteException;
+ public void dropDataset(JobId jobId, String dataverseName, String datasetName)
+ throws MetadataException, RemoteException;
/**
* Inserts an index into the metadata, acquiring local locks on behalf of
@@ -313,8 +313,8 @@
* For example, if the datatype does not exist.
* @throws RemoteException
*/
- public Datatype getDatatype(JobId jobId, String dataverseName, String datatypeName) throws MetadataException,
- RemoteException;
+ public Datatype getDatatype(JobId jobId, String dataverseName, String datatypeName)
+ throws MetadataException, RemoteException;
/**
* Deletes the given datatype in given dataverse, acquiring local locks on
@@ -331,8 +331,8 @@
* deleted.
* @throws RemoteException
*/
- public void dropDatatype(JobId jobId, String dataverseName, String datatypeName) throws MetadataException,
- RemoteException;
+ public void dropDatatype(JobId jobId, String dataverseName, String datatypeName)
+ throws MetadataException, RemoteException;
/**
* Inserts a node group, acquiring local locks on behalf of the given
@@ -400,8 +400,8 @@
* @throws MetadataException
* @throws RemoteException
*/
- public Function getFunction(JobId jobId, FunctionSignature functionSignature) throws MetadataException,
- RemoteException;
+ public Function getFunction(JobId jobId, FunctionSignature functionSignature)
+ throws MetadataException, RemoteException;
/**
* Deletes a function, acquiring local locks on behalf of the given
@@ -416,8 +416,8 @@
* group to be deleted.
* @throws RemoteException
*/
- public void dropFunction(JobId jobId, FunctionSignature functionSignature) throws MetadataException,
- RemoteException;
+ public void dropFunction(JobId jobId, FunctionSignature functionSignature)
+ throws MetadataException, RemoteException;
/**
* @param jobId
@@ -438,8 +438,8 @@
* @throws MetadataException
* @throws RemoteException
*/
- public List<Function> getDataverseFunctions(JobId jobId, String dataverseName) throws MetadataException,
- RemoteException;
+ public List<Function> getDataverseFunctions(JobId jobId, String dataverseName)
+ throws MetadataException, RemoteException;
/**
* @param ctx
@@ -448,8 +448,8 @@
* @throws MetadataException
* @throws RemoteException
*/
- public List<DatasourceAdapter> getDataverseAdapters(JobId jobId, String dataverseName) throws MetadataException,
- RemoteException;
+ public List<DatasourceAdapter> getDataverseAdapters(JobId jobId, String dataverseName)
+ throws MetadataException, RemoteException;
/**
* @param jobId
@@ -475,8 +475,8 @@
* if the adapter does not exists.
* @throws RemoteException
*/
- public void dropAdapter(JobId jobId, String dataverseName, String adapterName) throws MetadataException,
- RemoteException;
+ public void dropAdapter(JobId jobId, String dataverseName, String adapterName)
+ throws MetadataException, RemoteException;
/**
* @param jobId
@@ -495,8 +495,8 @@
* @throws MetadataException
* @throws RemoteException
*/
- public void addCompactionPolicy(JobId jobId, CompactionPolicy compactionPolicy) throws MetadataException,
- RemoteException;
+ public void addCompactionPolicy(JobId jobId, CompactionPolicy compactionPolicy)
+ throws MetadataException, RemoteException;
/**
* @param jobId
@@ -506,8 +506,8 @@
* @throws MetadataException
* @throws RemoteException
*/
- public CompactionPolicy getCompactionPolicy(JobId jobId, String dataverse, String policy) throws MetadataException,
- RemoteException;
+ public CompactionPolicy getCompactionPolicy(JobId jobId, String dataverse, String policy)
+ throws MetadataException, RemoteException;
/**
* @param jobId
@@ -550,7 +550,6 @@
*/
public void dropFeed(JobId jobId, String dataverse, String feedName) throws MetadataException, RemoteException;
-
/**
* @param jobId
* @param feedPolicy
@@ -567,9 +566,8 @@
* @throws MetadataException
* @throws RemoteException
*/
- public FeedPolicyEntity getFeedPolicy(JobId jobId, String dataverse, String policy) throws MetadataException,
- RemoteException;
-
+ public FeedPolicyEntity getFeedPolicy(JobId jobId, String dataverse, String policy)
+ throws MetadataException, RemoteException;
/**
* Removes a library , acquiring local locks on behalf of the given
@@ -584,8 +582,8 @@
* if the library does not exists.
* @throws RemoteException
*/
- public void dropLibrary(JobId jobId, String dataverseName, String libraryName) throws MetadataException,
- RemoteException;
+ public void dropLibrary(JobId jobId, String dataverseName, String libraryName)
+ throws MetadataException, RemoteException;
/**
* Adds a library, acquiring local locks on behalf of the given
@@ -612,8 +610,8 @@
* @throws MetadataException
* @throws RemoteException
*/
- public Library getLibrary(JobId jobId, String dataverseName, String libraryName) throws MetadataException,
- RemoteException;
+ public Library getLibrary(JobId jobId, String dataverseName, String libraryName)
+ throws MetadataException, RemoteException;
/**
* Retireve libraries installed in a given dataverse.
@@ -626,8 +624,8 @@
* @throws MetadataException
* @throws RemoteException
*/
- public List<Library> getDataverseLibraries(JobId jobId, String dataverseName) throws MetadataException,
- RemoteException;
+ public List<Library> getDataverseLibraries(JobId jobId, String dataverseName)
+ throws MetadataException, RemoteException;
/**
* @param jobId
@@ -648,8 +646,8 @@
* @throws RemoteException
* @throws MetadataException
*/
- public void dropFeedPolicy(JobId jobId, String dataverseName, String policyName) throws MetadataException,
- RemoteException;
+ public void dropFeedPolicy(JobId jobId, String dataverseName, String policyName)
+ throws MetadataException, RemoteException;
/**
* @param jobId
@@ -658,8 +656,8 @@
* @throws MetadataException
* @throws RemoteException
*/
- public List<FeedPolicyEntity> getDataversePolicies(JobId jobId, String dataverse) throws MetadataException,
- RemoteException;
+ public List<FeedPolicyEntity> getDataversePolicies(JobId jobId, String dataverse)
+ throws MetadataException, RemoteException;
/**
* @param jobId
@@ -731,7 +729,6 @@
public ExternalFile getExternalFile(JobId jobId, String dataverseName, String datasetName, Integer fileNumber)
throws MetadataException, RemoteException;
-
/**
* update an existing dataset in the metadata, acquiring local locks on behalf
* of the given transaction id.
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java
index 9fe72a4..5e2e227 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java
@@ -110,7 +110,6 @@
import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexInstantSearchOperationCallbackFactory;
import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexModificationOperationCallbackFactory;
import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexOperationTrackerProvider;
-import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexSearchOperationCallbackFactory;
import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexModificationOperationCallbackFactory;
import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexSearchOperationCallbackFactory;
@@ -730,17 +729,15 @@
primaryKeyFields[i] = i;
}
- AqlMetadataImplConfig aqlMetadataImplConfig = (AqlMetadataImplConfig) implConfig;
ITransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
- if (aqlMetadataImplConfig != null && aqlMetadataImplConfig.isInstantLock()) {
- searchCallbackFactory = temp ? NoOpOperationCallbackFactory.INSTANCE
- : new PrimaryIndexInstantSearchOperationCallbackFactory(jobId, datasetId, primaryKeyFields,
- txnSubsystemProvider, ResourceType.LSM_BTREE);
- } else {
- searchCallbackFactory = temp ? NoOpOperationCallbackFactory.INSTANCE
- : new PrimaryIndexSearchOperationCallbackFactory(jobId, datasetId, primaryKeyFields,
- txnSubsystemProvider, ResourceType.LSM_BTREE);
- }
+
+ /**
+ * Due to the read-committed isolation level,
+ * we may acquire very short duration lock(i.e., instant lock) for readers.
+ */
+ searchCallbackFactory = temp ? NoOpOperationCallbackFactory.INSTANCE
+ : new PrimaryIndexInstantSearchOperationCallbackFactory(jobId, datasetId, primaryKeyFields,
+ txnSubsystemProvider, ResourceType.LSM_BTREE);
}
Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
.getMergePolicyFactory(dataset, mdTxnCtx);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/AsterixLSMPrimaryUpsertOperatorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/AsterixLSMPrimaryUpsertOperatorNodePushable.java
index 83c6e34..7785978 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/AsterixLSMPrimaryUpsertOperatorNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/AsterixLSMPrimaryUpsertOperatorNodePushable.java
@@ -128,10 +128,10 @@
appender = new FrameTupleAppender(new VSizeFrame(ctx), true);
modCallback = opDesc.getModificationOpCallbackFactory().createModificationOperationCallback(
indexHelper.getResourcePath(), indexHelper.getResourceID(), indexHelper.getResourcePartition(),
- index, ctx);
+ index, ctx, this);
indexAccessor = index.createAccessor(modCallback, opDesc.getSearchOpCallbackFactory()
- .createSearchOperationCallback(indexHelper.getResourceID(), ctx));
+ .createSearchOperationCallback(indexHelper.getResourceID(), ctx, this));
cursor = indexAccessor.createSearchCursor(false);
frameTuple = new FrameTupleReference();
IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext()
@@ -191,6 +191,7 @@
//TODO: use tryDelete/tryInsert in order to prevent deadlocks
@Override
public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+
accessor.reset(buffer);
LSMTreeIndexAccessor lsmAccessor = (LSMTreeIndexAccessor) indexAccessor;
int tupleCount = accessor.getTupleCount();
@@ -239,6 +240,15 @@
}
}
+ /**
+ * Flushes tuples (which have already been written to tuple appender's buffer in writeOutput() method)
+ * to the next operator/consumer.
+ */
+ @Override
+ public void flushPartialFrame() throws HyracksDataException {
+ appender.write(writer, true);
+ }
+
private ITupleReference getPrevTupleWithFilter(ITupleReference prevTuple) throws IOException, AsterixException {
prevRecWithPKWithFilterValue.reset();
for (int i = 0; i < prevTuple.getFieldCount(); i++) {
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/LockThenSearchOperationCallback.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/LockThenSearchOperationCallback.java
index 49cea94..ef3b218 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/LockThenSearchOperationCallback.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/LockThenSearchOperationCallback.java
@@ -20,18 +20,40 @@
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.transactions.AbstractOperationCallback;
-import org.apache.asterix.common.transactions.ILockManager;
+import org.apache.asterix.common.transactions.ILogManager;
+import org.apache.asterix.common.transactions.ILogRecord;
import org.apache.asterix.common.transactions.ITransactionContext;
+import org.apache.asterix.common.transactions.ITransactionSubsystem;
+import org.apache.asterix.common.transactions.LogRecord;
+import org.apache.asterix.common.transactions.LogSource;
+import org.apache.asterix.common.transactions.LogType;
import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
import org.apache.hyracks.storage.am.common.api.ISearchOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.dataflow.LSMIndexInsertUpdateDeleteOperatorNodePushable;
public class LockThenSearchOperationCallback extends AbstractOperationCallback implements ISearchOperationCallback {
- public LockThenSearchOperationCallback(int datasetId, int[] entityIdFields, ILockManager lockManager,
- ITransactionContext txnCtx) {
- super(datasetId, entityIdFields, txnCtx, lockManager);
+ /**
+ * variables used for deadlock-free locking protocol
+ */
+ private final LSMIndexInsertUpdateDeleteOperatorNodePushable operatorNodePushable;
+ private final ILogManager logManager;
+ private final ILogRecord logRecord;
+
+ public LockThenSearchOperationCallback(int datasetId, int[] entityIdFields, ITransactionSubsystem txnSubsystem,
+ ITransactionContext txnCtx, IOperatorNodePushable operatorNodePushable) {
+ super(datasetId, entityIdFields, txnCtx, txnSubsystem.getLockManager());
+ this.operatorNodePushable = (LSMIndexInsertUpdateDeleteOperatorNodePushable) operatorNodePushable;
+ this.logManager = txnSubsystem.getLogManager();
+ this.logRecord = new LogRecord();
+ logRecord.setTxnCtx(txnCtx);
+ logRecord.setLogSource(LogSource.LOCAL);
+ logRecord.setLogType(LogType.WAIT);
+ logRecord.setJobId(txnCtx.getJobId().getId());
+ logRecord.computeAndSetLogSize();
}
@Override
@@ -55,9 +77,49 @@
public void before(ITupleReference tuple) throws HyracksDataException {
int pkHash = computePrimaryKeyHashValue(tuple, primaryKeyFields);
try {
- lockManager.lock(datasetId, pkHash, LockMode.X, txnCtx);
+ if (operatorNodePushable != null) {
+
+ /**********************************************************************************
+ * In order to achieve deadlock-free locking protocol during any write (insert/delete/upsert) operations,
+ * the following logic is implemented.
+ * See https://cwiki.apache.org/confluence/display/ASTERIXDB/Deadlock-Free+Locking+Protocol for more details.
+ * 1. for each entry in a frame
+ * 2. returnValue = tryLock() for an entry
+ * 3. if returnValue == false
+ * 3-1. flush all entries (which already acquired locks) to the next operator
+ * : this will make all those entries reach commit operator so that corresponding commit logs will be created.
+ * 3-2. create a WAIT log and wait until logFlusher thread will flush the WAIT log and gives notification
+ * : this notification guarantees that all locks acquired by this transactor (or all locks acquired for the entries)
+ * were released.
+ * 3-3. acquire lock using lock() instead of tryLock() for the failed entry
+ * : we know for sure this lock call will not cause deadlock since the transactor doesn't hold any other locks.
+ * 4. create an update log and insert the entry
+ * From the above logic, step 2 and 3 are implemented in this before() method.
+ **********************/
+
+ //release all locks held by this actor (which is a thread) by flushing partial frame.
+ boolean tryLockSucceed = lockManager.tryLock(datasetId, pkHash, LockMode.X, txnCtx);
+ if (!tryLockSucceed) {
+ //flush entries which have been inserted already to release locks hold by them
+ operatorNodePushable.flushPartialFrame();
+
+ //create WAIT log and wait until the WAIT log is flushed and notified by LogFlusher thread
+ logWait();
+
+ //acquire lock
+ lockManager.lock(datasetId, pkHash, LockMode.X, txnCtx);
+ }
+
+ } else {
+ //operatorNodePushable can be null when metadata node operation is executed
+ lockManager.lock(datasetId, pkHash, LockMode.X, txnCtx);
+ }
} catch (ACIDException e) {
throw new HyracksDataException(e);
}
}
+
+ private void logWait() throws ACIDException {
+ logManager.log(logRecord);
+ }
}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/LockThenSearchOperationCallbackFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/LockThenSearchOperationCallbackFactory.java
index 6bfb6cd..0d65c16 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/LockThenSearchOperationCallbackFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/LockThenSearchOperationCallbackFactory.java
@@ -25,6 +25,7 @@
import org.apache.asterix.common.transactions.ITransactionSubsystem;
import org.apache.asterix.common.transactions.JobId;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.common.api.ISearchOperationCallback;
import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
@@ -40,13 +41,13 @@
}
@Override
- public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx)
- throws HyracksDataException {
+ public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx,
+ IOperatorNodePushable operatorNodePushable) throws HyracksDataException {
ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
try {
ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
- return new LockThenSearchOperationCallback(datasetId, primaryKeyFields, txnSubsystem.getLockManager(),
- txnCtx);
+ return new LockThenSearchOperationCallback(datasetId, primaryKeyFields, txnSubsystem, txnCtx,
+ operatorNodePushable);
} catch (ACIDException e) {
throw new HyracksDataException(e);
}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallbackFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallbackFactory.java
index 9b14807..ba97287 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallbackFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallbackFactory.java
@@ -26,6 +26,7 @@
import org.apache.asterix.common.transactions.ITransactionSubsystem;
import org.apache.asterix.common.transactions.JobId;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.common.api.ISearchOperationCallback;
import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
@@ -41,7 +42,7 @@
}
@Override
- public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx)
+ public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx, IOperatorNodePushable operatorNodePushable)
throws HyracksDataException {
ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
try {
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java
index 780f294..4bde490 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java
@@ -19,11 +19,14 @@
package org.apache.asterix.transaction.management.opcallbacks;
+import org.apache.asterix.common.dataflow.AsterixLSMInsertDeleteOperatorNodePushable;
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.transactions.ILockManager;
import org.apache.asterix.common.transactions.ITransactionContext;
import org.apache.asterix.common.transactions.ITransactionSubsystem;
+import org.apache.asterix.common.transactions.LogType;
import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
import org.apache.hyracks.storage.am.common.api.IModificationOperationCallback;
@@ -36,18 +39,57 @@
public class PrimaryIndexModificationOperationCallback extends AbstractIndexModificationOperationCallback
implements IModificationOperationCallback {
+ private final AsterixLSMInsertDeleteOperatorNodePushable operatorNodePushable;
+
public PrimaryIndexModificationOperationCallback(int datasetId, int[] primaryKeyFields, ITransactionContext txnCtx,
ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId, int resourcePartition,
- byte resourceType, IndexOperation indexOp) {
+ byte resourceType, IndexOperation indexOp, IOperatorNodePushable operatorNodePushable) {
super(datasetId, primaryKeyFields, txnCtx, lockManager, txnSubsystem, resourceId, resourcePartition,
resourceType, indexOp);
+ this.operatorNodePushable = (AsterixLSMInsertDeleteOperatorNodePushable) operatorNodePushable;
}
@Override
public void before(ITupleReference tuple) throws HyracksDataException {
int pkHash = computePrimaryKeyHashValue(tuple, primaryKeyFields);
try {
- lockManager.lock(datasetId, pkHash, LockMode.X, txnCtx);
+ if (operatorNodePushable != null) {
+
+ /**********************************************************************************
+ * In order to achieve deadlock-free locking protocol during any write (insert/delete/upsert) operations,
+ * the following logic is implemented.
+ * See https://cwiki.apache.org/confluence/display/ASTERIXDB/Deadlock-Free+Locking+Protocol for more details.
+ * 1. for each entry in a frame
+ * 2. returnValue = tryLock() for an entry
+ * 3. if returnValue == false
+ * 3-1. flush all entries (which already acquired locks) to the next operator
+ * : this will make all those entries reach commit operator so that corresponding commit logs will be created.
+ * 3-2. create a WAIT log and wait until logFlusher thread will flush the WAIT log and gives notification
+ * : this notification guarantees that all locks acquired by this transactor (or all locks acquired for the entries)
+ * were released.
+ * 3-3. acquire lock using lock() instead of tryLock() for the failed entry
+ * : we know for sure this lock call will not cause deadlock since the transactor doesn't hold any other locks.
+ * 4. create an update log and insert the entry
+ * From the above logic, step 2 and 3 are implemented in this before() method.
+ **********************/
+
+ //release all locks held by this actor (which is a thread) by flushing partial frame.
+ boolean tryLockSucceed = lockManager.tryLock(datasetId, pkHash, LockMode.X, txnCtx);
+ if (!tryLockSucceed) {
+ //flush entries which have been inserted already to release locks hold by them
+ operatorNodePushable.flushPartialFrame();
+
+ //create WAIT log and wait until the WAIT log is flushed and notified by LogFlusher thread
+ logWait();
+
+ //acquire lock
+ lockManager.lock(datasetId, pkHash, LockMode.X, txnCtx);
+ }
+
+ } else {
+ //operatorNodePushable can be null when metadata node operation is executed
+ lockManager.lock(datasetId, pkHash, LockMode.X, txnCtx);
+ }
} catch (ACIDException e) {
throw new HyracksDataException(e);
}
@@ -62,4 +104,10 @@
throw new HyracksDataException(e);
}
}
+
+ private void logWait() throws ACIDException {
+ logRecord.setLogType(LogType.WAIT);
+ logRecord.computeAndSetLogSize();
+ txnSubsystem.getLogManager().log(logRecord);
+ }
}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
index db68b26..c406812 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
@@ -27,6 +27,7 @@
import org.apache.asterix.common.transactions.ITransactionSubsystem;
import org.apache.asterix.common.transactions.JobId;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManager;
import org.apache.hyracks.storage.am.common.api.IModificationOperationCallback;
@@ -51,7 +52,8 @@
@Override
public IModificationOperationCallback createModificationOperationCallback(String resourcePath, long resourceId,
- int resourcePartition, Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
+ int resourcePartition, Object resource, IHyracksTaskContext ctx, IOperatorNodePushable operatorNodePushable)
+ throws HyracksDataException {
ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
@@ -65,7 +67,7 @@
ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
IModificationOperationCallback modCallback = new PrimaryIndexModificationOperationCallback(datasetId,
primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId,
- resourcePartition, resourceType, indexOp);
+ resourcePartition, resourceType, indexOp, operatorNodePushable);
txnCtx.registerIndexAndCallback(resourceId, index, (AbstractOperationCallback) modCallback, true);
return modCallback;
} catch (ACIDException e) {
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java
index b483674..1dd8368 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java
@@ -26,6 +26,7 @@
import org.apache.asterix.common.transactions.ITransactionSubsystem;
import org.apache.asterix.common.transactions.JobId;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.common.api.ISearchOperationCallback;
import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
@@ -41,7 +42,7 @@
}
@Override
- public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx)
+ public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx, IOperatorNodePushable operatorNodePushable)
throws HyracksDataException {
ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
try {
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
index ef2b498..168da99 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
@@ -27,6 +27,7 @@
import org.apache.asterix.common.transactions.ITransactionSubsystem;
import org.apache.asterix.common.transactions.JobId;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManager;
import org.apache.hyracks.storage.am.common.api.IModificationOperationCallback;
@@ -48,7 +49,8 @@
@Override
public IModificationOperationCallback createModificationOperationCallback(String resourcePath, long resourceId,
- int resourcePartition, Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
+ int resourcePartition, Object resource, IHyracksTaskContext ctx, IOperatorNodePushable operatorNodePushable)
+ throws HyracksDataException {
ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
.getDatasetLifecycleManager();
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexSearchOperationCallbackFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexSearchOperationCallbackFactory.java
index 4e1ee63..5dfdcdc 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexSearchOperationCallbackFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexSearchOperationCallbackFactory.java
@@ -20,6 +20,7 @@
package org.apache.asterix.transaction.management.opcallbacks;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.common.api.ISearchOperationCallback;
import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
@@ -29,7 +30,7 @@
private static final long serialVersionUID = 1L;
@Override
- public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx)
+ public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx, IOperatorNodePushable operatorNodePushable)
throws HyracksDataException {
return new SecondaryIndexSearchOperationCallback();
}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
index b08798c..8c91c1a 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
@@ -27,6 +27,7 @@
import org.apache.asterix.common.transactions.ITransactionSubsystem;
import org.apache.asterix.common.transactions.JobId;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManager;
import org.apache.hyracks.storage.am.common.api.IModificationOperationCallback;
@@ -49,7 +50,8 @@
@Override
public IModificationOperationCallback createModificationOperationCallback(String resourcePath, long resourceId,
- int resourcePartition, Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
+ int resourcePartition, Object resource, IHyracksTaskContext ctx, IOperatorNodePushable operatorNodePushable)
+ throws HyracksDataException {
ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
.getDatasetLifecycleManager();
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
index 403d68d..3e11531 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
@@ -27,6 +27,7 @@
import org.apache.asterix.common.transactions.ITransactionSubsystem;
import org.apache.asterix.common.transactions.JobId;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManager;
import org.apache.hyracks.storage.am.common.api.IModificationOperationCallback;
@@ -49,7 +50,8 @@
@Override
public IModificationOperationCallback createModificationOperationCallback(String resourcePath, long resourceId,
- int resourcePartition, Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
+ int resourcePartition, Object resource, IHyracksTaskContext ctx, IOperatorNodePushable operatorNodePushable)
+ throws HyracksDataException {
ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
.getDatasetLifecycleManager();
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/UpsertOperationCallbackFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/UpsertOperationCallbackFactory.java
index 707f986..87cb8e7 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/UpsertOperationCallbackFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/UpsertOperationCallbackFactory.java
@@ -26,6 +26,7 @@
import org.apache.asterix.common.transactions.ITransactionSubsystem;
import org.apache.asterix.common.transactions.JobId;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManager;
import org.apache.hyracks.storage.am.common.api.IModificationOperationCallback;
@@ -47,7 +48,8 @@
@Override
public IModificationOperationCallback createModificationOperationCallback(String resourceName, long resourceId,
- int resourcePartition, Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
+ int resourcePartition, Object resource, IHyracksTaskContext ctx, IOperatorNodePushable operatorNodePushable)
+ throws HyracksDataException {
ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ConcurrentLockManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ConcurrentLockManager.java
index e268134..8dcc1be 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ConcurrentLockManager.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ConcurrentLockManager.java
@@ -22,7 +22,6 @@
import java.io.IOException;
import java.io.OutputStream;
import java.util.ArrayList;
-import java.util.HashMap;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
import java.util.logging.Level;
@@ -46,6 +45,10 @@
static final Logger LOGGER = Logger.getLogger(ConcurrentLockManager.class.getName());
static final Level LVL = Level.FINER;
+ public static final boolean ENABLED_DEADLOCK_FREE_LOCKING_PROTOCOL = true;
+
+ public static final int NIL = -1;
+ public static final long NILL = -1L;
public static final boolean DEBUG_MODE = false;//true
public static final boolean CHECK_CONSISTENCY = false;
@@ -54,8 +57,7 @@
private ResourceArenaManager resArenaMgr;
private RequestArenaManager reqArenaMgr;
private JobArenaManager jobArenaMgr;
- private ConcurrentHashMap<Integer, Long> jobIdSlotMap;
- private ThreadLocal<DatasetLockCache> dsLockCache;
+ private ConcurrentHashMap<Integer, Long> jobId2JobSlotMap;
private LockManagerStats stats = new LockManagerStats(10000);
enum LockAction {
@@ -94,13 +96,7 @@
resArenaMgr = new ResourceArenaManager(noArenas, lockManagerShrinkTimer);
reqArenaMgr = new RequestArenaManager(noArenas, lockManagerShrinkTimer);
jobArenaMgr = new JobArenaManager(noArenas, lockManagerShrinkTimer);
- jobIdSlotMap = new ConcurrentHashMap<>();
- dsLockCache = new ThreadLocal<DatasetLockCache>() {
- @Override
- protected DatasetLockCache initialValue() {
- return new DatasetLockCache();
- }
- };
+ jobId2JobSlotMap = new ConcurrentHashMap<Integer, Long>();
}
@Override
@@ -111,17 +107,7 @@
final int dsId = datasetId.getId();
final int jobId = txnContext.getJobId().getId();
-
- if (entityHashValue != -1) {
- lock(datasetId, -1, LockMode.intentionMode(lockMode), txnContext);
- } else {
- if (dsLockCache.get().contains(jobId, dsId, lockMode)) {
- return;
- }
- }
-
final long jobSlot = findOrAllocJobSlot(jobId);
-
final ResourceGroup group = table.get(dsId, entityHashValue);
group.getLatch();
try {
@@ -160,9 +146,6 @@
throw new IllegalStateException();
}
}
- if (entityHashValue == -1) {
- dsLockCache.get().put(jobId, dsId, lockMode);
- }
} catch (InterruptedException e) {
throw new WaitInterruptedException(txnContext, "interrupted", e);
} finally {
@@ -282,7 +265,15 @@
* @return true if a cycle would be introduced, false otherwise
*/
private boolean introducesDeadlock(final long resSlot, final long jobSlot, final DeadlockTracker tracker) {
- return introducesDeadlock(resSlot, jobSlot, tracker, 0);
+ if (ENABLED_DEADLOCK_FREE_LOCKING_PROTOCOL) {
+ /**
+ * Due to the deadlock-free locking protocol, deadlock is not possible.
+ * So, this method always returns false.
+ */
+ return false;
+ } else {
+ return introducesDeadlock(resSlot, jobSlot, tracker, 0);
+ }
}
private boolean introducesDeadlock(final long resSlot, final long jobSlot, final DeadlockTracker tracker,
@@ -336,15 +327,8 @@
final int dsId = datasetId.getId();
final int jobId = txnContext.getJobId().getId();
-
- if (entityHashValue != -1) {
- lock(datasetId, -1, LockMode.intentionMode(lockMode), txnContext);
- } else {
- throw new UnsupportedOperationException("instant locks are not supported on datasets");
- }
-
final ResourceGroup group = table.get(dsId, entityHashValue);
- if (group.firstResourceIndex.get() == -1l) {
+ if (group.firstResourceIndex.get() == NILL) {
validateJob(txnContext);
// if we do not have a resource in the group, we know that the
// resource that we are looking for is not locked
@@ -352,7 +336,7 @@
}
// we only allocate a request slot if we actually have to wait
- long reqSlot = -1;
+ long reqSlot = NILL;
group.getLatch();
try {
@@ -374,7 +358,7 @@
return;
case WAIT:
case CONV:
- if (reqSlot == -1) {
+ if (reqSlot == NILL) {
reqSlot = allocRequestSlot(resSlot, jobSlot, lockMode);
}
enqueueWaiter(group, reqSlot, resSlot, jobSlot, act, txnContext);
@@ -387,7 +371,7 @@
} catch (InterruptedException e) {
throw new WaitInterruptedException(txnContext, "interrupted", e);
} finally {
- if (reqSlot != -1) {
+ if (reqSlot != NILL) {
// deallocate request, if we allocated one earlier
if (DEBUG_MODE)
LOGGER.finer("del req slot " + TypeUtil.Global.toString(reqSlot));
@@ -405,19 +389,7 @@
final int dsId = datasetId.getId();
final int jobId = txnContext.getJobId().getId();
-
- if (entityHashValue != -1) {
- if (!tryLock(datasetId, -1, LockMode.intentionMode(lockMode), txnContext)) {
- return false;
- }
- } else {
- if (dsLockCache.get().contains(jobId, dsId, lockMode)) {
- return true;
- }
- }
-
final long jobSlot = findOrAllocJobSlot(jobId);
-
final ResourceGroup group = table.get(dsId, entityHashValue);
group.getLatch();
@@ -434,9 +406,6 @@
// no break
case GET:
addHolder(reqSlot, resSlot, jobSlot);
- if (entityHashValue == -1) {
- dsLockCache.get().put(jobId, dsId, lockMode);
- }
return true;
case WAIT:
case CONV:
@@ -447,9 +416,6 @@
} finally {
group.releaseLatch();
}
-
- // if we did acquire the dataset lock, but not the entity lock, we keep
- // it anyway and clean it up at the end of the job
}
@Override
@@ -460,17 +426,8 @@
final int dsId = datasetId.getId();
final int jobId = txnContext.getJobId().getId();
-
- if (entityHashValue != -1) {
- if (!tryLock(datasetId, -1, LockMode.intentionMode(lockMode), txnContext)) {
- return false;
- }
- } else {
- throw new UnsupportedOperationException("instant locks are not supported on datasets");
- }
-
final ResourceGroup group = table.get(dsId, entityHashValue);
- if (group.firstResourceIndex.get() == -1l) {
+ if (group.firstResourceIndex.get() == NILL) {
validateJob(txnContext);
// if we do not have a resource in the group, we know that the
// resource that we are looking for is not locked
@@ -511,7 +468,8 @@
throws ACIDException {
log("unlock", datasetId.getId(), entityHashValue, lockMode, txnContext);
final int jobId = txnContext.getJobId().getId();
- final long jobSlot = jobIdSlotMap.get(jobId);
+ final long jobSlot = jobId2JobSlotMap.get(jobId);
+
final int dsId = datasetId.getId();
unlock(dsId, entityHashValue, lockMode, jobSlot);
}
@@ -561,19 +519,15 @@
} finally {
group.releaseLatch();
}
-
- // dataset intention locks are
- // a) kept in dsLockCache and
- // b) cleaned up only in releaseLocks at the end of the job
}
@Override
public void releaseLocks(ITransactionContext txnContext) throws ACIDException {
- log("releaseLocks", -1, -1, LockMode.ANY, txnContext);
+ log("releaseLocks", NIL, NIL, LockMode.ANY, txnContext);
stats.releaseLocks();
int jobId = txnContext.getJobId().getId();
- Long jobSlot = jobIdSlotMap.get(jobId);
+ Long jobSlot = jobId2JobSlotMap.get(jobId);
if (jobSlot == null) {
// we don't know the job, so there are no locks for it - we're done
return;
@@ -588,7 +542,7 @@
synchronized (jobArenaMgr) {
holder = jobArenaMgr.getLastHolder(jobSlot);
}
- while (holder != -1) {
+ while (holder != NILL) {
long resource = reqArenaMgr.getResourceId(holder);
int dsId = resArenaMgr.getDatasetId(resource);
int pkHashVal = resArenaMgr.getPkHashVal(resource);
@@ -600,20 +554,20 @@
if (DEBUG_MODE)
LOGGER.finer("del job slot " + TypeUtil.Global.toString(jobSlot));
jobArenaMgr.deallocate(jobSlot);
- jobIdSlotMap.remove(jobId);
+ jobId2JobSlotMap.remove(jobId);
stats.logCounters(LOGGER, Level.INFO, true);
}
private long findOrAllocJobSlot(int jobId) {
- Long jobSlot = jobIdSlotMap.get(jobId);
+ Long jobSlot = jobId2JobSlotMap.get(jobId);
if (jobSlot == null) {
jobSlot = new Long(jobArenaMgr.allocate());
if (DEBUG_MODE)
LOGGER.finer("new job slot " + TypeUtil.Global.toString(jobSlot) + " (" + jobId + ")");
jobArenaMgr.setJobId(jobSlot, jobId);
- Long oldSlot = jobIdSlotMap.putIfAbsent(jobId, jobSlot);
+ Long oldSlot = jobId2JobSlotMap.putIfAbsent(jobId, jobSlot);
if (oldSlot != null) {
- // if another thread allocated a slot for this jobId between
+ // if another thread allocated a slot for this jobThreadId between
// get(..) and putIfAbsent(..), we'll use that slot and
// deallocate the one we allocated
if (DEBUG_MODE)
@@ -629,7 +583,7 @@
private long findOrAllocResourceSlot(ResourceGroup group, int dsId, int entityHashValue) {
long resSlot = findResourceInGroup(group, dsId, entityHashValue);
- if (resSlot == -1) {
+ if (resSlot == NILL) {
// we don't know about this resource, let's alloc a slot
resSlot = resArenaMgr.allocate();
resArenaMgr.setDatasetId(resSlot, dsId);
@@ -690,11 +644,15 @@
// carefully distinguishing the different lock modes
long holder = resArenaMgr.getLastHolder(resource);
LockAction res = LockAction.WAIT;
- while (holder != -1) {
+ while (holder != NILL) {
if (job == reqArenaMgr.getJobSlot(holder)) {
if (reqArenaMgr.getLockMode(holder) == lockMode) {
return LockAction.GET;
} else {
+ if (ENABLED_DEADLOCK_FREE_LOCKING_PROTOCOL) {
+ throw new IllegalStateException(
+ "Lock conversion is not supported when deadlock-free locking protocol is enabled!");
+ }
res = LockAction.CONV;
}
}
@@ -706,7 +664,7 @@
private long findResourceInGroup(ResourceGroup group, int dsId, int entityHashValue) {
stats.logCounters(LOGGER, Level.INFO, false);
long resSlot = group.firstResourceIndex.get();
- while (resSlot != -1) {
+ while (resSlot != NILL) {
// either we already have a lock on this resource or we have a
// hash collision
if (resArenaMgr.getDatasetId(resSlot) == dsId && resArenaMgr.getPkHashVal(resSlot) == entityHashValue) {
@@ -715,7 +673,7 @@
resSlot = resArenaMgr.getNext(resSlot);
}
}
- return -1;
+ return NILL;
}
private void addHolder(long request, long resource, long job) {
@@ -732,7 +690,7 @@
private boolean hasOtherHolders(long resSlot, long jobSlot) {
long holder = resArenaMgr.getLastHolder(resSlot);
- while (holder != -1) {
+ while (holder != NILL) {
if (reqArenaMgr.getJobSlot(holder) != jobSlot) {
return true;
}
@@ -771,10 +729,10 @@
private long removeRequestFromJob(long holder, long unmodified) {
long prevForJob = reqArenaMgr.getPrevJobRequest(holder);
long nextForJob = reqArenaMgr.getNextJobRequest(holder);
- if (nextForJob != -1) {
+ if (nextForJob != NILL) {
reqArenaMgr.setPrevJobRequest(nextForJob, prevForJob);
}
- if (prevForJob == -1) {
+ if (prevForJob == NILL) {
return nextForJob;
} else {
reqArenaMgr.setNextJobRequest(prevForJob, nextForJob);
@@ -792,8 +750,8 @@
@Override
public void add(long request, long resource, long job) {
long waiter = resArenaMgr.getFirstWaiter(resource);
- reqArenaMgr.setNextRequest(request, -1);
- if (waiter == -1) {
+ reqArenaMgr.setNextRequest(request, NILL);
+ if (waiter == NILL) {
resArenaMgr.setFirstWaiter(resource, request);
} else {
appendToRequestQueue(waiter, request);
@@ -858,7 +816,7 @@
private void insertIntoJobQueue(long newRequest, long oldRequest) {
reqArenaMgr.setNextJobRequest(newRequest, oldRequest);
- reqArenaMgr.setPrevJobRequest(newRequest, -1);
+ reqArenaMgr.setPrevJobRequest(newRequest, NILL);
if (oldRequest >= 0) {
reqArenaMgr.setPrevJobRequest(oldRequest, newRequest);
}
@@ -866,7 +824,7 @@
private void appendToRequestQueue(long head, long appendee) {
long next = reqArenaMgr.getNextRequest(head);
- while (next != -1) {
+ while (next != NILL) {
head = next;
next = reqArenaMgr.getNextRequest(head);
}
@@ -876,9 +834,9 @@
private long removeRequestFromQueueForSlot(long head, long reqSlot) {
long cur = head;
long prev = cur;
- while (prev != -1) {
+ while (prev != NILL) {
cur = reqArenaMgr.getNextRequest(prev);
- if (cur == -1) {
+ if (cur == NILL) {
throw new IllegalStateException("request " + reqSlot + " not in queue");
}
if (cur == reqSlot) {
@@ -907,9 +865,9 @@
private long removeRequestFromQueueForJob(long head, long jobSlot, byte lockMode) {
long holder = head;
long prev = holder;
- while (prev != -1) {
+ while (prev != NILL) {
holder = reqArenaMgr.getNextRequest(prev);
- if (holder == -1) {
+ if (holder == NILL) {
throw new IllegalStateException("no entry for job " + jobSlot + " in queue");
}
if (requestMatches(holder, jobSlot, lockMode)) {
@@ -925,7 +883,7 @@
private int determineNewMaxMode(long resource, int oldMaxMode) {
int newMaxMode = LockMode.NL;
long holder = resArenaMgr.getLastHolder(resource);
- while (holder != -1) {
+ while (holder != NILL) {
int curLockMode = reqArenaMgr.getLockMode(holder);
if (curLockMode == oldMaxMode) {
// we have another lock of the same mode - we're done
@@ -976,10 +934,10 @@
}
StringBuilder sb = new StringBuilder();
sb.append("{ op : ").append(string);
- if (id != -1) {
+ if (id != NIL) {
sb.append(" , dataset : ").append(id);
}
- if (entityHashValue != -1) {
+ if (entityHashValue != NIL) {
sb.append(" , entity : ").append(entityHashValue);
}
if (lockMode != LockMode.NL) {
@@ -1000,11 +958,11 @@
if (group.tryLatch(100, TimeUnit.MILLISECONDS)) {
try {
long resSlot = group.firstResourceIndex.get();
- while (resSlot != -1) {
+ while (resSlot != NILL) {
int dsId = resArenaMgr.getDatasetId(resSlot);
int entityHashValue = resArenaMgr.getPkHashVal(resSlot);
long reqSlot = resArenaMgr.getLastHolder(resSlot);
- while (reqSlot != -1) {
+ while (reqSlot != NILL) {
byte lockMode = (byte) reqArenaMgr.getLockMode(reqSlot);
long jobSlot = reqArenaMgr.getJobSlot(reqSlot);
int jobId = jobArenaMgr.getJobId(jobSlot);
@@ -1026,7 +984,7 @@
}
private void assertLockCanBeFoundInJobQueue(int dsId, int entityHashValue, byte lockMode, int jobId) {
- if (findLockInJobQueue(dsId, entityHashValue, jobId, lockMode) == -1) {
+ if (findLockInJobQueue(dsId, entityHashValue, jobId, lockMode) == NILL) {
String msg = "request for " + LockMode.toString(lockMode) + " lock on dataset " + dsId + " entity "
+ entityHashValue + " not found for job " + jobId + " in thread "
+ Thread.currentThread().getName();
@@ -1046,19 +1004,19 @@
* job id
* @param lockMode
* lock mode
- * @return the slot of the request, if the lock request is found, -1 otherwise
+ * @return the slot of the request, if the lock request is found, NILL otherwise
*/
private long findLockInJobQueue(final int dsId, final int entityHashValue, final int jobId, byte lockMode) {
- Long jobSlot = jobIdSlotMap.get(jobId);
+ Long jobSlot = jobId2JobSlotMap.get(jobId);
if (jobSlot == null) {
- return -1;
+ return NILL;
}
long holder;
synchronized (jobArenaMgr) {
holder = jobArenaMgr.getLastHolder(jobSlot);
}
- while (holder != -1) {
+ while (holder != NILL) {
long resource = reqArenaMgr.getResourceId(holder);
if (dsId == resArenaMgr.getDatasetId(resource) && entityHashValue == resArenaMgr.getPkHashVal(resource)
&& jobSlot == reqArenaMgr.getJobSlot(holder)
@@ -1069,7 +1027,7 @@
holder = reqArenaMgr.getNextJobRequest(holder);
}
}
- return -1;
+ return NILL;
}
private TablePrinter getResourceTablePrinter() {
@@ -1077,7 +1035,7 @@
}
private TablePrinter getDumpTablePrinter() {
- return new DumpTablePrinter(table, resArenaMgr, reqArenaMgr, jobArenaMgr, jobIdSlotMap);
+ return new DumpTablePrinter(table, resArenaMgr, reqArenaMgr, jobArenaMgr, jobId2JobSlotMap);
}
public String printByResource() {
@@ -1115,46 +1073,4 @@
dumpState(os);
}
}
-
- private static class DatasetLockCache {
- private long jobId = -1;
- private HashMap<Integer, Byte> lockCache = new HashMap<Integer, Byte>();
- // size 1 cache to avoid the boxing/unboxing that comes with the
- // access to the HashMap
- private int cDsId = -1;
- private byte cDsLockMode = -1;
-
- public boolean contains(final int jobId, final int dsId, byte dsLockMode) {
- if (this.jobId == jobId) {
- if (this.cDsId == dsId && this.cDsLockMode == dsLockMode) {
- return true;
- }
- final Byte cachedLockMode = this.lockCache.get(dsId);
- if (cachedLockMode != null && cachedLockMode == dsLockMode) {
- this.cDsId = dsId;
- this.cDsLockMode = dsLockMode;
- return true;
- }
- } else {
- this.jobId = -1;
- this.cDsId = -1;
- this.cDsLockMode = -1;
- this.lockCache.clear();
- }
- return false;
- }
-
- public void put(final int jobId, final int dsId, byte dsLockMode) {
- this.jobId = jobId;
- this.cDsId = dsId;
- this.cDsLockMode = dsLockMode;
- this.lockCache.put(dsId, dsLockMode);
- }
-
- @Override
- public String toString() {
- return "[ " + jobId + " : " + lockCache.toString() + "]";
- }
- }
-
}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DatasetLockInfo.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DatasetLockInfo.java
deleted file mode 100644
index dfb5d1e..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DatasetLockInfo.java
+++ /dev/null
@@ -1,536 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you 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 at
- *
- * 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 org.apache.asterix.transaction.management.service.locking;
-
-import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
-
-public class DatasetLockInfo {
- private EntityLockInfoManager entityLockInfoManager;
- private EntityInfoManager entityInfoManager;
- private LockWaiterManager lockWaiterManager;
- private PrimitiveIntHashMap entityResourceHT;
- private int IXCount;
- private int ISCount;
- private int XCount;
- private int SCount;
- private int lastHolder;
- private int firstWaiter;
- private int firstUpgrader;
-
- public DatasetLockInfo(EntityLockInfoManager entityLockInfoManager, EntityInfoManager entityInfoManager,
- LockWaiterManager lockWaiterManager) {
- this.entityLockInfoManager = entityLockInfoManager;
- this.entityInfoManager = entityInfoManager;
- this.lockWaiterManager = lockWaiterManager;
- entityResourceHT = new PrimitiveIntHashMap();
- lastHolder = -1; //-1 stands for end of list
- firstWaiter = -1;
- firstUpgrader = -1;
- }
-
- public void increaseLockCount(byte lockMode) {
- switch (lockMode) {
- case LockMode.IX:
- IXCount++;
- break;
- case LockMode.IS:
- ISCount++;
- break;
- case LockMode.X:
- XCount++;
- break;
- case LockMode.S:
- SCount++;
- break;
- default:
- throw new IllegalStateException("Invalid dataset lock mode");
- }
- }
-
- public void decreaseLockCount(byte lockMode) {
- switch (lockMode) {
- case LockMode.IX:
- IXCount--;
- break;
- case LockMode.IS:
- ISCount--;
- break;
- case LockMode.X:
- XCount--;
- break;
- case LockMode.S:
- SCount--;
- break;
- default:
- throw new IllegalStateException("Invalid dataset lock mode");
- }
- }
-
- public void increaseLockCount(byte lockMode, int count) {
- switch (lockMode) {
- case LockMode.IX:
- IXCount += count;
- break;
- case LockMode.IS:
- ISCount += count;
- break;
- case LockMode.X:
- XCount += count;
- break;
- case LockMode.S:
- SCount += count;
- break;
- default:
- throw new IllegalStateException("Invalid dataset lock mode");
- }
- }
-
- public void decreaseLockCount(byte lockMode, int count) {
- switch (lockMode) {
- case LockMode.IX:
- IXCount -= count;
- break;
- case LockMode.IS:
- ISCount -= count;
- break;
- case LockMode.X:
- XCount -= count;
- break;
- case LockMode.S:
- SCount -= count;
- break;
- default:
- throw new IllegalStateException("Invalid dataset lock mode");
- }
- }
-
- public boolean isUpgradeCompatible(byte lockMode, int entityInfo) {
- switch (lockMode) {
- //upgrade from IS -> IX
- //XCount is guaranteed to be 0.
- //upgrade is allowed if SCount is 0.
- case LockMode.IX:
- return SCount == 0;
-
- //upgrade from S -> X
- //XCount and IXCount are guaranteed to be 0.
- //upgrade is allowed if ISCount is 0.
- case LockMode.X:
- return ISCount == 0;
-
- default:
- throw new IllegalStateException("Invalid upgrade lock mode");
- }
- }
-
- public boolean isCompatible(byte lockMode) {
- switch (lockMode) {
- case LockMode.IX:
- return SCount == 0 && XCount == 0;
-
- case LockMode.IS:
- return XCount == 0;
-
- case LockMode.X:
- return ISCount == 0 && IXCount == 0 && SCount == 0 && XCount == 0;
-
- case LockMode.S:
- return IXCount == 0 && XCount == 0;
-
- default:
- throw new IllegalStateException("Invalid upgrade lock mode");
- }
- }
-
- public int findEntityInfoFromHolderList(int jobId, int hashVal) {
- int entityInfo;
- int eLockInfo;
- int waiterObjId;
- if (hashVal == -1) {//dataset-granule lock
- entityInfo = lastHolder;
- while (entityInfo != -1) {
- if (jobId == entityInfoManager.getJobId(entityInfo)) {
- return entityInfo;
- }
- entityInfo = entityInfoManager.getPrevEntityActor(entityInfo);
- }
- return -1;
- } else { //entity-granule lock
- eLockInfo = entityResourceHT.get(hashVal);
- if (eLockInfo == -1) {
- return -1;
- }
- entityInfo = entityLockInfoManager.findEntityInfoFromHolderList(eLockInfo, jobId, hashVal);
- if (entityInfo == -1) {
- //find the entityInfo from the waiter list of entityLockInfo.
- //There is a case where dataset-granule lock is acquired, but entity-granule lock is not acquired yet.
- //In this case, the waiter of the entityLockInfo represents the holder of the datasetLockInfo.
- waiterObjId = entityLockInfoManager.findWaiterFromWaiterList(eLockInfo, jobId, hashVal);
- if (waiterObjId != -1) {
- entityInfo = lockWaiterManager.getLockWaiter(waiterObjId).getEntityInfoSlot();
- }
- }
- return entityInfo;
- }
- }
-
- public int findWaiterFromWaiterList(int jobId, int hashVal) {
- int waiterObjId;
- LockWaiter waiterObj;
- int entityInfo = 0;
-
- waiterObjId = firstWaiter;
- while (waiterObjId != -1) {
- waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
- entityInfo = waiterObj.getEntityInfoSlot();
- if (jobId == entityInfoManager.getJobId(entityInfo)
- && hashVal == entityInfoManager.getPKHashVal(entityInfo)) {
- return waiterObjId;
- }
- waiterObjId = waiterObj.getNextWaiterObjId();
- }
-
- return -1;
- }
-
- public int findUpgraderFromUpgraderList(int jobId, int hashVal) {
- int waiterObjId;
- LockWaiter waiterObj;
- int entityInfo = 0;
-
- waiterObjId = firstUpgrader;
- while (waiterObjId != -1) {
- waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
- entityInfo = waiterObj.getEntityInfoSlot();
- if (jobId == entityInfoManager.getJobId(entityInfo)
- && hashVal == entityInfoManager.getPKHashVal(entityInfo)) {
- return waiterObjId;
- }
- waiterObjId = waiterObj.getNextWaiterObjId();
- }
-
- return -1;
- }
-
- public boolean isNoHolder() {
- return ISCount == 0 && IXCount == 0 && SCount == 0 && XCount == 0;
- }
-
- public void addHolder(int holder) {
- entityInfoManager.setPrevEntityActor(holder, lastHolder);
- lastHolder = holder;
- }
-
- /**
- * Remove holder from linked list of Actor.
- * Also, remove the corresponding resource from linked list of resource
- * in order to minimize JobInfo's resource link traversal.
- *
- * @param holder
- * @param jobInfo
- */
- public void removeHolder(int holder, JobInfo jobInfo) {
- int prev = lastHolder;
- int current = -1;
- int next;
-
- //remove holder from linked list of Actor
- while (prev != holder) {
- if (LockManager.IS_DEBUG_MODE) {
- if (prev == -1) {
- //shouldn't occur: debugging purpose
- try {
- throw new Exception();
- } catch (Exception e) {
- // TODO Auto-generated catch block
- e.printStackTrace();
- }
- }
- }
-
- current = prev;
- prev = entityInfoManager.getPrevEntityActor(current);
- }
-
- if (current != -1) {
- //current->prev = prev->prev
- entityInfoManager.setPrevEntityActor(current, entityInfoManager.getPrevEntityActor(prev));
- } else {
- //lastHolder = prev->prev
- lastHolder = entityInfoManager.getPrevEntityActor(prev);
- }
-
- //Notice!!
- //remove the corresponding resource from linked list of resource.
- //it is guaranteed that there is no waiter or upgrader in the JobInfo when this function is called.
- prev = entityInfoManager.getPrevJobResource(holder);
- next = entityInfoManager.getNextJobResource(holder);
-
- if (prev != -1) {
- entityInfoManager.setNextJobResource(prev, next);
- }
-
- if (next != -1) {
- entityInfoManager.setPrevJobResource(next, prev);
- } else {
- //This entityInfo(i.e., holder) is the last resource held by this job.
- jobInfo.setlastHoldingResource(holder);
- }
-
- //jobInfo.decreaseDatasetLockCount(holder);
- }
-
- /**
- * append new waiter to the end of waiters
- *
- * @param waiterObjId
- */
- public void addWaiter(int waiterObjId) {
- int lastObjId;
- LockWaiter lastObj = null;
-
- if (firstWaiter != -1) {
- //find the lastWaiter
- lastObjId = firstWaiter;
- while (lastObjId != -1) {
- lastObj = lockWaiterManager.getLockWaiter(lastObjId);
- lastObjId = lastObj.getNextWaiterObjId();
- }
- //last->next = new_waiter
- lastObj.setNextWaiterObjId(waiterObjId);
- } else {
- firstWaiter = waiterObjId;
- }
- //new_waiter->next = -1
- lastObj = lockWaiterManager.getLockWaiter(waiterObjId);
- lastObj.setNextWaiterObjId(-1);
-
- // if (LockManager.IS_DEBUG_MODE) {
- // System.out.println(printWaiters());
- // }
- }
-
- public void removeWaiter(int waiterObjId) {
- int currentObjId = firstWaiter;
- LockWaiter currentObj;
- LockWaiter prevObj = null;
- int prevObjId = -1;
- int nextObjId;
-
- while (currentObjId != waiterObjId) {
-
- if (LockManager.IS_DEBUG_MODE) {
- if (currentObjId == -1) {
- //shouldn't occur: debugging purpose
- try {
- throw new Exception();
- } catch (Exception e) {
- // TODO Auto-generated catch block
- e.printStackTrace();
- }
- }
- }
-
- prevObj = lockWaiterManager.getLockWaiter(currentObjId);
- prevObjId = currentObjId;
- currentObjId = prevObj.getNextWaiterObjId();
- }
-
- //get current waiter object
- currentObj = lockWaiterManager.getLockWaiter(currentObjId);
-
- //get next waiterObjId
- nextObjId = currentObj.getNextWaiterObjId();
-
- if (prevObjId != -1) {
- //prev->next = next
- prevObj.setNextWaiterObjId(nextObjId);
- } else {
- //removed first waiter. firstWaiter = current->next
- firstWaiter = nextObjId;
- }
-
- // if (LockManager.IS_DEBUG_MODE) {
- // System.out.println(printWaiters());
- // }
- }
-
- public void addUpgrader(int waiterObjId) {
- int lastObjId;
- LockWaiter lastObj = null;
-
- if (firstUpgrader != -1) {
- //find the lastWaiter
- lastObjId = firstUpgrader;
- while (lastObjId != -1) {
- lastObj = lockWaiterManager.getLockWaiter(lastObjId);
- lastObjId = lastObj.getNextWaiterObjId();
- }
- //last->next = new_waiter
- lastObj.setNextWaiterObjId(waiterObjId);
- } else {
- firstUpgrader = waiterObjId;
- }
- //new_waiter->next = -1
- lastObj = lockWaiterManager.getLockWaiter(waiterObjId);
- lastObj.setNextWaiterObjId(-1);
- }
-
- public void removeUpgrader(int waiterObjId) {
- int currentObjId = firstUpgrader;
- LockWaiter currentObj;
- LockWaiter prevObj = null;
- int prevObjId = -1;
- int nextObjId;
-
- while (currentObjId != waiterObjId) {
-
- if (LockManager.IS_DEBUG_MODE) {
- if (currentObjId == -1) {
- //shouldn't occur: debugging purpose
- try {
- throw new Exception();
- } catch (Exception e) {
- // TODO Auto-generated catch block
- e.printStackTrace();
- }
- }
- }
-
- prevObj = lockWaiterManager.getLockWaiter(currentObjId);
- prevObjId = currentObjId;
- currentObjId = prevObj.getNextWaiterObjId();
- }
-
- //get current waiter object
- currentObj = lockWaiterManager.getLockWaiter(currentObjId);
-
- //get next waiterObjId
- nextObjId = currentObj.getNextWaiterObjId();
-
- if (prevObjId != -1) {
- //prev->next = next
- prevObj.setNextWaiterObjId(nextObjId);
- } else {
- //removed first waiter. firstWaiter = current->next
- firstUpgrader = nextObjId;
- }
- }
-
- //debugging method
- public String printWaiters() {
- StringBuilder s = new StringBuilder();
- int waiterObjId;
- LockWaiter waiterObj;
- int entityInfo;
-
- s.append("WID\tWCT\tEID\tJID\tDID\tPK\n");
-
- waiterObjId = firstWaiter;
- while (waiterObjId != -1) {
- waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
- entityInfo = waiterObj.getEntityInfoSlot();
- s.append(waiterObjId).append("\t").append(waiterObj.getWaiterCount()).append("\t").append(entityInfo)
- .append("\t").append(entityInfoManager.getJobId(entityInfo)).append("\t")
- .append(entityInfoManager.getDatasetId(entityInfo)).append("\t")
- .append(entityInfoManager.getPKHashVal(entityInfo)).append("\n");
- waiterObjId = waiterObj.getNextWaiterObjId();
- }
-
- return s.toString();
- }
-
- public String coreDump() {
- StringBuilder sb = new StringBuilder();
- sb.append("\n\t firstUpgrader: " + firstUpgrader);
- sb.append("\n\t firstWaiter: " + firstWaiter);
- sb.append("\n\t lastHolder: " + lastHolder);
- sb.append("\n\t ISCount: " + ISCount);
- sb.append("\n\t IXCount: " + IXCount);
- sb.append("\n\t SCount: " + SCount);
- sb.append("\n\t XCount: " + XCount);
- sb.append("\n\t entityResourceHT");
- sb.append(entityResourceHT.prettyPrint());
- return sb.toString();
- }
-
- /////////////////////////////////////////////////////////
- // set/get method for private variable
- /////////////////////////////////////////////////////////
- public void setIXCount(int count) {
- IXCount = count;
- }
-
- public int getIXCount() {
- return IXCount;
- }
-
- public void setISCount(int count) {
- ISCount = count;
- }
-
- public int getISCount() {
- return ISCount;
- }
-
- public void setXCount(int count) {
- XCount = count;
- }
-
- public int getXCount() {
- return XCount;
- }
-
- public void setSCount(int count) {
- SCount = count;
- }
-
- public int getSCount() {
- return SCount;
- }
-
- public void setLastHolder(int holder) {
- lastHolder = holder;
- }
-
- public int getLastHolder() {
- return lastHolder;
- }
-
- public void setFirstWaiter(int waiter) {
- firstWaiter = waiter;
- }
-
- public int getFirstWaiter() {
- return firstWaiter;
- }
-
- public void setFirstUpgrader(int upgrader) {
- firstUpgrader = upgrader;
- }
-
- public int getFirstUpgrader() {
- return firstUpgrader;
- }
-
- public PrimitiveIntHashMap getEntityResourceHT() {
- return entityResourceHT;
- }
-
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DeadlockDetector.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DeadlockDetector.java
deleted file mode 100644
index f8a4f5e..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DeadlockDetector.java
+++ /dev/null
@@ -1,255 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you 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 at
- *
- * 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 org.apache.asterix.transaction.management.service.locking;
-
-import java.util.HashMap;
-
-import org.apache.asterix.common.transactions.DatasetId;
-import org.apache.asterix.common.transactions.JobId;
-
-/**
- * @author pouria, kisskys Performing a BFS search, upon adding each waiter to a waiting
- * list to avoid deadlocks this class implements such a loop-detector in
- * the wait-for-graph
- */
-
-public class DeadlockDetector {
-
- public static final boolean IS_DEBUG_MODE = true;//false
- private HashMap<JobId, JobInfo> jobHT;
- private HashMap<DatasetId, DatasetLockInfo> datasetResourceHT;
- private EntityLockInfoManager entityLockInfoManager;
- private EntityInfoManager entityInfoManager;
- private LockWaiterManager lockWaiterManager;
-
- private PrimitiveIntHashMap holderList;
- private PrimitiveIntHashMap nextHolderList;
- private PrimitiveIntHashMap resourceList;
- private PrimitiveIntHashMap visitedHolderList;
- private JobId tempJobIdObj; //temporary object to avoid object creation
- private DatasetId tempDatasetIdObj; //temporary object to avoid object creation
-
- public DeadlockDetector(HashMap<JobId, JobInfo> jobHT, HashMap<DatasetId, DatasetLockInfo> datasetResourceHT,
- EntityLockInfoManager entityLockInfoManager, EntityInfoManager entityInfoManager,
- LockWaiterManager lockWaiterManager) {
- this.jobHT = jobHT;
- this.datasetResourceHT = datasetResourceHT;
- this.entityLockInfoManager = entityLockInfoManager;
- this.entityInfoManager = entityInfoManager;
- this.lockWaiterManager = lockWaiterManager;
- holderList = new PrimitiveIntHashMap(1 << 6, 1 << 3, 180000);
- nextHolderList = new PrimitiveIntHashMap(1 << 6, 1 << 3, 180000);
- resourceList = new PrimitiveIntHashMap(1, 1 << 4, 180000);
- visitedHolderList = new PrimitiveIntHashMap(1 << 6, 1 << 3, 180000);
- tempJobIdObj = new JobId(0);
- tempDatasetIdObj = new DatasetId(0);
- }
-
- public boolean isSafeToAdd(DatasetLockInfo dLockInfo, int eLockInfo, int entityInfo, boolean isDatasetLockInfo,
- boolean isUpgrade) {
- int holder;
- int visitedHolder;
- int callerId = entityInfoManager.getJobId(entityInfo);
- int datasetId = entityInfoManager.getDatasetId(entityInfo);
- int hashValue = entityInfoManager.getPKHashVal(entityInfo);
- int resource;
- PrimitiveIntHashMap tempHolderList;
-
- holderList.clear(true);
- visitedHolderList.clear(true);
-
- //holderlist contains jobId
- //resourceList contains entityInfo's slot numbers instead of resourceId in order to avoid object creation
- //since resourceId consists of datasetId and PKHashValue
-
- //get holder list(jobId list)
- if (isDatasetLockInfo) {
- getHolderList(datasetId, -1, holderList);
- } else {
- getHolderList(datasetId, hashValue, holderList);
- }
-
- //check whether this caller is upgrader or not
- //if it is upgrader, then handle it as special case in the following manner
- //if there is another upgrader or waiter of which lock mode is not compatible with the caller's lock mode,
- //then this caller's wait causes deadlock.
- if (holderList.get(callerId) != -1) {
- if (isUpgrade && dLockInfo.getFirstUpgrader() != -1) {
- return false;
- }
- //there is no case such that while a job is holding any mode of lock on a dataset and waits for the same dataset as an waiter.
- //But the job may wait for the same dataset as an upgrader.
- }
-
- //TODO
- //check whether when there are multiple resources, the waiter and upgrader should be distinguished or not.
- //The current logic doesn't distinguish these two types of waiter.
-
- //while holderList is not empty
- holderList.beginIterate();
- holder = holderList.getNextKey();
- while (holder != -1) {
-
- nextHolderList.clear(true);
-
- while (holder != -1) {
- resourceList.clear(true);
- getWaitingResourceList(holder, resourceList);
- resourceList.beginIterate();
- resource = resourceList.getNextKey();
-
- while (resource != -1) {
- //get dataset holder
- getHolderList(entityInfoManager.getDatasetId(resource), -1, nextHolderList);
- //get entity holder
- getHolderList(entityInfoManager.getDatasetId(resource), entityInfoManager.getPKHashVal(resource),
- nextHolderList);
- if (nextHolderList.get(callerId) != -1) {
- return false;
- }
- resource = resourceList.getNextKey();
- }
-
- visitedHolderList.put(holder, -1);
- holder = holderList.getNextKey();
- }
-
- //remove visitedHolder for nextHolderList;
- visitedHolderList.beginIterate();
- visitedHolder = visitedHolderList.getNextKey();
- while (visitedHolder != -1) {
- nextHolderList.remove(visitedHolder);
- visitedHolder = visitedHolderList.getNextKey();
- }
-
- //swap holder list
- //set holderList to nextHolderList and nextHolderList to holderList
- tempHolderList = holderList;
- holderList = nextHolderList;
- nextHolderList = tempHolderList;
- holderList.beginIterate();
- holder = holderList.getNextKey();
- }
-
- return true;
- }
-
- /**
- * Get holder list of dataset if hashValue == -1. Get holder list of entity otherwise.
- * Where, a holder is a jobId, not entityInfo's slotNum
- *
- * @param datasetId
- * @param hashValue
- * @param holderList
- */
- private void getHolderList(int datasetId, int hashValue, PrimitiveIntHashMap holderList) {
- PrimitiveIntHashMap entityHT;
- DatasetLockInfo dLockInfo;
- int entityLockInfo;
- int entityInfo;
- int waiterObjId;
- LockWaiter waiterObj;
-
- //get datasetLockInfo
- tempDatasetIdObj.setId(datasetId);
- dLockInfo = datasetResourceHT.get(tempDatasetIdObj);
- if (dLockInfo == null) {
- return;
- }
-
- if (hashValue == -1) {
- //get S/X-lock holders of dataset
- entityInfo = dLockInfo.getLastHolder();
- while (entityInfo != -1) {
- holderList.put(entityInfoManager.getJobId(entityInfo), 0);
- entityInfo = entityInfoManager.getPrevEntityActor(entityInfo);
- }
-
- //get IS/IX-lock holders of dataset
- entityHT = dLockInfo.getEntityResourceHT();
- entityHT.beginIterate();
- entityLockInfo = entityHT.getNextValue();
- while (entityLockInfo != -1) {
-
- //1. add holder of eLockInfo to holerList
- entityInfo = entityLockInfoManager.getLastHolder(entityLockInfo);
- while (entityInfo != -1) {
- holderList.put(entityInfoManager.getJobId(entityInfo), 0);
- entityInfo = entityInfoManager.getPrevEntityActor(entityInfo);
- }
-
- //2. add waiter of eLockInfo to holderList since waiter of entityLock is a holder of datasetLock
- //(Upgraders need not to be added since upgraders are also holders)
- waiterObjId = entityLockInfoManager.getFirstWaiter(entityLockInfo);
- while (waiterObjId != -1) {
- waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
- entityInfo = waiterObj.getEntityInfoSlot();
- holderList.put(entityInfoManager.getJobId(entityInfo), 0);
- waiterObjId = waiterObj.getNextWaiterObjId();
- }
-
- entityLockInfo = entityHT.getNextValue();
- }
- } else {
- //get S/X-lock holders of entity
- entityHT = dLockInfo.getEntityResourceHT();
- entityLockInfo = entityHT.get(hashValue);
- if (entityLockInfo != -1) {
- entityInfo = entityLockInfoManager.getLastHolder(entityLockInfo);
- while (entityInfo != -1) {
- holderList.put(entityInfoManager.getJobId(entityInfo), 0);
- entityInfo = entityInfoManager.getPrevEntityActor(entityInfo);
- }
- }
- }
- return;
- }
-
- /**
- * Get waiting resource list of jobId, where a resource is represented with entityInfo's slot number
- *
- * @param jobId
- * @param resourceList
- */
- private void getWaitingResourceList(int jobId, PrimitiveIntHashMap resourceList) {
- JobInfo jobInfo;
- int waiterId;
- LockWaiter waiterObj;
- int entityInfo;
-
- //get JobInfo
- tempJobIdObj.setId(jobId);
- jobInfo = jobHT.get(tempJobIdObj);
- if (IS_DEBUG_MODE) {
- if (jobInfo == null) {
- System.out.println(Thread.currentThread().getName() + "jobId:" + jobId);
- }
- }
-
- //get WaiterObj
- waiterId = jobInfo.getFirstWaitingResource();
- while (waiterId != -1) {
- waiterObj = lockWaiterManager.getLockWaiter(waiterId);
- entityInfo = waiterObj.getEntityInfoSlot();
- resourceList.put(entityInfo, -1);
- waiterId = waiterObj.getNextWaitingResourceObjId();
- }
- return;
- }
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DummyLockManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DummyLockManager.java
deleted file mode 100644
index 8ffa775..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DummyLockManager.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you 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 at
- *
- * 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 org.apache.asterix.transaction.management.service.locking;
-
-import java.io.IOException;
-import java.io.OutputStream;
-
-import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.transactions.DatasetId;
-import org.apache.asterix.common.transactions.ILockManager;
-import org.apache.asterix.common.transactions.ITransactionContext;
-import org.apache.asterix.transaction.management.service.transaction.TransactionSubsystem;
-import org.apache.hyracks.api.lifecycle.ILifeCycleComponent;
-
-
-/**
- * A dummy implementation of the ILockManager interface. It assumes that all
- * requests are successful. It can be used to for jobs that are known to be
- * conflict free, but it'll yield terrible results if there are conflicts.
- *
- * @author tillw
- *
- */
-public class DummyLockManager implements ILockManager, ILifeCycleComponent {
-
- public DummyLockManager(TransactionSubsystem transactionSubsystem) {
- }
-
- @Override
- public void start() {
- }
-
- @Override
- public void stop(boolean dumpState, OutputStream ouputStream) throws IOException {
- }
-
- @Override
- public void lock(DatasetId datasetId, int entityHashValue, byte lockMode, ITransactionContext txnContext)
- throws ACIDException {
- }
-
- @Override
- public void releaseLocks(ITransactionContext txnContext) throws ACIDException {
- }
-
- @Override
- public void unlock(DatasetId datasetId, int entityHashValue, byte lockMode, ITransactionContext txnContext)
- throws ACIDException {
- }
-
- @Override
- public void instantLock(DatasetId datasetId, int entityHashValue, byte lockMode, ITransactionContext context)
- throws ACIDException {
- }
-
- @Override
- public boolean tryLock(DatasetId datasetId, int entityHashValue, byte lockMode, ITransactionContext context)
- throws ACIDException {
- return true;
- }
-
- @Override
- public boolean instantTryLock(DatasetId datasetId, int entityHashValue, byte lockMode,
- ITransactionContext txnContext) throws ACIDException {
- return true;
- }
-
- @Override
- public String prettyPrint() throws ACIDException {
- return "DummyLockManager";
- }
-
- @Override
- public void dumpState(OutputStream os) throws IOException {
- }
-
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DumpTablePrinter.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DumpTablePrinter.java
index ffdb151..a75f756 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DumpTablePrinter.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DumpTablePrinter.java
@@ -26,18 +26,15 @@
private ResourceArenaManager resArenaMgr;
private RequestArenaManager reqArenaMgr;
private JobArenaManager jobArenaMgr;
- private ConcurrentHashMap<Integer, Long> jobIdSlotMap;
+ private ConcurrentHashMap<Integer, Long> jobId2JobSlotMap;
- DumpTablePrinter(ResourceGroupTable table,
- ResourceArenaManager resArenaMgr,
- RequestArenaManager reqArenaMgr,
- JobArenaManager jobArenaMgr,
- ConcurrentHashMap<Integer, Long> jobIdSlotMap) {
+ DumpTablePrinter(ResourceGroupTable table, ResourceArenaManager resArenaMgr, RequestArenaManager reqArenaMgr,
+ JobArenaManager jobArenaMgr, ConcurrentHashMap<Integer, Long> jobId2JobSlotMap) {
this.table = table;
this.resArenaMgr = resArenaMgr;
this.reqArenaMgr = reqArenaMgr;
this.jobArenaMgr = jobArenaMgr;
- this.jobIdSlotMap = jobIdSlotMap;
+ this.jobId2JobSlotMap = jobId2JobSlotMap;
}
public StringBuilder append(StringBuilder sb) {
@@ -56,9 +53,9 @@
sb.append(">>dump_end\t>>----- [reqArenaMgr] -----\n");
sb.append(">>dump_begin\t>>----- [jobIdSlotMap] -----\n");
- for (Integer i : jobIdSlotMap.keySet()) {
+ for (Integer i : jobId2JobSlotMap.keySet()) {
sb.append(i).append(" : ");
- TypeUtil.Global.append(sb, jobIdSlotMap.get(i));
+ TypeUtil.Global.append(sb, jobId2JobSlotMap.get(i));
sb.append("\n");
}
sb.append(">>dump_end\t>>----- [jobIdSlotMap] -----\n");
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/EntityInfoManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/EntityInfoManager.java
deleted file mode 100644
index 0e607ee..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/EntityInfoManager.java
+++ /dev/null
@@ -1,730 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you 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 at
- *
- * 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 org.apache.asterix.transaction.management.service.locking;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-
-/**
- * EntityInfoManager provides EntityInfo arrays backed by ByteBuffer.
- * The array grows when the slots are overflowed.
- * Also, the array shrinks according to the following shrink policy
- * : Shrink when the resource under-utilization lasts for a certain threshold time.
- *
- * @author kisskys
- */
-public class EntityInfoManager {
-
- private ArrayList<ChildEntityInfoArrayManager> pArray;
- private int allocChild; //used to allocate the next free EntityInfo slot.
- private long shrinkTimer;
- private boolean isShrinkTimerOn;
- private int occupiedSlots;
- private int shrinkTimerThreshold;
-
- // ////////////////////////////////////////////////
- // // begin of unit test
- // ////////////////////////////////////////////////
- //
- // public static final int SHRINK_TIMER_THRESHOLD = 0; //for unit test
- //
- // /**
- // * @param args
- // */
- // public static void main(String[] args) {
- // final int DataSize = 5000;
- //
- // int i, j;
- // int slots = ChildEntityInfoArrayManager.NUM_OF_SLOTS;
- // int data[] = new int[DataSize];
- // EntityInfoManager eiMgr = new EntityInfoManager();
- //
- // //allocate: 50
- // System.out.println("allocate: 50");
- // for (i = 0; i < 5; i++) {
- // for (j = i * slots; j < i * slots + slots; j++) {
- // data[j] = eiMgr.allocate();
- // }
- //
- // System.out.println(eiMgr.prettyPrint());
- // }
- //
- // //deallocate from the last child to the first child
- // System.out.println("deallocate from the last child to the first child");
- // for (i = 4; i >= 0; i--) {
- // for (j = i * slots + slots - 1; j >= i * slots; j--) {
- // eiMgr.deallocate(data[j]);
- // }
- // System.out.println(eiMgr.prettyPrint());
- // }
- //
- // //allocate: 50
- // System.out.println("allocate: 50");
- // for (i = 0; i < 5; i++) {
- // for (j = i * slots; j < i * slots + slots; j++) {
- // data[j] = eiMgr.allocate();
- // }
- //
- // System.out.println(eiMgr.prettyPrint());
- // }
- //
- // //deallocate from the first child to last child
- // System.out.println("deallocate from the first child to last child");
- // for (i = 0; i < 5; i++) {
- // for (j = i * slots; j < i * slots + slots; j++) {
- // eiMgr.deallocate(data[j]);
- // }
- //
- // System.out.println(eiMgr.prettyPrint());
- // }
- //
- // //allocate: 50
- // System.out.println("allocate: 50");
- // for (i = 0; i < 5; i++) {
- // for (j = i * slots; j < i * slots + slots; j++) {
- // data[j] = eiMgr.allocate();
- // }
- //
- // System.out.println(eiMgr.prettyPrint());
- // }
- //
- // //deallocate from the first child to 4th child
- // System.out.println("deallocate from the first child to 4th child");
- // for (i = 0; i < 4; i++) {
- // for (j = i * slots; j < i * slots + slots; j++) {
- // eiMgr.deallocate(data[j]);
- // }
- //
- // System.out.println(eiMgr.prettyPrint());
- // }
- //
- // //allocate: 40
- // System.out.println("allocate: 40");
- // for (i = 0; i < 4; i++) {
- // for (j = i * slots; j < i * slots + slots; j++) {
- // data[j] = eiMgr.allocate();
- // }
- //
- // System.out.println(eiMgr.prettyPrint());
- // }
- // }
- //
- // ////////////////////////////////////////////////
- // // end of unit test
- // ////////////////////////////////////////////////
-
- public EntityInfoManager(int shrinkTimerThreshold) {
- pArray = new ArrayList<ChildEntityInfoArrayManager>();
- pArray.add(new ChildEntityInfoArrayManager());
- allocChild = 0;
- occupiedSlots = 0;
- isShrinkTimerOn = false;
- this.shrinkTimerThreshold = shrinkTimerThreshold;
- }
-
- public int allocate(int jobId, int datasetId, int entityHashVal, byte lockMode) {
- int slotNum = allocate();
- initEntityInfo(slotNum, jobId, datasetId, entityHashVal, lockMode);
- return slotNum;
- }
-
- public int allocate() {
- if (pArray.get(allocChild).isFull()) {
- int size = pArray.size();
- boolean bAlloc = false;
- ChildEntityInfoArrayManager child;
-
- //find a deinitialized child and initialze it
- for (int i = 0; i < size; i++) {
- child = pArray.get(i);
- if (child.isDeinitialized()) {
- child.initialize();
- allocChild = i;
- bAlloc = true;
- break;
- }
- }
-
- //allocate new child when there is no deinitialized child
- if (!bAlloc) {
- pArray.add(new ChildEntityInfoArrayManager());
- allocChild = pArray.size() - 1;
- }
- }
-
- occupiedSlots++;
- return pArray.get(allocChild).allocate() + allocChild * ChildEntityInfoArrayManager.NUM_OF_SLOTS;
- }
-
- void deallocate(int slotNum) {
- pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).deallocate(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
- occupiedSlots--;
-
- if (needShrink()) {
- shrink();
- }
- }
-
- /**
- * Shrink policy:
- * Shrink when the resource under-utilization lasts for a certain amount of time.
- * TODO Need to figure out which of the policies is better
- * case1.
- * pArray status : O x x x x x O (O is initialized, x is deinitialized)
- * In the above status, 'CURRENT' needShrink() returns 'TRUE'
- * even if there is nothing to shrink or deallocate.
- * It doesn't distinguish the deinitialized children from initialized children
- * by calculating totalNumOfSlots = pArray.size() * ChildEntityInfoArrayManager.NUM_OF_SLOTS.
- * In other words, it doesn't subtract the deinitialized children's slots.
- * case2.
- * pArray status : O O x x x x x
- * However, in the above case, if we subtract the deinitialized children's slots,
- * needShrink() will return false even if we shrink the pArray at this case.
- *
- * @return
- */
- private boolean needShrink() {
- int size = pArray.size();
- int usedSlots = occupiedSlots;
- if (usedSlots == 0) {
- usedSlots = 1;
- }
-
- if (size > 1 && size * ChildEntityInfoArrayManager.NUM_OF_SLOTS / usedSlots >= 3) {
- if (isShrinkTimerOn) {
- if (System.currentTimeMillis() - shrinkTimer >= shrinkTimerThreshold) {
- isShrinkTimerOn = false;
- return true;
- }
- } else {
- //turn on timer
- isShrinkTimerOn = true;
- shrinkTimer = System.currentTimeMillis();
- }
- } else {
- //turn off timer
- isShrinkTimerOn = false;
- }
-
- return false;
- }
-
- /**
- * Shrink() may
- * deinitialize(:deallocates ByteBuffer of child) Children(s) or
- * shrink pArray according to the deinitialized children's contiguity status.
- * It doesn't deinitialze or shrink more than half of children at a time.
- */
- private void shrink() {
- int i;
- int removeCount = 0;
- int size = pArray.size();
- int maxDecreaseCount = size / 2;
- ChildEntityInfoArrayManager child;
-
- //The first buffer never be deinitialized.
- for (i = 1; i < size; i++) {
- if (pArray.get(i).isEmpty()) {
- pArray.get(i).deinitialize();
- }
- }
-
- //remove the empty buffers from the end
- for (i = size - 1; i >= 1; i--) {
- child = pArray.get(i);
- if (child.isDeinitialized()) {
- pArray.remove(i);
- if (++removeCount == maxDecreaseCount) {
- break;
- }
- } else {
- break;
- }
- }
-
- //reset allocChild to the first buffer
- allocChild = 0;
-
- isShrinkTimerOn = false;
- }
-
- public String prettyPrint() {
- StringBuilder s = new StringBuilder("\n########### EntityInfoManager Status #############\n");
- int size = pArray.size();
- ChildEntityInfoArrayManager child;
-
- for (int i = 0; i < size; i++) {
- child = pArray.get(i);
- if (child.isDeinitialized()) {
- continue;
- }
- s.append("child[" + i + "]");
- s.append(child.prettyPrint());
- }
- return s.toString();
- }
-
- public void coreDump(OutputStream os) {
- StringBuilder sb = new StringBuilder("\n\t########### EntityLockInfoManager Status #############\n");
- int size = pArray.size();
- ChildEntityInfoArrayManager child;
-
- sb.append("Number of Child: " + size + "\n");
- for (int i = 0; i < size; i++) {
- try {
- child = pArray.get(i);
- sb.append("child[" + i + "]");
- sb.append(child.prettyPrint());
-
- os.write(sb.toString().getBytes());
- } catch (IOException e) {
- //ignore IOException
- }
- sb = new StringBuilder();
- }
- }
-
- public int getShrinkTimerThreshold() {
- return shrinkTimerThreshold;
- }
-
- public void initEntityInfo(int slotNum, int jobId, int datasetId, int PKHashVal, byte lockMode) {
- pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).initEntityInfo(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, jobId, datasetId, PKHashVal, lockMode);
- }
-
- public boolean compareEntityInfo(int slotNum, int jobId, int datasetId, int PKHashVal) {
- return getPKHashVal(slotNum) == PKHashVal && getDatasetId(slotNum) == datasetId && getJobId(slotNum) == jobId;
- }
-
- public void increaseDatasetLockCount(int slotNum) {
- setDatasetLockCount(slotNum, (byte) (getDatasetLockCount(slotNum) + 1));
- }
-
- public void decreaseDatasetLockCount(int slotNum) {
- setDatasetLockCount(slotNum, (byte) (getDatasetLockCount(slotNum) - 1));
- }
-
- public void increaseEntityLockCount(int slotNum) {
- setEntityLockCount(slotNum, (byte) (getEntityLockCount(slotNum) + 1));
- }
-
- public void decreaseEntityLockCount(int slotNum) {
- setEntityLockCount(slotNum, (byte) (getEntityLockCount(slotNum) - 1));
- }
-
- public void increaseDatasetLockCount(int slotNum, int count) {
- setDatasetLockCount(slotNum, (byte) (getDatasetLockCount(slotNum) + count));
- }
-
- public void decreaseDatasetLockCount(int slotNum, int count) {
- setDatasetLockCount(slotNum, (byte) (getDatasetLockCount(slotNum) - count));
- }
-
- public void increaseEntityLockCount(int slotNum, int count) {
- setEntityLockCount(slotNum, (byte) (getEntityLockCount(slotNum) + count));
- }
-
- public void decreaseEntityLockCount(int slotNum, int count) {
- setEntityLockCount(slotNum, (byte) (getEntityLockCount(slotNum) - count));
- }
-
- //////////////////////////////////////////////////////////////////
- // set/get method for each field of EntityInfo
- //////////////////////////////////////////////////////////////////
-
- public void setJobId(int slotNum, int id) {
- pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setJobId(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, id);
- }
-
- public int getJobId(int slotNum) {
- return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getJobId(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
- }
-
- public void setDatasetId(int slotNum, int id) {
- pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setDatasetId(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, id);
- }
-
- public int getDatasetId(int slotNum) {
- return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getDatasetId(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
- }
-
- public void setPKHashVal(int slotNum, int hashVal) {
- pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setPKHashVal(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, hashVal);
- }
-
- public int getPKHashVal(int slotNum) {
- return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getPKHashVal(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
- }
-
- public void setDatasetLockMode(int slotNum, byte mode) {
- pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setDatasetLockMode(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, mode);
- }
-
- public byte getDatasetLockMode(int slotNum) {
- return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getDatasetLockMode(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
- }
-
- public void setDatasetLockCount(int slotNum, byte count) {
- pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setDatasetLockCount(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, count);
- }
-
- public byte getDatasetLockCount(int slotNum) {
- return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getDatasetLockCount(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
- }
-
- public void setEntityLockMode(int slotNum, byte mode) {
- pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setEntityLockMode(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, mode);
- }
-
- public byte getEntityLockMode(int slotNum) {
- return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getEntityLockMode(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
- }
-
- public void setEntityLockCount(int slotNum, byte count) {
- pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setEntityLockCount(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, count);
- }
-
- public byte getEntityLockCount(int slotNum) {
- return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getEntityLockCount(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
- }
-
- //Used for Waiter/Upgrader
- public void setNextEntityActor(int slotNum, int nextActorSlotNum) {
- pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setNextEntityActor(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, nextActorSlotNum);
- }
-
- //Used for Waiter/Upgrader
- public int getNextEntityActor(int slotNum) {
- return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getNextEntityActor(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
- }
-
- //Used for Holder
- public void setPrevEntityActor(int slotNum, int nextActorSlotNum) {
- pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setPrevEntityActor(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, nextActorSlotNum);
- }
-
- //Used for Holder
- public int getPrevEntityActor(int slotNum) {
- return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getPrevEntityActor(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
- }
-
- public void setPrevJobResource(int slotNum, int prevResourceSlotNum) {
- pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setPrevJobResource(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, prevResourceSlotNum);
- }
-
- public int getPrevJobResource(int slotNum) {
- return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getPrevJobResource(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
- }
-
- public void setNextJobResource(int slotNum, int nextResourceSlotNum) {
- pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setNextJobResource(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, nextResourceSlotNum);
- }
-
- public int getNextJobResource(int slotNum) {
- return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getNextJobResource(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
- }
-
- // public void setNextDatasetActor(int slotNum, int nextActorSlotNum) {
- // pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setNextDatasetActor(
- // slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, nextActorSlotNum);
- // }
- //
- // public int getNextDatasetActor(int slotNum) {
- // return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getNextDatasetActor(
- // slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
- // }
-}
-
-/******************************************
- * EntityInfo (28 bytes)
- * ****************************************
- * int jobId
- * int datasetId
- * int PKHashValue
- * byte datasetLockMode
- * byte datasetLockCount
- * byte enitityLockMode
- * byte entityLockCount
- * int nextEntityActor : actor can be either holder/waiter/upgrader
- * int prevJobResource : resource can be either dataset or entity and a job is holding/waiting/upgrading lock(s) on it.
- * int nextJobResource : resource can be either dataset or entity and a job is holding/waiting/upgrading lock(s) on it.
- * (int nextDatasetActor : actor can be either holder/waiter/upgrader) --> not used.
- *******************************************/
-
-class ChildEntityInfoArrayManager {
- public static final int ENTITY_INFO_SIZE = 28; //28bytes
- public static final int NUM_OF_SLOTS = 1024; //number of entities in a buffer
- // public static final int NUM_OF_SLOTS = 10; //for unit test
- public static final int BUFFER_SIZE = ENTITY_INFO_SIZE * NUM_OF_SLOTS;
-
- //byte offset of each field of EntityInfo
- public static final int JOB_ID_OFFSET = 0;
- public static final int DATASET_ID_OFFSET = 4;
- public static final int PKHASH_VAL_OFFSET = 8;
- public static final int DATASET_LOCK_MODE_OFFSET = 12;
- public static final int DATASET_LOCK_COUNT_OFFSET = 13;
- public static final int ENTITY_LOCK_MODE_OFFSET = 14;
- public static final int ENTITY_LOCK_COUNT_OFFSET = 15;
- public static final int ENTITY_ACTOR_OFFSET = 16;
- public static final int PREV_JOB_RESOURCE_OFFSET = 20;
- public static final int NEXT_JOB_RESOURCE_OFFSET = 24;
- //public static final int DATASET_ACTOR_OFFSET = 28;
-
- //byte offset of nextFreeSlotNum which shares the same space of JobId
- //If a slot is in use, the space is used for JobId. Otherwise, it is used for nextFreeSlotNum.
- public static final int NEXT_FREE_SLOT_OFFSET = 0;
-
- private ByteBuffer buffer;
- private int freeSlotNum;
- private int occupiedSlots; //-1 represents 'deinitialized' state.
-
- public ChildEntityInfoArrayManager() {
- initialize();
- }
-
- public void initialize() {
- this.buffer = ByteBuffer.allocate(BUFFER_SIZE);
- this.freeSlotNum = 0;
- this.occupiedSlots = 0;
-
- for (int i = 0; i < NUM_OF_SLOTS - 1; i++) {
- setNextFreeSlot(i, i + 1);
- }
- setNextFreeSlot(NUM_OF_SLOTS - 1, -1); //-1 represents EOL(end of link)
- }
-
- public int allocate() {
- int currentSlot = freeSlotNum;
- freeSlotNum = getNextFreeSlot(currentSlot);
- occupiedSlots++;
- if (LockManager.IS_DEBUG_MODE) {
- System.out.println(Thread.currentThread().getName() + " entity allocate: " + currentSlot);
- }
- return currentSlot;
- }
-
- public void deallocate(int slotNum) {
- setNextFreeSlot(slotNum, freeSlotNum);
- freeSlotNum = slotNum;
- occupiedSlots--;
- if (LockManager.IS_DEBUG_MODE) {
- System.out.println(Thread.currentThread().getName() + " entity deallocate: " + slotNum);
- }
- }
-
- public void deinitialize() {
- buffer = null;
- occupiedSlots = -1;
- }
-
- public boolean isDeinitialized() {
- return occupiedSlots == -1;
- }
-
- public boolean isFull() {
- return occupiedSlots == NUM_OF_SLOTS;
- }
-
- public boolean isEmpty() {
- return occupiedSlots == 0;
- }
-
- public int getNumOfOccupiedSlots() {
- return occupiedSlots;
- }
-
- public int getFreeSlotNum() {
- return freeSlotNum;
- }
-
- public String prettyPrint() {
- StringBuilder sb = new StringBuilder();
- sb.append("\n\toccupiedSlots:" + getNumOfOccupiedSlots());
- sb.append("\n\tfreeSlotNum:" + getFreeSlotNum() + "\n");
- sb.append("\tjid\t").append("did\t").append("PK\t").append("DLM\t").append("DLC\t").append("ELM\t")
- .append("ELC\t").append("NEA\t").append("PJR\t").append("NJR\n");
- for (int j = 0; j < ChildEntityInfoArrayManager.NUM_OF_SLOTS; j++) {
- sb.append(j).append(": ");
- sb.append("\t" + getJobId(j));
- sb.append("\t" + getDatasetId(j));
- sb.append("\t" + getPKHashVal(j));
- sb.append("\t" + getDatasetLockMode(j));
- sb.append("\t" + getDatasetLockCount(j));
- sb.append("\t" + getEntityLockMode(j));
- sb.append("\t" + getEntityLockCount(j));
- sb.append("\t" + getNextEntityActor(j));
- sb.append("\t" + getPrevJobResource(j));
- sb.append("\t" + getNextJobResource(j));
- sb.append("\n");
- }
- return sb.toString();
- }
-
- //////////////////////////////////////////////////////////////////
- // set/get method for each field of EntityInfo plus freeSlot
- //////////////////////////////////////////////////////////////////
- public void initEntityInfo(int slotNum, int jobId, int datasetId, int PKHashVal, byte lockMode) {
- buffer.putInt(slotNum * ENTITY_INFO_SIZE + JOB_ID_OFFSET, jobId);
- buffer.putInt(slotNum * ENTITY_INFO_SIZE + DATASET_ID_OFFSET, datasetId);
- buffer.putInt(slotNum * ENTITY_INFO_SIZE + PKHASH_VAL_OFFSET, PKHashVal);
- buffer.put(slotNum * ENTITY_INFO_SIZE + DATASET_LOCK_MODE_OFFSET, lockMode);
- buffer.put(slotNum * ENTITY_INFO_SIZE + DATASET_LOCK_COUNT_OFFSET, (byte) 0);
- buffer.put(slotNum * ENTITY_INFO_SIZE + ENTITY_LOCK_MODE_OFFSET, lockMode);
- buffer.put(slotNum * ENTITY_INFO_SIZE + ENTITY_LOCK_COUNT_OFFSET, (byte) 0);
- buffer.putInt(slotNum * ENTITY_INFO_SIZE + ENTITY_ACTOR_OFFSET, -1);
- buffer.putInt(slotNum * ENTITY_INFO_SIZE + PREV_JOB_RESOURCE_OFFSET, -1);
- buffer.putInt(slotNum * ENTITY_INFO_SIZE + NEXT_JOB_RESOURCE_OFFSET, -1);
- //buffer.putInt(slotNum * ENTITY_INFO_SIZE + DATASET_ACTOR_OFFSET, -1);
- }
-
- public void setNextFreeSlot(int slotNum, int nextFreeSlot) {
- buffer.putInt(slotNum * ENTITY_INFO_SIZE + NEXT_FREE_SLOT_OFFSET, nextFreeSlot);
- }
-
- public int getNextFreeSlot(int slotNum) {
- return buffer.getInt(slotNum * ENTITY_INFO_SIZE + NEXT_FREE_SLOT_OFFSET);
- }
-
- public void setJobId(int slotNum, int id) {
- buffer.putInt(slotNum * ENTITY_INFO_SIZE + JOB_ID_OFFSET, id);
- }
-
- public int getJobId(int slotNum) {
- return buffer.getInt(slotNum * ENTITY_INFO_SIZE + JOB_ID_OFFSET);
- }
-
- public void setDatasetId(int slotNum, int id) {
- buffer.putInt(slotNum * ENTITY_INFO_SIZE + DATASET_ID_OFFSET, id);
- }
-
- public int getDatasetId(int slotNum) {
- return buffer.getInt(slotNum * ENTITY_INFO_SIZE + DATASET_ID_OFFSET);
- }
-
- public void setPKHashVal(int slotNum, int hashVal) {
- buffer.putInt(slotNum * ENTITY_INFO_SIZE + PKHASH_VAL_OFFSET, hashVal);
- }
-
- public int getPKHashVal(int slotNum) {
- return buffer.getInt(slotNum * ENTITY_INFO_SIZE + PKHASH_VAL_OFFSET);
- }
-
- public void setDatasetLockMode(int slotNum, byte mode) {
- buffer.put(slotNum * ENTITY_INFO_SIZE + DATASET_LOCK_MODE_OFFSET, mode);
- }
-
- public byte getDatasetLockMode(int slotNum) {
- return buffer.get(slotNum * ENTITY_INFO_SIZE + DATASET_LOCK_MODE_OFFSET);
- }
-
- public void setDatasetLockCount(int slotNum, byte count) {
- buffer.put(slotNum * ENTITY_INFO_SIZE + DATASET_LOCK_COUNT_OFFSET, count);
- }
-
- public byte getDatasetLockCount(int slotNum) {
- return buffer.get(slotNum * ENTITY_INFO_SIZE + DATASET_LOCK_COUNT_OFFSET);
- }
-
- public void setEntityLockMode(int slotNum, byte mode) {
- buffer.put(slotNum * ENTITY_INFO_SIZE + ENTITY_LOCK_MODE_OFFSET, mode);
- }
-
- public byte getEntityLockMode(int slotNum) {
- return buffer.get(slotNum * ENTITY_INFO_SIZE + ENTITY_LOCK_MODE_OFFSET);
- }
-
- public void setEntityLockCount(int slotNum, byte count) {
- buffer.put(slotNum * ENTITY_INFO_SIZE + ENTITY_LOCK_COUNT_OFFSET, count);
- }
-
- public byte getEntityLockCount(int slotNum) {
- return buffer.get(slotNum * ENTITY_INFO_SIZE + ENTITY_LOCK_COUNT_OFFSET);
- }
-
- //Used for Waiter/Upgrader
- public void setNextEntityActor(int slotNum, int nextActorSlotNum) {
- buffer.putInt(slotNum * ENTITY_INFO_SIZE + ENTITY_ACTOR_OFFSET, nextActorSlotNum);
- }
-
- //Used for Waiter/Upgrader
- public int getNextEntityActor(int slotNum) {
- return buffer.getInt(slotNum * ENTITY_INFO_SIZE + ENTITY_ACTOR_OFFSET);
- }
-
- //Used for Holder
- public void setPrevEntityActor(int slotNum, int nextActorSlotNum) {
- buffer.putInt(slotNum * ENTITY_INFO_SIZE + ENTITY_ACTOR_OFFSET, nextActorSlotNum);
- }
-
- //Used for Holder
- public int getPrevEntityActor(int slotNum) {
- return buffer.getInt(slotNum * ENTITY_INFO_SIZE + ENTITY_ACTOR_OFFSET);
- }
-
- public void setPrevJobResource(int slotNum, int prevResourceSlotNum) {
- buffer.putInt(slotNum * ENTITY_INFO_SIZE + PREV_JOB_RESOURCE_OFFSET, prevResourceSlotNum);
- }
-
- public int getPrevJobResource(int slotNum) {
- return buffer.getInt(slotNum * ENTITY_INFO_SIZE + PREV_JOB_RESOURCE_OFFSET);
- }
-
- public void setNextJobResource(int slotNum, int prevResourceSlotNum) {
- buffer.putInt(slotNum * ENTITY_INFO_SIZE + NEXT_JOB_RESOURCE_OFFSET, prevResourceSlotNum);
- }
-
- public int getNextJobResource(int slotNum) {
- return buffer.getInt(slotNum * ENTITY_INFO_SIZE + NEXT_JOB_RESOURCE_OFFSET);
- }
-
- // public void setNextDatasetActor(int slotNum, int nextActorSlotNum) {
- // buffer.putInt(slotNum * ENTITY_INFO_SIZE + DATASET_ACTOR_OFFSET, nextActorSlotNum);
- // }
- //
- // public int getNextDatasetActor(int slotNum) {
- // return buffer.getInt(slotNum * ENTITY_INFO_SIZE + DATASET_ACTOR_OFFSET);
- // }
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/EntityLockInfoManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/EntityLockInfoManager.java
deleted file mode 100644
index c3de3c4..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/EntityLockInfoManager.java
+++ /dev/null
@@ -1,827 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you 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 at
- *
- * 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 org.apache.asterix.transaction.management.service.locking;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-
-import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
-
-/**
- * EntityLockInfoManager provides EntityLockInfo arrays backed by ByteBuffer.
- * The array grows when the slots are overflowed.
- * Also, the array shrinks according to the following shrink policy
- * : Shrink when the resource under-utilization lasts for a certain threshold time.
- *
- * @author kisskys
- */
-public class EntityLockInfoManager {
-
- public static final int SHRINK_TIMER_THRESHOLD = 120000; //2min
-
- private ArrayList<ChildEntityLockInfoArrayManager> pArray;
- private int allocChild; //used to allocate the next free EntityInfo slot.
- private long shrinkTimer;
- private boolean isShrinkTimerOn;
- private int occupiedSlots;
- private EntityInfoManager entityInfoManager;
- LockWaiterManager lockWaiterManager;
-
- // ////////////////////////////////////////////////
- // // begin of unit test
- // ////////////////////////////////////////////////
- //
- // public static final int SHRINK_TIMER_THRESHOLD = 0; //for unit test
- //
- // /**
- // * @param args
- // */
- // public static void main(String[] args) {
- // final int DataSize = 5000;
- //
- // int i, j;
- // int slots = ChildEntityLockInfoArrayManager.NUM_OF_SLOTS;
- // int data[] = new int[DataSize];
- // EntityLockInfoManager eliMgr = new EntityLockInfoManager();
- //
- // //allocate: 50
- // System.out.println("allocate: 50");
- // for (i = 0; i < 5; i++) {
- // for (j = i * slots; j < i * slots + slots; j++) {
- // data[j] = eliMgr.allocate();
- // }
- //
- // System.out.println(eliMgr.prettyPrint());
- // }
- //
- // //deallocate from the last child to the first child
- // System.out.println("deallocate from the last child to the first child");
- // for (i = 4; i >= 0; i--) {
- // for (j = i * slots + slots - 1; j >= i * slots; j--) {
- // eliMgr.deallocate(data[j]);
- // }
- // System.out.println(eliMgr.prettyPrint());
- // }
- //
- // //allocate: 50
- // System.out.println("allocate: 50");
- // for (i = 0; i < 5; i++) {
- // for (j = i * slots; j < i * slots + slots; j++) {
- // data[j] = eliMgr.allocate();
- // }
- //
- // System.out.println(eliMgr.prettyPrint());
- // }
- //
- // //deallocate from the first child to last child
- // System.out.println("deallocate from the first child to last child");
- // for (i = 0; i < 5; i++) {
- // for (j = i * slots; j < i * slots + slots; j++) {
- // eliMgr.deallocate(data[j]);
- // }
- //
- // System.out.println(eliMgr.prettyPrint());
- // }
- //
- // //allocate: 50
- // System.out.println("allocate: 50");
- // for (i = 0; i < 5; i++) {
- // for (j = i * slots; j < i * slots + slots; j++) {
- // data[j] = eliMgr.allocate();
- // }
- //
- // System.out.println(eliMgr.prettyPrint());
- // }
- //
- // //deallocate from the first child to 4th child
- // System.out.println("deallocate from the first child to 4th child");
- // for (i = 0; i < 4; i++) {
- // for (j = i * slots; j < i * slots + slots; j++) {
- // eliMgr.deallocate(data[j]);
- // }
- //
- // System.out.println(eliMgr.prettyPrint());
- // }
- //
- // //allocate: 40
- // System.out.println("allocate: 40");
- // for (i = 0; i < 4; i++) {
- // for (j = i * slots; j < i * slots + slots; j++) {
- // data[j] = eliMgr.allocate();
- // }
- //
- // System.out.println(eliMgr.prettyPrint());
- // }
- // }
- //
- // ////////////////////////////////////////////////
- // // end of unit test
- // ////////////////////////////////////////////////
-
- public EntityLockInfoManager(EntityInfoManager entityInfoManager, LockWaiterManager lockWaiterManager) {
- pArray = new ArrayList<ChildEntityLockInfoArrayManager>();
- pArray.add(new ChildEntityLockInfoArrayManager());
- allocChild = 0;
- occupiedSlots = 0;
- isShrinkTimerOn = false;
- this.entityInfoManager = entityInfoManager;
- this.lockWaiterManager = lockWaiterManager;
- }
-
- public int allocate() {
- if (pArray.get(allocChild).isFull()) {
- int size = pArray.size();
- boolean bAlloc = false;
- ChildEntityLockInfoArrayManager child;
-
- //find a deinitialized child and initialze it
- for (int i = 0; i < size; i++) {
- child = pArray.get(i);
- if (child.isDeinitialized()) {
- child.initialize();
- allocChild = i;
- bAlloc = true;
- break;
- }
- }
-
- //allocate new child when there is no deinitialized child
- if (!bAlloc) {
- pArray.add(new ChildEntityLockInfoArrayManager());
- allocChild = pArray.size() - 1;
- }
- }
- occupiedSlots++;
- return pArray.get(allocChild).allocate() + allocChild * ChildEntityLockInfoArrayManager.NUM_OF_SLOTS;
- }
-
- void deallocate(int slotNum) {
- pArray.get(slotNum / ChildEntityLockInfoArrayManager.NUM_OF_SLOTS).deallocate(
- slotNum % ChildEntityLockInfoArrayManager.NUM_OF_SLOTS);
- occupiedSlots--;
-
- if (needShrink()) {
- shrink();
- }
- }
-
- /**
- * Shrink policy:
- * Shrink when the resource under-utilization lasts for a certain amount of time.
- * TODO Need to figure out which of the policies is better
- * case1.
- * pArray status : O x x x x x O (O is initialized, x is deinitialized)
- * In the above status, 'CURRENT' needShrink() returns 'TRUE'
- * even if there is nothing to shrink or deallocate.
- * It doesn't distinguish the deinitialized children from initialized children
- * by calculating totalNumOfSlots = pArray.size() * ChildEntityLockInfoArrayManager.NUM_OF_SLOTS.
- * In other words, it doesn't subtract the deinitialized children's slots.
- * case2.
- * pArray status : O O x x x x x
- * However, in the above case, if we subtract the deinitialized children's slots,
- * needShrink() will return false even if we shrink the pArray at this case.
- *
- * @return
- */
- private boolean needShrink() {
- int size = pArray.size();
- int usedSlots = occupiedSlots;
- if (usedSlots == 0) {
- usedSlots = 1;
- }
-
- if (size > 1 && size * ChildEntityLockInfoArrayManager.NUM_OF_SLOTS / usedSlots >= 3) {
- if (isShrinkTimerOn) {
- if (System.currentTimeMillis() - shrinkTimer >= SHRINK_TIMER_THRESHOLD) {
- isShrinkTimerOn = false;
- return true;
- }
- } else {
- //turn on timer
- isShrinkTimerOn = true;
- shrinkTimer = System.currentTimeMillis();
- }
- } else {
- //turn off timer
- isShrinkTimerOn = false;
- }
-
- return false;
- }
-
- /**
- * Shrink() may
- * deinitialize(:deallocates ByteBuffer of child) Children(s) or
- * shrink pArray according to the deinitialized children's contiguity status.
- * It doesn't deinitialze or shrink more than half of children at a time.
- */
- private void shrink() {
- int i;
- int removeCount = 0;
- int size = pArray.size();
- int maxDecreaseCount = size / 2;
- ChildEntityLockInfoArrayManager child;
-
- //The first buffer never be deinitialized.
- for (i = 1; i < size; i++) {
- if (pArray.get(i).isEmpty()) {
- pArray.get(i).deinitialize();
- }
- }
-
- //remove the empty buffers from the end
- for (i = size - 1; i >= 1; i--) {
- child = pArray.get(i);
- if (child.isDeinitialized()) {
- pArray.remove(i);
- if (++removeCount == maxDecreaseCount) {
- break;
- }
- } else {
- break;
- }
- }
-
- //reset allocChild to the first buffer
- allocChild = 0;
-
- isShrinkTimerOn = false;
- }
-
- public String prettyPrint() {
- StringBuilder s = new StringBuilder("\n########### EntityLockInfoManager Status #############\n");
- int size = pArray.size();
- ChildEntityLockInfoArrayManager child;
-
- for (int i = 0; i < size; i++) {
- child = pArray.get(i);
- if (child.isDeinitialized()) {
- continue;
- }
- s.append("child[" + i + "]");
- s.append(child.prettyPrint());
- }
- return s.toString();
- }
-
- public void coreDump(OutputStream os) {
- StringBuilder sb = new StringBuilder("\n\t########### EntityLockInfoManager Status #############\n");
- int size = pArray.size();
- ChildEntityLockInfoArrayManager child;
-
- sb.append("Number of Child: " + size + "\n");
- for (int i = 0; i < size; i++) {
- try {
- child = pArray.get(i);
- sb.append("child[" + i + "]");
- sb.append(child.prettyPrint());
-
- os.write(sb.toString().getBytes());
- } catch (IOException e) {
- //ignore IOException
- }
- sb = new StringBuilder();
- }
- }
-
- public int getShrinkTimerThreshold() {
- return SHRINK_TIMER_THRESHOLD;
- }
-
- //debugging method
- public String printWaiters(int slotNum) {
- StringBuilder s = new StringBuilder();
- int waiterObjId;
- LockWaiter waiterObj;
- int entityInfo;
-
- s.append("WID\tWCT\tEID\tJID\tDID\tPK\n");
-
- waiterObjId = getFirstWaiter(slotNum);
- while (waiterObjId != -1) {
- waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
- entityInfo = waiterObj.getEntityInfoSlot();
- s.append(waiterObjId).append("\t").append(waiterObj.getWaiterCount()).append("\t").append(entityInfo)
- .append("\t").append(entityInfoManager.getJobId(entityInfo)).append("\t")
- .append(entityInfoManager.getDatasetId(entityInfo)).append("\t")
- .append(entityInfoManager.getPKHashVal(entityInfo)).append("\n");
- waiterObjId = waiterObj.getNextWaiterObjId();
- }
-
- return s.toString();
- }
-
- public void addHolder(int slotNum, int holder) {
- entityInfoManager.setPrevEntityActor(holder, getLastHolder(slotNum));
- setLastHolder(slotNum, holder);
- }
-
- /**
- * Remove holder from linked list of Actor.
- * Also, remove the corresponding resource from linked list of resource
- * in order to minimize JobInfo's resource link traversal.
- *
- * @param slotNum
- * @param holder
- * @param jobInfo
- */
- public void removeHolder(int slotNum, int holder, JobInfo jobInfo) {
- int prev = getLastHolder(slotNum);
- int current = -1;
- int next;
-
- //remove holder from linked list of Actor
- while (prev != holder) {
- if (LockManager.IS_DEBUG_MODE) {
- if (prev == -1) {
- //shouldn't occur: debugging purpose
- try {
- throw new Exception();
- } catch (Exception e) {
- // TODO Auto-generated catch block
- e.printStackTrace();
- }
- }
- }
-
- current = prev;
- prev = entityInfoManager.getPrevEntityActor(current);
- }
-
- if (current != -1) {
- //current->prev = prev->prev
- entityInfoManager.setPrevEntityActor(current, entityInfoManager.getPrevEntityActor(prev));
- } else {
- //lastHolder = prev->prev
- setLastHolder(slotNum, entityInfoManager.getPrevEntityActor(prev));
- }
-
- //Notice!!
- //remove the corresponding resource from linked list of resource.
- prev = entityInfoManager.getPrevJobResource(holder);
- next = entityInfoManager.getNextJobResource(holder);
-
- if (prev != -1) {
- entityInfoManager.setNextJobResource(prev, next);
- }
-
- if (next != -1) {
- entityInfoManager.setPrevJobResource(next, prev);
- } else {
- //This entityInfo(i.e., holder) is the last resource held by this job.
- jobInfo.setlastHoldingResource(prev);
- }
-
- //jobInfo.decreaseDatasetLockCount(holder);
- }
-
- public void addWaiter(int slotNum, int waiterObjId) {
- int lastObjId;
- LockWaiter lastObj = null;
- int firstWaiter = getFirstWaiter(slotNum);
-
- if (firstWaiter != -1) {
- //find the lastWaiter
- lastObjId = firstWaiter;
- while (lastObjId != -1) {
- lastObj = lockWaiterManager.getLockWaiter(lastObjId);
- lastObjId = lastObj.getNextWaiterObjId();
- }
- //last->next = new_waiter
- lastObj.setNextWaiterObjId(waiterObjId);
- } else {
- setFirstWaiter(slotNum, waiterObjId);
- }
- //new_waiter->next = -1
- lastObj = lockWaiterManager.getLockWaiter(waiterObjId);
- lastObj.setNextWaiterObjId(-1);
- }
-
- public void removeWaiter(int slotNum, int waiterObjId) {
- int currentObjId = getFirstWaiter(slotNum);
- LockWaiter currentObj;
- LockWaiter prevObj = null;
- int prevObjId = -1;
- int nextObjId;
-
- while (currentObjId != waiterObjId) {
-
- if (LockManager.IS_DEBUG_MODE) {
- if (currentObjId == -1) {
- //shouldn't occur: debugging purpose
- try {
- throw new Exception();
- } catch (Exception e) {
- // TODO Auto-generated catch block
- e.printStackTrace();
- }
- }
- }
-
- prevObj = lockWaiterManager.getLockWaiter(currentObjId);
- prevObjId = currentObjId;
- currentObjId = prevObj.getNextWaiterObjId();
- }
-
- //get current waiter object
- currentObj = lockWaiterManager.getLockWaiter(currentObjId);
-
- //get next waiterObjId
- nextObjId = currentObj.getNextWaiterObjId();
-
- if (prevObjId != -1) {
- //prev->next = next
- prevObj.setNextWaiterObjId(nextObjId);
- } else {
- //removed first waiter. firstWaiter = current->next
- setFirstWaiter(slotNum, nextObjId);
- }
- }
-
- public void addUpgrader(int slotNum, int waiterObjId) {
- //[Notice]
- //Even if there are multiple threads in a job try to upgrade lock mode on same resource which is entity-granule,
- //while the first upgrader is waiting, all the incoming upgrade requests from other threads should be rejected by aborting them.
- //Therefore, there is no actual "ADD" upgrader method. Instead, it only has "SET" upgrader method.
- if (LockManager.IS_DEBUG_MODE) {
- if (getUpgrader(slotNum) != -1) {
- throw new IllegalStateException("Invalid lock upgrade request. This call should be handled as deadlock");
- }
- }
-
- setUpgrader(slotNum, waiterObjId);
- }
-
- public void removeUpgrader(int slotNum, int waiterObjId) {
- setUpgrader(slotNum, -1);
- }
-
- public boolean isUpgradeCompatible(int slotNum, byte lockMode, int entityInfo) {
- switch (lockMode) {
- case LockMode.X:
- return getSCount(slotNum) - entityInfoManager.getEntityLockCount(entityInfo) == 0;
-
- default:
- throw new IllegalStateException("Invalid upgrade lock mode");
- }
- }
-
- public boolean isCompatible(int slotNum, byte lockMode) {
- switch (lockMode) {
- case LockMode.X:
- return getSCount(slotNum) == 0 && getXCount(slotNum) == 0;
-
- case LockMode.S:
- return getXCount(slotNum) == 0;
-
- default:
- throw new IllegalStateException("Invalid upgrade lock mode");
- }
- }
-
- public int findEntityInfoFromHolderList(int eLockInfo, int jobId, int hashVal) {
- int entityInfo = getLastHolder(eLockInfo);
-
- while (entityInfo != -1) {
- if (jobId == entityInfoManager.getJobId(entityInfo)
- && hashVal == entityInfoManager.getPKHashVal(entityInfo)) {
- return entityInfo;
- }
- // if (LockManager.IS_DEBUG_MODE) {
- // System.out.println("eLockInfo(" + eLockInfo + "),entityInfo(" + entityInfo + "), Request[" + jobId
- // + "," + hashVal + "]:Result[" + entityInfoManager.getJobId(entityInfo) + ","
- // + entityInfoManager.getPKHashVal(entityInfo) + "]");
- // }
- entityInfo = entityInfoManager.getPrevEntityActor(entityInfo);
- }
-
- return -1;
- }
-
- public int findWaiterFromWaiterList(int eLockInfo, int jobId, int hashVal) {
- int waiterObjId = getFirstWaiter(eLockInfo);
- LockWaiter waiterObj;
- int entityInfo;
-
- while (waiterObjId != -1) {
- waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
- entityInfo = waiterObj.getEntityInfoSlot();
- if (jobId == entityInfoManager.getJobId(entityInfo)
- && hashVal == entityInfoManager.getPKHashVal(entityInfo)) {
- return waiterObjId;
- }
- waiterObjId = waiterObj.getNextWaiterObjId();
- }
-
- return -1;
- }
-
- public int findUpgraderFromUpgraderList(int eLockInfo, int jobId, int hashVal) {
- int waiterObjId = getUpgrader(eLockInfo);
- LockWaiter waiterObj;
- int entityInfo;
-
- if (waiterObjId != -1) {
- waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
- entityInfo = waiterObj.getEntityInfoSlot();
- if (jobId == entityInfoManager.getJobId(entityInfo)
- && hashVal == entityInfoManager.getPKHashVal(entityInfo)) {
- return waiterObjId;
- }
- }
-
- return -1;
- }
-
- public void increaseLockCount(int slotNum, byte lockMode) {
- switch (lockMode) {
- case LockMode.X:
- setXCount(slotNum, (short) (getXCount(slotNum) + 1));
- break;
- case LockMode.S:
- setSCount(slotNum, (short) (getSCount(slotNum) + 1));
- break;
- default:
- throw new IllegalStateException("Invalid entity lock mode " + lockMode);
- }
- }
-
- public void decreaseLockCount(int slotNum, byte lockMode) {
- switch (lockMode) {
- case LockMode.X:
- setXCount(slotNum, (short) (getXCount(slotNum) - 1));
- break;
- case LockMode.S:
- setSCount(slotNum, (short) (getSCount(slotNum) - 1));
- break;
- default:
- throw new IllegalStateException("Invalid entity lock mode " + lockMode);
- }
- }
-
- public void increaseLockCount(int slotNum, byte lockMode, short count) {
- switch (lockMode) {
- case LockMode.X:
- setXCount(slotNum, (short) (getXCount(slotNum) + count));
- break;
- case LockMode.S:
- setSCount(slotNum, (short) (getSCount(slotNum) + count));
- break;
- default:
- throw new IllegalStateException("Invalid entity lock mode " + lockMode);
- }
- }
-
- public void decreaseLockCount(int slotNum, byte lockMode, short count) {
- switch (lockMode) {
- case LockMode.X:
- setXCount(slotNum, (short) (getXCount(slotNum) - count));
- break;
- case LockMode.S:
- setSCount(slotNum, (short) (getSCount(slotNum) - count));
- break;
- default:
- throw new IllegalStateException("Invalid entity lock mode " + lockMode);
- }
- }
-
- //////////////////////////////////////////////////////////////////
- // set/get method for each field of EntityLockInfo
- //////////////////////////////////////////////////////////////////
-
- public void setXCount(int slotNum, short count) {
- pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setXCount(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, count);
- }
-
- public short getXCount(int slotNum) {
- return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getXCount(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
- }
-
- public void setSCount(int slotNum, short count) {
- pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setSCount(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, count);
- }
-
- public short getSCount(int slotNum) {
- return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getSCount(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
- }
-
- public void setLastHolder(int slotNum, int holder) {
- pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setLastHolder(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, holder);
- }
-
- public int getLastHolder(int slotNum) {
- return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getLastHolder(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
- }
-
- public void setFirstWaiter(int slotNum, int waiter) {
- pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setFirstWaiter(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, waiter);
- }
-
- public int getFirstWaiter(int slotNum) {
- return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getFirstWaiter(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
- }
-
- public void setUpgrader(int slotNum, int upgrader) {
- pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setUpgrader(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, upgrader);
- }
-
- public int getUpgrader(int slotNum) {
- return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getUpgrader(
- slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
- }
-
-}
-
-/******************************************
- * EntityLockInfo (16 bytes)
- * ****************************************
- * short XCount : used to represent the count of X mode lock if it is allocated. Otherwise, it represents next free slot.
- * short SCount
- * int lastHolder
- * int firstWaiter
- * int upgrader : may exist only one since there are only S and X mode lock in Entity-level
- *******************************************/
-
-class ChildEntityLockInfoArrayManager {
- public static final int ENTITY_LOCK_INFO_SIZE = 16; //16bytes
- public static final int NUM_OF_SLOTS = 1024; //number of entityLockInfos in a buffer
- //public static final int NUM_OF_SLOTS = 10; //for unit test
- public static final int BUFFER_SIZE = ENTITY_LOCK_INFO_SIZE * NUM_OF_SLOTS;
-
- //byte offset of each field of EntityLockInfo
- public static final int XCOUNT_OFFSET = 0;
- public static final int SCOUNT_OFFSET = 2;
- public static final int LAST_HOLDER_OFFSET = 4;
- public static final int FIRST_WAITER_OFFSET = 8;
- public static final int UPGRADER_OFFSET = 12;
-
- //byte offset of nextFreeSlotNum which shares the same space with LastHolder field
- //If a slot is in use, the space is used for LastHolder. Otherwise, it is used for nextFreeSlotNum.
- public static final int NEXT_FREE_SLOT_OFFSET = 4;
-
- private ByteBuffer buffer;
- private int freeSlotNum;
- private int occupiedSlots; //-1 represents 'deinitialized' state.
-
- public ChildEntityLockInfoArrayManager() {
- initialize();
- }
-
- public void initialize() {
- this.buffer = ByteBuffer.allocate(BUFFER_SIZE);
- this.freeSlotNum = 0;
- this.occupiedSlots = 0;
-
- for (int i = 0; i < NUM_OF_SLOTS - 1; i++) {
- setNextFreeSlot(i, i + 1);
- }
- setNextFreeSlot(NUM_OF_SLOTS - 1, -1); //-1 represents EOL(end of link)
- }
-
- public int allocate() {
- int currentSlot = freeSlotNum;
- freeSlotNum = getNextFreeSlot(currentSlot);
- //initialize values
- setXCount(currentSlot, (short) 0);
- setSCount(currentSlot, (short) 0);
- setLastHolder(currentSlot, -1);
- setFirstWaiter(currentSlot, -1);
- setUpgrader(currentSlot, -1);
- occupiedSlots++;
- if (LockManager.IS_DEBUG_MODE) {
- System.out.println(Thread.currentThread().getName() + " Allocated ELockInfo[" + currentSlot + "]");
- }
- return currentSlot;
- }
-
- public void deallocate(int slotNum) {
- setNextFreeSlot(slotNum, freeSlotNum);
- freeSlotNum = slotNum;
- occupiedSlots--;
- if (LockManager.IS_DEBUG_MODE) {
- System.out.println(Thread.currentThread().getName() + " Deallocated ELockInfo[" + slotNum + "]");
- }
- }
-
- public void deinitialize() {
- buffer = null;
- occupiedSlots = -1;
- }
-
- public boolean isDeinitialized() {
- return occupiedSlots == -1;
- }
-
- public boolean isFull() {
- return occupiedSlots == NUM_OF_SLOTS;
- }
-
- public boolean isEmpty() {
- return occupiedSlots == 0;
- }
-
- public int getNumOfOccupiedSlots() {
- return occupiedSlots;
- }
-
- public int getFreeSlotNum() {
- return freeSlotNum;
- }
-
- public String prettyPrint() {
- StringBuilder sb = new StringBuilder();
- sb.append("\n\toccupiedSlots:" + getNumOfOccupiedSlots());
- sb.append("\n\tfreeSlotNum:" + getFreeSlotNum());
- sb.append("\n\tX\t").append("S\t").append("LH\t").append("FW\t").append("UP\n");
- for (int j = 0; j < ChildEntityLockInfoArrayManager.NUM_OF_SLOTS; j++) {
- sb.append(j).append(": ");
- sb.append("\t" + getXCount(j));
- sb.append("\t" + getSCount(j));
- sb.append("\t" + getLastHolder(j));
- sb.append("\t" + getFirstWaiter(j));
- sb.append("\t" + getUpgrader(j));
- sb.append("\n");
- }
- return sb.toString();
- }
-
- //////////////////////////////////////////////////////////////////
- // set/get method for each field of EntityLockInfo plus freeSlot
- //////////////////////////////////////////////////////////////////
-
- public void setNextFreeSlot(int slotNum, int nextFreeSlot) {
- buffer.putInt(slotNum * ENTITY_LOCK_INFO_SIZE + NEXT_FREE_SLOT_OFFSET, nextFreeSlot);
- }
-
- public int getNextFreeSlot(int slotNum) {
- return buffer.getInt(slotNum * ENTITY_LOCK_INFO_SIZE + NEXT_FREE_SLOT_OFFSET);
- }
-
- public void setXCount(int slotNum, short count) {
- buffer.putShort(slotNum * ENTITY_LOCK_INFO_SIZE + XCOUNT_OFFSET, count);
- }
-
- public short getXCount(int slotNum) {
- return buffer.getShort(slotNum * ENTITY_LOCK_INFO_SIZE + XCOUNT_OFFSET);
- }
-
- public void setSCount(int slotNum, short count) {
- buffer.putShort(slotNum * ENTITY_LOCK_INFO_SIZE + SCOUNT_OFFSET, count);
- }
-
- public short getSCount(int slotNum) {
- return buffer.getShort(slotNum * ENTITY_LOCK_INFO_SIZE + SCOUNT_OFFSET);
- }
-
- public void setLastHolder(int slotNum, int holder) {
- buffer.putInt(slotNum * ENTITY_LOCK_INFO_SIZE + LAST_HOLDER_OFFSET, holder);
- }
-
- public int getLastHolder(int slotNum) {
- return buffer.getInt(slotNum * ENTITY_LOCK_INFO_SIZE + LAST_HOLDER_OFFSET);
- }
-
- public void setFirstWaiter(int slotNum, int waiter) {
- buffer.putInt(slotNum * ENTITY_LOCK_INFO_SIZE + FIRST_WAITER_OFFSET, waiter);
- }
-
- public int getFirstWaiter(int slotNum) {
- return buffer.getInt(slotNum * ENTITY_LOCK_INFO_SIZE + FIRST_WAITER_OFFSET);
- }
-
- public void setUpgrader(int slotNum, int upgrader) {
- buffer.putInt(slotNum * ENTITY_LOCK_INFO_SIZE + UPGRADER_OFFSET, upgrader);
- }
-
- public int getUpgrader(int slotNum) {
- return buffer.getInt(slotNum * ENTITY_LOCK_INFO_SIZE + UPGRADER_OFFSET);
- }
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ILockHashTable.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ILockHashTable.java
deleted file mode 100644
index bf85e69..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ILockHashTable.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you 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 at
- *
- * 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 org.apache.asterix.transaction.management.service.locking;
-
-/**
- * @author pouria Interface for a hashTable, used in the internal data
- * structures of lockManager
- * @param <K>
- * Type of the objects, used as keys
- * @param <V>
- * Type of the objects, used as values
- */
-public interface ILockHashTable<K, V> {
-
- public void put(K key, V value);
-
- public V get(K key);
-
- public V remove(K key);
-
- public int getKeysetSize();
-
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ILockMatrix.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ILockMatrix.java
deleted file mode 100644
index e2e2e3a..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ILockMatrix.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you 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 at
- *
- * 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 org.apache.asterix.transaction.management.service.locking;
-
-/**
- * @author pouria
- * Shows: - The conflict matrix for the locking protocol (whether two
- * lock modes conflict with each other or not on a single resource) -
- * Whether request to convert a lock mode to a new one is a conversion
- * (i.e. the new lock mode is stringer than the current one) or not
- * Each lock mode is shown/interpreted as an integer
- */
-
-public interface ILockMatrix {
-
- /**
- * @param mask
- * (current/expected) lock mask on the resource
- * @param reqLockMode
- * index of the requested lockMode
- * @return true if the lock request conflicts with the mask
- */
- public boolean conflicts(int mask, int reqLockMode);
-
- /**
- * @param currentLockMode
- * @param reqLockMode
- * @return true if the request is a conversion
- */
- public boolean isConversion(int currentLockMode, int reqLockMode);
-
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/JobInfo.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/JobInfo.java
deleted file mode 100644
index ebbb0b8..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/JobInfo.java
+++ /dev/null
@@ -1,334 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you 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 at
- *
- * 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 org.apache.asterix.transaction.management.service.locking;
-
-import org.apache.asterix.common.transactions.ITransactionContext;
-import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
-
-public class JobInfo {
- private EntityInfoManager entityInfoManager;
- private LockWaiterManager lockWaiterManager;
- private ITransactionContext jobCtx;
- private int lastHoldingResource; //resource(entity or dataset) which is held by this job lastly
- private int firstWaitingResource; //resource(entity or dataset) which this job is waiting for
- private int upgradingResource; //resource(entity or dataset) which this job is waiting for to upgrade
-
- private PrimitiveIntHashMap datasetISLockHT; //used for keeping dataset-granule-lock's count acquired by this job.
-
- public JobInfo(EntityInfoManager entityInfoManager, LockWaiterManager lockWaiterManager, ITransactionContext txnCtx) {
- this.entityInfoManager = entityInfoManager;
- this.lockWaiterManager = lockWaiterManager;
- this.jobCtx = txnCtx;
- this.lastHoldingResource = -1;
- this.firstWaitingResource = -1;
- this.upgradingResource = -1;
- if (LockManager.ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
- //This table maintains the number of locks acquired by this jobInfo.
- //[Notice] But this doesn't decrease the count even if the lock is released.
- this.datasetISLockHT = new PrimitiveIntHashMap(1 << 4, 1 << 2, Integer.MAX_VALUE);
- }
- }
-
- public void addHoldingResource(int resource) {
-
- if (LockManager.IS_DEBUG_MODE) {
- if (entityInfoManager.getJobId(resource) != jobCtx.getJobId().getId()) {
- throw new IllegalStateException("JobInfo(" + jobCtx.getJobId().getId() + ") has diffrent Job(JID:"
- + entityInfoManager.getJobId(resource) + "'s resource!!!");
- }
- //System.out.println(Thread.currentThread().getName()+"\tJobInfo_AddHolder:"+ resource);
- }
-
- if (lastHoldingResource != -1) {
- entityInfoManager.setNextJobResource(lastHoldingResource, resource);
- }
- entityInfoManager.setPrevJobResource(resource, lastHoldingResource);
- entityInfoManager.setNextJobResource(resource, -1);
- lastHoldingResource = resource;
- }
-
- public void removeHoldingResource(int resource) {
- int current = lastHoldingResource;
- int prev;
- int next;
-
- if (LockManager.IS_DEBUG_MODE) {
- if (entityInfoManager.getJobId(resource) != jobCtx.getJobId().getId()) {
- throw new IllegalStateException("JobInfo(" + jobCtx.getJobId().getId() + ") has diffrent Job(JID:"
- + entityInfoManager.getJobId(resource) + "'s resource!!!");
- }
- //System.out.println(Thread.currentThread().getName()+"\tJobInfo_RemoveHolder:"+ resource);
- }
-
- while (current != resource) {
-
- if (LockManager.IS_DEBUG_MODE) {
- if (current == -1) {
- //shouldn't occur: debugging purpose
- try {
- throw new Exception();
- } catch (Exception e) {
- // TODO Auto-generated catch block
- e.printStackTrace();
- }
- }
- }
-
- current = entityInfoManager.getPrevJobResource(current);
- }
-
- prev = entityInfoManager.getPrevJobResource(current);
- next = entityInfoManager.getNextJobResource(current);
- //update prev->next = next
- if (prev != -1) {
- entityInfoManager.setNextJobResource(prev, next);
- }
- if (next != -1) {
- entityInfoManager.setPrevJobResource(next, prev);
- }
- if (lastHoldingResource == resource) {
- lastHoldingResource = prev;
- }
-
- //decreaseDatasetLockCount(resource);
- }
-
- public void addWaitingResource(int waiterObjId) {
- int lastObjId;
- LockWaiter lastObj = null;
-
- if (firstWaitingResource != -1) {
- //find the lastWaiter
- lastObjId = firstWaitingResource;
- while (lastObjId != -1) {
- lastObj = lockWaiterManager.getLockWaiter(lastObjId);
- if (LockManager.IS_DEBUG_MODE) {
- int entityInfo = lastObj.getEntityInfoSlot();
- if (entityInfoManager.getJobId(entityInfo) != jobCtx.getJobId().getId()) {
- throw new IllegalStateException("JobInfo(" + jobCtx.getJobId().getId()
- + ") has diffrent Job(JID:" + entityInfoManager.getJobId(entityInfo) + "'s resource!!!");
- }
- }
- lastObjId = lastObj.getNextWaitingResourceObjId();
- }
- //last->next = new_waiter
- lastObj.setNextWaitingResourceObjId(waiterObjId);
- } else {
- firstWaitingResource = waiterObjId;
- }
- //new_waiter->next = -1
- lastObj = lockWaiterManager.getLockWaiter(waiterObjId);
- if (LockManager.IS_DEBUG_MODE) {
- int entityInfo = lastObj.getEntityInfoSlot();
- if (entityInfoManager.getJobId(entityInfo) != jobCtx.getJobId().getId()) {
- throw new IllegalStateException("JobInfo(" + jobCtx.getJobId().getId() + ") has diffrent Job(JID:"
- + entityInfoManager.getJobId(entityInfo) + "'s resource!!!");
- }
- }
- lastObj.setNextWaitingResourceObjId(-1);
-
- // if (LockManager.IS_DEBUG_MODE) {
- // System.out.println(Thread.currentThread().getName()+"\tJobInfo_AddWaiter:"+ waiterObjId + ", FirstWaiter:"+firstWaitingResource);
- // }
- }
-
- public void removeWaitingResource(int waiterObjId) {
- int currentObjId = firstWaitingResource;
- LockWaiter currentObj;
- LockWaiter prevObj = null;
- int prevObjId = -1;
- int nextObjId;
-
- while (currentObjId != waiterObjId) {
-
- if (LockManager.IS_DEBUG_MODE) {
- if (currentObjId == -1) {
- //shouldn't occur: debugging purpose
- try {
- throw new Exception();
- } catch (Exception e) {
- // TODO Auto-generated catch block
- e.printStackTrace();
- }
- }
- }
-
- prevObj = lockWaiterManager.getLockWaiter(currentObjId);
- prevObjId = currentObjId;
- currentObjId = prevObj.getNextWaitingResourceObjId();
- }
-
- //get current waiter object
- currentObj = lockWaiterManager.getLockWaiter(currentObjId);
-
- if (LockManager.IS_DEBUG_MODE) {
- int entityInfo = currentObj.getEntityInfoSlot();
- if (entityInfoManager.getJobId(entityInfo) != jobCtx.getJobId().getId()) {
- throw new IllegalStateException("JobInfo(" + jobCtx.getJobId().getId() + ") has diffrent Job(JID:"
- + entityInfoManager.getJobId(entityInfo) + "'s resource!!!");
- }
- }
-
- //get next waiterObjId
- nextObjId = currentObj.getNextWaitingResourceObjId();
-
- if (prevObjId != -1) {
- //prev->next = next
- prevObj.setNextWaitingResourceObjId(nextObjId);
- } else {
- //removed first waiter. firstWaiter = current->next
- firstWaitingResource = nextObjId;
- }
-
- // if (LockManager.IS_DEBUG_MODE) {
- // System.out.println(Thread.currentThread().getName()+"\tJobInfo_RemoveWaiter:"+ waiterObjId + ", FirstWaiter:"+firstWaitingResource);
- // }
- }
-
- public void increaseDatasetISLockCount(int datasetId) {
- int count = datasetISLockHT.get(datasetId);
- if (count == -1) {
- datasetISLockHT.put(datasetId, 1);
- } else {
- datasetISLockHT.upsert(datasetId, count + 1);
- }
- }
-
- public void decreaseDatasetISLockCount(int datasetId, int entityToDatasetLockEscalationThreshold) {
- int count = datasetISLockHT.get(datasetId);
- if (count >= entityToDatasetLockEscalationThreshold) {
- //do not decrease the count since it is already escalated.
- } else if (count > 1) {
- datasetISLockHT.upsert(datasetId, count - 1);
- } else if (count == 1) {
- datasetISLockHT.remove(datasetId);
- } else if (count <= 0) {
- throw new IllegalStateException("Illegal state of datasetLock count(" + count + ") in JobInfo's dLockHT");
- }
- }
-
- public int getDatasetISLockCount(int datasetId) {
- int count = datasetISLockHT.get(datasetId);
- if (count == -1) {
- return 0;
- } else {
- return count;
- }
- }
-
- /**********************************************************************************
- * public boolean isDatasetLockGranted(int datasetId) {
- * return dLockHT.get(datasetId) == -1 ? false : true;
- * }
- **********************************************************************************/
-
- public boolean isDatasetLockGranted(int datasetId, byte lockMode) {
- int entityInfo = lastHoldingResource;
- byte datasetLockMode;
-
- while (entityInfo != -1) {
- datasetLockMode = entityInfoManager.getDatasetLockMode(entityInfo);
- datasetLockMode = entityInfoManager.getPKHashVal(entityInfo) == -1 ? datasetLockMode
- : datasetLockMode == LockMode.S ? LockMode.IS : LockMode.IX;
- if (entityInfoManager.getDatasetId(entityInfo) == datasetId
- && isStrongerOrEqualToLockMode(datasetLockMode, lockMode)) {
- return true;
- }
- entityInfo = entityInfoManager.getPrevJobResource(entityInfo);
- }
- return false;
- }
-
- //check whether LockMode modeA is stronger than or equal to LockMode modeB
- private boolean isStrongerOrEqualToLockMode(byte modeA, byte modeB) {
- switch (modeB) {
- case LockMode.X:
- return modeA == LockMode.X;
-
- case LockMode.IX:
- return modeA == LockMode.IX || modeA == LockMode.X;
-
- case LockMode.S:
- return modeA == LockMode.S || modeA == LockMode.X;
-
- case LockMode.IS:
- return true;
-
- default:
- throw new IllegalStateException("Unsupported dataset lock mode.");
- }
- }
-
- public String printHoldingResource() {
- StringBuilder s = new StringBuilder();
- int entityInfo = lastHoldingResource;
-
- while (entityInfo != -1) {
- s.append("entityInfo[").append(entityInfo).append("] ");
- s.append(entityInfoManager.getJobId(entityInfo)).append(" ");
- s.append(entityInfoManager.getDatasetId(entityInfo)).append(" ");
- s.append(entityInfoManager.getPKHashVal(entityInfo)).append(" ");
- s.append(entityInfoManager.getDatasetLockMode(entityInfo)).append(" ");
- s.append(entityInfoManager.getDatasetLockCount(entityInfo)).append(" ");
- s.append(entityInfoManager.getEntityLockCount(entityInfo)).append(" ");
- s.append(entityInfoManager.getEntityLockMode(entityInfo)).append(" ");
- s.append("\n");
- entityInfo = entityInfoManager.getPrevJobResource(entityInfo);
- }
- return s.toString();
- }
-
- public String coreDump() {
- StringBuilder sb = new StringBuilder();
- sb.append("\n\t datasetISLockHT");
- sb.append(datasetISLockHT.prettyPrint());
- sb.append("\n\t firstWaitingResource: " + firstWaitingResource);
- sb.append("\n\t lastHoldingResource: " + lastHoldingResource);
- sb.append("\n\t upgradingResource: " + upgradingResource);
- sb.append("\n\t jobCtx.jobId: " + jobCtx.getJobId());
- return sb.toString();
- }
-
- /////////////////////////////////////////////////////////
- // set/get method for private variable
- /////////////////////////////////////////////////////////
- public void setlastHoldingResource(int resource) {
- lastHoldingResource = resource;
- }
-
- public int getLastHoldingResource() {
- return lastHoldingResource;
- }
-
- public void setFirstWaitingResource(int resource) {
- firstWaitingResource = resource;
- }
-
- public int getFirstWaitingResource() {
- return firstWaitingResource;
- }
-
- public void setUpgradingResource(int resource) {
- upgradingResource = resource;
- }
-
- public int getUpgradingResource() {
- return upgradingResource;
- }
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockManager.java
deleted file mode 100644
index f1e1915..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockManager.java
+++ /dev/null
@@ -1,2289 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you 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 at
- *
- * 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 org.apache.asterix.transaction.management.service.locking;
-
-import java.io.OutputStream;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import org.apache.asterix.common.config.AsterixTransactionProperties;
-import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.transactions.DatasetId;
-import org.apache.asterix.common.transactions.ILockManager;
-import org.apache.asterix.common.transactions.ITransactionContext;
-import org.apache.asterix.common.transactions.ITransactionManager;
-import org.apache.asterix.common.transactions.JobId;
-import org.apache.asterix.common.transactions.LogRecord;
-import org.apache.asterix.common.transactions.LogType;
-import org.apache.asterix.transaction.management.service.logging.LogBuffer;
-import org.apache.asterix.transaction.management.service.logging.LogBufferTailReader;
-import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
-import org.apache.asterix.transaction.management.service.transaction.TransactionSubsystem;
-import org.apache.hyracks.api.lifecycle.ILifeCycleComponent;
-
-/**
- * An implementation of the ILockManager interface for the
- * specific case of locking protocol with two lock modes: (S) and (X),
- * where S lock mode is shown by 0, and X lock mode is shown by 1.
- *
- * @author pouria, kisskys
- */
-
-public class LockManager implements ILockManager, ILifeCycleComponent {
-
- public static final boolean IS_DEBUG_MODE = false;//true
- //This variable indicates that the dataset granule X lock request is allowed when
- //there are concurrent lock requests. As of 4/16/2013, we only allow the dataset granule X lock
- //during DDL operation which is preceded by holding X latch on metadata.
- //Therefore, we don't allow the concurrent lock requests with the dataset granule X lock.
- public static final boolean ALLOW_DATASET_GRANULE_X_LOCK_WITH_OTHER_CONCURRENT_LOCK_REQUESTS = false;
-
- public static final boolean ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET = true;
- private static final int DO_ESCALATE = 0;
- private static final int ESCALATED = 1;
- private static final int DONOT_ESCALATE = 2;
-
- private final TransactionSubsystem txnSubsystem;
-
- //all threads accessing to LockManager's tables such as jobHT and datasetResourceHT
- //are serialized through LockTableLatch. All threads waiting the latch will be fairly served
- //in FIFO manner when the latch is available.
- private final ReadWriteLock lockTableLatch;
- private final ReadWriteLock waiterLatch;
- private final HashMap<JobId, JobInfo> jobHT;
- private final HashMap<DatasetId, DatasetLockInfo> datasetResourceHT;
-
- private final EntityLockInfoManager entityLockInfoManager;
- private final EntityInfoManager entityInfoManager;
- private final LockWaiterManager lockWaiterManager;
-
- private final DeadlockDetector deadlockDetector;
- private final TimeOutDetector toutDetector;
- private final DatasetId tempDatasetIdObj; //temporary object to avoid object creation
- private final JobId tempJobIdObj;
-
- private int tryLockDatasetGranuleRevertOperation;
-
- private LockRequestTracker lockRequestTracker; //for debugging
- private final ConsecutiveWakeupContext consecutiveWakeupContext;
-
- 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.lockWaiterManager = new LockWaiterManager();
- this.entityLockInfoManager = new EntityLockInfoManager(entityInfoManager, lockWaiterManager);
- this.deadlockDetector = new DeadlockDetector(jobHT, datasetResourceHT, entityLockInfoManager,
- entityInfoManager, lockWaiterManager);
- this.toutDetector = new TimeOutDetector(this, txnSubsystem.getAsterixAppRuntimeContextProvider()
- .getThreadExecutor());
- this.tempDatasetIdObj = new DatasetId(0);
- this.tempJobIdObj = new JobId(0);
- this.consecutiveWakeupContext = new ConsecutiveWakeupContext();
- if (IS_DEBUG_MODE) {
- this.lockRequestTracker = new LockRequestTracker();
- }
- }
-
- public AsterixTransactionProperties getTransactionProperties() {
- return this.txnSubsystem.getTransactionProperties();
- }
-
- @Override
- public void lock(DatasetId datasetId, int entityHashValue, byte lockMode, ITransactionContext txnContext)
- throws ACIDException {
- internalLock(datasetId, entityHashValue, lockMode, txnContext, false);
- }
-
- private void internalLock(DatasetId datasetId, int entityHashValue, byte lockMode, ITransactionContext txnContext,
- boolean isInstant) throws ACIDException {
-
- JobId jobId = txnContext.getJobId();
- int jId = jobId.getId(); //int-type jobId
- int dId = datasetId.getId(); //int-type datasetId
- int entityInfo;
- int eLockInfo = -1;
- DatasetLockInfo dLockInfo = null;
- JobInfo jobInfo;
- byte datasetLockMode = entityHashValue == -1 ? lockMode : lockMode == LockMode.S ? LockMode.IS : LockMode.IX;
- boolean doEscalate = false;
- boolean caughtLockMgrLatchException = false;
-
- latchLockTable();
- try {
- validateJob(txnContext);
-
- if (IS_DEBUG_MODE) {
- trackLockRequest("Requested", RequestType.LOCK, datasetId, entityHashValue, lockMode, txnContext,
- dLockInfo, eLockInfo);
- }
-
- dLockInfo = datasetResourceHT.get(datasetId);
- jobInfo = jobHT.get(jobId);
-
- if (ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
- if (!isInstant && datasetLockMode == LockMode.IS && jobInfo != null && dLockInfo != null) {
- int escalateStatus = needEscalateFromEntityToDataset(jobInfo, dId, lockMode);
- switch (escalateStatus) {
- case DO_ESCALATE:
- entityHashValue = -1;
- doEscalate = true;
- break;
-
- case ESCALATED:
- return;
-
- default:
- break;
- }
- }
- }
-
- //#. if the datasetLockInfo doesn't exist in datasetResourceHT
- if (dLockInfo == null || dLockInfo.isNoHolder()) {
- if (dLockInfo == null) {
- dLockInfo = new DatasetLockInfo(entityLockInfoManager, entityInfoManager, lockWaiterManager);
- datasetResourceHT.put(new DatasetId(dId), dLockInfo); //datsetId obj should be created
- }
- entityInfo = entityInfoManager.allocate(jId, dId, entityHashValue, lockMode);
-
- //if dataset-granule lock
- if (entityHashValue == -1) { //-1 stands for dataset-granule
- entityInfoManager.increaseDatasetLockCount(entityInfo);
- dLockInfo.increaseLockCount(datasetLockMode);
- dLockInfo.addHolder(entityInfo);
- } else {
- entityInfoManager.increaseDatasetLockCount(entityInfo);
- dLockInfo.increaseLockCount(datasetLockMode);
- //add entityLockInfo
- eLockInfo = entityLockInfoManager.allocate();
- dLockInfo.getEntityResourceHT().put(entityHashValue, eLockInfo);
- entityInfoManager.increaseEntityLockCount(entityInfo);
- entityLockInfoManager.increaseLockCount(eLockInfo, lockMode);
- entityLockInfoManager.addHolder(eLockInfo, entityInfo);
- }
-
- if (jobInfo == null) {
- jobInfo = new JobInfo(entityInfoManager, lockWaiterManager, txnContext);
- jobHT.put(jobId, jobInfo); //jobId obj doesn't have to be created
- }
- jobInfo.addHoldingResource(entityInfo);
-
- if (ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
- if (!isInstant && datasetLockMode == LockMode.IS) {
- jobInfo.increaseDatasetISLockCount(dId);
- if (doEscalate) {
- throw new IllegalStateException(
- "ESCALATE_TRHESHOLD_ENTITY_TO_DATASET should not be set to "
- + txnSubsystem.getTransactionProperties()
- .getEntityToDatasetLockEscalationThreshold());
- }
- }
- }
-
- if (IS_DEBUG_MODE) {
- trackLockRequest("Granted", RequestType.LOCK, datasetId, entityHashValue, lockMode, txnContext,
- dLockInfo, eLockInfo);
- }
-
- return;
- }
-
- //#. the datasetLockInfo exists in datasetResourceHT.
- //1. handle dataset-granule lock
- entityInfo = lockDatasetGranule(datasetId, entityHashValue, lockMode, txnContext);
-
- //2. handle entity-granule lock
- if (entityHashValue != -1) {
- lockEntityGranule(datasetId, entityHashValue, lockMode, entityInfo, txnContext);
- }
-
- if (ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
- if (!isInstant) {
- if (doEscalate) {
- //jobInfo must not be null.
- assert jobInfo != null;
- jobInfo.increaseDatasetISLockCount(dId);
- //release pre-acquired locks
- releaseDatasetISLocks(jobInfo, jobId, datasetId, txnContext);
- } else if (datasetLockMode == LockMode.IS) {
- if (jobInfo == null) {
- jobInfo = jobHT.get(jobId);
- //jobInfo must not be null;
- assert jobInfo != null;
- }
- jobInfo.increaseDatasetISLockCount(dId);
- }
- }
- }
-
- if (IS_DEBUG_MODE) {
- trackLockRequest("Granted", RequestType.LOCK, datasetId, entityHashValue, lockMode, txnContext,
- dLockInfo, eLockInfo);
- }
- } catch (Exception e) {
- if (e instanceof LockMgrLatchHandlerException) {
- // don't unlatch
- caughtLockMgrLatchException = true;
- throw new ACIDException(((LockMgrLatchHandlerException) e).getInternalException());
- }
- } finally {
- if (!caughtLockMgrLatchException) {
- unlatchLockTable();
- }
- }
-
- return;
- }
-
- private void releaseDatasetISLocks(JobInfo jobInfo, JobId jobId, DatasetId datasetId, ITransactionContext txnContext)
- throws ACIDException {
- int entityInfo;
- int prevEntityInfo;
- int entityHashValue;
- int did;//int-type dataset id
-
- //while traversing all holding resources,
- //release IS locks on the escalated dataset and
- //release S locks on the corresponding enttites
- //by calling unlock() method.
- entityInfo = jobInfo.getLastHoldingResource();
- while (entityInfo != -1) {
- prevEntityInfo = entityInfoManager.getPrevJobResource(entityInfo);
-
- //release a lock only if the datset is the escalated dataset and
- //the entityHashValue is not -1("not -1" means a non-dataset-level lock)
- did = entityInfoManager.getDatasetId(entityInfo);
- entityHashValue = entityInfoManager.getPKHashVal(entityInfo);
- if (did == datasetId.getId() && entityHashValue != -1) {
- this.unlock(datasetId, entityHashValue, LockMode.ANY, txnContext);
- }
-
- entityInfo = prevEntityInfo;
- }
- }
-
- private int needEscalateFromEntityToDataset(JobInfo jobInfo, int datasetId, byte lockMode) {
- //we currently allow upgrade only if the lockMode is S.
- if (lockMode != LockMode.S) {
- return DONOT_ESCALATE;
- }
-
- int count = jobInfo.getDatasetISLockCount(datasetId);
- if (count == txnSubsystem.getTransactionProperties().getEntityToDatasetLockEscalationThreshold()) {
- return DO_ESCALATE;
- } else if (count > txnSubsystem.getTransactionProperties().getEntityToDatasetLockEscalationThreshold()) {
- return ESCALATED;
- } else {
- return DONOT_ESCALATE;
- }
- }
-
- private void validateJob(ITransactionContext txnContext) throws ACIDException {
- if (txnContext.getTxnState() == ITransactionManager.ABORTED) {
- throw new ACIDException("" + txnContext.getJobId() + " is in ABORTED state.");
- } else if (txnContext.isTimeout()) {
- requestAbort(txnContext);
- }
- }
-
- private int lockDatasetGranule(DatasetId datasetId, int entityHashValue, byte lockMode,
- ITransactionContext txnContext) throws ACIDException {
- JobId jobId = txnContext.getJobId();
- int jId = jobId.getId(); //int-type jobId
- int dId = datasetId.getId(); //int-type datasetId
- int waiterObjId;
- int entityInfo = -1;
- DatasetLockInfo dLockInfo;
- JobInfo jobInfo;
- boolean isUpgrade = false;
- int weakerModeLockCount;
- int waiterCount = 0;
- byte datasetLockMode = entityHashValue == -1 ? lockMode : lockMode == LockMode.S ? LockMode.IS : LockMode.IX;
-
- dLockInfo = datasetResourceHT.get(datasetId);
- jobInfo = jobHT.get(jobId);
-
- //check duplicated call
-
- //1. lock request causing duplicated upgrading requests from different threads in a same job
- waiterObjId = dLockInfo.findUpgraderFromUpgraderList(jId, entityHashValue);
- if (waiterObjId != -1) {
- //make the caller wait on the same LockWaiter object
- entityInfo = lockWaiterManager.getLockWaiter(waiterObjId).getEntityInfoSlot();
- waiterCount = handleLockWaiter(dLockInfo, -1, entityInfo, true, true, txnContext, jobInfo, waiterObjId);
-
- //Only for the first-get-up thread, the waiterCount will be more than 0 and
- //the thread updates lock count on behalf of the all other waiting threads.
- //Therefore, all the next-get-up threads will not update any lock count.
- if (waiterCount > 0) {
- //add ((the number of waiting upgrader) - 1) to entityInfo's dataset lock count and datasetLockInfo's lock count
- //where -1 is for not counting the first upgrader's request since the lock count for the first upgrader's request
- //is already counted.
- weakerModeLockCount = entityInfoManager.getDatasetLockCount(entityInfo);
- entityInfoManager.setDatasetLockMode(entityInfo, lockMode);
- entityInfoManager.increaseDatasetLockCount(entityInfo, waiterCount - 1);
-
- if (entityHashValue == -1) { //dataset-granule lock
- dLockInfo.increaseLockCount(LockMode.X, weakerModeLockCount + waiterCount - 1);//new lock mode
- dLockInfo.decreaseLockCount(LockMode.S, weakerModeLockCount);//current lock mode
- } else {
- dLockInfo.increaseLockCount(LockMode.IX, weakerModeLockCount + waiterCount - 1);
- dLockInfo.decreaseLockCount(LockMode.IS, weakerModeLockCount);
- }
- }
-
- return entityInfo;
- }
-
- //2. lock request causing duplicated waiting requests from different threads in a same job
- waiterObjId = dLockInfo.findWaiterFromWaiterList(jId, entityHashValue);
- if (waiterObjId != -1) {
- //make the caller wait on the same LockWaiter object
- entityInfo = lockWaiterManager.getLockWaiter(waiterObjId).getEntityInfoSlot();
- waiterCount = handleLockWaiter(dLockInfo, -1, entityInfo, false, true, txnContext, jobInfo, waiterObjId);
-
- if (waiterCount > 0) {
- entityInfoManager.increaseDatasetLockCount(entityInfo, waiterCount);
- if (entityHashValue == -1) {
- dLockInfo.increaseLockCount(datasetLockMode, waiterCount);
- dLockInfo.addHolder(entityInfo);
- } else {
- dLockInfo.increaseLockCount(datasetLockMode, waiterCount);
- //IS and IX holders are implicitly handled.
- }
- //add entityInfo to JobInfo's holding-resource list
- jobInfo.addHoldingResource(entityInfo);
- }
-
- return entityInfo;
- }
-
- //3. lock request causing duplicated holding requests from different threads or a single thread in a same job
- entityInfo = dLockInfo.findEntityInfoFromHolderList(jId, entityHashValue);
- if (entityInfo == -1) {
-
- entityInfo = entityInfoManager.allocate(jId, dId, entityHashValue, lockMode);
- if (jobInfo == null) {
- jobInfo = new JobInfo(entityInfoManager, lockWaiterManager, txnContext);
- jobHT.put(jobId, jobInfo);
- }
-
- //wait if any upgrader exists or upgrading lock mode is not compatible
- if (dLockInfo.getFirstUpgrader() != -1 || dLockInfo.getFirstWaiter() != -1
- || !dLockInfo.isCompatible(datasetLockMode)) {
-
- /////////////////////////////////////////////////////////////////////////////////////////////
- //[Notice] Mimicking SIX mode
- //When the lock escalation from IS to S in dataset-level is allowed, the following case occurs
- //DatasetLockInfo's SCount = 1 and the same job who carried out the escalation tries to insert,
- //then the job should be able to insert without being blocked by itself.
- //Our approach is to introduce SIX mode, but we don't have currently,
- //so I simply mimicking SIX by allowing S and IX coexist in the dataset level
- //only if their job id is identical for the requests.
- if (ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
- if (datasetLockMode == LockMode.IX && dLockInfo.getSCount() == 1
- && jobInfo.isDatasetLockGranted(dId, LockMode.S)) {
- entityInfoManager.increaseDatasetLockCount(entityInfo);
- //IX holders are implicitly handled without adding holder
- dLockInfo.increaseLockCount(datasetLockMode);
- //add entityInfo to JobInfo's holding-resource list
- jobInfo.addHoldingResource(entityInfo);
- return entityInfo;
- }
- }
- ///////////////////////////////////////////////////////////////////////////////////////////////
-
- /////////////////////////////////////////////////////////////////////////////////////////////
- if (ALLOW_DATASET_GRANULE_X_LOCK_WITH_OTHER_CONCURRENT_LOCK_REQUESTS) {
- //The following case only may occur when the dataset level X lock is requested
- //with the other lock
-
- //[Notice]
- //There has been no same caller as (jId, dId, entityHashValue) triplet.
- //But there could be the same caller in terms of (jId, dId) pair.
- //For example,
- //1) (J1, D1, E1) acquires IS in Dataset D1
- //2) (J2, D1, -1) requests X in Dataset D1, but waits
- //3) (J1, D1, E2) requests IS in Dataset D1, but should wait
- //The 3) may cause deadlock if 1) and 3) are under the same thread.
- //Even if (J1, D1, E1) and (J1, D1, E2) are two different thread, instead of
- //aborting (J1, D1, E1) triggered by the deadlock, we give higher priority to 3) than 2)
- //as long as the dataset level lock D1 is being held by the same jobId.
- //The above consideration is covered in the following code.
- //find the same dataset-granule lock request, that is, (J1, D1) pair in the above example.
- if (jobInfo.isDatasetLockGranted(dId, LockMode.IS)) {
- if (dLockInfo.isCompatible(datasetLockMode)) {
- //this is duplicated call
- entityInfoManager.increaseDatasetLockCount(entityInfo);
- if (entityHashValue == -1) {
- dLockInfo.increaseLockCount(datasetLockMode);
- dLockInfo.addHolder(entityInfo);
- } else {
- dLockInfo.increaseLockCount(datasetLockMode);
- //IS and IX holders are implicitly handled.
- }
- //add entityInfo to JobInfo's holding-resource list
- jobInfo.addHoldingResource(entityInfo);
-
- return entityInfo;
- } else {
- //considered as upgrader
- waiterCount = handleLockWaiter(dLockInfo, -1, entityInfo, true, true, txnContext, jobInfo,
- -1);
- if (waiterCount > 0) {
- entityInfoManager.increaseDatasetLockCount(entityInfo);
- if (entityHashValue == -1) {
- dLockInfo.increaseLockCount(datasetLockMode);
- dLockInfo.addHolder(entityInfo);
- } else {
- dLockInfo.increaseLockCount(datasetLockMode);
- //IS and IX holders are implicitly handled.
- }
- //add entityInfo to JobInfo's holding-resource list
- jobInfo.addHoldingResource(entityInfo);
- }
- return entityInfo;
- }
- }
- }
- /////////////////////////////////////////////////////////////////////////////////////////////
-
- waiterCount = handleLockWaiter(dLockInfo, -1, entityInfo, false, true, txnContext, jobInfo, -1);
- } else {
- waiterCount = 1;
- }
-
- if (waiterCount > 0) {
- entityInfoManager.increaseDatasetLockCount(entityInfo);
- if (entityHashValue == -1) {
- dLockInfo.increaseLockCount(datasetLockMode);
- dLockInfo.addHolder(entityInfo);
- } else {
- dLockInfo.increaseLockCount(datasetLockMode);
- //IS and IX holders are implicitly handled.
- }
- //add entityInfo to JobInfo's holding-resource list
- jobInfo.addHoldingResource(entityInfo);
- }
- } else {
- isUpgrade = isLockUpgrade(entityInfoManager.getDatasetLockMode(entityInfo), lockMode);
- if (isUpgrade) { //upgrade call
- //wait if any upgrader exists or upgrading lock mode is not compatible
- if (dLockInfo.getFirstUpgrader() != -1 || !dLockInfo.isUpgradeCompatible(datasetLockMode, entityInfo)) {
- waiterCount = handleLockWaiter(dLockInfo, -1, entityInfo, true, true, txnContext, jobInfo, -1);
- } else {
- waiterCount = 1;
- }
-
- if (waiterCount > 0) {
- //add ((the number of waiting upgrader) - 1) to entityInfo's dataset lock count and datasetLockInfo's lock count
- //where -1 is for not counting the first upgrader's request since the lock count for the first upgrader's request
- //is already counted.
- weakerModeLockCount = entityInfoManager.getDatasetLockCount(entityInfo);
- entityInfoManager.setDatasetLockMode(entityInfo, lockMode);
- entityInfoManager.increaseDatasetLockCount(entityInfo, waiterCount - 1);
-
- if (entityHashValue == -1) { //dataset-granule lock
- dLockInfo.increaseLockCount(LockMode.X, weakerModeLockCount + waiterCount - 1);//new lock mode
- dLockInfo.decreaseLockCount(LockMode.S, weakerModeLockCount);//current lock mode
- } else {
- dLockInfo.increaseLockCount(LockMode.IX, weakerModeLockCount + waiterCount - 1);
- dLockInfo.decreaseLockCount(LockMode.IS, weakerModeLockCount);
- }
- }
- } else { //duplicated call
- entityInfoManager.increaseDatasetLockCount(entityInfo);
- datasetLockMode = entityInfoManager.getDatasetLockMode(entityInfo);
-
- if (entityHashValue == -1) { //dataset-granule
- dLockInfo.increaseLockCount(datasetLockMode);
- } else { //entity-granule
- datasetLockMode = datasetLockMode == LockMode.S ? LockMode.IS : LockMode.IX;
- dLockInfo.increaseLockCount(datasetLockMode);
- }
- }
- }
-
- return entityInfo;
- }
-
- private void lockEntityGranule(DatasetId datasetId, int entityHashValue, byte lockMode,
- int entityInfoFromDLockInfo, ITransactionContext txnContext) throws ACIDException {
- JobId jobId = txnContext.getJobId();
- int jId = jobId.getId(); //int-type jobId
- int waiterObjId;
- int eLockInfo = -1;
- int entityInfo;
- DatasetLockInfo dLockInfo;
- JobInfo jobInfo;
- boolean isUpgrade = false;
- int waiterCount = 0;
- int weakerModeLockCount;
-
- dLockInfo = datasetResourceHT.get(datasetId);
- jobInfo = jobHT.get(jobId);
- eLockInfo = dLockInfo.getEntityResourceHT().get(entityHashValue);
-
- if (eLockInfo != -1) {
- //check duplicated call
-
- //1. lock request causing duplicated upgrading requests from different threads in a same job
- waiterObjId = entityLockInfoManager.findUpgraderFromUpgraderList(eLockInfo, jId, entityHashValue);
- if (waiterObjId != -1) {
- entityInfo = lockWaiterManager.getLockWaiter(waiterObjId).getEntityInfoSlot();
- waiterCount = handleLockWaiter(dLockInfo, eLockInfo, -1, true, false, txnContext, jobInfo, waiterObjId);
-
- if (waiterCount > 0) {
- weakerModeLockCount = entityInfoManager.getEntityLockCount(entityInfo);
- entityInfoManager.setEntityLockMode(entityInfo, LockMode.X);
- entityInfoManager.increaseEntityLockCount(entityInfo, waiterCount - 1);
-
- entityLockInfoManager.increaseLockCount(eLockInfo, LockMode.X, (short) (weakerModeLockCount
- + waiterCount - 1));//new lock mode
- entityLockInfoManager.decreaseLockCount(eLockInfo, LockMode.S, (short) weakerModeLockCount);//old lock mode
- }
- return;
- }
-
- //2. lock request causing duplicated waiting requests from different threads in a same job
- waiterObjId = entityLockInfoManager.findWaiterFromWaiterList(eLockInfo, jId, entityHashValue);
- if (waiterObjId != -1) {
- entityInfo = lockWaiterManager.getLockWaiter(waiterObjId).getEntityInfoSlot();
- waiterCount = handleLockWaiter(dLockInfo, eLockInfo, -1, false, false, txnContext, jobInfo, waiterObjId);
-
- if (waiterCount > 0) {
- entityInfoManager.increaseEntityLockCount(entityInfo, waiterCount);
- entityLockInfoManager.increaseLockCount(eLockInfo, lockMode, (short) waiterCount);
- entityLockInfoManager.addHolder(eLockInfo, entityInfo);
- }
- return;
- }
-
- //3. lock request causing duplicated holding requests from different threads or a single thread in a same job
- entityInfo = entityLockInfoManager.findEntityInfoFromHolderList(eLockInfo, jId, entityHashValue);
- if (entityInfo != -1) {//duplicated call or upgrader
-
- isUpgrade = isLockUpgrade(entityInfoManager.getEntityLockMode(entityInfo), lockMode);
- if (isUpgrade) {//upgrade call
- //wait if any upgrader exists or upgrading lock mode is not compatible
- if (entityLockInfoManager.getUpgrader(eLockInfo) != -1
- || !entityLockInfoManager.isUpgradeCompatible(eLockInfo, lockMode, entityInfo)) {
- waiterCount = handleLockWaiter(dLockInfo, eLockInfo, entityInfo, true, false, txnContext,
- jobInfo, -1);
- } else {
- waiterCount = 1;
- }
-
- if (waiterCount > 0) {
- weakerModeLockCount = entityInfoManager.getEntityLockCount(entityInfo);
- entityInfoManager.setEntityLockMode(entityInfo, lockMode);
- entityInfoManager.increaseEntityLockCount(entityInfo, waiterCount - 1);
-
- entityLockInfoManager.increaseLockCount(eLockInfo, LockMode.X, (short) (weakerModeLockCount
- + waiterCount - 1));//new lock mode
- entityLockInfoManager.decreaseLockCount(eLockInfo, LockMode.S, (short) weakerModeLockCount);//old lock mode
- }
-
- } else {//duplicated call
- entityInfoManager.increaseEntityLockCount(entityInfo);
- entityLockInfoManager.increaseLockCount(eLockInfo, entityInfoManager.getEntityLockMode(entityInfo));
- }
- } else {//new call from this job, but still eLockInfo exists since other threads hold it or wait on it
- entityInfo = entityInfoFromDLockInfo;
- if (entityLockInfoManager.getUpgrader(eLockInfo) != -1
- || entityLockInfoManager.getFirstWaiter(eLockInfo) != -1
- || !entityLockInfoManager.isCompatible(eLockInfo, lockMode)) {
- waiterCount = handleLockWaiter(dLockInfo, eLockInfo, entityInfo, false, false, txnContext, jobInfo,
- -1);
- } else {
- waiterCount = 1;
- }
-
- if (waiterCount > 0) {
- entityInfoManager.increaseEntityLockCount(entityInfo, waiterCount);
- entityLockInfoManager.increaseLockCount(eLockInfo, lockMode, (short) waiterCount);
- entityLockInfoManager.addHolder(eLockInfo, entityInfo);
- }
- }
- } else {//eLockInfo doesn't exist, so this lock request is the first request and can be granted without waiting.
- eLockInfo = entityLockInfoManager.allocate();
- dLockInfo.getEntityResourceHT().put(entityHashValue, eLockInfo);
- entityInfoManager.increaseEntityLockCount(entityInfoFromDLockInfo);
- entityLockInfoManager.increaseLockCount(eLockInfo, lockMode);
- entityLockInfoManager.addHolder(eLockInfo, entityInfoFromDLockInfo);
- }
- }
-
- @Override
- public void unlock(DatasetId datasetId, int entityHashValue, byte lockMode, 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);
- }
-
- private void internalUnlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext,
- boolean isInstant) throws ACIDException {
- JobId jobId = txnContext.getJobId();
- int eLockInfo = -1;
- DatasetLockInfo dLockInfo = null;
- JobInfo jobInfo;
- int entityInfo = -1;
- byte datasetLockMode;
-
- if (IS_DEBUG_MODE) {
- if (entityHashValue == -1) {
- throw new UnsupportedOperationException(
- "Unsupported unlock request: dataset-granule unlock is not supported");
- }
- }
-
- latchLockTable();
- try {
- if (IS_DEBUG_MODE) {
- trackLockRequest("Requested", RequestType.UNLOCK, datasetId, entityHashValue, (byte) 0, txnContext,
- dLockInfo, eLockInfo);
- }
-
- //find the resource to be unlocked
- dLockInfo = datasetResourceHT.get(datasetId);
- jobInfo = jobHT.get(jobId);
- if (dLockInfo == null || jobInfo == null) {
- throw new IllegalStateException("Invalid unlock request: Corresponding lock info doesn't exist.");
- }
-
- eLockInfo = dLockInfo.getEntityResourceHT().get(entityHashValue);
-
- if (eLockInfo == -1) {
- throw new IllegalStateException("Invalid unlock request: Corresponding lock info doesn't exist.");
- }
-
- //find the corresponding entityInfo
- entityInfo = entityLockInfoManager.findEntityInfoFromHolderList(eLockInfo, jobId.getId(), entityHashValue);
- if (entityInfo == -1) {
- throw new IllegalStateException("Invalid unlock request[" + jobId.getId() + "," + datasetId.getId()
- + "," + entityHashValue + "]: Corresponding lock info doesn't exist.");
- }
-
- datasetLockMode = entityInfoManager.getDatasetLockMode(entityInfo) == LockMode.S ? LockMode.IS
- : LockMode.IX;
-
- //decrease the corresponding count of dLockInfo/eLockInfo/entityInfo
- dLockInfo.decreaseLockCount(datasetLockMode);
- entityLockInfoManager.decreaseLockCount(eLockInfo, entityInfoManager.getEntityLockMode(entityInfo));
- entityInfoManager.decreaseDatasetLockCount(entityInfo);
- entityInfoManager.decreaseEntityLockCount(entityInfo);
-
- if (entityInfoManager.getEntityLockCount(entityInfo) == 0
- && entityInfoManager.getDatasetLockCount(entityInfo) == 0) {
- int threadCount = 0; //number of threads(in the same job) waiting on the same resource
- int waiterObjId = jobInfo.getFirstWaitingResource();
- int waitingEntityInfo;
- LockWaiter waiterObj;
-
- //1) wake up waiters and remove holder
- //wake up waiters of dataset-granule lock
- wakeUpDatasetLockWaiters(dLockInfo);
- //wake up waiters of entity-granule lock
- wakeUpEntityLockWaiters(eLockInfo);
- //remove the holder from eLockInfo's holder list and remove the holding resource from jobInfo's holding resource list
- //this can be done in the following single function call.
- entityLockInfoManager.removeHolder(eLockInfo, entityInfo, jobInfo);
-
- //2) if
- // there is no waiting thread on the same resource (this can be checked through jobInfo)
- // then
- // a) delete the corresponding entityInfo
- // b) write commit log for the unlocked resource(which is a committed txn).
- while (waiterObjId != -1) {
- waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
- waitingEntityInfo = waiterObj.getEntityInfoSlot();
- if (entityInfoManager.getDatasetId(waitingEntityInfo) == datasetId.getId()
- && entityInfoManager.getPKHashVal(waitingEntityInfo) == entityHashValue) {
- threadCount++;
- break;
- }
- waiterObjId = waiterObj.getNextWaiterObjId();
- }
- if (threadCount == 0) {
- entityInfoManager.deallocate(entityInfo);
- }
- }
-
- //deallocate entityLockInfo's slot if there is no txn referring to the entityLockInfo.
- if (entityLockInfoManager.getFirstWaiter(eLockInfo) == -1
- && entityLockInfoManager.getLastHolder(eLockInfo) == -1
- && entityLockInfoManager.getUpgrader(eLockInfo) == -1) {
- dLockInfo.getEntityResourceHT().remove(entityHashValue);
- entityLockInfoManager.deallocate(eLockInfo);
- }
-
- //we don't deallocate datasetLockInfo even if there is no txn referring to the datasetLockInfo
- //since the datasetLockInfo is likely to be referred to again.
-
- if (ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
- if (!isInstant && datasetLockMode == LockMode.IS) {
- jobInfo.decreaseDatasetISLockCount(datasetId.getId(), txnSubsystem.getTransactionProperties()
- .getEntityToDatasetLockEscalationThreshold());
- }
- }
-
- if (IS_DEBUG_MODE) {
- trackLockRequest("Granted", RequestType.UNLOCK, datasetId, entityHashValue, (byte) 0, txnContext,
- dLockInfo, eLockInfo);
- }
- } finally {
- unlatchLockTable();
- }
- }
-
- @Override
- public void releaseLocks(ITransactionContext txnContext) throws ACIDException {
- LockWaiter waiterObj;
- int entityInfo;
- int prevEntityInfo;
- int entityHashValue;
- DatasetLockInfo dLockInfo = null;
- int eLockInfo = -1;
- int did;//int-type dataset id
- int datasetLockCount;
- int entityLockCount;
- byte lockMode;
- boolean existWaiter = false;
-
- JobId jobId = txnContext.getJobId();
-
- latchLockTable();
- try {
- if (IS_DEBUG_MODE) {
- trackLockRequest("Requested", RequestType.RELEASE_LOCKS, new DatasetId(0), 0, (byte) 0, txnContext,
- dLockInfo, eLockInfo);
- }
-
- JobInfo jobInfo = jobHT.get(jobId);
- if (jobInfo == null) {
- return;
- }
-
- //remove waiterObj of JobInfo
- //[Notice]
- //waiterObjs may exist if aborted thread is the caller of this function.
- //Even if there are the waiterObjs, there is no waiting thread on the objects.
- //If the caller of this function is an aborted thread, it is guaranteed that there is no waiting threads
- //on the waiterObjs since when the aborted caller thread is waken up, all other waiting threads are
- //also waken up at the same time through 'notifyAll()' call.
- //In contrast, if the caller of this function is not an aborted thread, then there is no waiting object.
- int waiterObjId = jobInfo.getFirstWaitingResource();
- int nextWaiterObjId;
- while (waiterObjId != -1) {
- existWaiter = true;
- waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
- nextWaiterObjId = waiterObj.getNextWaitingResourceObjId();
- entityInfo = waiterObj.getEntityInfoSlot();
- if (IS_DEBUG_MODE) {
- if (jobId.getId() != entityInfoManager.getJobId(entityInfo)) {
- throw new IllegalStateException("JobInfo(" + jobId + ") has diffrent Job(JID:"
- + entityInfoManager.getJobId(entityInfo) + "'s lock request!!!");
- }
- }
-
- //1. remove from waiter(or upgrader)'s list of dLockInfo or eLockInfo.
- did = entityInfoManager.getDatasetId(entityInfo);
- tempDatasetIdObj.setId(did);
- dLockInfo = datasetResourceHT.get(tempDatasetIdObj);
-
- if (waiterObj.isWaitingOnEntityLock()) {
- entityHashValue = entityInfoManager.getPKHashVal(entityInfo);
- eLockInfo = dLockInfo.getEntityResourceHT().get(entityHashValue);
- if (waiterObj.isWaiter()) {
- entityLockInfoManager.removeWaiter(eLockInfo, waiterObjId);
- } else {
- entityLockInfoManager.removeUpgrader(eLockInfo, waiterObjId);
- }
- } else {
- if (waiterObj.isWaiter()) {
- dLockInfo.removeWaiter(waiterObjId);
- } else {
- dLockInfo.removeUpgrader(waiterObjId);
- }
- }
-
- //2. wake-up waiters
- latchWaitNotify();
- synchronized (waiterObj) {
- unlatchWaitNotify();
- waiterObj.setWait(false);
- if (IS_DEBUG_MODE) {
- System.out.println("" + Thread.currentThread().getName() + "\twake-up(D): WID(" + waiterObjId
- + "),EID(" + waiterObj.getEntityInfoSlot() + ")");
- }
- waiterObj.notifyAll();
- }
-
- //3. deallocate waiterObj
- lockWaiterManager.deallocate(waiterObjId);
-
- //4. deallocate entityInfo only if this waiter is not an upgrader
- if (entityInfoManager.getDatasetLockCount(entityInfo) == 0
- && entityInfoManager.getEntityLockCount(entityInfo) == 0) {
- entityInfoManager.deallocate(entityInfo);
- }
- waiterObjId = nextWaiterObjId;
- }
-
- //release holding resources
- entityInfo = jobInfo.getLastHoldingResource();
- while (entityInfo != -1) {
- prevEntityInfo = entityInfoManager.getPrevJobResource(entityInfo);
-
- //decrease lock count of datasetLock and entityLock
- did = entityInfoManager.getDatasetId(entityInfo);
- tempDatasetIdObj.setId(did);
- dLockInfo = datasetResourceHT.get(tempDatasetIdObj);
- entityHashValue = entityInfoManager.getPKHashVal(entityInfo);
-
- if (entityHashValue == -1) {
- //decrease datasetLockCount
- lockMode = entityInfoManager.getDatasetLockMode(entityInfo);
- datasetLockCount = entityInfoManager.getDatasetLockCount(entityInfo);
- if (datasetLockCount != 0) {
- dLockInfo.decreaseLockCount(lockMode, datasetLockCount);
-
- //wakeup waiters of datasetLock and remove holder from datasetLockInfo
- wakeUpDatasetLockWaiters(dLockInfo);
-
- //remove the holder from datasetLockInfo only if the lock is dataset-granule lock.
- //--> this also removes the holding resource from jobInfo
- //(Because the IX and IS lock's holders are handled implicitly,
- //those are not in the holder list of datasetLockInfo.)
- dLockInfo.removeHolder(entityInfo, jobInfo);
- }
- } else {
- //decrease datasetLockCount
- lockMode = entityInfoManager.getDatasetLockMode(entityInfo);
- lockMode = lockMode == LockMode.S ? LockMode.IS : LockMode.IX;
- datasetLockCount = entityInfoManager.getDatasetLockCount(entityInfo);
-
- if (datasetLockCount != 0) {
- dLockInfo.decreaseLockCount(lockMode, datasetLockCount);
- }
-
- //decrease entityLockCount
- lockMode = entityInfoManager.getEntityLockMode(entityInfo);
- entityLockCount = entityInfoManager.getEntityLockCount(entityInfo);
- eLockInfo = dLockInfo.getEntityResourceHT().get(entityHashValue);
- if (IS_DEBUG_MODE) {
- if (eLockInfo < 0) {
- System.out.println("eLockInfo:" + eLockInfo);
- }
- }
-
- if (entityLockCount != 0) {
- entityLockInfoManager.decreaseLockCount(eLockInfo, lockMode, (short) entityLockCount);
- }
-
- if (datasetLockCount != 0) {
- //wakeup waiters of datasetLock and don't remove holder from datasetLockInfo
- wakeUpDatasetLockWaiters(dLockInfo);
- }
-
- if (entityLockCount != 0) {
- //wakeup waiters of entityLock
- wakeUpEntityLockWaiters(eLockInfo);
-
- //remove the holder from entityLockInfo
- //--> this also removes the holding resource from jobInfo
- entityLockInfoManager.removeHolder(eLockInfo, entityInfo, jobInfo);
- }
-
- //deallocate entityLockInfo if there is no holder and waiter.
- if (entityLockInfoManager.getLastHolder(eLockInfo) == -1
- && entityLockInfoManager.getFirstWaiter(eLockInfo) == -1
- && entityLockInfoManager.getUpgrader(eLockInfo) == -1) {
- dLockInfo.getEntityResourceHT().remove(entityHashValue);
- entityLockInfoManager.deallocate(eLockInfo);
- }
- }
-
- //deallocate entityInfo
- entityInfoManager.deallocate(entityInfo);
-
- entityInfo = prevEntityInfo;
- }
-
- //remove JobInfo
- jobHT.remove(jobId);
-
- if (existWaiter) {
- txnContext.setTimeout(true);
- txnContext.setTxnState(ITransactionManager.ABORTED);
- }
-
- if (IS_DEBUG_MODE) {
- trackLockRequest("Granted", RequestType.RELEASE_LOCKS, new DatasetId(0), 0, (byte) 0, txnContext,
- dLockInfo, eLockInfo);
- }
- } finally {
- unlatchLockTable();
- }
- }
-
- @Override
- public void instantLock(DatasetId datasetId, int entityHashValue, byte lockMode, ITransactionContext txnContext)
- throws ACIDException {
-
- // try {
- // internalLock(datasetId, entityHashValue, lockMode, txnContext);
- // return;
- // } finally {
- // unlock(datasetId, entityHashValue, txnContext);
- // }
- internalLock(datasetId, entityHashValue, lockMode, txnContext, true);
- instantUnlock(datasetId, entityHashValue, txnContext);
- }
-
- @Override
- public boolean tryLock(DatasetId datasetId, int entityHashValue, byte lockMode, ITransactionContext txnContext)
- throws ACIDException {
- return internalTryLock(datasetId, entityHashValue, lockMode, txnContext, false);
- }
-
- @Override
- public boolean instantTryLock(DatasetId datasetId, int entityHashValue, byte lockMode,
- ITransactionContext txnContext) throws ACIDException {
- return internalInstantTryLock(datasetId, entityHashValue, lockMode, txnContext);
- }
-
- private boolean internalInstantTryLock(DatasetId datasetId, int entityHashValue, byte lockMode,
- ITransactionContext txnContext) throws ACIDException {
- DatasetLockInfo dLockInfo = null;
- boolean isSuccess = true;
-
- latchLockTable();
- try {
- validateJob(txnContext);
-
- if (IS_DEBUG_MODE) {
- trackLockRequest("Requested", RequestType.INSTANT_TRY_LOCK, datasetId, entityHashValue, lockMode,
- txnContext, dLockInfo, -1);
- }
-
- dLockInfo = datasetResourceHT.get(datasetId);
-
- //#. if the datasetLockInfo doesn't exist in datasetResourceHT
- if (dLockInfo == null || dLockInfo.isNoHolder()) {
- if (IS_DEBUG_MODE) {
- trackLockRequest("Granted", RequestType.INSTANT_TRY_LOCK, datasetId, entityHashValue, lockMode,
- txnContext, dLockInfo, -1);
- }
- return true;
- }
-
- //#. the datasetLockInfo exists in datasetResourceHT.
- //1. handle dataset-granule lock
- byte datasetLockMode = entityHashValue == -1 ? lockMode : lockMode == LockMode.S ? LockMode.IS
- : LockMode.IX;
- if (datasetLockMode == LockMode.IS) {
- //[Notice]
- //Skip checking the dataset level lock compatibility if the requested LockMode is IS lock.
- //We know that this internalInstantTryLock() call with IS lock mode will be always granted
- //because we don't allow X lock on dataset-level except DDL operation.
- //During DDL operation, all other operations will be pending, so there is no conflict.
- isSuccess = true;
- } else {
- isSuccess = instantTryLockDatasetGranule(datasetId, entityHashValue, lockMode, txnContext, dLockInfo,
- datasetLockMode);
- }
-
- if (isSuccess && entityHashValue != -1) {
- //2. handle entity-granule lock
- isSuccess = instantTryLockEntityGranule(datasetId, entityHashValue, lockMode, txnContext, dLockInfo);
- }
-
- if (IS_DEBUG_MODE) {
- if (isSuccess) {
- trackLockRequest("Granted", RequestType.INSTANT_TRY_LOCK, datasetId, entityHashValue, lockMode,
- txnContext, dLockInfo, -1);
- } else {
- trackLockRequest("Failed", RequestType.INSTANT_TRY_LOCK, datasetId, entityHashValue, lockMode,
- txnContext, dLockInfo, -1);
- }
- }
-
- } finally {
- unlatchLockTable();
- }
-
- return isSuccess;
- }
-
- private boolean instantTryLockDatasetGranule(DatasetId datasetId, int entityHashValue, byte lockMode,
- ITransactionContext txnContext, DatasetLockInfo dLockInfo, byte datasetLockMode) throws ACIDException {
- JobId jobId = txnContext.getJobId();
- int jId = jobId.getId(); //int-type jobId
- int dId = datasetId.getId(); //int-type datasetId
- int waiterObjId;
- int entityInfo = -1;
- JobInfo jobInfo;
- boolean isUpgrade = false;
-
- jobInfo = jobHT.get(jobId);
-
- //check duplicated call
-
- //1. lock request causing duplicated upgrading requests from different threads in a same job
- waiterObjId = dLockInfo.findUpgraderFromUpgraderList(jId, entityHashValue);
- if (waiterObjId != -1) {
- return false;
- }
-
- //2. lock request causing duplicated waiting requests from different threads in a same job
- waiterObjId = dLockInfo.findWaiterFromWaiterList(jId, entityHashValue);
- if (waiterObjId != -1) {
- return false;
- }
-
- //3. lock request causing duplicated holding requests from different threads or a single thread in a same job
- entityInfo = dLockInfo.findEntityInfoFromHolderList(jId, entityHashValue);
- if (entityInfo == -1) { //new call from this job -> doesn't mean that eLockInfo doesn't exist since another thread might have create the eLockInfo already.
-
- //return fail if any upgrader exists or upgrading lock mode is not compatible
- if (dLockInfo.getFirstUpgrader() != -1 || dLockInfo.getFirstWaiter() != -1
- || !dLockInfo.isCompatible(datasetLockMode)) {
-
- if (ALLOW_DATASET_GRANULE_X_LOCK_WITH_OTHER_CONCURRENT_LOCK_REQUESTS) {
- //The following case only may occur when the dataset level X lock is requested
- //with the other lock
-
- //[Notice]
- //There has been no same caller as (jId, dId, entityHashValue) triplet.
- //But there could be the same caller in terms of (jId, dId) pair.
- //For example,
- //1) (J1, D1, E1) acquires IS in Dataset D1
- //2) (J2, D1, -1) requests X in Dataset D1, but waits
- //3) (J1, D1, E2) requests IS in Dataset D1, but should wait
- //The 3) may cause deadlock if 1) and 3) are under the same thread.
- //Even if (J1, D1, E1) and (J1, D1, E2) are two different thread, instead of
- //aborting (J1, D1, E1) triggered by the deadlock, we give higher priority to 3) than 2)
- //as long as the dataset level lock D1 is being held by the same jobId.
- //The above consideration is covered in the following code.
- //find the same dataset-granule lock request, that is, (J1, D1) pair in the above example.
- if (jobInfo != null && jobInfo.isDatasetLockGranted(dId, LockMode.IS)) {
- if (dLockInfo.isCompatible(datasetLockMode)) {
- //this is duplicated call
- return true;
- }
- }
- }
-
- return false;
- }
- } else {
- isUpgrade = isLockUpgrade(entityInfoManager.getDatasetLockMode(entityInfo), lockMode);
- if (isUpgrade) { //upgrade call
- //return fail if any upgrader exists or upgrading lock mode is not compatible
- if (dLockInfo.getFirstUpgrader() != -1 || !dLockInfo.isUpgradeCompatible(datasetLockMode, entityInfo)) {
- return false;
- }
- }
- /************************************
- * else { //duplicated call
- * //do nothing
- * }
- *************************************/
- }
-
- return true;
- }
-
- private boolean instantTryLockEntityGranule(DatasetId datasetId, int entityHashValue, byte lockMode,
- ITransactionContext txnContext, DatasetLockInfo dLockInfo) throws ACIDException {
- JobId jobId = txnContext.getJobId();
- int jId = jobId.getId(); //int-type jobId
- int waiterObjId;
- int eLockInfo = -1;
- int entityInfo;
- boolean isUpgrade = false;
-
- dLockInfo = datasetResourceHT.get(datasetId);
- eLockInfo = dLockInfo.getEntityResourceHT().get(entityHashValue);
-
- if (eLockInfo != -1) {
- //check duplicated call
-
- //1. lock request causing duplicated upgrading requests from different threads in a same job
- waiterObjId = entityLockInfoManager.findUpgraderFromUpgraderList(eLockInfo, jId, entityHashValue);
- if (waiterObjId != -1) {
- return false;
- }
-
- //2. lock request causing duplicated waiting requests from different threads in a same job
- waiterObjId = entityLockInfoManager.findWaiterFromWaiterList(eLockInfo, jId, entityHashValue);
- if (waiterObjId != -1) {
- return false;
- }
-
- //3. lock request causing duplicated holding requests from different threads or a single thread in a same job
- entityInfo = entityLockInfoManager.findEntityInfoFromHolderList(eLockInfo, jId, entityHashValue);
- if (entityInfo != -1) {//duplicated call or upgrader
-
- isUpgrade = isLockUpgrade(entityInfoManager.getEntityLockMode(entityInfo), lockMode);
- if (isUpgrade) {//upgrade call
- //wait if any upgrader exists or upgrading lock mode is not compatible
- if (entityLockInfoManager.getUpgrader(eLockInfo) != -1
- || !entityLockInfoManager.isUpgradeCompatible(eLockInfo, lockMode, entityInfo)) {
- return false;
- }
- }
- /***************************
- * else {//duplicated call
- * //do nothing
- * }
- ****************************/
- } else {//new call from this job, but still eLockInfo exists since other threads hold it or wait on it
- if (entityLockInfoManager.getUpgrader(eLockInfo) != -1
- || entityLockInfoManager.getFirstWaiter(eLockInfo) != -1
- || !entityLockInfoManager.isCompatible(eLockInfo, lockMode)) {
- return false;
- }
- }
- }
- /*******************************
- * else {//eLockInfo doesn't exist, so this lock request is the first request and can be granted without waiting.
- * //do nothing
- * }
- *********************************/
-
- return true;
- }
-
- private boolean internalTryLock(DatasetId datasetId, int entityHashValue, byte lockMode,
- ITransactionContext txnContext, boolean isInstant) throws ACIDException {
- JobId jobId = txnContext.getJobId();
- int jId = jobId.getId(); //int-type jobId
- int dId = datasetId.getId(); //int-type datasetId
- int entityInfo;
- int eLockInfo = -1;
- DatasetLockInfo dLockInfo = null;
- JobInfo jobInfo;
- byte datasetLockMode = entityHashValue == -1 ? lockMode : lockMode == LockMode.S ? LockMode.IS : LockMode.IX;
- boolean isSuccess = true;
- boolean doEscalate = false;
-
- latchLockTable();
- try {
- validateJob(txnContext);
-
- if (IS_DEBUG_MODE) {
- trackLockRequest("Requested", RequestType.TRY_LOCK, datasetId, entityHashValue, lockMode, txnContext,
- dLockInfo, eLockInfo);
- }
-
- dLockInfo = datasetResourceHT.get(datasetId);
- jobInfo = jobHT.get(jobId);
-
- if (ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
- if (!isInstant && datasetLockMode == LockMode.IS && jobInfo != null && dLockInfo != null) {
- int upgradeStatus = needEscalateFromEntityToDataset(jobInfo, dId, lockMode);
- switch (upgradeStatus) {
- case DO_ESCALATE:
- entityHashValue = -1;
- doEscalate = true;
- break;
-
- case ESCALATED:
- return true;
-
- default:
- break;
- }
- }
- }
-
- //#. if the datasetLockInfo doesn't exist in datasetResourceHT
- if (dLockInfo == null || dLockInfo.isNoHolder()) {
- if (dLockInfo == null) {
- dLockInfo = new DatasetLockInfo(entityLockInfoManager, entityInfoManager, lockWaiterManager);
- datasetResourceHT.put(new DatasetId(dId), dLockInfo); //datsetId obj should be created
- }
- entityInfo = entityInfoManager.allocate(jId, dId, entityHashValue, lockMode);
-
- //if dataset-granule lock
- if (entityHashValue == -1) { //-1 stands for dataset-granule
- entityInfoManager.increaseDatasetLockCount(entityInfo);
- dLockInfo.increaseLockCount(datasetLockMode);
- dLockInfo.addHolder(entityInfo);
- } else {
- entityInfoManager.increaseDatasetLockCount(entityInfo);
- dLockInfo.increaseLockCount(datasetLockMode);
- //add entityLockInfo
- eLockInfo = entityLockInfoManager.allocate();
- dLockInfo.getEntityResourceHT().put(entityHashValue, eLockInfo);
- entityInfoManager.increaseEntityLockCount(entityInfo);
- entityLockInfoManager.increaseLockCount(eLockInfo, lockMode);
- entityLockInfoManager.addHolder(eLockInfo, entityInfo);
- }
-
- if (jobInfo == null) {
- jobInfo = new JobInfo(entityInfoManager, lockWaiterManager, txnContext);
- jobHT.put(jobId, jobInfo); //jobId obj doesn't have to be created
- }
- jobInfo.addHoldingResource(entityInfo);
-
- if (ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
- if (!isInstant && datasetLockMode == LockMode.IS) {
- jobInfo.increaseDatasetISLockCount(dId);
- if (doEscalate) {
- //This exception is thrown when the threshold value is set to 1.
- //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());
- }
- }
- }
-
- if (IS_DEBUG_MODE) {
- trackLockRequest("Granted", RequestType.TRY_LOCK, datasetId, entityHashValue, lockMode, txnContext,
- dLockInfo, eLockInfo);
- }
-
- return true;
- }
-
- //#. the datasetLockInfo exists in datasetResourceHT.
- //1. handle dataset-granule lock
- tryLockDatasetGranuleRevertOperation = 0;
- entityInfo = tryLockDatasetGranule(datasetId, entityHashValue, lockMode, txnContext);
- if (entityInfo == -2) {//-2 represents fail
- isSuccess = false;
- } else {
- //2. handle entity-granule lock
- if (entityHashValue != -1) {
- isSuccess = tryLockEntityGranule(datasetId, entityHashValue, lockMode, entityInfo, txnContext);
- if (!isSuccess) {
- revertTryLockDatasetGranuleOperation(datasetId, entityHashValue, lockMode, entityInfo,
- txnContext);
- }
- }
- }
-
- if (ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
- if (!isInstant) {
- if (doEscalate) {
- //jobInfo must not be null.
- assert jobInfo != null;
- jobInfo.increaseDatasetISLockCount(dId);
- //release pre-acquired locks
- releaseDatasetISLocks(jobInfo, jobId, datasetId, txnContext);
- } else if (datasetLockMode == LockMode.IS) {
- if (jobInfo == null) {
- jobInfo = jobHT.get(jobId);
- //jobInfo must not be null;
- assert jobInfo != null;
- }
- jobInfo.increaseDatasetISLockCount(dId);
- }
- }
- }
-
- if (IS_DEBUG_MODE) {
- if (isSuccess) {
- trackLockRequest("Granted", RequestType.TRY_LOCK, datasetId, entityHashValue, lockMode, txnContext,
- dLockInfo, eLockInfo);
- } else {
- trackLockRequest("Failed", RequestType.TRY_LOCK, datasetId, entityHashValue, lockMode, txnContext,
- dLockInfo, eLockInfo);
- }
- }
-
- } finally {
- unlatchLockTable();
- }
-
- return isSuccess;
- }
-
- private void trackLockRequest(String msg, int requestType, DatasetId datasetIdObj, int entityHashValue,
- byte lockMode, ITransactionContext txnContext, DatasetLockInfo dLockInfo, int eLockInfo) {
- StringBuilder s = new StringBuilder();
- LockRequest request = new LockRequest(Thread.currentThread().getName(), requestType, datasetIdObj,
- entityHashValue, lockMode, txnContext);
- s.append(Thread.currentThread().getId() + ":");
- s.append(msg);
- if (msg.equals("Granted")) {
- if (dLockInfo != null) {
- s.append("\t|D| ");
- s.append(dLockInfo.getIXCount()).append(",");
- s.append(dLockInfo.getISCount()).append(",");
- s.append(dLockInfo.getXCount()).append(",");
- s.append(dLockInfo.getSCount()).append(",");
- if (dLockInfo.getFirstUpgrader() != -1) {
- s.append("+");
- } else {
- s.append("-");
- }
- s.append(",");
- if (dLockInfo.getFirstWaiter() != -1) {
- s.append("+");
- } else {
- s.append("-");
- }
- }
-
- if (eLockInfo != -1) {
- s.append("\t|E| ");
- s.append(entityLockInfoManager.getXCount(eLockInfo)).append(",");
- s.append(entityLockInfoManager.getSCount(eLockInfo)).append(",");
- if (entityLockInfoManager.getUpgrader(eLockInfo) != -1) {
- s.append("+");
- } else {
- s.append("-");
- }
- s.append(",");
- if (entityLockInfoManager.getFirstWaiter(eLockInfo) != -1) {
- s.append("+");
- } else {
- s.append("-");
- }
- }
- }
-
- lockRequestTracker.addEvent(s.toString(), request);
- if (msg.equals("Requested")) {
- lockRequestTracker.addRequest(request);
- }
- System.out.println(request.prettyPrint() + "--> " + s.toString());
- }
-
- public String getHistoryForAllJobs() {
- if (IS_DEBUG_MODE) {
- return lockRequestTracker.getHistoryForAllJobs();
- }
- return null;
- }
-
- public String getHistoryPerJob() {
- if (IS_DEBUG_MODE) {
- return lockRequestTracker.getHistoryPerJob();
- }
- return null;
- }
-
- public String getRequestHistoryForAllJobs() {
- if (IS_DEBUG_MODE) {
- return lockRequestTracker.getRequestHistoryForAllJobs();
- }
- return null;
- }
-
- private void revertTryLockDatasetGranuleOperation(DatasetId datasetId, int entityHashValue, byte lockMode,
- int entityInfo, ITransactionContext txnContext) {
- JobId jobId = txnContext.getJobId();
- DatasetLockInfo dLockInfo;
- JobInfo jobInfo;
- int lockCount;
- byte datasetLockMode = entityHashValue == -1 ? lockMode : lockMode == LockMode.S ? LockMode.IS : LockMode.IX;
-
- dLockInfo = datasetResourceHT.get(datasetId);
- jobInfo = jobHT.get(jobId);
-
- //see tryLockDatasetGranule() function to know the revert operation
- switch (tryLockDatasetGranuleRevertOperation) {
-
- case 1://[revertOperation1]: reverting 'adding a holder'
-
- if (entityHashValue == -1) {
- dLockInfo.decreaseLockCount(datasetLockMode);
- dLockInfo.removeHolder(entityInfo, jobInfo); //--> this call removes entityInfo from JobInfo's holding-resource-list as well.
- } else {
- dLockInfo.decreaseLockCount(datasetLockMode);
- jobInfo.removeHoldingResource(entityInfo);
- }
- entityInfoManager.decreaseDatasetLockCount(entityInfo);
- if (jobInfo.getLastHoldingResource() == -1 && jobInfo.getFirstWaitingResource() == -1) {
- jobHT.remove(jobId);
- }
- entityInfoManager.deallocate(entityInfo);
- break;
-
- case 2://[revertOperation2]: reverting 'adding an upgrader'
- lockCount = entityInfoManager.getDatasetLockCount(entityInfo);
- if (entityHashValue == -1) { //dataset-granule lock
- dLockInfo.decreaseLockCount(LockMode.X, lockCount);
- dLockInfo.increaseLockCount(LockMode.S, lockCount);
- } else {
- dLockInfo.decreaseLockCount(LockMode.IX, lockCount);
- dLockInfo.increaseLockCount(LockMode.IS, lockCount);
- }
- entityInfoManager.setDatasetLockMode(entityInfo, LockMode.S);
- break;
-
- case 3://[revertOperation3]: reverting 'adding a duplicated call'
- entityInfoManager.decreaseDatasetLockCount(entityInfo);
- datasetLockMode = entityInfoManager.getDatasetLockMode(entityInfo);
- if (entityHashValue == -1) { //dataset-granule
- dLockInfo.decreaseLockCount(datasetLockMode);
- } else { //entity-granule
- datasetLockMode = datasetLockMode == LockMode.S ? LockMode.IS : LockMode.IX;
- dLockInfo.decreaseLockCount(datasetLockMode);
- }
-
- break;
- default:
- //do nothing;
- }
- }
-
- private int tryLockDatasetGranule(DatasetId datasetId, int entityHashValue, byte lockMode,
- ITransactionContext txnContext) throws ACIDException {
- JobId jobId = txnContext.getJobId();
- int jId = jobId.getId(); //int-type jobId
- int dId = datasetId.getId(); //int-type datasetId
- int waiterObjId;
- int entityInfo = -1;
- DatasetLockInfo dLockInfo;
- JobInfo jobInfo;
- boolean isUpgrade = false;
- int weakerModeLockCount;
- byte datasetLockMode = entityHashValue == -1 ? lockMode : lockMode == LockMode.S ? LockMode.IS : LockMode.IX;
-
- dLockInfo = datasetResourceHT.get(datasetId);
- jobInfo = jobHT.get(jobId);
-
- //check duplicated call
-
- //1. lock request causing duplicated upgrading requests from different threads in a same job
- waiterObjId = dLockInfo.findUpgraderFromUpgraderList(jId, entityHashValue);
- if (waiterObjId != -1) {
- return -2;
- }
-
- //2. lock request causing duplicated waiting requests from different threads in a same job
- waiterObjId = dLockInfo.findWaiterFromWaiterList(jId, entityHashValue);
- if (waiterObjId != -1) {
- return -2;
- }
-
- //3. lock request causing duplicated holding requests from different threads or a single thread in a same job
- entityInfo = dLockInfo.findEntityInfoFromHolderList(jId, entityHashValue);
- if (entityInfo == -1) { //new call from this job -> doesn't mean that eLockInfo doesn't exist since another thread might have create the eLockInfo already.
-
- //////////////////////////////////////////////////////////////////////////////////////
- //[part of revertOperation1]
- entityInfo = entityInfoManager.allocate(jId, dId, entityHashValue, lockMode);
- if (jobInfo == null) {
- jobInfo = new JobInfo(entityInfoManager, lockWaiterManager, txnContext);
- jobHT.put(jobId, jobInfo);
- }
- //////////////////////////////////////////////////////////////////////////////////////
-
- //return fail if any upgrader exists or upgrading lock mode is not compatible
- if (dLockInfo.getFirstUpgrader() != -1 || dLockInfo.getFirstWaiter() != -1
- || !dLockInfo.isCompatible(datasetLockMode)) {
-
- if (ALLOW_DATASET_GRANULE_X_LOCK_WITH_OTHER_CONCURRENT_LOCK_REQUESTS) {
- //The following case only may occur when the dataset level X lock is requested
- //with the other lock
-
- //[Notice]
- //There has been no same caller as (jId, dId, entityHashValue) triplet.
- //But there could be the same caller in terms of (jId, dId) pair.
- //For example,
- //1) (J1, D1, E1) acquires IS in Dataset D1
- //2) (J2, D1, -1) requests X in Dataset D1, but waits
- //3) (J1, D1, E2) requests IS in Dataset D1, but should wait
- //The 3) may cause deadlock if 1) and 3) are under the same thread.
- //Even if (J1, D1, E1) and (J1, D1, E2) are two different thread, instead of
- //aborting (J1, D1, E1) triggered by the deadlock, we give higher priority to 3) than 2)
- //as long as the dataset level lock D1 is being held by the same jobId.
- //The above consideration is covered in the following code.
- //find the same dataset-granule lock request, that is, (J1, D1) pair in the above example.
- if (jobInfo.isDatasetLockGranted(dId, LockMode.IS)) {
- if (dLockInfo.isCompatible(datasetLockMode)) {
- //this is duplicated call
- entityInfoManager.increaseDatasetLockCount(entityInfo);
- if (entityHashValue == -1) {
- dLockInfo.increaseLockCount(datasetLockMode);
- dLockInfo.addHolder(entityInfo);
- } else {
- dLockInfo.increaseLockCount(datasetLockMode);
- //IS and IX holders are implicitly handled.
- }
- //add entityInfo to JobInfo's holding-resource list
- jobInfo.addHoldingResource(entityInfo);
-
- tryLockDatasetGranuleRevertOperation = 1;
-
- return entityInfo;
- }
- }
- }
-
- //revert [part of revertOperation1] before return
- if (jobInfo.getLastHoldingResource() == -1 && jobInfo.getFirstWaitingResource() == -1) {
- jobHT.remove(jobId);
- }
- entityInfoManager.deallocate(entityInfo);
-
- return -2;
- }
-
- //////////////////////////////////////////////////////////////////////////////////////
- //revert the following operations if the caller thread has to wait during this call.
- //[revertOperation1]
- entityInfoManager.increaseDatasetLockCount(entityInfo);
- if (entityHashValue == -1) {
- dLockInfo.increaseLockCount(datasetLockMode);
- dLockInfo.addHolder(entityInfo);
- } else {
- dLockInfo.increaseLockCount(datasetLockMode);
- //IS and IX holders are implicitly handled.
- }
- //add entityInfo to JobInfo's holding-resource list
- jobInfo.addHoldingResource(entityInfo);
-
- //set revert operation to be reverted when tryLock() fails
- tryLockDatasetGranuleRevertOperation = 1;
- //////////////////////////////////////////////////////////////////////////////////////
-
- } else {
- isUpgrade = isLockUpgrade(entityInfoManager.getDatasetLockMode(entityInfo), lockMode);
- if (isUpgrade) { //upgrade call
- //return fail if any upgrader exists or upgrading lock mode is not compatible
- if (dLockInfo.getFirstUpgrader() != -1 || !dLockInfo.isUpgradeCompatible(datasetLockMode, entityInfo)) {
- return -2;
- }
-
- //update entityInfo's dataset lock count and datasetLockInfo's lock count
- weakerModeLockCount = entityInfoManager.getDatasetLockCount(entityInfo);
-
- //////////////////////////////////////////////////////////////////////////////////////
- //revert the following operations if the caller thread has to wait during this call.
- //[revertOperation2]
- entityInfoManager.setDatasetLockMode(entityInfo, lockMode);
-
- if (entityHashValue == -1) { //dataset-granule lock
- dLockInfo.increaseLockCount(LockMode.X, weakerModeLockCount);//new lock mode
- dLockInfo.decreaseLockCount(LockMode.S, weakerModeLockCount);//current lock mode
- } else {
- dLockInfo.increaseLockCount(LockMode.IX, weakerModeLockCount);
- dLockInfo.decreaseLockCount(LockMode.IS, weakerModeLockCount);
- }
- tryLockDatasetGranuleRevertOperation = 2;
- //////////////////////////////////////////////////////////////////////////////////////
-
- } else { //duplicated call
-
- //////////////////////////////////////////////////////////////////////////////////////
- //revert the following operations if the caller thread has to wait during this call.
- //[revertOperation3]
- entityInfoManager.increaseDatasetLockCount(entityInfo);
- datasetLockMode = entityInfoManager.getDatasetLockMode(entityInfo);
-
- if (entityHashValue == -1) { //dataset-granule
- dLockInfo.increaseLockCount(datasetLockMode);
- } else { //entity-granule
- datasetLockMode = datasetLockMode == LockMode.S ? LockMode.IS : LockMode.IX;
- dLockInfo.increaseLockCount(datasetLockMode);
- }
-
- tryLockDatasetGranuleRevertOperation = 3;
- //////////////////////////////////////////////////////////////////////////////////////
-
- }
- }
-
- return entityInfo;
- }
-
- private boolean tryLockEntityGranule(DatasetId datasetId, int entityHashValue, byte lockMode,
- int entityInfoFromDLockInfo, ITransactionContext txnContext) throws ACIDException {
- JobId jobId = txnContext.getJobId();
- int jId = jobId.getId(); //int-type jobId
- int waiterObjId;
- int eLockInfo = -1;
- int entityInfo;
- DatasetLockInfo dLockInfo;
- boolean isUpgrade = false;
- int weakerModeLockCount;
-
- dLockInfo = datasetResourceHT.get(datasetId);
- eLockInfo = dLockInfo.getEntityResourceHT().get(entityHashValue);
-
- if (eLockInfo != -1) {
- //check duplicated call
-
- //1. lock request causing duplicated upgrading requests from different threads in a same job
- waiterObjId = entityLockInfoManager.findUpgraderFromUpgraderList(eLockInfo, jId, entityHashValue);
- if (waiterObjId != -1) {
- return false;
- }
-
- //2. lock request causing duplicated waiting requests from different threads in a same job
- waiterObjId = entityLockInfoManager.findWaiterFromWaiterList(eLockInfo, jId, entityHashValue);
- if (waiterObjId != -1) {
- return false;
- }
-
- //3. lock request causing duplicated holding requests from different threads or a single thread in a same job
- entityInfo = entityLockInfoManager.findEntityInfoFromHolderList(eLockInfo, jId, entityHashValue);
- if (entityInfo != -1) {//duplicated call or upgrader
-
- isUpgrade = isLockUpgrade(entityInfoManager.getEntityLockMode(entityInfo), lockMode);
- if (isUpgrade) {//upgrade call
- //wait if any upgrader exists or upgrading lock mode is not compatible
- if (entityLockInfoManager.getUpgrader(eLockInfo) != -1
- || !entityLockInfoManager.isUpgradeCompatible(eLockInfo, lockMode, entityInfo)) {
- return false;
- }
-
- weakerModeLockCount = entityInfoManager.getEntityLockCount(entityInfo);
- entityInfoManager.setEntityLockMode(entityInfo, lockMode);
-
- entityLockInfoManager.increaseLockCount(eLockInfo, LockMode.X, (short) weakerModeLockCount);//new lock mode
- entityLockInfoManager.decreaseLockCount(eLockInfo, LockMode.S, (short) weakerModeLockCount);//old lock mode
-
- } else {//duplicated call
- entityInfoManager.increaseEntityLockCount(entityInfo);
- entityLockInfoManager.increaseLockCount(eLockInfo, entityInfoManager.getEntityLockMode(entityInfo));
- }
- } else {//new call from this job, but still eLockInfo exists since other threads hold it or wait on it
- entityInfo = entityInfoFromDLockInfo;
- if (entityLockInfoManager.getUpgrader(eLockInfo) != -1
- || entityLockInfoManager.getFirstWaiter(eLockInfo) != -1
- || !entityLockInfoManager.isCompatible(eLockInfo, lockMode)) {
- return false;
- }
-
- entityInfoManager.increaseEntityLockCount(entityInfo);
- entityLockInfoManager.increaseLockCount(eLockInfo, lockMode);
- entityLockInfoManager.addHolder(eLockInfo, entityInfo);
- }
- } else {//eLockInfo doesn't exist, so this lock request is the first request and can be granted without waiting.
- eLockInfo = entityLockInfoManager.allocate();
- dLockInfo.getEntityResourceHT().put(entityHashValue, eLockInfo);
- entityInfoManager.increaseEntityLockCount(entityInfoFromDLockInfo);
- entityLockInfoManager.increaseLockCount(eLockInfo, lockMode);
- entityLockInfoManager.addHolder(eLockInfo, entityInfoFromDLockInfo);
- }
-
- return true;
- }
-
- private void latchLockTable() {
- lockTableLatch.writeLock().lock();
- }
-
- private void unlatchLockTable() {
- lockTableLatch.writeLock().unlock();
- }
-
- private void latchWaitNotify() {
- waiterLatch.writeLock().lock();
- }
-
- private void unlatchWaitNotify() {
- waiterLatch.writeLock().unlock();
- }
-
- private int handleLockWaiter(DatasetLockInfo dLockInfo, int eLockInfo, int entityInfo, boolean isUpgrade,
- boolean isDatasetLockInfo, ITransactionContext txnContext, JobInfo jobInfo, int duplicatedWaiterObjId)
- throws ACIDException {
- int waiterId = -1;
- LockWaiter waiter;
- int waiterCount = 0;
- boolean isInterruptedExceptionOccurred = false;
-
- if (duplicatedWaiterObjId != -1
- || isDeadlockFree(dLockInfo, eLockInfo, entityInfo, isDatasetLockInfo, isUpgrade)) {//deadlock free -> wait
- if (duplicatedWaiterObjId == -1) {
- waiterId = lockWaiterManager.allocate(); //initial value of waiterObj: wait = true, victim = false
- waiter = lockWaiterManager.getLockWaiter(waiterId);
- waiter.setEntityInfoSlot(entityInfo);
- jobInfo.addWaitingResource(waiterId);
- waiter.setBeginWaitTime(System.currentTimeMillis());
- } else {
- waiterId = duplicatedWaiterObjId;
- waiter = lockWaiterManager.getLockWaiter(waiterId);
- }
-
- if (duplicatedWaiterObjId == -1) {
- //add actor properly
- if (isDatasetLockInfo) {
- waiter.setWaitingOnEntityLock(false);
- if (isUpgrade) {
- dLockInfo.addUpgrader(waiterId);
- waiter.setWaiter(false);
- } else {
- dLockInfo.addWaiter(waiterId);
- waiter.setWaiter(true);
- }
- } else {
- waiter.setWaitingOnEntityLock(true);
- if (isUpgrade) {
- waiter.setWaiter(false);
- entityLockInfoManager.addUpgrader(eLockInfo, waiterId);
- } else {
- waiter.setWaiter(true);
- entityLockInfoManager.addWaiter(eLockInfo, waiterId);
- }
- }
- }
- waiter.increaseWaiterCount();
- waiter.setFirstGetUp(true);
-
- latchWaitNotify();
- unlatchLockTable();
- try {
- synchronized (waiter) {
- unlatchWaitNotify();
- while (waiter.needWait()) {
- try {
- if (IS_DEBUG_MODE) {
- System.out.println("" + Thread.currentThread().getName() + "\twaits("
- + waiter.getWaiterCount() + "): WID(" + waiterId + "),EID("
- + waiter.getEntityInfoSlot() + ")");
- }
- waiter.wait();
- } catch (InterruptedException e) {
- //TODO figure-out what is the appropriate way to handle this exception
- e.printStackTrace();
- isInterruptedExceptionOccurred = true;
- waiter.setWait(false);
- }
- }
- }
-
- if (isInterruptedExceptionOccurred) {
- throw new ACIDException("InterruptedException is caught");
- }
- } catch (Exception e) {
- throw new LockMgrLatchHandlerException(e);
- }
-
- //waiter woke up -> remove/deallocate waiter object and abort if timeout
- latchLockTable();
-
- if (txnContext.isTimeout() || waiter.isVictim()) {
- requestAbort(txnContext);
- }
-
- if (waiter.isFirstGetUp()) {
- waiter.setFirstGetUp(false);
- waiterCount = waiter.getWaiterCount();
- } else {
- waiterCount = 0;
- }
-
- waiter.decreaseWaiterCount();
- if (IS_DEBUG_MODE) {
- System.out.println("" + Thread.currentThread().getName() + "\tgot-up!(" + waiter.getWaiterCount()
- + "): WID(" + waiterId + "),EID(" + waiter.getEntityInfoSlot() + ")");
- }
- if (waiter.getWaiterCount() == 0) {
- //remove actor properly
- if (isDatasetLockInfo) {
- if (isUpgrade) {
- dLockInfo.removeUpgrader(waiterId);
- } else {
- dLockInfo.removeWaiter(waiterId);
- }
- } else {
- if (isUpgrade) {
- entityLockInfoManager.removeUpgrader(eLockInfo, waiterId);
- } else {
- entityLockInfoManager.removeWaiter(eLockInfo, waiterId);
- }
- }
-
- //if (!isUpgrade && isDatasetLockInfo) {
- jobInfo.removeWaitingResource(waiterId);
- //}
- lockWaiterManager.deallocate(waiterId);
- }
-
- } else { //deadlock -> abort
- //[Notice]
- //Before requesting abort, the entityInfo for waiting datasetLock request is deallocated.
- if (!isUpgrade && isDatasetLockInfo) {
- //deallocate the entityInfo
- entityInfoManager.deallocate(entityInfo);
- }
- requestAbort(txnContext);
- }
-
- return waiterCount;
- }
-
- private boolean isDeadlockFree(DatasetLockInfo dLockInfo, int eLockInfo, int entityInfo, boolean isDatasetLockInfo,
- boolean isUpgrade) {
- return deadlockDetector.isSafeToAdd(dLockInfo, eLockInfo, entityInfo, isDatasetLockInfo, isUpgrade);
- }
-
- private void requestAbort(ITransactionContext txnContext) throws ACIDException {
- txnContext.setTimeout(true);
- throw new ACIDException("Transaction " + txnContext.getJobId()
- + " should abort (requested by the Lock Manager)");
- }
-
- /**
- * For now, upgrading lock granule from entity-granule to dataset-granule is not supported!!
- *
- * @param fromLockMode
- * @param toLockMode
- * @return
- */
- private boolean isLockUpgrade(byte fromLockMode, byte toLockMode) {
- return fromLockMode == LockMode.S && toLockMode == LockMode.X;
- }
-
- /**
- * wake up upgraders first, then waiters.
- * Criteria to wake up upgraders: if the upgrading lock mode is compatible, then wake up the upgrader.
- */
- private void wakeUpDatasetLockWaiters(DatasetLockInfo dLockInfo) {
- int waiterObjId = dLockInfo.getFirstUpgrader();
- int entityInfo;
- LockWaiter waiterObj;
- byte datasetLockMode;
- byte lockMode;
- boolean areAllUpgradersAwaken = true;
-
- consecutiveWakeupContext.reset();
- while (waiterObjId != -1) {
- //wake up upgraders
- waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
- entityInfo = waiterObj.getEntityInfoSlot();
- datasetLockMode = entityInfoManager.getPKHashVal(entityInfo) == -1 ? LockMode.X : LockMode.IX;
- if (dLockInfo.isUpgradeCompatible(datasetLockMode, entityInfo)
- && consecutiveWakeupContext.isCompatible(datasetLockMode)) {
- consecutiveWakeupContext.setLockMode(datasetLockMode);
- //compatible upgrader is waken up
- latchWaitNotify();
- synchronized (waiterObj) {
- unlatchWaitNotify();
- waiterObj.setWait(false);
- if (IS_DEBUG_MODE) {
- System.out.println("" + Thread.currentThread().getName() + "\twake-up(D): WID(" + waiterObjId
- + "),EID(" + waiterObj.getEntityInfoSlot() + ")");
- }
- waiterObj.notifyAll();
- }
- waiterObjId = waiterObj.getNextWaiterObjId();
- } else {
- areAllUpgradersAwaken = false;
- break;
- }
- }
-
- if (areAllUpgradersAwaken) {
- //wake up waiters
- waiterObjId = dLockInfo.getFirstWaiter();
- while (waiterObjId != -1) {
- waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
- entityInfo = waiterObj.getEntityInfoSlot();
- lockMode = entityInfoManager.getDatasetLockMode(entityInfo);
- datasetLockMode = entityInfoManager.getPKHashVal(entityInfo) == -1 ? lockMode
- : lockMode == LockMode.S ? LockMode.IS : LockMode.IX;
- if (dLockInfo.isCompatible(datasetLockMode) && consecutiveWakeupContext.isCompatible(datasetLockMode)) {
- consecutiveWakeupContext.setLockMode(datasetLockMode);
- //compatible waiter is waken up
- latchWaitNotify();
- synchronized (waiterObj) {
- unlatchWaitNotify();
- waiterObj.setWait(false);
- if (IS_DEBUG_MODE) {
- System.out.println("" + Thread.currentThread().getName() + "\twake-up(D): WID("
- + waiterObjId + "),EID(" + waiterObj.getEntityInfoSlot() + ")");
- }
- waiterObj.notifyAll();
- }
- waiterObjId = waiterObj.getNextWaiterObjId();
- } else {
- break;
- }
- }
- }
- }
-
- private void wakeUpEntityLockWaiters(int eLockInfo) {
- boolean areAllUpgradersAwaken = true;
- int waiterObjId = entityLockInfoManager.getUpgrader(eLockInfo);
- int entityInfo;
- LockWaiter waiterObj;
- byte entityLockMode;
-
- consecutiveWakeupContext.reset();
- while (waiterObjId != -1) {
- //wake up upgraders
- waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
- entityInfo = waiterObj.getEntityInfoSlot();
- if (entityLockInfoManager.isUpgradeCompatible(eLockInfo, LockMode.X, entityInfo)
- && consecutiveWakeupContext.isCompatible(LockMode.X)) {
- consecutiveWakeupContext.setLockMode(LockMode.X);
- latchWaitNotify();
- synchronized (waiterObj) {
- unlatchWaitNotify();
- waiterObj.setWait(false);
- if (IS_DEBUG_MODE) {
- System.out.println("" + Thread.currentThread().getName() + "\twake-up(E): WID(" + waiterObjId
- + "),EID(" + waiterObj.getEntityInfoSlot() + ")");
- }
- waiterObj.notifyAll();
- }
- waiterObjId = waiterObj.getNextWaiterObjId();
- } else {
- areAllUpgradersAwaken = false;
- break;
- }
- }
-
- if (areAllUpgradersAwaken) {
- //wake up waiters
- waiterObjId = entityLockInfoManager.getFirstWaiter(eLockInfo);
- while (waiterObjId != -1) {
- waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
- entityInfo = waiterObj.getEntityInfoSlot();
- entityLockMode = entityInfoManager.getEntityLockMode(entityInfo);
- if (entityLockInfoManager.isCompatible(eLockInfo, entityLockMode)
- && consecutiveWakeupContext.isCompatible(entityLockMode)) {
- consecutiveWakeupContext.setLockMode(entityLockMode);
- //compatible waiter is waken up
- latchWaitNotify();
- synchronized (waiterObj) {
- unlatchWaitNotify();
- waiterObj.setWait(false);
- if (IS_DEBUG_MODE) {
- System.out.println("" + Thread.currentThread().getName() + "\twake-up(E): WID("
- + waiterObjId + "),EID(" + waiterObj.getEntityInfoSlot() + ")");
- }
- waiterObj.notifyAll();
- }
- } else {
- break;
- }
- waiterObjId = waiterObj.getNextWaiterObjId();
- }
- }
- }
-
- @Override
- public String prettyPrint() throws ACIDException {
- StringBuilder s = new StringBuilder("\n########### LockManager Status #############\n");
- return s + "\n";
- }
-
- public void sweepForTimeout() throws ACIDException {
- JobInfo jobInfo;
- int waiterObjId;
- LockWaiter waiterObj;
-
- latchLockTable();
- try {
-
- Iterator<Entry<JobId, JobInfo>> iter = jobHT.entrySet().iterator();
- while (iter.hasNext()) {
- Map.Entry<JobId, JobInfo> pair = iter.next();
- jobInfo = pair.getValue();
- waiterObjId = jobInfo.getFirstWaitingResource();
- while (waiterObjId != -1) {
- waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
- toutDetector.checkAndSetVictim(waiterObj);
- waiterObjId = waiterObj.getNextWaiterObjId();
- }
- }
- } finally {
- unlatchLockTable();
- }
- }
-
- @Override
- public void start() {
- //no op
- }
-
- @Override
- public void stop(boolean dumpState, OutputStream os) {
- if (dumpState) {
- dumpState(os);
- }
- }
-
- @Override
- public void dumpState(OutputStream os) {
- //#. dump Configurable Variables
- dumpConfVars(os);
-
- //#. dump jobHT
- dumpJobInfo(os);
-
- //#. dump datasetResourceHT
- dumpDatasetLockInfo(os);
-
- //#. dump entityLockInfoManager
- dumpEntityLockInfo(os);
-
- //#. dump entityInfoManager
- dumpEntityInfo(os);
-
- //#. dump lockWaiterManager
- dumpLockWaiterInfo(os);
- }
-
- private void dumpConfVars(OutputStream os) {
- try {
- StringBuilder sb = new StringBuilder();
- sb.append("\n>>dump_begin\t>>----- [ConfVars] -----");
- 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());
- sb.append("\nSHRINK_TIMER_THRESHOLD (lockWaiterManager): " + lockWaiterManager.getShrinkTimerThreshold());
- 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.
- if (IS_DEBUG_MODE) {
- e.printStackTrace();
- }
- }
- }
-
- private void dumpJobInfo(OutputStream os) {
- JobId jobId;
- JobInfo jobInfo;
- StringBuilder sb = new StringBuilder();
-
- try {
- sb.append("\n>>dump_begin\t>>----- [JobInfo] -----");
- Set<Map.Entry<JobId, JobInfo>> entrySet = jobHT.entrySet();
- if (entrySet != null) {
- for (Map.Entry<JobId, JobInfo> entry : entrySet) {
- if (entry != null) {
- jobId = entry.getKey();
- if (jobId != null) {
- sb.append("\n" + jobId);
- } else {
- sb.append("\nJID:null");
- }
-
- jobInfo = entry.getValue();
- if (jobInfo != null) {
- sb.append(jobInfo.coreDump());
- } else {
- sb.append("\nJobInfo:null");
- }
- }
- }
- }
- sb.append("\n>>dump_end\t>>----- [JobInfo] -----\n");
- os.write(sb.toString().getBytes());
- } catch (Exception e) {
- //ignore exception and continue dumping as much as possible.
- if (IS_DEBUG_MODE) {
- e.printStackTrace();
- }
- }
- }
-
- private void dumpDatasetLockInfo(OutputStream os) {
- DatasetId datasetId;
- DatasetLockInfo datasetLockInfo;
- StringBuilder sb = new StringBuilder();
-
- try {
- sb.append("\n>>dump_begin\t>>----- [DatasetLockInfo] -----");
- Set<Map.Entry<DatasetId, DatasetLockInfo>> entrySet = datasetResourceHT.entrySet();
- if (entrySet != null) {
- for (Map.Entry<DatasetId, DatasetLockInfo> entry : entrySet) {
- if (entry != null) {
- datasetId = entry.getKey();
- if (datasetId != null) {
- sb.append("\nDatasetId:" + datasetId.getId());
- } else {
- sb.append("\nDatasetId:null");
- }
-
- datasetLockInfo = entry.getValue();
- if (datasetLockInfo != null) {
- sb.append(datasetLockInfo.coreDump());
- } else {
- sb.append("\nDatasetLockInfo:null");
- }
- }
- sb.append("\n>>dump_end\t>>----- [DatasetLockInfo] -----\n");
- os.write(sb.toString().getBytes());
-
- //create a new sb to avoid possible OOM exception
- sb = new StringBuilder();
- }
- }
- } catch (Exception e) {
- //ignore exception and continue dumping as much as possible.
- if (IS_DEBUG_MODE) {
- e.printStackTrace();
- }
- }
- }
-
- private void dumpEntityLockInfo(OutputStream os) {
- StringBuilder sb = new StringBuilder();
- try {
- sb.append("\n>>dump_begin\t>>----- [EntityLockInfo] -----");
- entityLockInfoManager.coreDump(os);
- sb.append("\n>>dump_end\t>>----- [EntityLockInfo] -----\n");
- os.write(sb.toString().getBytes());
- } catch (Exception e) {
- //ignore exception and continue dumping as much as possible.
- if (IS_DEBUG_MODE) {
- e.printStackTrace();
- }
- }
- }
-
- private void dumpEntityInfo(OutputStream os) {
- StringBuilder sb = new StringBuilder();
- try {
- sb.append("\n>>dump_begin\t>>----- [EntityInfo] -----");
- entityInfoManager.coreDump(os);
- sb.append("\n>>dump_end\t>>----- [EntityInfo] -----\n");
- os.write(sb.toString().getBytes());
- } catch (Exception e) {
- //ignore exception and continue dumping as much as possible.
- if (IS_DEBUG_MODE) {
- e.printStackTrace();
- }
- }
- }
-
- private void dumpLockWaiterInfo(OutputStream os) {
- StringBuilder sb = new StringBuilder();
- try {
- sb.append("\n>>dump_begin\t>>----- [LockWaiterInfo] -----");
- lockWaiterManager.coreDump(os);
- sb.append("\n>>dump_end\t>>----- [LockWaiterInfo] -----\n");
- os.write(sb.toString().getBytes());
- } catch (Exception e) {
- //ignore exception and continue dumping as much as possible.
- if (IS_DEBUG_MODE) {
- e.printStackTrace();
- }
- }
- }
-
- public void batchUnlock(LogBuffer logPage, LogBufferTailReader logBufferTailReader) throws ACIDException {
- latchLockTable();
- try {
- ITransactionContext txnCtx = null;
- LogRecord logRecord = logBufferTailReader.next();
- while (logRecord != null) {
- if (logRecord.getLogType() == LogType.ENTITY_COMMIT) {
- tempDatasetIdObj.setId(logRecord.getDatasetId());
- tempJobIdObj.setId(logRecord.getJobId());
- txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(tempJobIdObj, false);
- unlock(tempDatasetIdObj, logRecord.getPKHashValue(), LockMode.ANY, txnCtx);
- txnCtx.notifyOptracker(false);
- } else if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT) {
- tempJobIdObj.setId(logRecord.getJobId());
- txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(tempJobIdObj, false);
- txnCtx.notifyOptracker(true);
- logPage.notifyJobTerminator();
- }
- logRecord = logBufferTailReader.next();
- }
- } finally {
- unlatchLockTable();
- }
- }
-}
-
-class ConsecutiveWakeupContext {
- private boolean IS;
- private boolean IX;
- private boolean S;
- private boolean X;
-
- public void reset() {
- IS = false;
- IX = false;
- S = false;
- X = false;
- }
-
- public boolean isCompatible(byte lockMode) {
- switch (lockMode) {
- case LockMode.IX:
- return !S && !X;
-
- case LockMode.IS:
- return !X;
-
- case LockMode.X:
- return !IS && !IX && !S && !X;
-
- case LockMode.S:
- return !IX && !X;
-
- default:
- throw new IllegalStateException("Invalid upgrade lock mode");
- }
- }
-
- public void setLockMode(byte lockMode) {
- switch (lockMode) {
- case LockMode.IX:
- IX = true;
- return;
-
- case LockMode.IS:
- IS = true;
- return;
-
- case LockMode.X:
- X = true;
- return;
-
- case LockMode.S:
- S = true;
- return;
-
- default:
- throw new IllegalStateException("Invalid lock mode");
- }
-
- }
-
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockManagerDeterministicUnitTest.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockManagerDeterministicUnitTest.java
deleted file mode 100644
index 7ccd6b9..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockManagerDeterministicUnitTest.java
+++ /dev/null
@@ -1,664 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you 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 at
- *
- * 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 org.apache.asterix.transaction.management.service.locking;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.NoSuchElementException;
-import java.util.Scanner;
-
-import org.apache.commons.io.FileUtils;
-
-import org.apache.asterix.common.config.AsterixPropertiesAccessor;
-import org.apache.asterix.common.config.AsterixTransactionProperties;
-import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.transactions.DatasetId;
-import org.apache.asterix.common.transactions.ILockManager;
-import org.apache.asterix.common.transactions.ITransactionManager;
-import org.apache.asterix.common.transactions.JobId;
-import org.apache.asterix.transaction.management.service.logging.LogManager;
-import org.apache.asterix.transaction.management.service.transaction.TransactionContext;
-import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
-import org.apache.asterix.transaction.management.service.transaction.TransactionSubsystem;
-
-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/org.apache/asterix/transaction/management/service/locking/LockRequestFile");
- Thread t = new Thread(new LockRequestController(requestFileName));
- t.start();
- }
-}
-
-class LockRequestController implements Runnable {
-
- public static final boolean IS_DEBUG_MODE = false;
- TransactionSubsystem txnProvider;
- WorkerReadyQueue workerReadyQueue;
- ArrayList<LockRequest> requestList;
- ArrayList<ArrayList<Integer>> expectedResultList;
- int resultListIndex;
- ILockManager lockMgr;
- String requestFileName;
- long defaultWaitTime;
-
- public LockRequestController(String requestFileName) throws ACIDException, AsterixException {
- this.txnProvider = new TransactionSubsystem("nc1", new TestRuntimeContextProvider(), new AsterixTransactionProperties(
- new AsterixPropertiesAccessor()));
- this.workerReadyQueue = new WorkerReadyQueue();
- this.requestList = new ArrayList<LockRequest>();
- this.expectedResultList = new ArrayList<ArrayList<Integer>>();
- this.lockMgr = txnProvider.getLockManager();
- this.requestFileName = new String(requestFileName);
- this.resultListIndex = 0;
- this.defaultWaitTime = 10;
- }
-
- @Override
- public void run() {
- Thread.currentThread().setName("Thread-0");
- HashMap<String, Thread> threadMap = new HashMap<String, Thread>();
- Thread t = null;
- LockRequest lockRequest = null;
- boolean isSuccess = true;
-
- try {
- readRequest();
- } catch (IOException e) {
- e.printStackTrace();
- System.exit(-1);
- } catch (ACIDException e) {
- e.printStackTrace();
- System.exit(-1);
- }
-
- //initialize workerThread
- int size = requestList.size();
- for (int i = 0; i < size; i++) {
- lockRequest = requestList.get(i);
- if (lockRequest.threadName.equals("Thread-0")) {
- //Thread-0 is controller thread.
- continue;
- }
- t = threadMap.get(lockRequest.threadName);
- if (t == null) {
- t = new Thread(new LockRequestWorker(txnProvider, workerReadyQueue, lockRequest.threadName),
- lockRequest.threadName);
- threadMap.put(lockRequest.threadName, t);
- t.start();
- log("Created " + lockRequest.threadName);
- }
- }
-
- //wait for all workerThreads to be ready
- try {
- log("waiting for all workerThreads to complete initialization ...");
- Thread.sleep(5);
- } catch (InterruptedException e1) {
- e1.printStackTrace();
- }
- while (workerReadyQueue.size() != threadMap.size()) {
- try {
- log(" .");
- Thread.sleep(5);
- } catch (InterruptedException e) {
- e.printStackTrace();
- }
- }
-
- //make workerThread work
- while (requestList.size() != 0) {
- lockRequest = requestList.remove(0);
- log("Processing: " + lockRequest.prettyPrint());
- try {
- if (!handleRequest(lockRequest)) {
- log("\n*** Test Failed ***");
- isSuccess = false;
- break;
- } else {
- log("Processed: " + lockRequest.prettyPrint());
- }
- } catch (ACIDException e) {
- e.printStackTrace();
- break;
- }
- }
-
- if (isSuccess) {
- log("\n*** Test Passed ***");
- }
- ((LogManager) txnProvider.getLogManager()).stop(false, null);
- }
-
- public boolean handleRequest(LockRequest request) throws ACIDException {
- LockRequestWorker worker = null;
- int i = 0;
-
- if (request.requestType == RequestType.CHECK_SEQUENCE) {
- return validateExpectedResult(true);
- } else if (request.requestType == RequestType.CHECK_SET) {
- return validateExpectedResult(false);
- } else if (request.requestType == RequestType.WAIT) {
- try {
- Thread.sleep((long) request.entityHashValue);
- } catch (InterruptedException e) {
- e.printStackTrace();
- return false;
- }
- } else if (request.requestType == RequestType.END) {
- worker = workerReadyQueue.pop(request.threadName);
- while (worker == null) {
- if (!IS_DEBUG_MODE) {
- log(request.threadName + " is not in the workerReadyQueue");
- return false;
- }
- log(Thread.currentThread().getName() + " waiting for " + request.threadName
- + " to be in the workerReadyQueue[" + i++ + "].");
- try {
- Thread.sleep((long) 10);
- } catch (InterruptedException e) {
- e.printStackTrace();
- return false;
- }
- worker = workerReadyQueue.pop(request.threadName);
- }
- synchronized (worker) {
- worker.setDone(true);
- worker.setWait(false);
- worker.notify();
- }
- try {
- Thread.sleep((long) defaultWaitTime);
- } catch (InterruptedException e) {
- e.printStackTrace();
- }
- } else {
- worker = workerReadyQueue.pop(request.threadName);
- while (worker == null) {
- if (!IS_DEBUG_MODE) {
- log(request.threadName + " is not in the workerReadyQueue");
- return false;
- }
- log(Thread.currentThread().getName() + " waiting for " + request.threadName
- + " to be in the workerReadyQueue[" + i++ + "].");
- try {
- Thread.sleep((long) 10);
- } catch (InterruptedException e) {
- e.printStackTrace();
- }
- worker = workerReadyQueue.pop(request.threadName);
- }
-
- synchronized (worker) {
- worker.setLockRequest(request);
- worker.setWait(false);
- worker.notify();
- }
-
- try {
- Thread.sleep((long) defaultWaitTime);
- } catch (InterruptedException e) {
- e.printStackTrace();
- }
- }
-
- return true;
- }
-
- public boolean validateExpectedResult(boolean isSequence) {
-
- if (isSequence) {
- return workerReadyQueue.checkSequence(expectedResultList.get(resultListIndex++));
- } else {
- return workerReadyQueue.checkSet(expectedResultList.get(resultListIndex++));
- }
-
- }
-
- public void readRequest() throws IOException, ACIDException {
- int i = 0;
- LockRequest lockRequest = null;
- TransactionContext txnContext = null;
- HashMap<Integer, TransactionContext> jobMap = new HashMap<Integer, TransactionContext>();
-
- int threadId;
- String requestType;
- int jobId;
- int datasetId;
- int PKHashVal;
- int waitTime;
- ArrayList<Integer> list = null;
- String lockMode;
-
- Scanner scanner = new Scanner(new FileInputStream(requestFileName));
- while (scanner.hasNextLine()) {
- try {
- threadId = Integer.parseInt(scanner.next().substring(1));
- requestType = scanner.next();
- if (requestType.equals("CSQ") || requestType.equals("CST") || requestType.equals("END")) {
- log("LockRequest[" + i++ + "]:T" + threadId + "," + requestType);
- lockRequest = new LockRequest("Thread-" + threadId, getRequestType(requestType));
- if (requestType.equals("CSQ") || requestType.equals("CST")) {
- list = new ArrayList<Integer>();
- while (scanner.hasNextInt()) {
- threadId = scanner.nextInt();
- if (threadId < 0) {
- break;
- }
- list.add(threadId);
- }
- expectedResultList.add(list);
- }
- } else if (requestType.equals("DW")) {
- defaultWaitTime = scanner.nextInt();
- log("LockRequest[" + i++ + "]:T" + threadId + "," + requestType + "," + defaultWaitTime);
- continue;
- } else if (requestType.equals("W")) {
- waitTime = scanner.nextInt();
- log("LockRequest[" + i++ + "]:T" + threadId + "," + requestType);
- lockRequest = new LockRequest("Thread-" + threadId, getRequestType(requestType), waitTime);
- } else {
- jobId = Integer.parseInt(scanner.next().substring(1));
- datasetId = Integer.parseInt(scanner.next().substring(1));
- PKHashVal = Integer.parseInt(scanner.next().substring(1));
- lockMode = scanner.next();
- txnContext = jobMap.get(jobId);
- if (txnContext == null) {
- txnContext = new TransactionContext(new JobId(jobId), txnProvider);
- jobMap.put(jobId, txnContext);
- }
- log("LockRequest[" + i++ + "]:T" + threadId + "," + requestType + ",J" + jobId + ",D" + datasetId
- + ",E" + PKHashVal + "," + lockMode);
- lockRequest = new LockRequest("Thread-" + threadId, getRequestType(requestType), new DatasetId(
- datasetId), PKHashVal, getLockMode(lockMode), txnContext);
- }
-
- requestList.add(lockRequest);
- } catch (NoSuchElementException e) {
- scanner.close();
- break;
- }
- }
- }
-
- public void log(String s) {
- System.out.println(s);
- }
-
- private int getRequestType(String s) {
- if (s.equals("L")) {
- return RequestType.LOCK;
- }
-
- if (s.equals("TL")) {
- return RequestType.TRY_LOCK;
- }
-
- if (s.equals("IL")) {
- return RequestType.INSTANT_LOCK;
- }
-
- if (s.equals("ITL")) {
- return RequestType.INSTANT_TRY_LOCK;
- }
-
- if (s.equals("UL")) {
- return RequestType.UNLOCK;
- }
-
- if (s.equals("RL")) {
- return RequestType.RELEASE_LOCKS;
- }
-
- if (s.equals("CSQ")) {
- return RequestType.CHECK_SEQUENCE;
- }
-
- if (s.equals("CST")) {
- return RequestType.CHECK_SET;
- }
-
- if (s.equals("END")) {
- return RequestType.END;
- }
-
- if (s.equals("W")) {
- return RequestType.WAIT;
- }
-
- try {
- throw new UnsupportedOperationException("Invalid request type:" + s);
- } catch (Exception e) {
- e.printStackTrace();
- } finally {
- System.exit(0);
- }
-
- return -1;
-
- }
-
- private byte getLockMode(String s) {
- if (s.equals("S")) {
- return LockMode.S;
- }
-
- if (s.equals("X")) {
- return LockMode.X;
- }
-
- try {
- throw new UnsupportedOperationException("Invalid lock mode type:" + s);
- } catch (Exception e) {
- e.printStackTrace();
- } finally {
- System.exit(0);
- }
-
- return -1;
- }
-}
-
-class LockRequestWorker implements Runnable {
-
- String threadName;
- ILockManager lockMgr;
- WorkerReadyQueue workerReadyQueue;
- LockRequest lockRequest;
- boolean needWait;
- boolean isAwaken;
- boolean isDone;
-
- public LockRequestWorker(TransactionSubsystem txnProvider, WorkerReadyQueue workerReadyQueue, String threadName) {
- this.lockMgr = txnProvider.getLockManager();
- this.workerReadyQueue = workerReadyQueue;
- this.threadName = new String(threadName);
- this.lockRequest = null;
- needWait = true;
- isDone = false;
- isAwaken = false;
- }
-
- public boolean isAwaken() {
- return isAwaken;
- }
-
- @Override
- public void run() {
- //initial wait
- needWait = true;
- isAwaken = false;
-
- while (!isDone) {
- while (needWait) {
- synchronized (this) {
- workerReadyQueue.push(this);
- try {
- this.wait();
- isAwaken = true;
- } catch (InterruptedException e) {
- e.printStackTrace();
- }
- }
- }
-
- if (isDone) {
- break;
- }
-
- try {
- sendRequest(lockRequest);
- } catch (ACIDException e) {
- if (lockRequest.txnContext.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 {
- lockMgr.releaseLocks(lockRequest.txnContext);
- } catch (ACIDException e1) {
- e1.printStackTrace();
- }
- log("Abort --> Released all locks acquired by " + lockRequest.txnContext.getJobId());
- }
- isDone = true;
- } else {
- e.printStackTrace();
- System.exit(-1);
- }
- }
-
- try {
- Thread.sleep(1);
- } catch (InterruptedException e) {
- e.printStackTrace();
- }
-
- needWait = true;
- isAwaken = false;
- }
- }
-
- public void sendRequest(LockRequest request) throws ACIDException {
-
- switch (request.requestType) {
- case RequestType.LOCK:
- lockMgr.lock(request.datasetIdObj, request.entityHashValue, request.lockMode, request.txnContext);
- break;
- case RequestType.INSTANT_LOCK:
- lockMgr.instantLock(request.datasetIdObj, request.entityHashValue, request.lockMode, request.txnContext);
- break;
- case RequestType.TRY_LOCK:
- request.isTryLockFailed = !lockMgr.tryLock(request.datasetIdObj, request.entityHashValue,
- request.lockMode, request.txnContext);
- break;
- case RequestType.INSTANT_TRY_LOCK:
- lockMgr.instantTryLock(request.datasetIdObj, request.entityHashValue, request.lockMode,
- request.txnContext);
- break;
- case RequestType.UNLOCK:
- lockMgr.unlock(request.datasetIdObj, request.entityHashValue, request.lockMode, request.txnContext);
- break;
- case RequestType.RELEASE_LOCKS:
- lockMgr.releaseLocks(request.txnContext);
- break;
- default:
- throw new UnsupportedOperationException("Unsupported lock method");
- }
- }
-
- public void setLockRequest(LockRequest request) {
- this.lockRequest = request;
- }
-
- public void setWait(boolean wait) {
- needWait = wait;
- }
-
- public void setDone(boolean done) {
- isDone = done;
- }
-
- public String getThreadName() {
- return threadName;
- }
-
- public void log(String s) {
- System.out.println(s);
- }
-
- public String toString() {
- StringBuilder sb = new StringBuilder();
- sb.append("{ t : \"").append(threadName).append("\", r : ");
- if (lockRequest == null) {
- sb.append("null");
- } else {
- sb.append("\"").append(lockRequest.toString()).append("\"");
- }
- sb.append(" }");
- return sb.toString();
- }
-}
-
-class WorkerReadyQueue {
- ArrayList<LockRequestWorker> workerReadyQueue;
-
- public WorkerReadyQueue() {
- workerReadyQueue = new ArrayList<LockRequestWorker>();
- }
-
- public synchronized void push(LockRequestWorker worker) {
- workerReadyQueue.add(worker);
- }
-
- public synchronized LockRequestWorker pop(String threadName) {
- int i;
- LockRequestWorker worker = null;
- int size = workerReadyQueue.size();
- for (i = 0; i < size; i++) {
- worker = workerReadyQueue.get(i);
- if (worker.getThreadName().equals(threadName)) {
- workerReadyQueue.remove(i);
- break;
- }
- }
-
- if (i == size) {
- return null;
- } else {
- return worker;
- }
- }
-
- public synchronized int size() {
- return workerReadyQueue.size();
- }
-
- public boolean checkSet(ArrayList<Integer> threadIdList) {
- int i;
- int j;
- StringBuilder s = new StringBuilder();
- LockRequestWorker worker = null;
- int resultListSize = 0;
- int queueSize = workerReadyQueue.size();
- int listSize = threadIdList.size();
-
- s.append("ExpectedList(Set):\t");
- for (i = 0; i < listSize; i++) {
- s.append(threadIdList.get(i)).append(" ");
- }
- s.append("\n");
-
- while (queueSize < listSize) {
- //wait until workers finish its task
- try {
- Thread.sleep(1);
- } catch (InterruptedException e) {
- e.printStackTrace();
- }
- log(Thread.currentThread().getName() + " waiting for worker to finish its task...");
- queueSize = workerReadyQueue.size();
- }
-
- if (listSize != queueSize) {
- log("listSize:" + listSize + ", queueSize:" + queueSize);
- return false;
- }
-
- s.append("ResultList(Set):\t");
- for (i = 0; i < listSize; i++) {
- for (j = 0; j < queueSize; j++) {
- worker = workerReadyQueue.get(j);
- if (worker.getThreadName().equals("Thread-" + threadIdList.get(i))) {
- s.append(threadIdList.get(i)).append(" ");
- resultListSize++;
- break;
- }
- }
- }
-
- log(s.toString());
- if (listSize != resultListSize) {
- return false;
- }
-
- return true;
- }
-
- public boolean checkSequence(ArrayList<Integer> threadIdList) {
- int i;
- StringBuilder s = new StringBuilder();
- LockRequestWorker worker = null;
- int queueSize = workerReadyQueue.size();
- int listSize = threadIdList.size();
-
- s.append("ExpectedList(Sequence):\t");
- for (i = 0; i < listSize; i++) {
- s.append(threadIdList.get(i)).append(" ");
- }
- s.append("\n");
-
- while (queueSize < listSize) {
- //wait until workers finish its task
- try {
- Thread.sleep(1);
- } catch (InterruptedException e) {
- e.printStackTrace();
- }
- log(Thread.currentThread().getName() + " Waiting for worker to finish its task...");
- queueSize = workerReadyQueue.size();
- }
-
- if (queueSize != listSize) {
- return false;
- }
-
- s.append("ResultList(Sequence):\t");
- for (i = 0; i < listSize; i++) {
- worker = workerReadyQueue.get(i);
- if (!worker.getThreadName().equals("Thread-" + threadIdList.get(i))) {
- log(s.toString());
- return false;
- } else {
- s.append(threadIdList.get(i)).append(" ");
- }
- }
-
- log(s.toString());
- return true;
- }
-
- public void log(String s) {
- System.out.println(s);
- }
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockManagerRandomUnitTest.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockManagerRandomUnitTest.java
deleted file mode 100644
index eab8f46..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockManagerRandomUnitTest.java
+++ /dev/null
@@ -1,636 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you 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 at
- *
- * 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 org.apache.asterix.transaction.management.service.locking;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Random;
-
-import org.apache.asterix.common.config.AsterixPropertiesAccessor;
-import org.apache.asterix.common.config.AsterixTransactionProperties;
-import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.transactions.DatasetId;
-import org.apache.asterix.common.transactions.ILockManager;
-import org.apache.asterix.common.transactions.ITransactionContext;
-import org.apache.asterix.common.transactions.ITransactionManager;
-import org.apache.asterix.common.transactions.JobId;
-import org.apache.asterix.transaction.management.service.logging.LogManager;
-import org.apache.asterix.transaction.management.service.transaction.TransactionContext;
-import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
-import org.apache.asterix.transaction.management.service.transaction.TransactionSubsystem;
-import org.apache.commons.io.FileUtils;
-
-/**
- * LockManagerUnitTest: unit test of LockManager
- *
- * @author kisskys
- */
-
-public class LockManagerRandomUnitTest {
-
- private static final int MAX_NUM_OF_UPGRADE_JOB = 2;//2
- private static final int MAX_NUM_OF_ENTITY_LOCK_JOB = 8;//8
- private static final int MAX_NUM_OF_DATASET_LOCK_JOB = 2;//2
- private static final int MAX_NUM_OF_THREAD_IN_A_JOB = 2; //4
- private static int jobId = 0;
- private static Random rand;
-
- public static void main(String args[]) throws ACIDException, AsterixException, IOException {
- int i;
- //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);
-
- TransactionSubsystem txnProvider = new TransactionSubsystem("nc1", null,
- new AsterixTransactionProperties(new AsterixPropertiesAccessor()));
- rand = new Random(System.currentTimeMillis());
- for (i = 0; i < MAX_NUM_OF_ENTITY_LOCK_JOB; i++) {
- System.out.println("Creating " + i + "th EntityLockJob..");
- generateEntityLockThread(txnProvider);
- }
-
- for (i = 0; i < MAX_NUM_OF_DATASET_LOCK_JOB; i++) {
- System.out.println("Creating " + i + "th DatasetLockJob..");
- generateDatasetLockThread(txnProvider);
- }
-
- for (i = 0; i < MAX_NUM_OF_UPGRADE_JOB; i++) {
- System.out.println("Creating " + i + "th EntityLockUpgradeJob..");
- generateEntityLockUpgradeThread(txnProvider);
- }
- ((LogManager) txnProvider.getLogManager()).stop(false, null);
- }
-
- private static void generateEntityLockThread(TransactionSubsystem txnProvider) {
- Thread t;
- int childCount = rand.nextInt(MAX_NUM_OF_THREAD_IN_A_JOB);
- if (MAX_NUM_OF_THREAD_IN_A_JOB != 0 && childCount == 0) {
- childCount = 1;
- }
- TransactionContext txnContext = generateTxnContext(txnProvider);
-
- for (int i = 0; i < childCount; i++) {
- System.out.println("Creating " + txnContext.getJobId() + "," + i + "th EntityLockThread..");
- t = new Thread(new LockRequestProducer(txnProvider.getLockManager(), txnContext, false, false, false));
- t.start();
- }
- }
-
- private static void generateDatasetLockThread(TransactionSubsystem txnProvider) {
- Thread t;
- // int childCount = rand.nextInt(MAX_NUM_OF_THREAD_IN_A_JOB);
- // if (MAX_NUM_OF_THREAD_IN_A_JOB != 0 && childCount == 0) {
- // childCount = 1;
- // }
- int childCount = 1;
-
- TransactionContext txnContext = generateTxnContext(txnProvider);
-
- for (int i = 0; i < childCount; i++) {
- System.out.println("Creating " + txnContext.getJobId() + "," + i + "th DatasetLockThread..");
- t = new Thread(new LockRequestProducer(txnProvider.getLockManager(), txnContext, true, false, false));
- t.start();
- }
- }
-
- private static void generateEntityLockUpgradeThread(TransactionSubsystem txnProvider) {
- int i;
- Thread t;
- int childCount = MAX_NUM_OF_THREAD_IN_A_JOB;
- if (MAX_NUM_OF_THREAD_IN_A_JOB != 0 && childCount == 0) {
- childCount = 1;
- }
- TransactionContext txnContext = generateTxnContext(txnProvider);
-
- for (i = 0; i < childCount - 1; i++) {
- System.out.println("Creating " + txnContext.getJobId() + "," + i + "th EntityLockUpgradeThread(false)..");
- t = new Thread(new LockRequestProducer(txnProvider.getLockManager(), txnContext, false, true, false));
- t.start();
- }
- System.out.println("Creating " + txnContext.getJobId() + "," + i + "th EntityLockUpgradeThread(true)..");
- t = new Thread(new LockRequestProducer(txnProvider.getLockManager(), txnContext, false, true, true));
- t.start();
- }
-
- private static TransactionContext generateTxnContext(TransactionSubsystem txnProvider) {
- try {
- return new TransactionContext(new JobId(jobId++), txnProvider);
- } catch (ACIDException e) {
- e.printStackTrace();
- return null;
- }
- }
-
-}
-
-class LockRequestProducer implements Runnable {
-
- private static final int MAX_DATASET_NUM = 10;//10
- private static final int MAX_ENTITY_NUM = 30;//30
- private static final int MAX_LOCK_MODE_NUM = 2;
- private static final long DATASET_LOCK_THREAD_SLEEP_TIME = 1000;
- private static final int MAX_LOCK_REQUEST_TYPE_NUM = 4;
-
- private ILockManager lockMgr;
- private TransactionContext txnContext;
- private Random rand;
- private boolean isDatasetLock; //dataset or entity
- private ArrayList<LockRequest> requestQueue;
- private StringBuilder requestHistory;
- private int unlockIndex;
- private int upgradeIndex;
- private boolean isUpgradeThread;
- private boolean isUpgradeThreadJob;
- private boolean isDone;
-
- public LockRequestProducer(ILockManager lockMgr, TransactionContext txnContext, boolean isDatasetLock,
- boolean isUpgradeThreadJob, boolean isUpgradeThread) {
- this.lockMgr = lockMgr;
- this.txnContext = txnContext;
- this.isDatasetLock = isDatasetLock;
- this.isUpgradeThreadJob = isUpgradeThreadJob;
- this.isUpgradeThread = isUpgradeThread;
-
- this.rand = new Random(System.currentTimeMillis());
- requestQueue = new ArrayList<LockRequest>();
- requestHistory = new StringBuilder();
- unlockIndex = 0;
- upgradeIndex = 0;
- isDone = false;
- }
-
- @Override
- public void run() {
- try {
- if (isDatasetLock) {
- System.out.println("DatasetLockThread(" + Thread.currentThread().getName() + ") is running...");
- runDatasetLockTask();
- } else {
- System.out.println("EntityLockThread(" + Thread.currentThread().getName() + "," + isUpgradeThreadJob
- + "," + isUpgradeThread + ") is running...");
- runEntityLockTask();
- }
- } catch (Exception e) {
- e.printStackTrace();
- return;
- } finally {
-
- /*
- System.out.println("" + Thread.currentThread().getName() + "\n" + requestHistory.toString() + ""
- + Thread.currentThread().getName() + "\n");
- System.out.println("RequestHistoryPerJobId\n" + ((LockManager) lockMgr).getLocalRequestHistory());
- System.out.println("");
- System.out.println("GlobalRequestHistory\n" + ((LockManager) lockMgr).getGlobalRequestHistory());
- System.out.println("");
- */
- }
- }
-
- private void runDatasetLockTask() {
- try {
- produceDatasetLockRequest();
- if (isDone) {
- return;
- }
- } catch (ACIDException e) {
- e.printStackTrace();
- return;
- }
-
- try {
- Thread.sleep(DATASET_LOCK_THREAD_SLEEP_TIME);
- } catch (InterruptedException e) {
- e.printStackTrace();
- }
-
- try {
- produceDatasetUnlockRequest();
- if (isDone) {
- return;
- }
- } catch (ACIDException e) {
- e.printStackTrace();
- return;
- }
- }
-
- private void runEntityLockTask() {
- int i;
- byte lockMode;
- int lockCount;
- int upgradeCount;
- int releaseCount;
- boolean mayRelease = false;
-
- lockCount = 1 + rand.nextInt(20);
- if (isUpgradeThreadJob) {
- if (isUpgradeThread) {
- upgradeCount = 1; //rand.nextInt(4) + 1;
- if (upgradeCount > lockCount) {
- upgradeCount = lockCount;
- }
- } else {
- upgradeCount = 0;
- }
- lockMode = LockMode.S;
- } else {
- upgradeCount = 0;
- lockMode = (byte) (this.txnContext.getJobId().getId() % 2);
- }
- releaseCount = rand.nextInt(5) % 3 == 0 ? 1 : 0;
-
- //lock
- for (i = 0; i < lockCount; i++) {
- try {
- produceEntityLockRequest(lockMode);
- if (isDone) {
- return;
- }
- } catch (ACIDException e) {
- e.printStackTrace();
- return;
- }
- }
-
- //upgrade
- for (i = 0; i < upgradeCount; i++) {
- try {
- produceEntityLockUpgradeRequest();
- if (isDone) {
- return;
- }
- } catch (ACIDException e) {
- e.printStackTrace();
- return;
- }
- }
-
- //unlock or releaseLocks
- if (releaseCount == 0) {
- //unlock
- for (i = 0; i < lockCount; i++) {
- try {
- produceEntityUnlockRequest();
- if (isDone) {
- return;
- }
- } catch (ACIDException e) {
- e.printStackTrace();
- return;
- }
- }
- } else {
- try {
- synchronized (txnContext) {
- if (txnContext.getTxnState() != ITransactionManager.ABORTED) {
- txnContext.setTxnState(ITransactionManager.ABORTED);
- mayRelease = true;
- }
- }
- if (mayRelease) {
- produceEntityReleaseLocksRequest();
- }
- } catch (ACIDException e) {
- e.printStackTrace();
- return;
- }
- }
- }
-
- private void produceDatasetLockRequest() throws ACIDException {
- int requestType = RequestType.LOCK;
- int datasetId = rand.nextInt(MAX_DATASET_NUM);
- int entityHashValue = -1;
- byte lockMode = (byte) (rand.nextInt(MAX_LOCK_MODE_NUM));
- LockRequest request = new LockRequest(Thread.currentThread().getName(), requestType, new DatasetId(datasetId),
- entityHashValue, lockMode, txnContext);
- requestQueue.add(request);
- requestHistory.append(request.prettyPrint());
- sendRequest(request);
- }
-
- private void produceDatasetUnlockRequest() throws ACIDException {
- LockRequest lockRequest = requestQueue.get(0);
-
- int requestType = RequestType.RELEASE_LOCKS;
- int datasetId = lockRequest.datasetIdObj.getId();
- int entityHashValue = -1;
- byte lockMode = LockMode.S;//lockMode is not used for unlock() call.
- LockRequest request = new LockRequest(Thread.currentThread().getName(), requestType, new DatasetId(datasetId),
- entityHashValue, lockMode, txnContext);
- requestQueue.add(request);
- requestHistory.append(request.prettyPrint());
- sendRequest(request);
- }
-
- private void produceEntityLockRequest(byte lockMode) throws ACIDException {
- int requestType = rand.nextInt(MAX_LOCK_REQUEST_TYPE_NUM);
- int datasetId = rand.nextInt(MAX_DATASET_NUM);
- int entityHashValue = rand.nextInt(MAX_ENTITY_NUM);
- LockRequest request = new LockRequest(Thread.currentThread().getName(), requestType, new DatasetId(datasetId),
- entityHashValue, lockMode, txnContext);
- requestQueue.add(request);
- requestHistory.append(request.prettyPrint());
- sendRequest(request);
- }
-
- private void produceEntityLockUpgradeRequest() throws ACIDException {
- LockRequest lockRequest = null;
- int size = requestQueue.size();
- boolean existLockRequest = false;
-
- while (upgradeIndex < size) {
- lockRequest = requestQueue.get(upgradeIndex++);
- if (lockRequest.isUpgrade || lockRequest.isTryLockFailed) {
- continue;
- }
- if (lockRequest.requestType == RequestType.UNLOCK || lockRequest.requestType == RequestType.RELEASE_LOCKS
- || lockRequest.requestType == RequestType.INSTANT_LOCK
- || lockRequest.requestType == RequestType.INSTANT_TRY_LOCK) {
- continue;
- }
- if (lockRequest.lockMode == LockMode.X) {
- continue;
- }
- existLockRequest = true;
- break;
- }
-
- if (existLockRequest) {
- int requestType = lockRequest.requestType;
- int datasetId = lockRequest.datasetIdObj.getId();
- int entityHashValue = lockRequest.entityHashValue;
- byte lockMode = LockMode.X;
- LockRequest request = new LockRequest(Thread.currentThread().getName(), requestType,
- new DatasetId(datasetId), entityHashValue, lockMode, txnContext);
- request.isUpgrade = true;
- requestQueue.add(request);
- requestHistory.append(request.prettyPrint());
- sendRequest(request);
- }
- }
-
- private void produceEntityUnlockRequest() throws ACIDException {
- LockRequest lockRequest = null;
- int size = requestQueue.size();
- boolean existLockRequest = false;
-
- while (unlockIndex < size) {
- lockRequest = requestQueue.get(unlockIndex++);
- if (lockRequest.isUpgrade || lockRequest.isTryLockFailed) {
- continue;
- }
- if (lockRequest.requestType == RequestType.UNLOCK || lockRequest.requestType == RequestType.RELEASE_LOCKS
- || lockRequest.requestType == RequestType.INSTANT_LOCK
- || lockRequest.requestType == RequestType.INSTANT_TRY_LOCK) {
- continue;
- }
- existLockRequest = true;
- break;
- }
-
- if (existLockRequest) {
- int requestType = RequestType.UNLOCK;
- int datasetId = lockRequest.datasetIdObj.getId();
- int entityHashValue = lockRequest.entityHashValue;
- byte lockMode = lockRequest.lockMode;
- LockRequest request = new LockRequest(Thread.currentThread().getName(), requestType,
- new DatasetId(datasetId), entityHashValue, lockMode, txnContext);
- requestQueue.add(request);
- requestHistory.append(request.prettyPrint());
- sendRequest(request);
- }
- }
-
- private void produceEntityReleaseLocksRequest() throws ACIDException {
- LockRequest lockRequest = null;
- int size = requestQueue.size();
- boolean existLockRequest = false;
-
- while (unlockIndex < size) {
- lockRequest = requestQueue.get(unlockIndex++);
- if (lockRequest.isUpgrade || lockRequest.isTryLockFailed) {
- continue;
- }
- if (lockRequest.requestType == RequestType.UNLOCK || lockRequest.requestType == RequestType.RELEASE_LOCKS
- || lockRequest.requestType == RequestType.INSTANT_LOCK
- || lockRequest.requestType == RequestType.INSTANT_TRY_LOCK) {
- continue;
- }
- existLockRequest = true;
- break;
- }
-
- if (existLockRequest) {
- int requestType = RequestType.RELEASE_LOCKS;
- int datasetId = lockRequest.datasetIdObj.getId();
- int entityHashValue = lockRequest.entityHashValue;
- byte lockMode = lockRequest.lockMode;
- LockRequest request = new LockRequest(Thread.currentThread().getName(), requestType,
- new DatasetId(datasetId), entityHashValue, lockMode, txnContext);
- requestQueue.add(request);
- requestHistory.append(request.prettyPrint());
- sendRequest(request);
- }
- }
-
- private void sendRequest(LockRequest request) throws ACIDException {
-
- switch (request.requestType) {
- case RequestType.LOCK:
- try {
- lockMgr.lock(request.datasetIdObj, request.entityHashValue, request.lockMode, request.txnContext);
- } catch (ACIDException e) {
- if (request.txnContext.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 {
- lockMgr.releaseLocks(request.txnContext);
- } catch (ACIDException e1) {
- e1.printStackTrace();
- }
- log("Abort --> Released all locks acquired by " + request.txnContext.getJobId());
- }
- isDone = true;
- } else {
- throw e;
- }
- }
- break;
- case RequestType.INSTANT_LOCK:
- try {
- lockMgr.instantLock(request.datasetIdObj, request.entityHashValue, request.lockMode,
- request.txnContext);
- } catch (ACIDException e) {
- if (request.txnContext.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 {
- lockMgr.releaseLocks(request.txnContext);
- } catch (ACIDException e1) {
- e1.printStackTrace();
- }
- log("Abort --> Released all locks acquired by " + request.txnContext.getJobId());
- }
- isDone = true;
- } else {
- throw e;
- }
- }
- break;
- case RequestType.TRY_LOCK:
- request.isTryLockFailed = !lockMgr.tryLock(request.datasetIdObj, request.entityHashValue,
- request.lockMode, request.txnContext);
- break;
- case RequestType.INSTANT_TRY_LOCK:
- lockMgr.instantTryLock(request.datasetIdObj, request.entityHashValue, request.lockMode,
- request.txnContext);
- break;
- case RequestType.UNLOCK:
- lockMgr.unlock(request.datasetIdObj, request.entityHashValue, request.lockMode, request.txnContext);
- break;
- case RequestType.RELEASE_LOCKS:
- lockMgr.releaseLocks(request.txnContext);
- break;
- default:
- throw new UnsupportedOperationException("Unsupported lock method");
- }
- try {
- Thread.sleep(0);
- } catch (InterruptedException e) {
- // TODO Auto-generated catch block
- e.printStackTrace();
- }
- }
-
- private void log(String s) {
- System.out.println(s);
- }
-}
-
-class LockRequest {
- public int requestType;
- public DatasetId datasetIdObj;
- public int entityHashValue;
- public byte lockMode;
- public ITransactionContext txnContext;
- public boolean isUpgrade;
- public boolean isTryLockFailed;
- public long requestTime;
- public String threadName;
-
- public LockRequest(String threadName, int requestType, DatasetId datasetIdObj, int entityHashValue, byte lockMode,
- ITransactionContext txnContext) {
- this.requestType = requestType;
- this.datasetIdObj = datasetIdObj;
- this.entityHashValue = entityHashValue;
- this.lockMode = lockMode;
- this.txnContext = txnContext;
- this.requestTime = System.currentTimeMillis();
- this.threadName = new String(threadName);
- isUpgrade = false;
- isTryLockFailed = false;//used for TryLock request not to call Unlock when the tryLock failed.
- }
-
- public LockRequest(String threadName, int requestType) {
- this.requestType = requestType;
- this.requestTime = System.currentTimeMillis();
- this.threadName = new String(threadName);
- }
-
- //used for "W" request type
- public LockRequest(String threadName, int requestType, int waitTime) {
- this.requestType = requestType;
- this.requestTime = System.currentTimeMillis();
- this.threadName = new String(threadName);
- this.entityHashValue = waitTime;
- }
-
- @Override
- public String toString() {
- return prettyPrint();
- }
-
- public String prettyPrint() {
- StringBuilder s = new StringBuilder();
- //s.append(threadName.charAt(7)).append("\t").append("\t");
- s.append("T").append(threadName.substring(7)).append("\t");
- switch (requestType) {
- case RequestType.LOCK:
- s.append("L");
- break;
- case RequestType.TRY_LOCK:
- s.append("TL");
- break;
- case RequestType.INSTANT_LOCK:
- s.append("IL");
- break;
- case RequestType.INSTANT_TRY_LOCK:
- s.append("ITL");
- break;
- case RequestType.UNLOCK:
- s.append("UL");
- break;
- case RequestType.RELEASE_LOCKS:
- s.append("RL");
- break;
- case RequestType.CHECK_SEQUENCE:
- s.append("CSQ");
- return s.toString();
- case RequestType.CHECK_SET:
- s.append("CST");
- return s.toString();
- case RequestType.END:
- s.append("END");
- return s.toString();
- case RequestType.WAIT:
- s.append("W").append("\t").append(entityHashValue);
- return s.toString();
- default:
- throw new UnsupportedOperationException("Unsupported method");
- }
- s.append("\tJ").append(txnContext.getJobId().getId()).append("\tD").append(datasetIdObj.getId()).append("\tE")
- .append(entityHashValue).append("\t");
- s.append(LockMode.toString(lockMode)).append("\n");
- return s.toString();
- }
-}
-
-class RequestType {
- public static final int LOCK = 0;
- public static final int TRY_LOCK = 1;
- public static final int INSTANT_LOCK = 2;
- public static final int INSTANT_TRY_LOCK = 3;
- public static final int UNLOCK = 4;
- public static final int RELEASE_LOCKS = 5;
- public static final int CHECK_SEQUENCE = 6;
- public static final int CHECK_SET = 7;
- public static final int END = 8;
- public static final int WAIT = 9;
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockRequestTracker.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockRequestTracker.java
deleted file mode 100644
index 6fb0675..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockRequestTracker.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you 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 at
- *
- * 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 org.apache.asterix.transaction.management.service.locking;
-
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-public class LockRequestTracker {
- HashMap<Integer, StringBuilder> historyPerJob; //per job
- StringBuilder historyForAllJobs;
- StringBuilder requestHistoryForAllJobs; //request only
-
- public LockRequestTracker() {
- historyForAllJobs = new StringBuilder();
- historyPerJob = new HashMap<Integer, StringBuilder>();
- requestHistoryForAllJobs = new StringBuilder();
- }
-
- public void addEvent(String msg, LockRequest request) {
- int jobId = request.txnContext.getJobId().getId();
- StringBuilder jobHistory = historyPerJob.get(jobId);
-
- //update jobHistory
- if (jobHistory == null) {
- jobHistory = new StringBuilder();
- }
- jobHistory.append(request.prettyPrint()).append("--> ").append(msg).append("\n");
- historyPerJob.put(jobId, jobHistory);
-
- //handle global request queue
- historyForAllJobs.append(request.prettyPrint()).append("--> ").append(msg).append("\n");
- }
-
- public void addRequest(LockRequest request) {
- requestHistoryForAllJobs.append(request.prettyPrint());
- }
-
- public String getHistoryForAllJobs() {
- return historyForAllJobs.toString();
- }
-
- public String getHistoryPerJob() {
- StringBuilder history = new StringBuilder();
- Set<Entry<Integer, StringBuilder>> s = historyPerJob.entrySet();
- Iterator<Entry<Integer, StringBuilder>> iter = s.iterator();
- while (iter.hasNext()) {
- Map.Entry<Integer, StringBuilder> entry = (Map.Entry<Integer, StringBuilder>) iter.next();
- history.append(entry.getValue().toString());
- }
- return history.toString();
- }
-
- public String getRequestHistoryForAllJobs() {
- return requestHistoryForAllJobs.toString();
- }
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockWaiter.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockWaiter.java
deleted file mode 100644
index 82f0877..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockWaiter.java
+++ /dev/null
@@ -1,149 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you 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 at
- *
- * 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 org.apache.asterix.transaction.management.service.locking;
-
-/**
- * LockWaiter object is used for keeping a lock waiter or a lock upgrader information on a certain resource.
- * The resource can be a dataset or an entity.
- *
- * @author kisskys
- */
-public class LockWaiter {
- /**
- * entityInfoSlotNum:
- * If this LockWaiter object is used, this variable is used to indicate the corresponding EntityInfoSlotNum.
- * Otherwise, the variable is used for nextFreeSlot Which indicates the next free waiter object.
- */
- private int entityInfoSlotNum;
- private boolean wait;
- private boolean victim;
- private byte waiterCount;
- private boolean firstGetUp;
- private int nextWaiterObjId; //used for DatasetLockInfo and EntityLockInfo
- private int nextWaitingResourceObjId; //used for JobInfo
- private long beginWaitTime;
- private boolean isWaiter; //is upgrader or waiter
- private boolean isWaitingOnEntityLock; //is waiting on datasetLock or entityLock
-
- public LockWaiter() {
- this.victim = false;
- this.wait = true;
- waiterCount = 0;
- nextWaiterObjId = -1;
- nextWaitingResourceObjId = -1;
- }
-
- public void setEntityInfoSlot(int slotNum) {
- this.entityInfoSlotNum = slotNum;
- }
-
- public int getEntityInfoSlot() {
- return this.entityInfoSlotNum;
- }
-
- public void setNextFreeSlot(int slotNum) {
- this.entityInfoSlotNum = slotNum;
- }
-
- public int getNextFreeSlot() {
- return this.entityInfoSlotNum;
- }
-
- public void setWait(boolean wait) {
- this.wait = wait;
- }
-
- public boolean needWait() {
- return this.wait;
- }
-
- public void setVictim(boolean victim) {
- this.victim = victim;
- }
-
- public boolean isVictim() {
- return this.victim;
- }
-
- public void increaseWaiterCount() {
- waiterCount++;
- }
-
- public void decreaseWaiterCount() {
- waiterCount--;
- }
-
- public byte getWaiterCount() {
- return waiterCount;
- }
-
- public void setWaiterCount(byte count) {
- waiterCount = count;
- }
-
- public void setFirstGetUp(boolean isFirst) {
- firstGetUp = isFirst;
- }
-
- public boolean isFirstGetUp() {
- return firstGetUp;
- }
-
- public void setNextWaiterObjId(int next) {
- nextWaiterObjId = next;
- }
-
- public int getNextWaiterObjId() {
- return nextWaiterObjId;
- }
-
- public void setNextWaitingResourceObjId(int next) {
- nextWaitingResourceObjId = next;
- }
-
- public int getNextWaitingResourceObjId() {
- return nextWaitingResourceObjId;
- }
-
- public void setBeginWaitTime(long time) {
- this.beginWaitTime = time;
- }
-
- public long getBeginWaitTime() {
- return beginWaitTime;
- }
-
- public boolean isWaiter() {
- return isWaiter;
- }
-
- public void setWaiter(boolean isWaiter) {
- this.isWaiter = isWaiter;
- }
-
- public boolean isWaitingOnEntityLock() {
- return isWaitingOnEntityLock;
- }
-
- public void setWaitingOnEntityLock(boolean isWaitingOnEntityLock) {
- this.isWaitingOnEntityLock = isWaitingOnEntityLock;
- }
-
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockWaiterManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockWaiterManager.java
deleted file mode 100644
index ae971fb..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockWaiterManager.java
+++ /dev/null
@@ -1,403 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you 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 at
- *
- * 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 org.apache.asterix.transaction.management.service.locking;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.ArrayList;
-
-/**
- * LockWaiterManager manages LockWaiter objects array.
- * The array grows when the slots are overflowed.
- * Also, the array shrinks according to the following shrink policy
- * : Shrink when the resource under-utilization lasts for a certain threshold time.
- *
- * @author kisskys
- */
-public class LockWaiterManager {
-
- public static final int SHRINK_TIMER_THRESHOLD = 120000; //2min
-
- private ArrayList<ChildLockWaiterArrayManager> pArray; //parent array
- private int allocChild; //used to allocate the next free LockWaiter object.
- private long shrinkTimer;
- private boolean isShrinkTimerOn;
- private int occupiedSlots;
-
- // ////////////////////////////////////////////////
- // // begin of unit test
- // ////////////////////////////////////////////////
- //
- // public static final int SHRINK_TIMER_THRESHOLD = 0; //for unit test
- //
- // /**
- // * @param args
- // */
- // public static void main(String[] args) {
- // final int DataSize = 5000;
- //
- // int i, j;
- // int slots = ChildLockWaiterArrayManager.NUM_OF_SLOTS;
- // int data[] = new int[DataSize];
- // LockWaiterManager lwMgr = new LockWaiterManager();
- //
- // //allocate: 50
- // System.out.println("allocate: 50");
- // for (i = 0; i < 5; i++) {
- // for (j = i * slots; j < i * slots + slots; j++) {
- // data[j] = lwMgr.allocate();
- // }
- //
- // System.out.println(lwMgr.prettyPrint());
- // }
- //
- // //deallocate from the last child to the first child
- // System.out.println("deallocate from the last child to the first child");
- // for (i = 4; i >= 0; i--) {
- // for (j = i * slots + slots - 1; j >= i * slots; j--) {
- // lwMgr.deallocate(data[j]);
- // }
- // System.out.println(lwMgr.prettyPrint());
- // }
- //
- // //allocate: 50
- // System.out.println("allocate: 50");
- // for (i = 0; i < 5; i++) {
- // for (j = i * slots; j < i * slots + slots; j++) {
- // data[j] = lwMgr.allocate();
- // }
- //
- // System.out.println(lwMgr.prettyPrint());
- // }
- //
- // //deallocate from the first child to last child
- // System.out.println("deallocate from the first child to last child");
- // for (i = 0; i < 5; i++) {
- // for (j = i * slots; j < i * slots + slots; j++) {
- // lwMgr.deallocate(data[j]);
- // }
- //
- // System.out.println(lwMgr.prettyPrint());
- // }
- //
- // //allocate: 50
- // System.out.println("allocate: 50");
- // for (i = 0; i < 5; i++) {
- // for (j = i * slots; j < i * slots + slots; j++) {
- // data[j] = lwMgr.allocate();
- // }
- //
- // System.out.println(lwMgr.prettyPrint());
- // }
- //
- // //deallocate from the first child to 4th child
- // System.out.println("deallocate from the first child to 4th child");
- // for (i = 0; i < 4; i++) {
- // for (j = i * slots; j < i * slots + slots; j++) {
- // lwMgr.deallocate(data[j]);
- // }
- //
- // System.out.println(lwMgr.prettyPrint());
- // }
- //
- // //allocate: 40
- // System.out.println("allocate: 40");
- // for (i = 0; i < 4; i++) {
- // for (j = i * slots; j < i * slots + slots; j++) {
- // data[j] = lwMgr.allocate();
- // }
- //
- // System.out.println(lwMgr.prettyPrint());
- // }
- // }
- //
- // ////////////////////////////////////////////////
- // // end of unit test
- // ////////////////////////////////////////////////
-
- public LockWaiterManager() {
- pArray = new ArrayList<ChildLockWaiterArrayManager>();
- pArray.add(new ChildLockWaiterArrayManager());
- allocChild = 0;
- occupiedSlots = 0;
- isShrinkTimerOn = false;
- }
-
- public int allocate() {
- if (pArray.get(allocChild).isFull()) {
- int size = pArray.size();
- boolean bAlloc = false;
- ChildLockWaiterArrayManager child;
-
- //find a deinitialized child and initialize it
- for (int i = 0; i < size; i++) {
- child = pArray.get(i);
- if (child.isDeinitialized()) {
- child.initialize();
- allocChild = i;
- bAlloc = true;
- break;
- }
- }
-
- //allocate new child when there is no deinitialized child
- if (!bAlloc) {
- pArray.add(new ChildLockWaiterArrayManager());
- allocChild = pArray.size() - 1;
- }
- }
- occupiedSlots++;
- return pArray.get(allocChild).allocate() + allocChild * ChildLockWaiterArrayManager.NUM_OF_SLOTS;
- }
-
- void deallocate(int slotNum) {
- pArray.get(slotNum / ChildLockWaiterArrayManager.NUM_OF_SLOTS).deallocate(
- slotNum % ChildLockWaiterArrayManager.NUM_OF_SLOTS);
- occupiedSlots--;
-
- if (needShrink()) {
- shrink();
- }
- }
-
- /**
- * Shrink policy:
- * Shrink when the resource under-utilization lasts for a certain amount of time.
- * TODO Need to figure out which of the policies is better
- * case1.
- * pArray status : O x x x x x O (O is initialized, x is deinitialized)
- * In the above status, 'CURRENT' needShrink() returns 'TRUE'
- * even if there is nothing to shrink or deallocate.
- * It doesn't distinguish the deinitialized children from initialized children
- * by calculating totalNumOfSlots = pArray.size() * ChildLockWaiterArrayManager.NUM_OF_SLOTS.
- * In other words, it doesn't subtract the deinitialized children's slots.
- * case2.
- * pArray status : O O x x x x x
- * However, in the above case, if we subtract the deinitialized children's slots,
- * needShrink() will return false even if we shrink the pArray at this case.
- *
- * @return
- */
- private boolean needShrink() {
- int size = pArray.size();
- int usedSlots = occupiedSlots;
- if (usedSlots == 0) {
- usedSlots = 1;
- }
-
- if (size > 1 && size * ChildLockWaiterArrayManager.NUM_OF_SLOTS / usedSlots >= 3) {
- if (isShrinkTimerOn) {
- if (System.currentTimeMillis() - shrinkTimer >= SHRINK_TIMER_THRESHOLD) {
- isShrinkTimerOn = false;
- return true;
- }
- } else {
- //turn on timer
- isShrinkTimerOn = true;
- shrinkTimer = System.currentTimeMillis();
- }
- } else {
- //turn off timer
- isShrinkTimerOn = false;
- }
-
- return false;
- }
-
- /**
- * Shrink() may
- * deinitialize(:deallocates array of LockWaiter objects in a child) Children(s) or
- * shrink pArray according to the deinitialized children's contiguity status.
- * It doesn't deinitialize or shrink more than half of children at a time.
- */
- private void shrink() {
- int i;
- int removeCount = 0;
- int size = pArray.size();
- int maxDecreaseCount = size / 2;
- ChildLockWaiterArrayManager child;
-
- //The first buffer never be deinitialized.
- for (i = 1; i < size; i++) {
- if (pArray.get(i).isEmpty()) {
- pArray.get(i).deinitialize();
- }
- }
-
- //remove the empty buffers from the end
- for (i = size - 1; i >= 1; i--) {
- child = pArray.get(i);
- if (child.isDeinitialized()) {
- pArray.remove(i);
- if (++removeCount == maxDecreaseCount) {
- break;
- }
- } else {
- break;
- }
- }
-
- //reset allocChild to the first buffer
- allocChild = 0;
-
- isShrinkTimerOn = false;
- }
-
- public String prettyPrint() {
- StringBuilder s = new StringBuilder("\n########### LockWaiterManager Status #############\n");
- int size = pArray.size();
- ChildLockWaiterArrayManager child;
-
- for (int i = 0; i < size; i++) {
- child = pArray.get(i);
- if (child.isDeinitialized()) {
- continue;
- }
- s.append("child[" + i + "]");
- s.append(child.prettyPrint());
- }
- return s.toString();
- }
-
- public void coreDump(OutputStream os) {
- StringBuilder sb = new StringBuilder("\n########### LockWaiterManager Status #############\n");
- int size = pArray.size();
- ChildLockWaiterArrayManager child;
-
- sb.append("Number of Child: " + size + "\n");
- for (int i = 0; i < size; i++) {
- try {
- child = pArray.get(i);
- sb.append("child[" + i + "]");
- sb.append(child.prettyPrint());
-
- os.write(sb.toString().getBytes());
- } catch (IOException e) {
- //ignore IOException
- }
- sb = new StringBuilder();
- }
- }
-
- public int getShrinkTimerThreshold() {
- return SHRINK_TIMER_THRESHOLD;
- }
-
- public LockWaiter getLockWaiter(int slotNum) {
- return pArray.get(slotNum / ChildLockWaiterArrayManager.NUM_OF_SLOTS).getLockWaiter(
- slotNum % ChildLockWaiterArrayManager.NUM_OF_SLOTS);
- }
-}
-
-class ChildLockWaiterArrayManager {
- public static final int NUM_OF_SLOTS = 100; //number of LockWaiter objects in 'childArray'.
- // public static final int NUM_OF_SLOTS = 10; //for unit test
-
- private int freeSlotNum;
- private int occupiedSlots; //-1 represents 'deinitialized' state.
- LockWaiter childArray[];//childArray
-
- public ChildLockWaiterArrayManager() {
- initialize();
- }
-
- public void initialize() {
- this.childArray = new LockWaiter[NUM_OF_SLOTS];
- this.freeSlotNum = 0;
- this.occupiedSlots = 0;
-
- for (int i = 0; i < NUM_OF_SLOTS - 1; i++) {
- childArray[i] = new LockWaiter();
- childArray[i].setNextFreeSlot(i + 1);
- }
- childArray[NUM_OF_SLOTS - 1] = new LockWaiter();
- childArray[NUM_OF_SLOTS - 1].setNextFreeSlot(-1); //-1 represents EOL(end of link)
- }
-
- public LockWaiter getLockWaiter(int slotNum) {
- return childArray[slotNum];
- }
-
- public int allocate() {
- int currentSlot = freeSlotNum;
- freeSlotNum = childArray[currentSlot].getNextFreeSlot();
- childArray[currentSlot].setWait(true);
- childArray[currentSlot].setVictim(false);
- childArray[currentSlot].setWaiterCount((byte) 0);
- childArray[currentSlot].setNextWaiterObjId(-1);
- childArray[currentSlot].setNextWaitingResourceObjId(-1);
- childArray[currentSlot].setBeginWaitTime(-1l);
- occupiedSlots++;
- if (LockManager.IS_DEBUG_MODE) {
- System.out.println(Thread.currentThread().getName() + " Alloc LockWaiterId(" + currentSlot + ")");
- }
- return currentSlot;
- }
-
- public void deallocate(int slotNum) {
- childArray[slotNum].setNextFreeSlot(freeSlotNum);
- freeSlotNum = slotNum;
- occupiedSlots--;
- if (LockManager.IS_DEBUG_MODE) {
- System.out.println(Thread.currentThread().getName() + " Dealloc LockWaiterId(" + slotNum + ")");
- }
- }
-
- public void deinitialize() {
- childArray = null;
- occupiedSlots = -1;
- }
-
- public boolean isDeinitialized() {
- return occupiedSlots == -1;
- }
-
- public boolean isFull() {
- return occupiedSlots == NUM_OF_SLOTS;
- }
-
- public boolean isEmpty() {
- return occupiedSlots == 0;
- }
-
- public int getNumOfOccupiedSlots() {
- return occupiedSlots;
- }
-
- public int getFreeSlotNum() {
- return freeSlotNum;
- }
-
- public String prettyPrint() {
- LockWaiter waiter;
- StringBuilder sb = new StringBuilder();
- sb.append("\n\toccupiedSlots:" + getNumOfOccupiedSlots());
- sb.append("\n\tfreeSlotNum:" + getFreeSlotNum() + "\n");
- for (int j = 0; j < ChildLockWaiterArrayManager.NUM_OF_SLOTS; j++) {
- waiter = getLockWaiter(j);
- sb.append(j).append(": ");
- sb.append("\t" + waiter.getEntityInfoSlot());
- sb.append("\t" + waiter.needWait());
- sb.append("\t" + waiter.isVictim());
- sb.append("\n");
- }
- return sb.toString();
- }
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/TimeOutDetector.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/TimeOutDetector.java
deleted file mode 100644
index bd20c43..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/TimeOutDetector.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you 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 at
- *
- * 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 org.apache.asterix.transaction.management.service.locking;
-
-import java.util.LinkedList;
-import java.util.concurrent.Executor;
-
-import org.apache.asterix.common.exceptions.ACIDException;
-
-/**
- * @author pouria, kisskys
- * Any transaction which has been waiting for a lock for more
- * than the predefined time-out threshold is considered to be deadlocked
- * (this can happen in distributed case for example) An instance of this
- * class triggers scanning (sweeping) lock manager's transactions table
- * periodically and detects such timed-out transactions
- */
-
-public class TimeOutDetector {
-
- LockManager lockMgr;
- Thread trigger;
- LinkedList<LockWaiter> victimList;
- int timeoutThreshold;
- int sweepThreshold;
-
- public TimeOutDetector(LockManager lockMgr, Executor threadExecutor) {
- this.victimList = new LinkedList<LockWaiter>();
- this.lockMgr = lockMgr;
- this.trigger = new Thread(new TimeoutTrigger(this));
- this.timeoutThreshold = lockMgr.getTransactionProperties().getTimeoutWaitThreshold();
- this.sweepThreshold = lockMgr.getTransactionProperties().getTimeoutSweepThreshold();
- trigger.setDaemon(true);
- threadExecutor.execute(trigger);
- }
-
- public void sweep() throws ACIDException {
- victimList.clear();
- // Initiates the time-out sweeping process
- // from the lockManager
- lockMgr.sweepForTimeout();
- notifyVictims();
- }
-
- public void checkAndSetVictim(LockWaiter waiterObj) {
- if (System.currentTimeMillis() - waiterObj.getBeginWaitTime() >= timeoutThreshold) {
- waiterObj.setVictim(true);
- waiterObj.setWait(false);
- victimList.add(waiterObj);
- }
- }
-
- private void notifyVictims() {
- for (LockWaiter waiterObj : victimList) {
- synchronized (waiterObj) {
- waiterObj.notifyAll();
- }
- }
- victimList.clear();
- }
-}
-
-class TimeoutTrigger implements Runnable {
-
- TimeOutDetector owner;
-
- public TimeoutTrigger(TimeOutDetector owner) {
- this.owner = owner;
- }
-
- @Override
- public void run() {
- while (true) {
- try {
- Thread.sleep(owner.sweepThreshold);
- owner.sweep(); // Trigger the timeout detector (the owner) to
- // initiate sweep
- } catch (InterruptedException e) {
- e.printStackTrace();
- } catch (ACIDException e) {
- throw new IllegalStateException(e);
- }
- }
- }
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java
index 502e9c7..c1180a4 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java
@@ -90,7 +90,7 @@
@Override
public void append(ILogRecord logRecord, long appendLSN) {
logRecord.writeLogRecord(appendBuffer);
- if (logRecord.getLogType() != LogType.FLUSH) {
+ if (logRecord.getLogType() != LogType.FLUSH && logRecord.getLogType() != LogType.WAIT) {
logRecord.getTxnCtx().setLastLSN(appendLSN);
}
synchronized (this) {
@@ -98,7 +98,8 @@
if (IS_DEBUG_MODE) {
LOGGER.info("append()| appendOffset: " + appendOffset);
}
- if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT) {
+ if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT
+ || logRecord.getLogType() == LogType.WAIT) {
logRecord.isFlushed(false);
syncCommitQ.offer(logRecord);
}
@@ -114,18 +115,19 @@
public void appendWithReplication(ILogRecord logRecord, long appendLSN) {
logRecord.writeLogRecord(appendBuffer, appendLSN);
- if (logRecord.getLogSource() == LogSource.LOCAL && logRecord.getLogType() != LogType.FLUSH) {
- if (logRecord.getLogType() != LogType.FLUSH) {
- logRecord.getTxnCtx().setLastLSN(appendLSN);
- }
+ if (logRecord.getLogSource() == LogSource.LOCAL && logRecord.getLogType() != LogType.FLUSH
+ && logRecord.getLogType() != LogType.WAIT) {
+ logRecord.getTxnCtx().setLastLSN(appendLSN);
}
+
synchronized (this) {
appendOffset += logRecord.getLogSize();
if (IS_DEBUG_MODE) {
LOGGER.info("append()| appendOffset: " + appendOffset);
}
if (logRecord.getLogSource() == LogSource.LOCAL) {
- if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT) {
+ if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT
+ || logRecord.getLogType() == LogType.WAIT) {
logRecord.isFlushed(false);
syncCommitQ.offer(logRecord);
}
@@ -265,18 +267,23 @@
// since this operation consisted of delete and insert, we need to notify the optracker twice
txnCtx.notifyOptracker(false);
}
+ if (TransactionSubsystem.IS_PROFILE_MODE) {
+ txnSubsystem.incrementEntityCommitCount();
+ }
} else if (logRecord.getLogType() == LogType.JOB_COMMIT
|| logRecord.getLogType() == LogType.ABORT) {
reusableJobId.setId(logRecord.getJobId());
txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(reusableJobId, false);
txnCtx.notifyOptracker(true);
- notifyJobTerminator();
+ notifyJobTermination();
} else if (logRecord.getLogType() == LogType.FLUSH) {
- notifyFlushTerminator();
+ notifyFlushTermination();
+ } else if (logRecord.getLogType() == LogType.WAIT) {
+ notifyWaitTermination();
}
} else if (logRecord.getLogSource() == LogSource.REMOTE) {
if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT) {
- notifyReplicationTerminator();
+ notifyReplicationTermination();
}
}
@@ -285,7 +292,15 @@
}
}
- public void notifyJobTerminator() {
+ public void notifyJobTermination() {
+ notifyToSyncCommitQWaiter();
+ }
+
+ public void notifyWaitTermination() {
+ notifyToSyncCommitQWaiter();
+ }
+
+ public void notifyToSyncCommitQWaiter() {
ILogRecord logRecord = null;
while (logRecord == null) {
try {
@@ -300,7 +315,7 @@
}
}
- public void notifyFlushTerminator() throws ACIDException {
+ public void notifyFlushTermination() throws ACIDException {
LogRecord logRecord = null;
try {
logRecord = (LogRecord) flushQ.take();
@@ -321,7 +336,7 @@
}
}
- public void notifyReplicationTerminator() {
+ public void notifyReplicationTermination() {
LogRecord logRecord = null;
try {
logRecord = (LogRecord) remoteJobsQ.take();
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java
index b8ccf04..776577a 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java
@@ -128,8 +128,8 @@
protected void appendToLogTail(ILogRecord logRecord) throws ACIDException {
syncAppendToLogTail(logRecord);
- if ((logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT)
- && !logRecord.isFlushed()) {
+ if ((logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT
+ || logRecord.getLogType() == LogType.WAIT) && !logRecord.isFlushed()) {
synchronized (logRecord) {
while (!logRecord.isFlushed()) {
try {
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
index efd66a8..4112529 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
@@ -42,7 +42,7 @@
}
//only locally generated logs should be replicated
- logRecord.setReplicated(logRecord.getLogSource() == LogSource.LOCAL);
+ logRecord.setReplicated(logRecord.getLogSource() == LogSource.LOCAL && logRecord.getLogType() != LogType.WAIT);
//Remote flush logs do not need to be flushed separately since they may not trigger local flush
if (logRecord.getLogType() == LogType.FLUSH && logRecord.getLogSource() == LogSource.LOCAL) {
@@ -62,8 +62,8 @@
}
if (logRecord.getLogSource() == LogSource.LOCAL) {
- if ((logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT)
- && !logRecord.isFlushed()) {
+ if ((logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT
+ || logRecord.getLogType() == LogType.WAIT) && !logRecord.isFlushed()) {
synchronized (logRecord) {
while (!logRecord.isFlushed()) {
try {
@@ -74,11 +74,13 @@
}
//wait for job Commit/Abort ACK from replicas
- while (!replicationManager.hasBeenReplicated(logRecord)) {
- try {
- logRecord.wait();
- } catch (InterruptedException e) {
- //ignore
+ if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT) {
+ while (!replicationManager.hasBeenReplicated(logRecord)) {
+ try {
+ logRecord.wait();
+ } catch (InterruptedException e) {
+ //ignore
+ }
}
}
}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/RecoveryManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/RecoveryManager.java
index 9a74a18..0ec6b10 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/RecoveryManager.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/RecoveryManager.java
@@ -270,6 +270,7 @@
abortLogCount++;
break;
case LogType.FLUSH:
+ case LogType.WAIT:
break;
default:
throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
@@ -289,8 +290,8 @@
return winnerJobSet;
}
- private synchronized void startRecoveryRedoPhase(Set<Integer> partitions, ILogReader logReader, long lowWaterMarkLSN,
- Set<Integer> winnerJobSet) throws IOException, ACIDException {
+ private synchronized void startRecoveryRedoPhase(Set<Integer> partitions, ILogReader logReader,
+ long lowWaterMarkLSN, Set<Integer> winnerJobSet) throws IOException, ACIDException {
int redoCount = 0;
int jobId = -1;
@@ -765,6 +766,7 @@
throw new ACIDException("Unexpected LogType(" + logRecord.getLogType() + ") during abort.");
case LogType.ABORT:
case LogType.FLUSH:
+ case LogType.WAIT:
//ignore
break;
default:
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionManagementConstants.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionManagementConstants.java
index 5df230b..2e72033 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionManagementConstants.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionManagementConstants.java
@@ -24,11 +24,6 @@
*/
public class TransactionManagementConstants {
- public static class ResourceMgrIds {
- public static final byte BTREE_RESOURCE_MGR_ID = 1;
- public static final byte METADATA_RESOURCE_MGR_ID = 2;
- }
-
public static class LogManagerConstants {
public static final int TERMINAL_LSN = -1;
}
@@ -36,30 +31,28 @@
public static class LockManagerConstants {
public static class LockMode {
public static final byte ANY = -1;
- public static final byte NL = 0;
- public static final byte IS = 1;
- public static final byte IX = 2;
- public static final byte S = 3;
- public static final byte X = 4;
-
- public static byte intentionMode(byte mode) {
- switch (mode) {
- case S: return IS;
- case X: return IX;
- default: throw new IllegalArgumentException(
- "no intention lock mode for " + toString(mode));
- }
- }
+ public static final byte NL = 0;
+ public static final byte IS = 1;
+ public static final byte IX = 2;
+ public static final byte S = 3;
+ public static final byte X = 4;
public static String toString(byte mode) {
switch (mode) {
- case ANY: return "ANY";
- case NL: return "NL";
- case IS: return "IS";
- case IX: return "IX";
- case S: return "S";
- case X: return "X";
- default: throw new IllegalArgumentException("no such lock mode");
+ case ANY:
+ return "ANY";
+ case NL:
+ return "NL";
+ case IS:
+ return "IS";
+ case IX:
+ return "IX";
+ case S:
+ return "S";
+ case X:
+ return "X";
+ default:
+ throw new IllegalArgumentException("no such lock mode");
}
}
}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionManager.java
index 2112097..f035029 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionManager.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionManager.java
@@ -117,7 +117,7 @@
}
throw ae;
} finally {
- txnSubsystem.getLockManager().releaseLocks(txnCtx); // release
+ txnSubsystem.getLockManager().releaseLocks(txnCtx);
transactionContextRepository.remove(txnCtx.getJobId());
txnCtx.setTxnState(ITransactionManager.COMMITTED);
}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionSubsystem.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionSubsystem.java
index 6650ac6..f7ed355 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionSubsystem.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionSubsystem.java
@@ -18,6 +18,10 @@
*/
package org.apache.asterix.transaction.management.service.transaction;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Future;
+import java.util.logging.Logger;
+
import org.apache.asterix.common.config.AsterixReplicationProperties;
import org.apache.asterix.common.config.AsterixTransactionProperties;
import org.apache.asterix.common.config.IAsterixPropertiesProvider;
@@ -48,6 +52,12 @@
private final CheckpointThread checkpointThread;
private final AsterixTransactionProperties txnProperties;
+ //for profiling purpose
+ public static final boolean IS_PROFILE_MODE = false;//true
+ public long profilerEntityCommitLogCount = 0;
+ private EntityCommitProfiler ecp;
+ private Future<Object> fecp;
+
public TransactionSubsystem(String id, IAsterixAppRuntimeContextProvider asterixAppRuntimeContextProvider,
AsterixTransactionProperties txnProperties) throws ACIDException {
this.asterixAppRuntimeContextProvider = asterixAppRuntimeContextProvider;
@@ -72,12 +82,17 @@
if (asterixAppRuntimeContextProvider != null) {
this.checkpointThread = new CheckpointThread(recoveryManager,
- asterixAppRuntimeContextProvider.getDatasetLifecycleManager(),logManager,
+ asterixAppRuntimeContextProvider.getDatasetLifecycleManager(), logManager,
this.txnProperties.getCheckpointLSNThreshold(), this.txnProperties.getCheckpointPollFrequency());
this.checkpointThread.start();
} else {
this.checkpointThread = null;
}
+
+ if (IS_PROFILE_MODE) {
+ ecp = new EntityCommitProfiler(this, this.txnProperties.getCommitProfilerReportInterval());
+ fecp = (Future<Object>) getAsterixAppRuntimeContextProvider().getThreadExecutor().submit(ecp);
+ }
}
public ILogManager getLogManager() {
@@ -108,4 +123,63 @@
return id;
}
+ public void incrementEntityCommitCount() {
+ ++profilerEntityCommitLogCount;
+ }
+
+ /**
+ * Thread for profiling entity level commit count
+ * This thread takes a report interval (in seconds) parameter and
+ * reports entity level commit count every report interval (in seconds)
+ * only if IS_PROFILE_MODE is set to true.
+ * However, the thread doesn't start reporting the count until the entityCommitCount > 0.
+ */
+ static class EntityCommitProfiler implements Callable<Boolean> {
+ private static final Logger LOGGER = Logger.getLogger(EntityCommitProfiler.class.getName());
+ private final long reportIntervalInMillisec;
+ private long lastEntityCommitCount;
+ private int reportIntervalInSeconds;
+ private TransactionSubsystem txnSubsystem;
+ private boolean firstReport = true;
+ private long startTimeStamp = 0;
+ private long reportRound = 1;
+
+ public EntityCommitProfiler(TransactionSubsystem txnSubsystem, int reportIntervalInSeconds) {
+ Thread.currentThread().setName("EntityCommitProfiler-Thread");
+ this.txnSubsystem = txnSubsystem;
+ this.reportIntervalInSeconds = reportIntervalInSeconds;
+ this.reportIntervalInMillisec = reportIntervalInSeconds * 1000;
+ lastEntityCommitCount = txnSubsystem.profilerEntityCommitLogCount;
+ }
+
+ @Override
+ public Boolean call() throws Exception {
+ while (true) {
+ Thread.sleep(reportIntervalInMillisec);
+ if (txnSubsystem.profilerEntityCommitLogCount > 0) {
+ if (firstReport) {
+ startTimeStamp = System.currentTimeMillis();
+ firstReport = false;
+ }
+ //output the count
+ outputCount();
+ }
+ }
+ }
+
+ private void outputCount() {
+ long currentTimeStamp = System.currentTimeMillis();
+ long currentEntityCommitCount = txnSubsystem.profilerEntityCommitLogCount;
+
+ LOGGER.severe("EntityCommitProfiler ReportRound[" + reportRound + "], AbsoluteTimeStamp[" + currentTimeStamp
+ + "], ActualRelativeTimeStamp[" + (currentTimeStamp - startTimeStamp)
+ + "], ExpectedRelativeTimeStamp[" + (reportIntervalInSeconds * reportRound) + "], IIPS["
+ + ((currentEntityCommitCount - lastEntityCommitCount) / reportIntervalInSeconds) + "], IPS["
+ + (currentEntityCommitCount / (reportRound * reportIntervalInSeconds)) + "]");
+
+ lastEntityCommitCount = currentEntityCommitCount;
+ ++reportRound;
+ }
+ }
+
}
diff --git a/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/LockManagerUnitTest.java b/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/LockManagerUnitTest.java
index 5cabd04..14e4020 100644
--- a/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/LockManagerUnitTest.java
+++ b/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/LockManagerUnitTest.java
@@ -19,15 +19,8 @@
package org.apache.asterix.transaction.management.service.locking;
-
-import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.transactions.DatasetId;
-import org.apache.asterix.common.transactions.ILockManager;
-import org.apache.asterix.common.transactions.ITransactionContext;
-import org.apache.asterix.common.transactions.JobId;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
import java.io.PrintStream;
import java.util.ArrayList;
@@ -40,10 +33,15 @@
import java.util.logging.ConsoleHandler;
import java.util.logging.Logger;
-import static org.apache.asterix.transaction.management.service.locking.Request.Kind;
-import static org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
+import org.apache.asterix.common.transactions.DatasetId;
+import org.apache.asterix.common.transactions.ILockManager;
+import org.apache.asterix.common.transactions.ITransactionContext;
+import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.transaction.management.service.locking.Request.Kind;
+import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
public class LockManagerUnitTest {
@@ -59,7 +57,7 @@
Logger.getLogger(ConcurrentLockManager.class.getName()).addHandler(new ConsoleHandler());
}
- Map<Integer, ITransactionContext> jobMap;
+ Map<Integer, ITransactionContext> jobId2TxnCtxMap;
ILockManager lockMgr;
// set to e.g. System.err to get some output
@@ -72,14 +70,14 @@
@Before
public void setUp() throws Exception {
- jobMap = new HashMap<>();
+ jobId2TxnCtxMap = new HashMap<>();
lockMgr = new ConcurrentLockManager(LOCK_MGR_SHRINK_TIMER, LOCK_MGR_ARENAS, LOCK_MGR_TABLE_SIZE);
}
@After
public void tearDown() throws Exception {
lockMgr = null;
- jobMap = null;
+ jobId2TxnCtxMap = null;
}
@Test
@@ -87,7 +85,6 @@
List<Request> reqs = new ArrayList<>();
reqs.add(req(Kind.LOCK, j(1), d(1), e(1), LockMode.S));
reqs.add(req(Kind.UNLOCK, j(1), d(1), e(1), LockMode.S));
- reqs.add(req(Kind.UNLOCK, j(1), d(1), e(-1), LockMode.IS));
reportErrors(execute(reqs));
}
@@ -139,55 +136,16 @@
}
@Test
- public void testDeadlock() throws Exception {
- List<Request> reqs = new ArrayList<>();
- reqs.add(req(Kind.LOCK, j(1), d(1), e(1), LockMode.X));
- reqs.add(req(Kind.LOCK, j(2), d(1), e(2), LockMode.X));
- reqs.add(req(Kind.LOCK, j(2), d(1), e(1), LockMode.X));
- reqs.add(req(Kind.LOCK, j(1), d(1), e(2), LockMode.X));
- reqs.add(req(Kind.RELEASE, j(1)));
- reqs.add(req(Kind.RELEASE, j(2)));
- expectError(execute(reqs), j(1), ACIDException.class);
- }
-
- @Test
+ /**
+ * lock conversion/upgrade is not supported when deadlock-free locking
+ * protocol is enabled.
+ */
public void testUpgrade() throws Exception {
List<Request> reqs = new ArrayList<>();
reqs.add(req(Kind.LOCK, j(1), d(1), e(1), LockMode.S));
reqs.add(req(Kind.LOCK, j(1), d(1), e(1), LockMode.X));
reqs.add(req(Kind.RELEASE, j(1)));
- reportErrors(execute(reqs));
- }
-
- @Test
- public void testUpgradeDeadlock() throws Exception {
- List<Request> reqs = new ArrayList<>();
- reqs.add(req(Kind.LOCK, j(1), d(1), e(1), LockMode.S));
- reqs.add(req(Kind.LOCK, j(2), d(1), e(1), LockMode.S));
- reqs.add(req(Kind.LOCK, j(1), d(1), e(1), LockMode.X));
- reqs.add(req(Kind.PRINT));
- reqs.add(req(Kind.LOCK, j(2), d(1), e(1), LockMode.X));
- reqs.add(req(Kind.RELEASE, j(1)));
- reqs.add(req(Kind.RELEASE, j(2)));
- expectError(execute(reqs), j(2), ACIDException.class);
- }
-
- @Test
- /**
- * Runs into a time-out and j(1) gets interrupted by
- * the test. This scenario happens only in this test as there
- * is additional synchronization between the locking threads
- * through the coordinator.
- */
- public void testTimeout() throws Exception {
- List<Request> reqs = new ArrayList<>();
- reqs.add(req(Kind.LOCK, j(1), d(1), e(1), LockMode.S));
- reqs.add(req(Kind.LOCK, j(2), d(1), e(1), LockMode.S));
- reqs.add(req(Kind.LOCK, j(1), d(1), e(1), LockMode.X));
- reqs.add(req(Kind.RELEASE, j(1)));
- reqs.add(req(Kind.RELEASE, j(2)));
- // this runs into a time-out and j(1) gets interrupted
- expectError(execute(reqs), j(1), WaitInterruptedException.class);
+ expectError(execute(reqs), j(1), IllegalStateException.class);
}
//--------------------------------------------------------------------
@@ -202,9 +160,10 @@
* LockManager in list order, however they are fulfilled in the order
* decided by the LockManager
*
- * @param reqs a list of requests that will be execute in order
+ * @param reqs
+ * a list of requests that will be execute in order
* @return a map of (JodId, exception) pairs that can either be handled
- * by the test or thrown using #reportErrors
+ * by the test or thrown using #reportErrors
*/
private Map<String, Throwable> execute(List<Request> reqs) throws InterruptedException {
if (err != null) {
@@ -258,7 +217,7 @@
private Set<Locker> createLockers(List<Request> reqs, AtomicInteger timeStamp) {
Set<Locker> lockers = new HashSet<>();
lockers.add(new Locker(lockMgr, null, reqs, timeStamp, err));
- for (ITransactionContext txnCtx : jobMap.values()) {
+ for (ITransactionContext txnCtx : jobId2TxnCtxMap.values()) {
Locker locker = new Locker(lockMgr, txnCtx, reqs, timeStamp, err);
lockers.add(locker);
}
@@ -275,8 +234,8 @@
return threads;
}
- private Map<String, Throwable> stopThreads(Set<Locker> lockers, Map<String, Thread> threads) throws
- InterruptedException {
+ private Map<String, Throwable> stopThreads(Set<Locker> lockers, Map<String, Thread> threads)
+ throws InterruptedException {
Map<String, Throwable> result = new HashMap<>();
for (Locker locker : lockers) {
stopThread(threads.get(locker.name));
@@ -317,7 +276,8 @@
* throws the first Throwable found in the map.
* This is the default way to handle the errors returned by #execute
*
- * @param errors a map of (JodId, exception) pairs
+ * @param errors
+ * a map of (JodId, exception) pairs
*/
void reportErrors(Map<String, Throwable> errors) {
for (String name : errors.keySet()) {
@@ -333,8 +293,10 @@
/**
* gets the error for a specific job from the errors map
*
- * @param errors a map of (JodId, throwable) pairs
- * @param txnCtx the transaction context of the job whose error is requested
+ * @param errors
+ * a map of (JodId, throwable) pairs
+ * @param txnCtx
+ * the transaction context of the job whose error is requested
* @return throwable for said error
*/
private static Throwable getError(Map<String, Throwable> errors, ITransactionContext txnCtx) {
@@ -344,16 +306,19 @@
/**
* asserts that the error for a specific job from the errors map is of a specific class
*
- * @param errors a map of (JodId, throwable) pairs
- * @param txnCtx the transaction context of the job whose error is requested
- * @param clazz the exception class
+ * @param errors
+ * a map of (JodId, throwable) pairs
+ * @param txnCtx
+ * the transaction context of the job whose error is requested
+ * @param clazz
+ * the exception class
*/
private void expectError(Map<String, Throwable> errors, ITransactionContext txnCtx,
- Class<? extends Throwable> clazz) throws Exception {
+ Class<? extends Throwable> clazz) throws Exception {
Throwable error = getError(errors, txnCtx);
if (error == null) {
- throw new AssertionError("expected " + clazz.getSimpleName() + " for " + txnCtx.getJobId() + ", got no " +
- "exception");
+ throw new AssertionError(
+ "expected " + clazz.getSimpleName() + " for " + txnCtx.getJobId() + ", got no " + "exception");
}
if (!clazz.isInstance(error)) {
throw new AssertionError(error);
@@ -365,8 +330,8 @@
// Convenience methods to make test description more compact
//--------------------------------------------------------------------
- private Request req(final Kind kind, final ITransactionContext txnCtx,
- final DatasetId dsId, final int hashValue, final byte lockMode) {
+ private Request req(final Kind kind, final ITransactionContext txnCtx, final DatasetId dsId, final int hashValue,
+ final byte lockMode) {
return Request.create(kind, txnCtx, dsId, hashValue, lockMode);
}
@@ -387,12 +352,11 @@
}
private ITransactionContext j(int jId) {
- if (!jobMap.containsKey(jId)) {
+ if (!jobId2TxnCtxMap.containsKey(jId)) {
ITransactionContext mockTxnContext = mock(ITransactionContext.class);
when(mockTxnContext.getJobId()).thenReturn(new JobId(jId));
- jobMap.put(jId, mockTxnContext);
+ jobId2TxnCtxMap.put(jId, mockTxnContext);
}
- return jobMap.get(jId);
+ return jobId2TxnCtxMap.get(jId);
}
-
}
diff --git a/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/Locker.java b/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/Locker.java
index 164fc07..97b4f8a 100644
--- a/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/Locker.java
+++ b/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/Locker.java
@@ -18,11 +18,6 @@
*/
package org.apache.asterix.transaction.management.service.locking;
-import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.transactions.ILockManager;
-import org.apache.asterix.common.transactions.ITransactionContext;
-import org.junit.Assert;
-
import java.io.PrintStream;
import java.util.ArrayList;
import java.util.Collections;
@@ -31,6 +26,11 @@
import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.transactions.ILockManager;
+import org.apache.asterix.common.transactions.ITransactionContext;
+import org.junit.Assert;
+
/**
* Executes a sequence of lock requests against an ILockManager.
* Lockers are run by different threads in the LockManagerUnitTest.
@@ -55,18 +55,22 @@
private PrintStream err;
/**
- * @param lockMgr the ILockManager to send requests to
- * @param txnCtx the ITransactionContext that identifies the transaction that this Locker represents
- * @param allRequests an ordered list of lock requests for multiple transactions, this Locker will only execute
- * requests for the transaction identified by txnCtx
- * @param time a global timestamp that is used to synchronize different lockers to ensure that requests are started
- * in the order given in allRequests
- * @param err a stream to write log/error information to
- *
+ * @param lockMgr
+ * the ILockManager to send requests to
+ * @param txnCtx
+ * the ITransactionContext that identifies the transaction that this Locker represents
+ * @param allRequests
+ * an ordered list of lock requests for multiple transactions, this Locker will only execute
+ * requests for the transaction identified by txnCtx
+ * @param time
+ * a global timestamp that is used to synchronize different lockers to ensure that requests are started
+ * in the order given in allRequests
+ * @param err
+ * a stream to write log/error information to
* @see Request
*/
Locker(ILockManager lockMgr, ITransactionContext txnCtx, List<Request> allRequests, AtomicInteger time,
- PrintStream err) {
+ PrintStream err) {
this.name = txnCtx == null ? "admin" : txnCtx.getJobId().toString();
this.lockMgr = lockMgr;
@@ -110,7 +114,7 @@
public void run() {
log("running");
try {
- while (! hasErrors() && reqIter.hasNext()) {
+ while (!hasErrors() && reqIter.hasNext()) {
curReq = reqIter.next();
int localTime = globalTime.get();
while (localTime < curReq.time) {
@@ -123,13 +127,14 @@
log("will exec at t=" + localTime + " " + curReq);
try {
reqStart = currentTime();
- Assert.assertEquals(localTime, globalTime.getAndIncrement());
- log("incremented");
+ Assert.assertEquals(localTime, globalTime.get());
curReq.setResult(curReq.request.execute(lockMgr) ? Requester.SUCCESS : Requester.FAIL);
} catch (ACIDException e) {
curReq.setResult(Requester.ERROR);
addError(e);
} finally {
+ globalTime.getAndIncrement();
+ log("incremented");
reqStart = -1;
}
log("time " + localTime);
diff --git a/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/Request.java b/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/Request.java
index c7e0c42..70a803b 100644
--- a/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/Request.java
+++ b/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/Request.java
@@ -18,14 +18,14 @@
*/
package org.apache.asterix.transaction.management.service.locking;
+import java.io.PrintStream;
+
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.transactions.DatasetId;
import org.apache.asterix.common.transactions.ILockManager;
import org.apache.asterix.common.transactions.ITransactionContext;
import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants;
-import java.io.PrintStream;
-
/**
* repesents a lock request for testing.
*/
@@ -56,16 +56,17 @@
this.txnCtx = txnCtx;
}
- String asString(final Kind kind, final ITransactionContext txnCtx,
- final DatasetId dsId, final int hashValue, final byte lockMode) {
- return txnCtx.getJobId().toString() + ":" + kind.name() + ":" + dsId.getId() + ":" + hashValue + ":"
+ String asString(final Kind kind, final ITransactionContext txnCtx, final DatasetId dsId, final int hashValue,
+ final byte lockMode) {
+ return txnCtx.getJobId() + ":" + kind.name() + ":" + dsId.getId() + ":" + hashValue + ":"
+ TransactionManagementConstants.LockManagerConstants.LockMode.toString(lockMode);
}
abstract boolean execute(ILockManager lockMgr) throws ACIDException;
- static Request create(final Kind kind, final ITransactionContext txnCtx,
- final DatasetId dsId, final int hashValue, final byte lockMode) {
+ static Request create(final Kind kind, final ITransactionContext txnCtx, final DatasetId dsId, final int hashValue,
+ final byte lockMode) {
+
switch (kind) {
case INSTANT_TRY_LOCK:
return new Request(kind, txnCtx) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java
index 6fc26a7..df3a211 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java
@@ -442,7 +442,7 @@
private ISearchOperationCallbackFactory[] mockSearchOpCallbackFactories() throws HyracksDataException {
ISearchOperationCallback searchOpCallback = mockSearchOpCallback();
ISearchOperationCallbackFactory searchOpCallbackFactory = Mockito.mock(ISearchOperationCallbackFactory.class);
- Mockito.when(searchOpCallbackFactory.createSearchOperationCallback(Mockito.anyLong(), Mockito.any()))
+ Mockito.when(searchOpCallbackFactory.createSearchOperationCallback(Mockito.anyLong(), Mockito.any(), null))
.thenReturn(searchOpCallback);
return new ISearchOperationCallbackFactory[] { searchOpCallbackFactory };
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IModificationOperationCallbackFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IModificationOperationCallbackFactory.java
index 8d618c8..05114f0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IModificationOperationCallbackFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IModificationOperationCallbackFactory.java
@@ -21,9 +21,11 @@
import java.io.Serializable;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
import org.apache.hyracks.api.exceptions.HyracksDataException;
public interface IModificationOperationCallbackFactory extends Serializable {
public IModificationOperationCallback createModificationOperationCallback(String resourcePath, long resourceId,
- int resourcePartition, Object resource, IHyracksTaskContext ctx) throws HyracksDataException;
+ int resourcePartition, Object resource, IHyracksTaskContext ctx, IOperatorNodePushable operatorNodePushable)
+ throws HyracksDataException;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ISearchOperationCallbackFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ISearchOperationCallbackFactory.java
index 1a629e9..c0d5128 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ISearchOperationCallbackFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ISearchOperationCallbackFactory.java
@@ -21,9 +21,10 @@
import java.io.Serializable;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
import org.apache.hyracks.api.exceptions.HyracksDataException;
public interface ISearchOperationCallbackFactory extends Serializable {
- public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx)
+ public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx, IOperatorNodePushable operatorNodePushable)
throws HyracksDataException;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexInsertUpdateDeleteOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexInsertUpdateDeleteOperatorNodePushable.java
index 9438632..33bad73 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexInsertUpdateDeleteOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexInsertUpdateDeleteOperatorNodePushable.java
@@ -78,7 +78,7 @@
writer.open();
modCallback = opDesc.getModificationOpCallbackFactory().createModificationOperationCallback(
indexHelper.getResourcePath(), indexHelper.getResourceID(), indexHelper.getResourcePartition(),
- index, ctx);
+ index, ctx, this);
indexAccessor = index.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
ITupleFilterFactory tupleFilterFactory = opDesc.getTupleFilterFactory();
if (tupleFilterFactory != null) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
index c4f3fca..650dcfa 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
@@ -133,7 +133,7 @@
dos = tb.getDataOutput();
appender = new FrameTupleAppender(new VSizeFrame(ctx), true);
ISearchOperationCallback searchCallback = opDesc.getSearchOpCallbackFactory()
- .createSearchOperationCallback(indexHelper.getResourceID(), ctx);
+ .createSearchOperationCallback(indexHelper.getResourceID(), ctx, null);
indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE, searchCallback);
cursor = createCursor();
if (retainInput) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
index 170950e..3ac6065 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
@@ -57,7 +57,7 @@
ITreeIndexFrame cursorFrame = treeIndex.getLeafFrameFactory().createFrame();
ITreeIndexCursor cursor = treeIndexHelper.createDiskOrderScanCursor(cursorFrame);
ISearchOperationCallback searchCallback = opDesc.getSearchOpCallbackFactory()
- .createSearchOperationCallback(treeIndexHelper.getResourceID(), ctx);
+ .createSearchOperationCallback(treeIndexHelper.getResourceID(), ctx, null);
ITreeIndexAccessor indexAccessor = (ITreeIndexAccessor) treeIndex
.createAccessor(NoOpOperationCallback.INSTANCE, searchCallback);
try {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/NoOpOperationCallbackFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/NoOpOperationCallbackFactory.java
index b42e619..c2cca41 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/NoOpOperationCallbackFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/NoOpOperationCallbackFactory.java
@@ -19,6 +19,7 @@
package org.apache.hyracks.storage.am.common.impls;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.common.api.IModificationOperationCallback;
import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
@@ -30,17 +31,18 @@
* Implemented as an enum to preserve singleton model while being serializable
*/
public enum NoOpOperationCallbackFactory
- implements ISearchOperationCallbackFactory,IModificationOperationCallbackFactory {
- INSTANCE;
+ implements ISearchOperationCallbackFactory, IModificationOperationCallbackFactory {
+ INSTANCE;
@Override
- public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx) {
+ public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx, IOperatorNodePushable operatorNodePushable) {
return NoOpOperationCallback.INSTANCE;
}
@Override
public IModificationOperationCallback createModificationOperationCallback(String resourcePath, long resourceId,
- int resourcePartition, Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
+ int resourcePartition, Object resource, IHyracksTaskContext ctx, IOperatorNodePushable operatorNodePushable)
+ throws HyracksDataException {
return NoOpOperationCallback.INSTANCE;
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexFrameWriter.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexFrameWriter.java
new file mode 100644
index 0000000..47c1854
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexFrameWriter.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 at
+ *
+ * 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 org.apache.hyracks.storage.am.lsm.common.api;
+
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public interface ILSMIndexFrameWriter extends IFrameWriter {
+ /**
+ * Push tuples in the frame from startTupleIndex(inclusive) to endTupleIndex(exclusive)
+ * forward to the next operator/consumer.
+ *
+ * @param startTupleIndex
+ * the first tuple's index to be pushed
+ * @param endTupleIndex
+ * the last tuple's index to be pushed
+ * @throws HyracksDataException
+ */
+ public default void flushPartialFrame(int startTupleIndex, int endTupleIndex) throws HyracksDataException {
+ throw new HyracksDataException("flushPartialFrame() is not supported in this ILSMIndexFrameWriter");
+ }
+
+ /**
+ * Push tuples in the frame forward to the next operator/consumer.
+ * The flushed tuples don't have to be all tuples in the frame.
+ *
+ * @throws HyracksDataException
+ */
+ public default void flushPartialFrame() throws HyracksDataException {
+ throw new HyracksDataException("flushPartialFrame() is not supported in this ILSMIndexFrameWriter");
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java
index 25f2382..de1719c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java
@@ -29,8 +29,10 @@
import org.apache.hyracks.storage.am.common.dataflow.IndexInsertUpdateDeleteOperatorNodePushable;
import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexFrameWriter;
-public class LSMIndexInsertUpdateDeleteOperatorNodePushable extends IndexInsertUpdateDeleteOperatorNodePushable {
+public class LSMIndexInsertUpdateDeleteOperatorNodePushable extends IndexInsertUpdateDeleteOperatorNodePushable
+ implements ILSMIndexFrameWriter {
protected FrameTupleAppender appender;
@@ -116,7 +118,8 @@
}
}
- private void flushPartialFrame(int startTupleIndex, int endTupleIndex) throws HyracksDataException {
+ @Override
+ public void flushPartialFrame(int startTupleIndex, int endTupleIndex) throws HyracksDataException {
for (int i = startTupleIndex; i < endTupleIndex; i++) {
FrameUtils.appendToWriter(writer, appender, accessor, i);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java
index aad5bf47..7340fdb 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java
@@ -131,7 +131,6 @@
* This method returns whether there is an ongoing merge operation or not by checking
* each component state of given components.
*
- * @param immutableComponents
* @return true if there is an ongoing merge operation, false otherwise.
*/
private boolean isMergeOngoing(List<ILSMComponent> immutableComponents) {