[NO ISSUE][TX] Rename Asterix JobId to TxnId

- user model changes: no
- storage format changes: yes
  - Txn log jobId changed from int to long.
- interface changes: yes
  Update APIs to use long TxnId instead of int JobId

Details:
- Rename TxnId -> TxnEntityId.
- Rename Asterix JobId -> TxnId.
- Rename Asterix JobIdFactory -> TxnIdFactory.
- Change TxnId size from int to long and update
  log sizes accordingly.

Change-Id: I0905595a50195b83c1afae5dde88e5502ad21b9f
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2149
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Till Westmann <tillw@apache.org>
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitPOperator.java
index 2dd4c3d..abd18aa 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitPOperator.java
@@ -21,7 +21,7 @@
 
 import java.util.List;
 
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -39,17 +39,16 @@
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.io.FileSplit;
 
 public class CommitPOperator extends AbstractPhysicalOperator {
 
     private final List<LogicalVariable> primaryKeyLogicalVars;
-    private final JobId jobId;
+    private final TxnId txnId;
     private final Dataset dataset;
     private final boolean isSink;
 
-    public CommitPOperator(JobId jobId, Dataset dataset, List<LogicalVariable> primaryKeyLogicalVars, boolean isSink) {
-        this.jobId = jobId;
+    public CommitPOperator(TxnId txnId, Dataset dataset, List<LogicalVariable> primaryKeyLogicalVars, boolean isSink) {
+        this.txnId = txnId;
         this.dataset = dataset;
         this.primaryKeyLogicalVars = primaryKeyLogicalVars;
         this.isSink = isSink;
@@ -88,7 +87,7 @@
         int[] primaryKeyFields = JobGenHelper.variablesToFieldIndexes(primaryKeyLogicalVars, inputSchemas[0]);
 
         //get dataset splits
-        IPushRuntimeFactory runtime = dataset.getCommitRuntimeFactory(metadataProvider, jobId, primaryKeyFields,
+        IPushRuntimeFactory runtime = dataset.getCommitRuntimeFactory(metadataProvider, txnId, primaryKeyFields,
                 isSink);
         builder.contributeMicroOperator(op, runtime, recDesc);
         ILogicalOperator src = op.getInputs().get(0).getValue();
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
index 12114f0..c941320 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
@@ -18,7 +18,6 @@
  */
 package org.apache.asterix.algebra.operators.physical;
 
-import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.declared.DataSourceId;
 import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -159,7 +158,7 @@
                 jobSpec, outputRecDesc, queryField, dataflowHelperFactory, queryTokenizerFactory, searchModifierFactory,
                 retainInput, retainMissing, context.getMissingWriterFactory(),
                 dataset.getSearchCallbackFactory(metadataProvider.getStorageComponentProvider(), secondaryIndex,
-                        ((JobEventListenerFactory) jobSpec.getJobletEventListenerFactory()).getJobId(),
+                        ((JobEventListenerFactory) jobSpec.getJobletEventListenerFactory()).getTxnId(),
                         IndexOperation.SEARCH, null),
                 minFilterFieldIndexes, maxFilterFieldIndexes, isFullTextSearchQuery, numPrimaryKeys,
                 propagateIndexFilter);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetupCommitExtensionOpRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetupCommitExtensionOpRule.java
index 18e5f5e..61339bf 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetupCommitExtensionOpRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetupCommitExtensionOpRule.java
@@ -23,7 +23,7 @@
 
 import org.apache.asterix.algebra.operators.CommitOperator;
 import org.apache.asterix.algebra.operators.physical.CommitPOperator;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.metadata.declared.DatasetDataSource;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
@@ -99,14 +99,14 @@
             primaryKeyLogicalVars.add(new LogicalVariable(varRefExpr.getVariableReference().getId()));
         }
 
-        //get JobId(TransactorId)
+        //get TxnId(TransactorId)
         MetadataProvider mp = (MetadataProvider) context.getMetadataProvider();
-        JobId jobId = mp.getJobId();
+        TxnId txnId = mp.getTxnId();
 
         //create the logical and physical operator
         CommitOperator commitOperator = new CommitOperator(primaryKeyLogicalVars, isSink);
         CommitPOperator commitPOperator =
-                new CommitPOperator(jobId, dataset, primaryKeyLogicalVars, isSink);
+                new CommitPOperator(txnId, dataset, primaryKeyLogicalVars, isSink);
         commitOperator.setPhysicalOperator(commitPOperator);
 
         //create ExtensionOperator and put the commitOperator in it.
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
index 2078288..f7ecefc 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -38,6 +38,7 @@
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.common.utils.Job;
 import org.apache.asterix.common.utils.Job.SubmissionMode;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
@@ -63,7 +64,7 @@
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.optimizer.base.FuzzyUtils;
 import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
-import org.apache.asterix.transaction.management.service.transaction.JobIdFactory;
+import org.apache.asterix.transaction.management.service.transaction.TxnIdFactory;
 import org.apache.asterix.translator.CompiledStatements.ICompiledDmlStatement;
 import org.apache.asterix.translator.IStatementExecutor.Stats;
 import org.apache.asterix.translator.SessionConfig;
@@ -211,8 +212,8 @@
             printPlanPostfix(output);
         }
 
-        org.apache.asterix.common.transactions.JobId asterixJobId = JobIdFactory.generateJobId();
-        metadataProvider.setJobId(asterixJobId);
+        TxnId txnId = TxnIdFactory.create();
+        metadataProvider.setTxnId(txnId);
         ILangExpressionToPlanTranslator t =
                 translatorFactory.createExpressionToPlanTranslator(metadataProvider, varCounter);
 
@@ -351,7 +352,7 @@
         builder.setNormalizedKeyComputerFactoryProvider(format.getNormalizedKeyComputerFactoryProvider());
 
         JobEventListenerFactory jobEventListenerFactory =
-                new JobEventListenerFactory(asterixJobId, metadataProvider.isWriteTransaction());
+                new JobEventListenerFactory(txnId, metadataProvider.isWriteTransaction());
         JobSpecification spec = compiler.createJob(metadataProvider.getApplicationContext(), jobEventListenerFactory);
 
         // When the top-level statement is a query, the statement parameter is null.
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
index 25385c6..2435b60 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
@@ -61,7 +61,7 @@
 import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
 import org.apache.asterix.transaction.management.service.logging.LogManager;
 import org.apache.asterix.transaction.management.service.recovery.AbstractCheckpointManager;
-import org.apache.asterix.transaction.management.service.recovery.TxnId;
+import org.apache.asterix.transaction.management.service.recovery.TxnEntityId;
 import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants;
 import org.apache.commons.io.FileUtils;
 import org.apache.hyracks.api.application.INCServiceContext;
@@ -87,7 +87,7 @@
     private final LogManager logMgr;
     private final boolean replicationEnabled;
     private static final String RECOVERY_FILES_DIR_NAME = "recovery_temp";
-    private Map<Integer, JobEntityCommits> jobId2WinnerEntitiesMap = null;
+    private Map<Long, JobEntityCommits> jobId2WinnerEntitiesMap = null;
     private final long cachedEntityCommitsPerJobSize;
     private final PersistentLocalResourceRepository localResourceRepository;
     private final ICheckpointManager checkpointManager;
@@ -183,7 +183,7 @@
     public synchronized void replayPartitionsLogs(Set<Integer> partitions, ILogReader logReader, long lowWaterMarkLSN)
             throws IOException, ACIDException {
         try {
-            Set<Integer> winnerJobSet = startRecoverysAnalysisPhase(partitions, logReader, lowWaterMarkLSN);
+            Set<Long> winnerJobSet = startRecoverysAnalysisPhase(partitions, logReader, lowWaterMarkLSN);
             startRecoveryRedoPhase(partitions, logReader, lowWaterMarkLSN, winnerJobSet);
         } finally {
             logReader.close();
@@ -191,13 +191,13 @@
         }
     }
 
-    private synchronized Set<Integer> startRecoverysAnalysisPhase(Set<Integer> partitions, ILogReader logReader,
+    private synchronized Set<Long> startRecoverysAnalysisPhase(Set<Integer> partitions, ILogReader logReader,
             long lowWaterMarkLSN) throws IOException, ACIDException {
         int updateLogCount = 0;
         int entityCommitLogCount = 0;
         int jobCommitLogCount = 0;
         int abortLogCount = 0;
-        Set<Integer> winnerJobSet = new HashSet<>();
+        Set<Long> winnerJobSet = new HashSet<>();
         jobId2WinnerEntitiesMap = new HashMap<>();
         //set log reader to the lowWaterMarkLsn
         ILogRecord logRecord;
@@ -214,8 +214,8 @@
                     }
                     break;
                 case LogType.JOB_COMMIT:
-                    winnerJobSet.add(logRecord.getJobId());
-                    cleanupJobCommits(logRecord.getJobId());
+                    winnerJobSet.add(logRecord.getTxnId());
+                    cleanupTxnCommits(logRecord.getTxnId());
                     jobCommitLogCount++;
                     break;
                 case LogType.ENTITY_COMMIT:
@@ -249,38 +249,38 @@
         return winnerJobSet;
     }
 
-    private void cleanupJobCommits(int jobId) {
-        if (jobId2WinnerEntitiesMap.containsKey(jobId)) {
-            JobEntityCommits jobEntityWinners = jobId2WinnerEntitiesMap.get(jobId);
+    private void cleanupTxnCommits(long txnId) {
+        if (jobId2WinnerEntitiesMap.containsKey(txnId)) {
+            JobEntityCommits jobEntityWinners = jobId2WinnerEntitiesMap.get(txnId);
             //to delete any spilled files as well
             jobEntityWinners.clear();
-            jobId2WinnerEntitiesMap.remove(jobId);
+            jobId2WinnerEntitiesMap.remove(txnId);
         }
     }
 
     private void analyzeEntityCommitLog(ILogRecord logRecord) throws IOException {
-        int jobId = logRecord.getJobId();
+        long txnId = logRecord.getTxnId();
         JobEntityCommits jobEntityWinners;
-        if (!jobId2WinnerEntitiesMap.containsKey(jobId)) {
-            jobEntityWinners = new JobEntityCommits(jobId);
+        if (!jobId2WinnerEntitiesMap.containsKey(txnId)) {
+            jobEntityWinners = new JobEntityCommits(txnId);
             if (needToFreeMemory()) {
                 // If we don't have enough memory for one more job,
                 // we will force all jobs to spill their cached entities to disk.
                 // This could happen only when we have many jobs with small
                 // number of records and none of them have job commit.
-                freeJobsCachedEntities(jobId);
+                freeJobsCachedEntities(txnId);
             }
-            jobId2WinnerEntitiesMap.put(jobId, jobEntityWinners);
+            jobId2WinnerEntitiesMap.put(txnId, jobEntityWinners);
         } else {
-            jobEntityWinners = jobId2WinnerEntitiesMap.get(jobId);
+            jobEntityWinners = jobId2WinnerEntitiesMap.get(txnId);
         }
         jobEntityWinners.add(logRecord);
     }
 
     private synchronized void startRecoveryRedoPhase(Set<Integer> partitions, ILogReader logReader,
-            long lowWaterMarkLSN, Set<Integer> winnerJobSet) throws IOException, ACIDException {
+            long lowWaterMarkLSN, Set<Long> winnerTxnSet) throws IOException, ACIDException {
         int redoCount = 0;
-        int jobId = -1;
+        long jobId;
 
         long resourceId;
         long maxDiskLastLsn;
@@ -296,7 +296,7 @@
 
         Map<Long, LocalResource> resourcesMap = localResourceRepository.loadAndGetAllResources();
         Map<Long, Long> resourceId2MaxLSNMap = new HashMap<>();
-        TxnId tempKeyTxnId = new TxnId(-1, -1, -1, null, -1, false);
+        TxnEntityId tempKeyTxnEntityId = new TxnEntityId(-1, -1, -1, null, -1, false);
 
         ILogRecord logRecord = null;
         try {
@@ -307,18 +307,18 @@
                     LOGGER.info(logRecord.getLogRecordForDisplay());
                 }
                 lsn = logRecord.getLSN();
-                jobId = logRecord.getJobId();
+                jobId = logRecord.getTxnId();
                 foundWinner = false;
                 switch (logRecord.getLogType()) {
                     case LogType.UPDATE:
                         if (partitions.contains(logRecord.getResourcePartition())) {
-                            if (winnerJobSet.contains(jobId)) {
+                            if (winnerTxnSet.contains(jobId)) {
                                 foundWinner = true;
                             } else if (jobId2WinnerEntitiesMap.containsKey(jobId)) {
                                 jobEntityWinners = jobId2WinnerEntitiesMap.get(jobId);
-                                tempKeyTxnId.setTxnId(jobId, logRecord.getDatasetId(), logRecord.getPKHashValue(),
+                                tempKeyTxnEntityId.setTxnId(jobId, logRecord.getDatasetId(), logRecord.getPKHashValue(),
                                         logRecord.getPKValue(), logRecord.getPKValueSize());
-                                if (jobEntityWinners.containsEntityCommitForTxnId(lsn, tempKeyTxnId)) {
+                                if (jobEntityWinners.containsEntityCommitForTxnId(lsn, tempKeyTxnEntityId)) {
                                     foundWinner = true;
                                 }
                             }
@@ -449,9 +449,9 @@
     }
 
     @Override
-    public File createJobRecoveryFile(int jobId, String fileName) throws IOException {
+    public File createJobRecoveryFile(long txnId, String fileName) throws IOException {
         String recoveryDirPath = getRecoveryDirPath();
-        Path jobRecoveryFolder = Paths.get(recoveryDirPath + File.separator + jobId);
+        Path jobRecoveryFolder = Paths.get(recoveryDirPath + File.separator + txnId);
         if (!Files.exists(jobRecoveryFolder)) {
             Files.createDirectories(jobRecoveryFolder);
         }
@@ -459,10 +459,10 @@
         File jobRecoveryFile = new File(jobRecoveryFolder.toString() + File.separator + fileName);
         if (!jobRecoveryFile.exists()) {
             if (!jobRecoveryFile.createNewFile()) {
-                throw new IOException("Failed to create file: " + fileName + " for job id(" + jobId + ")");
+                throw new IOException("Failed to create file: " + fileName + " for txn id(" + txnId + ")");
             }
         } else {
-            throw new IOException("File: " + fileName + " for job id(" + jobId + ") already exists");
+            throw new IOException("File: " + fileName + " for txn id(" + txnId + ") already exists");
         }
         return jobRecoveryFile;
     }
@@ -483,11 +483,11 @@
         return logDir + RECOVERY_FILES_DIR_NAME;
     }
 
-    private void freeJobsCachedEntities(int requestingJobId) throws IOException {
+    private void freeJobsCachedEntities(long requestingTxnId) throws IOException {
         if (jobId2WinnerEntitiesMap != null) {
-            for (Entry<Integer, JobEntityCommits> jobEntityCommits : jobId2WinnerEntitiesMap.entrySet()) {
+            for (Entry<Long, JobEntityCommits> jobEntityCommits : jobId2WinnerEntitiesMap.entrySet()) {
                 //if the job is not the requester, free its memory
-                if (jobEntityCommits.getKey() != requestingJobId) {
+                if (jobEntityCommits.getKey() != requestingTxnId) {
                     jobEntityCommits.getValue().spillToDiskAndfreeMemory();
                 }
             }
@@ -496,7 +496,7 @@
 
     @Override
     public void rollbackTransaction(ITransactionContext txnContext) throws ACIDException {
-        int abortedJobId = txnContext.getJobId().getId();
+        long abortedTxnId = txnContext.getTxnId().getId();
         // Obtain the first/last log record LSNs written by the Job
         long firstLSN = txnContext.getFirstLSN();
         /*
@@ -517,7 +517,7 @@
         // check if the transaction actually wrote some logs.
         if (firstLSN == TransactionManagementConstants.LogManagerConstants.TERMINAL_LSN || firstLSN > lastLSN) {
             if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("no need to roll back as there were no operations by the job " + txnContext.getJobId());
+                LOGGER.info("no need to roll back as there were no operations by the txn " + txnContext.getTxnId());
             }
             return;
         }
@@ -527,13 +527,13 @@
             LOGGER.info("collecting loser transaction's LSNs from " + firstLSN + " to " + lastLSN);
         }
 
-        Map<TxnId, List<Long>> jobLoserEntity2LSNsMap = new HashMap<>();
-        TxnId tempKeyTxnId = new TxnId(-1, -1, -1, null, -1, false);
+        Map<TxnEntityId, List<Long>> jobLoserEntity2LSNsMap = new HashMap<>();
+        TxnEntityId tempKeyTxnEntityId = new TxnEntityId(-1, -1, -1, null, -1, false);
         int updateLogCount = 0;
         int entityCommitLogCount = 0;
-        int logJobId = -1;
+        long logTxnId;
         long currentLSN = -1;
-        TxnId loserEntity = null;
+        TxnEntityId loserEntity;
         List<Long> undoLSNSet = null;
         //get active partitions on this node
         Set<Integer> activePartitions = localResourceRepository.getActivePartitions();
@@ -552,19 +552,20 @@
                         LOGGER.info(logRecord.getLogRecordForDisplay());
                     }
                 }
-                logJobId = logRecord.getJobId();
-                if (logJobId != abortedJobId) {
+                logTxnId = logRecord.getTxnId();
+                if (logTxnId != abortedTxnId) {
                     continue;
                 }
-                tempKeyTxnId.setTxnId(logJobId, logRecord.getDatasetId(), logRecord.getPKHashValue(),
+                tempKeyTxnEntityId.setTxnId(logTxnId, logRecord.getDatasetId(), logRecord.getPKHashValue(),
                         logRecord.getPKValue(), logRecord.getPKValueSize());
                 switch (logRecord.getLogType()) {
                     case LogType.UPDATE:
                         if (activePartitions.contains(logRecord.getResourcePartition())) {
-                            undoLSNSet = jobLoserEntity2LSNsMap.get(tempKeyTxnId);
+                            undoLSNSet = jobLoserEntity2LSNsMap.get(tempKeyTxnEntityId);
                             if (undoLSNSet == null) {
-                                loserEntity = new TxnId(logJobId, logRecord.getDatasetId(), logRecord.getPKHashValue(),
-                                        logRecord.getPKValue(), logRecord.getPKValueSize(), true);
+                                loserEntity =
+                                        new TxnEntityId(logTxnId, logRecord.getDatasetId(), logRecord.getPKHashValue(),
+                                                logRecord.getPKValue(), logRecord.getPKValueSize(), true);
                                 undoLSNSet = new LinkedList<>();
                                 jobLoserEntity2LSNsMap.put(loserEntity, undoLSNSet);
                             }
@@ -572,17 +573,17 @@
                             updateLogCount++;
                             if (IS_DEBUG_MODE) {
                                 LOGGER.info(Thread.currentThread().getId() + "======> update[" + currentLSN + "]:"
-                                        + tempKeyTxnId);
+                                        + tempKeyTxnEntityId);
                             }
                         }
                         break;
                     case LogType.ENTITY_COMMIT:
                         if (activePartitions.contains(logRecord.getResourcePartition())) {
-                            jobLoserEntity2LSNsMap.remove(tempKeyTxnId);
+                            jobLoserEntity2LSNsMap.remove(tempKeyTxnEntityId);
                             entityCommitLogCount++;
                             if (IS_DEBUG_MODE) {
                                 LOGGER.info(Thread.currentThread().getId() + "======> entity_commit[" + currentLSN + "]"
-                                        + tempKeyTxnId);
+                                        + tempKeyTxnEntityId);
                             }
                         }
                         break;
@@ -601,7 +602,7 @@
 
             if (currentLSN != lastLSN) {
                 throw new ACIDException("LastLSN mismatch: lastLSN(" + lastLSN + ") vs currentLSN(" + currentLSN
-                        + ") during abort( " + txnContext.getJobId() + ")");
+                        + ") during abort( " + txnContext.getTxnId() + ")");
             }
 
             //undo loserTxn's effect
@@ -610,10 +611,10 @@
             IDatasetLifecycleManager datasetLifecycleManager =
                     txnSubsystem.getAsterixAppRuntimeContextProvider().getDatasetLifecycleManager();
             //TODO sort loser entities by smallest LSN to undo in one pass.
-            Iterator<Entry<TxnId, List<Long>>> iter = jobLoserEntity2LSNsMap.entrySet().iterator();
+            Iterator<Entry<TxnEntityId, List<Long>>> iter = jobLoserEntity2LSNsMap.entrySet().iterator();
             int undoCount = 0;
             while (iter.hasNext()) {
-                Map.Entry<TxnId, List<Long>> loserEntity2LSNsMap = iter.next();
+                Map.Entry<TxnEntityId, List<Long>> loserEntity2LSNsMap = iter.next();
                 undoLSNSet = loserEntity2LSNsMap.getValue();
                 // The step below is important since the upsert operations must be done in reverse order.
                 Collections.reverse(undoLSNSet);
@@ -622,7 +623,7 @@
                     //read the corresponding log record to be undone.
                     logRecord = logReader.read(undoLSN);
                     if (logRecord == null) {
-                        throw new ACIDException("IllegalState exception during abort( " + txnContext.getJobId() + ")");
+                        throw new ACIDException("IllegalState exception during abort( " + txnContext.getTxnId() + ")");
                     }
                     if (IS_DEBUG_MODE) {
                         LOGGER.info(logRecord.getLogRecordForDisplay());
@@ -713,25 +714,25 @@
 
     private class JobEntityCommits {
         private static final String PARTITION_FILE_NAME_SEPARATOR = "_";
-        private final int jobId;
-        private final Set<TxnId> cachedEntityCommitTxns = new HashSet<>();
+        private final long txnId;
+        private final Set<TxnEntityId> cachedEntityCommitTxns = new HashSet<>();
         private final List<File> jobEntitCommitOnDiskPartitionsFiles = new ArrayList<>();
         //a flag indicating whether all the the commits for this jobs have been added.
         private boolean preparedForSearch = false;
-        private TxnId winnerEntity = null;
+        private TxnEntityId winnerEntity = null;
         private int currentPartitionSize = 0;
         private long partitionMaxLSN = 0;
         private String currentPartitonName;
 
-        public JobEntityCommits(int jobId) {
-            this.jobId = jobId;
+        public JobEntityCommits(long txnId) {
+            this.txnId = txnId;
         }
 
         public void add(ILogRecord logRecord) throws IOException {
             if (preparedForSearch) {
                 throw new IOException("Cannot add new entity commits after preparing for search.");
             }
-            winnerEntity = new TxnId(logRecord.getJobId(), logRecord.getDatasetId(), logRecord.getPKHashValue(),
+            winnerEntity = new TxnEntityId(logRecord.getTxnId(), logRecord.getDatasetId(), logRecord.getPKHashValue(),
                     logRecord.getPKValue(), logRecord.getPKValueSize(), true);
             cachedEntityCommitTxns.add(winnerEntity);
             //since log file is read sequentially, LSNs are always increasing
@@ -772,15 +773,15 @@
             preparedForSearch = true;
         }
 
-        public boolean containsEntityCommitForTxnId(long logLSN, TxnId txnId) throws IOException {
+        public boolean containsEntityCommitForTxnId(long logLSN, TxnEntityId txnEntityId) throws IOException {
             //if we don't have any partitions on disk, search only from memory
             if (jobEntitCommitOnDiskPartitionsFiles.size() == 0) {
-                return cachedEntityCommitTxns.contains(txnId);
+                return cachedEntityCommitTxns.contains(txnEntityId);
             } else {
                 //get candidate partitions from disk
                 ArrayList<File> candidatePartitions = getCandidiatePartitions(logLSN);
                 for (File partition : candidatePartitions) {
-                    if (serachPartition(partition, txnId)) {
+                    if (serachPartition(partition, txnEntityId)) {
                         return true;
                     }
                 }
@@ -814,17 +815,17 @@
             jobEntitCommitOnDiskPartitionsFiles.clear();
         }
 
-        private boolean serachPartition(File partition, TxnId txnId) throws IOException {
+        private boolean serachPartition(File partition, TxnEntityId txnEntityId) throws IOException {
             //load partition from disk if it is not  already in memory
             if (!partition.getName().equals(currentPartitonName)) {
                 loadPartitionToMemory(partition, cachedEntityCommitTxns);
                 currentPartitonName = partition.getName();
             }
-            return cachedEntityCommitTxns.contains(txnId);
+            return cachedEntityCommitTxns.contains(txnEntityId);
         }
 
         private String getPartitionName(long maxLSN) {
-            return jobId + PARTITION_FILE_NAME_SEPARATOR + maxLSN;
+            return txnId + PARTITION_FILE_NAME_SEPARATOR + maxLSN;
         }
 
         private long getPartitionMaxLSNFromName(String partitionName) {
@@ -835,18 +836,18 @@
             //if we don't have enough memory to allocate for this partition,
             // we will ask recovery manager to free memory
             if (needToFreeMemory()) {
-                freeJobsCachedEntities(jobId);
+                freeJobsCachedEntities(txnId);
             }
             //allocate a buffer that can hold the current partition
             ByteBuffer buffer = ByteBuffer.allocate(currentPartitionSize);
-            for (Iterator<TxnId> iterator = cachedEntityCommitTxns.iterator(); iterator.hasNext();) {
-                TxnId txnId = iterator.next();
+            for (Iterator<TxnEntityId> iterator = cachedEntityCommitTxns.iterator(); iterator.hasNext();) {
+                TxnEntityId txnEntityId = iterator.next();
                 //serialize the object and remove it from memory
-                txnId.serialize(buffer);
+                txnEntityId.serialize(buffer);
                 iterator.remove();
             }
             //name partition file based on job id and max lsn
-            File partitionFile = createJobRecoveryFile(jobId, getPartitionName(partitionMaxLSN));
+            File partitionFile = createJobRecoveryFile(txnId, getPartitionName(partitionMaxLSN));
             //write file to disk
             try (FileOutputStream fileOutputstream = new FileOutputStream(partitionFile, false);
                     FileChannel fileChannel = fileOutputstream.getChannel()) {
@@ -858,11 +859,11 @@
             jobEntitCommitOnDiskPartitionsFiles.add(partitionFile);
         }
 
-        private void loadPartitionToMemory(File partition, Set<TxnId> partitionTxn) throws IOException {
+        private void loadPartitionToMemory(File partition, Set<TxnEntityId> partitionTxn) throws IOException {
             partitionTxn.clear();
             //if we don't have enough memory to a load partition, we will ask recovery manager to free memory
             if (needToFreeMemory()) {
-                freeJobsCachedEntities(jobId);
+                freeJobsCachedEntities(txnId);
             }
             ByteBuffer buffer = ByteBuffer.allocateDirect((int) partition.length());
             //load partition to memory
@@ -873,9 +874,9 @@
                 }
             }
             buffer.flip();
-            TxnId temp = null;
+            TxnEntityId temp;
             while (buffer.remaining() != 0) {
-                temp = TxnId.deserialize(buffer);
+                temp = TxnEntityId.deserialize(buffer);
                 partitionTxn.add(temp);
             }
         }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
index 608769b..e42b5e5 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
@@ -42,7 +42,7 @@
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.common.messaging.api.ICCMessageBroker;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.asterix.compiler.provider.SqlppCompilationProvider;
 import org.apache.asterix.external.api.IAdapterFactory;
@@ -279,7 +279,7 @@
         Map<ConnectorDescriptorId, ConnectorDescriptorId> connectorIdMapping = new HashMap<>();
         Map<OperatorDescriptorId, List<LocationConstraint>> operatorLocations = new HashMap<>();
         Map<OperatorDescriptorId, Integer> operatorCounts = new HashMap<>();
-        List<JobId> jobIds = new ArrayList<>();
+        List<TxnId> txnIds = new ArrayList<>();
         FeedMetaOperatorDescriptor metaOp;
 
         for (int iter1 = 0; iter1 < jobsList.size(); iter1++) {
@@ -415,11 +415,11 @@
             for (OperatorDescriptorId root : subJob.getRoots()) {
                 jobSpec.addRoot(jobSpec.getOperatorMap().get(operatorIdMapping.get(root)));
             }
-            jobIds.add(((JobEventListenerFactory) subJob.getJobletEventListenerFactory()).getJobId());
+            txnIds.add(((JobEventListenerFactory) subJob.getJobletEventListenerFactory()).getTxnId());
         }
 
         // jobEventListenerFactory
-        jobSpec.setJobletEventListenerFactory(new MultiTransactionJobletEventListenerFactory(jobIds, true));
+        jobSpec.setJobletEventListenerFactory(new MultiTransactionJobletEventListenerFactory(txnIds, true));
         // useConnectorSchedulingPolicy
         jobSpec.setUseConnectorPolicyForScheduling(jobsList.get(0).isUseConnectorPolicyForScheduling());
         // connectorAssignmentPolicy
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FlushDatasetUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FlushDatasetUtil.java
index 958444c..4137fbd 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FlushDatasetUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FlushDatasetUtil.java
@@ -20,12 +20,13 @@
 package org.apache.asterix.utils;
 
 import org.apache.asterix.common.config.CompilerProperties;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.common.utils.JobUtils;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
 import org.apache.asterix.runtime.operators.std.FlushDatasetOperatorDescriptor;
-import org.apache.asterix.transaction.management.service.transaction.JobIdFactory;
+import org.apache.asterix.transaction.management.service.transaction.TxnIdFactory;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
 import org.apache.hyracks.algebricks.common.utils.Pair;
@@ -59,8 +60,8 @@
         AlgebricksMetaOperatorDescriptor emptySource = new AlgebricksMetaOperatorDescriptor(spec, 0, 1,
                 new IPushRuntimeFactory[] { new EmptyTupleSourceRuntimeFactory() }, rDescs);
 
-        org.apache.asterix.common.transactions.JobId jobId = JobIdFactory.generateJobId();
-        FlushDatasetOperatorDescriptor flushOperator = new FlushDatasetOperatorDescriptor(spec, jobId,
+        TxnId txnId = TxnIdFactory.create();
+        FlushDatasetOperatorDescriptor flushOperator = new FlushDatasetOperatorDescriptor(spec, txnId,
                 dataset.getDatasetId());
 
         spec.connect(new OneToOneConnectorDescriptor(spec), emptySource, 0, flushOperator, 0);
@@ -72,7 +73,7 @@
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, emptySource,
                 primaryPartitionConstraint);
 
-        JobEventListenerFactory jobEventListenerFactory = new JobEventListenerFactory(jobId, true);
+        JobEventListenerFactory jobEventListenerFactory = new JobEventListenerFactory(txnId, true);
         spec.setJobletEventListenerFactory(jobEventListenerFactory);
         JobUtils.runJob(hcc, spec, true);
     }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java
index 9e34d70..42f577f 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java
@@ -34,7 +34,7 @@
 import org.apache.asterix.app.active.ActiveNotificationHandler;
 import org.apache.asterix.common.api.IMetadataLockManager;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.common.utils.JobUtils;
 import org.apache.asterix.dataflow.data.nontagged.MissingWriterFactory;
 import org.apache.asterix.metadata.MetadataManager;
@@ -47,7 +47,7 @@
 import org.apache.asterix.metadata.utils.IndexUtil;
 import org.apache.asterix.rebalance.IDatasetRebalanceCallback;
 import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
-import org.apache.asterix.transaction.management.service.transaction.JobIdFactory;
+import org.apache.asterix.transaction.management.service.transaction.TxnIdFactory;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -275,21 +275,21 @@
     private static void populateDataToRebalanceTarget(Dataset source, Dataset target, MetadataProvider metadataProvider,
             IHyracksClientConnection hcc) throws Exception {
         JobSpecification spec = new JobSpecification();
-        JobId jobId = JobIdFactory.generateJobId();
-        JobEventListenerFactory jobEventListenerFactory = new JobEventListenerFactory(jobId, true);
+        TxnId txnId = TxnIdFactory.create();
+        JobEventListenerFactory jobEventListenerFactory = new JobEventListenerFactory(txnId, true);
         spec.setJobletEventListenerFactory(jobEventListenerFactory);
 
         // The pipeline starter.
         IOperatorDescriptor starter = DatasetUtil.createDummyKeyProviderOp(spec, source, metadataProvider);
 
         // Creates primary index scan op.
-        IOperatorDescriptor primaryScanOp = DatasetUtil.createPrimaryIndexScanOp(spec, metadataProvider, source, jobId);
+        IOperatorDescriptor primaryScanOp = DatasetUtil.createPrimaryIndexScanOp(spec, metadataProvider, source, txnId);
 
         // Creates secondary BTree upsert op.
         IOperatorDescriptor upsertOp = createPrimaryIndexUpsertOp(spec, metadataProvider, source, target);
 
         // The final commit operator.
-        IOperatorDescriptor commitOp = createUpsertCommitOp(spec, metadataProvider, jobId, target);
+        IOperatorDescriptor commitOp = createUpsertCommitOp(spec, metadataProvider, txnId, target);
 
         // Connects empty-tuple-source and scan.
         spec.connect(new OneToOneConnectorDescriptor(spec), starter, 0, primaryScanOp, 0);
@@ -326,11 +326,11 @@
 
     // Creates the commit operator for populating the target dataset.
     private static IOperatorDescriptor createUpsertCommitOp(JobSpecification spec, MetadataProvider metadataProvider,
-            JobId jobId, Dataset target) throws AlgebricksException {
+            TxnId txnId, Dataset target) throws AlgebricksException {
         int[] primaryKeyFields = getPrimaryKeyPermutationForUpsert(target);
         return new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
                 new IPushRuntimeFactory[] {
-                        target.getCommitRuntimeFactory(metadataProvider, jobId, primaryKeyFields, true) },
+                        target.getCommitRuntimeFactory(metadataProvider, txnId, primaryKeyFields, true) },
                 new RecordDescriptor[] { target.getPrimaryRecordDescriptor(metadataProvider) });
     }
 
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
index 1810517..df8eef7 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
@@ -40,6 +40,7 @@
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.transactions.IRecoveryManager.ResourceType;
 import org.apache.asterix.common.transactions.ITransactionManager;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.dataflow.data.nontagged.MissingWriterFactory;
 import org.apache.asterix.file.StorageComponentProvider;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
@@ -152,8 +153,8 @@
         ExecutionTestUtil.tearDown(cleanupOnStop);
     }
 
-    public org.apache.asterix.common.transactions.JobId getTxnJobId(IHyracksTaskContext ctx) {
-        return new org.apache.asterix.common.transactions.JobId((int) ctx.getJobletContext().getJobId().getId());
+    public TxnId getTxnJobId(IHyracksTaskContext ctx) {
+        return new TxnId(ctx.getJobletContext().getJobId().getId());
     }
 
     public Pair<LSMInsertDeleteOperatorNodePushable, CommitRuntime> getInsertPipeline(IHyracksTaskContext ctx,
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 d1d869e..d2b1276 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
@@ -26,15 +26,15 @@
 public abstract class AbstractOperationCallbackFactory implements Serializable {
     private static final long serialVersionUID = 1L;
 
-    protected final JobId jobId;
+    protected final TxnId txnId;
     protected final int datasetId;
     protected final int[] primaryKeyFields;
     protected final ITransactionSubsystemProvider txnSubsystemProvider;
     protected final byte resourceType;
 
-    public AbstractOperationCallbackFactory(JobId jobId, int datasetId, int[] primaryKeyFields,
+    public AbstractOperationCallbackFactory(TxnId txnId, int datasetId, int[] primaryKeyFields,
             ITransactionSubsystemProvider txnSubsystemProvider, byte resourceType) {
-        this.jobId = jobId;
+        this.txnId = txnId;
         this.datasetId = datasetId;
         this.primaryKeyFields = primaryKeyFields;
         this.txnSubsystemProvider = txnSubsystemProvider;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/Checkpoint.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/Checkpoint.java
index a4c41df..cb278a7 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/Checkpoint.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/Checkpoint.java
@@ -31,19 +31,19 @@
 
     private final long checkpointLsn;
     private final long minMCTFirstLsn;
-    private final int maxJobId;
+    private final long maxTxnId;
     private final long timeStamp;
     private final boolean sharp;
     private final int storageVersion;
 
     @JsonCreator
     public Checkpoint(@JsonProperty("checkpointLsn") long checkpointLsn,
-            @JsonProperty("minMCTFirstLsn") long minMCTFirstLsn, @JsonProperty("maxJobId") int maxJobId,
+            @JsonProperty("minMCTFirstLsn") long minMCTFirstLsn, @JsonProperty("maxJobId") long maxTxnId,
             @JsonProperty("timeStamp") long timeStamp, @JsonProperty("sharp") boolean sharp,
             @JsonProperty("storageVersion") int storageVersion) {
         this.checkpointLsn = checkpointLsn;
         this.minMCTFirstLsn = minMCTFirstLsn;
-        this.maxJobId = maxJobId;
+        this.maxTxnId = maxTxnId;
         this.timeStamp = timeStamp;
         this.sharp = sharp;
         this.storageVersion = storageVersion;
@@ -57,8 +57,8 @@
         return minMCTFirstLsn;
     }
 
-    public int getMaxJobId() {
-        return maxJobId;
+    public long getMaxJobId() {
+        return maxTxnId;
     }
 
     public long getTimeStamp() {
@@ -108,7 +108,7 @@
         final int prime = 31;
         int result = 1;
         result = prime * result + (int) (checkpointLsn ^ (checkpointLsn >>> 32));
-        result = prime * result + maxJobId;
+        result = prime * result + Long.hashCode(maxTxnId);
         result = prime * result + (int) (minMCTFirstLsn ^ (minMCTFirstLsn >>> 32));
         result = prime * result + (sharp ? 1231 : 1237);
         result = prime * result + storageVersion;
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 6ee0980..4090b65 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
@@ -32,36 +32,33 @@
         LARGE_RECORD
     }
 
-    public static final int CHKSUM_LEN = Long.BYTES;
-    public static final int FLDCNT_LEN = Integer.BYTES;
-    public static final int DS_LEN = Integer.BYTES;
-    public static final int LOG_SOURCE_LEN = Byte.BYTES;
-    public static final int LOGRCD_SZ_LEN = Integer.BYTES;
-    public static final int NEWOP_LEN = Byte.BYTES;
-    public static final int NEWVALSZ_LEN = Integer.BYTES;
-    public static final int PKHASH_LEN = Integer.BYTES;
-    public static final int PKSZ_LEN = Integer.BYTES;
-    public static final int PRVLSN_LEN = Long.BYTES;
-    public static final int RS_PARTITION_LEN = Integer.BYTES;
-    public static final int RSID_LEN = Long.BYTES;
-    public static final int SEQ_NUM_LEN = Long.BYTES;
-    public static final int TYPE_LEN = Byte.BYTES;
-    public static final int UUID_LEN = Long.BYTES;
+    int CHKSUM_LEN = Long.BYTES;
+    int FLDCNT_LEN = Integer.BYTES;
+    int DS_LEN = Integer.BYTES;
+    int LOG_SOURCE_LEN = Byte.BYTES;
+    int LOGRCD_SZ_LEN = Integer.BYTES;
+    int NEWOP_LEN = Byte.BYTES;
+    int NEWVALSZ_LEN = Integer.BYTES;
+    int PKHASH_LEN = Integer.BYTES;
+    int PKSZ_LEN = Integer.BYTES;
+    int PRVLSN_LEN = Long.BYTES;
+    int RS_PARTITION_LEN = Integer.BYTES;
+    int RSID_LEN = Long.BYTES;
+    int SEQ_NUM_LEN = Long.BYTES;
+    int TYPE_LEN = Byte.BYTES;
+    int UUID_LEN = Long.BYTES;
 
-    public static final int ALL_RECORD_HEADER_LEN = LOG_SOURCE_LEN + TYPE_LEN + JobId.BYTES;
-    public static final int ENTITYCOMMIT_UPDATE_HEADER_LEN = RS_PARTITION_LEN + DatasetId.BYTES + PKHASH_LEN + PKSZ_LEN;
-    public static final int UPDATE_LSN_HEADER = RSID_LEN + LOGRCD_SZ_LEN;
-    public static final int UPDATE_BODY_HEADER = FLDCNT_LEN + NEWOP_LEN + NEWVALSZ_LEN;
-    // What are these fields? vvvvv
-    public static final int REMOTE_FLUSH_LOG_EXTRA_FIELDS_LEN = Long.BYTES + Integer.BYTES + Integer.BYTES;
+    int ALL_RECORD_HEADER_LEN = LOG_SOURCE_LEN + TYPE_LEN + TxnId.BYTES;
+    int ENTITYCOMMIT_UPDATE_HEADER_LEN = RS_PARTITION_LEN + DatasetId.BYTES + PKHASH_LEN + PKSZ_LEN;
+    int UPDATE_LSN_HEADER = RSID_LEN + LOGRCD_SZ_LEN;
+    int UPDATE_BODY_HEADER = FLDCNT_LEN + NEWOP_LEN + NEWVALSZ_LEN;
 
-    // How are the following computed?
-    public static final int JOB_TERMINATE_LOG_SIZE = ALL_RECORD_HEADER_LEN + CHKSUM_LEN;
-    public static final int ENTITY_COMMIT_LOG_BASE_SIZE = 30; // ALL_RECORD_HEADER_LEN + CHKSUM_LEN +?
-    public static final int UPDATE_LOG_BASE_SIZE = 51; // ALL_RECORD_HEADER_LEN + CHKSUM_LEN +?
-    public static final int FLUSH_LOG_SIZE = 18; // ALL_RECORD_HEADER_LEN + CHKSUM_LEN +?
-    public static final int WAIT_LOG_SIZE = ALL_RECORD_HEADER_LEN + CHKSUM_LEN;
-    public static final int MARKER_BASE_LOG_SIZE =
+    int JOB_TERMINATE_LOG_SIZE = ALL_RECORD_HEADER_LEN + CHKSUM_LEN;
+    int ENTITY_COMMIT_LOG_BASE_SIZE = ALL_RECORD_HEADER_LEN + ENTITYCOMMIT_UPDATE_HEADER_LEN + CHKSUM_LEN;
+    int UPDATE_LOG_BASE_SIZE = ENTITY_COMMIT_LOG_BASE_SIZE + UPDATE_LSN_HEADER + UPDATE_BODY_HEADER;
+    int FLUSH_LOG_SIZE = ALL_RECORD_HEADER_LEN + DatasetId.BYTES + CHKSUM_LEN;
+    int WAIT_LOG_SIZE = ALL_RECORD_HEADER_LEN + CHKSUM_LEN;
+    int MARKER_BASE_LOG_SIZE =
             ALL_RECORD_HEADER_LEN + CHKSUM_LEN + DS_LEN + RS_PARTITION_LEN + PRVLSN_LEN + LOGRCD_SZ_LEN;
 
     public RecordReadStatus readLogRecord(ByteBuffer buffer);
@@ -80,9 +77,9 @@
 
     public void setLogType(byte logType);
 
-    public int getJobId();
+    long getTxnId();
 
-    public void setJobId(int jobId);
+    void setTxnId(long jobId);
 
     public int getDatasetId();
 
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
index 7965aa5..dea7a67 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
@@ -98,13 +98,13 @@
     /**
      * Creates a temporary file to be used during recovery
      *
-     * @param jobId
+     * @param txnId
      * @param fileName
      * @return A file to the created temporary file
      * @throws IOException
-     *             if the file for the specified {@code jobId} with the {@code fileName} already exists
+     *             if the file for the specified {@code txnId} with the {@code fileName} already exists
      */
-    File createJobRecoveryFile(int jobId, String fileName) throws IOException;
+    File createJobRecoveryFile(long txnId, String fileName) throws IOException;
 
     /**
      * Deletes all temporary recovery files
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITransactionContext.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITransactionContext.java
index f9d924f..3dda5d3 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITransactionContext.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITransactionContext.java
@@ -25,7 +25,7 @@
     public void registerIndexAndCallback(long resourceId, ILSMIndex index, AbstractOperationCallback callback,
             boolean isPrimaryIndex);
 
-    public JobId getJobId();
+    public TxnId getTxnId();
 
     public void setTimeout(boolean isTimeout);
 
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITransactionManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITransactionManager.java
index 0123814..77c6a9f 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITransactionManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITransactionManager.java
@@ -42,26 +42,26 @@
      * Begins a transaction identified by a transaction id and returns the
      * associated transaction context.
      *
-     * @param jobId
+     * @param txnId
      *            a unique value for the transaction id.
      * @return the transaction context associated with the initiated transaction
      * @see ITransactionContext
      * @throws ACIDException
      */
-    public ITransactionContext beginTransaction(JobId jobId) throws ACIDException;
+    public ITransactionContext beginTransaction(TxnId txnId) throws ACIDException;
 
     /**
      * Returns the transaction context of an active transaction given the
      * transaction id.
      *
-     * @param jobId
+     * @param txnId
      *            a unique value for the transaction id.
      * @param createIfNotExist
      *            TODO
      * @return
      * @throws ACIDException
      */
-    public ITransactionContext getTransactionContext(JobId jobId, boolean createIfNotExist) throws ACIDException;
+    public ITransactionContext getTransactionContext(TxnId txnId, boolean createIfNotExist) throws ACIDException;
 
     /**
      * Commits a transaction.
@@ -73,7 +73,7 @@
      * @param pkHash
      *            TODO
      * @throws ACIDException
-     * @see ITransactionContextimport org.apache.hyracks.api.job.JobId;
+     * @see ITransactionContextimport org.apache.hyracks.api.job.TxnId;
      * @see ACIDException
      */
     public void commitTransaction(ITransactionContext txnContext, DatasetId datasetId, int pkHash)
@@ -125,8 +125,8 @@
     public ITransactionSubsystem getTransactionSubsystem();
 
     /**
-     * @return The current max job id.
+     * @return The current max txn id.
      */
-    int getMaxJobId();
+    long getMaxTxnId();
 
 }
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 e80cfa6..68afb2a 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
@@ -32,10 +32,10 @@
 /**
  * == LogRecordFormat ==
  * ---------------------------
- * [Header1] (6 bytes) : for all log types
+ * [Header1] (10 bytes) : for all log types
  * LogSource(1)
  * LogType(1)
- * JobId(4)
+ * TxnId(8)
  * ---------------------------
  * [Header2] (16 bytes + PKValueSize) : for entity_commit, upsert_entity_commit, and update log types
  * DatasetId(4) //stored in dataset_dataset in Metadata Node
@@ -57,16 +57,6 @@
  * [Tail] (8 bytes) : for all log types
  * Checksum(8)
  * ---------------------------
- * = LogSize =
- * 1) JOB_COMMIT_LOG_SIZE: 14 bytes (Header1(6) + Tail(8))
- * 2) ENTITY_COMMIT || UPSERT_ENTITY_COMMIT: (Header1(6) + Header2(16) + Tail(8)) + PKValueSize
- * --> ENTITY_COMMIT_LOG_BASE_SIZE = 30
- * 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 {
@@ -74,7 +64,7 @@
     // ------------- fields in a log record (begin) ------------//
     private byte logSource;
     private byte logType;
-    private int jobId;
+    private long txnId;
     private int datasetId;
     private int PKHashValue;
     private int PKValueSize;
@@ -130,7 +120,7 @@
     private void doWriteLogRecord(ByteBuffer buffer) {
         buffer.put(logSource);
         buffer.put(logType);
-        buffer.putInt(jobId);
+        buffer.putLong(txnId);
         switch (logType) {
             case LogType.ENTITY_COMMIT:
                 writeEntityInfo(buffer);
@@ -248,7 +238,7 @@
         }
         logSource = buffer.get();
         logType = buffer.get();
-        jobId = buffer.getInt();
+        txnId = buffer.getLong();
         switch (logType) {
             case LogType.FLUSH:
                 if (buffer.remaining() < ILogRecord.DS_LEN) {
@@ -454,7 +444,7 @@
         builder.append(" LSN : ").append(LSN);
         builder.append(" LogType : ").append(LogType.toString(logType));
         builder.append(" LogSize : ").append(logSize);
-        builder.append(" JobId : ").append(jobId);
+        builder.append(" TxnId : ").append(txnId);
         if (logType == LogType.ENTITY_COMMIT || logType == LogType.UPDATE) {
             builder.append(" DatasetId : ").append(datasetId);
             builder.append(" ResourcePartition : ").append(resourcePartition);
@@ -503,13 +493,13 @@
     }
 
     @Override
-    public int getJobId() {
-        return jobId;
+    public long getTxnId() {
+        return txnId;
     }
 
     @Override
-    public void setJobId(int jobId) {
-        this.jobId = jobId;
+    public void setTxnId(long jobId) {
+        this.txnId = jobId;
     }
 
     @Override
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/JobId.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/TxnId.java
similarity index 75%
rename from asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/JobId.java
rename to asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/TxnId.java
index 9654a92..b0d38b8 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/JobId.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/TxnId.java
@@ -20,26 +20,26 @@
 
 import java.io.Serializable;
 
-public class JobId implements Serializable {
+public class TxnId implements Serializable {
     private static final long serialVersionUID = 1L;
     /**
-     * The number of bytes used to represent {@link JobId} value.
+     * The number of bytes used to represent {@link TxnId} value.
      */
-    public static final int BYTES = Integer.BYTES;
+    public static final int BYTES = Long.BYTES;
 
-    private int id;
+    protected long id;
 
-    public JobId(int id) {
+    public TxnId(long id) {
         this.id = id;
     }
 
-    public int getId() {
+    public long getId() {
         return id;
     }
 
     @Override
     public int hashCode() {
-        return id;
+        return Long.hashCode(id);
     }
 
     @Override
@@ -47,18 +47,15 @@
         if (o == this) {
             return true;
         }
-        if (!(o instanceof JobId)) {
+        if (!(o instanceof TxnId)) {
             return false;
         }
-        return ((JobId) o).id == id;
+        return ((TxnId) o).id == id;
     }
 
     @Override
     public String toString() {
-        return "JID:" + id;
+        return "TxnId:" + id;
     }
 
-    public void setId(int jobId) {
-        id = jobId;
-    }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/TransactionUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/TransactionUtil.java
index e9f96f9..be4b47f 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/TransactionUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/TransactionUtil.java
@@ -34,21 +34,21 @@
 
     public static void formJobTerminateLogRecord(ITransactionContext txnCtx, LogRecord logRecord, boolean isCommit) {
         logRecord.setTxnCtx(txnCtx);
-        TransactionUtil.formJobTerminateLogRecord(logRecord, txnCtx.getJobId().getId(), isCommit);
+        TransactionUtil.formJobTerminateLogRecord(logRecord, txnCtx.getTxnId().getId(), isCommit);
     }
 
-    public static void formJobTerminateLogRecord(LogRecord logRecord, int jobId, boolean isCommit) {
+    public static void formJobTerminateLogRecord(LogRecord logRecord, long txnId, boolean isCommit) {
         logRecord.setLogType(isCommit ? LogType.JOB_COMMIT : LogType.ABORT);
         logRecord.setDatasetId(-1);
         logRecord.setPKHashValue(-1);
-        logRecord.setJobId(jobId);
+        logRecord.setTxnId(txnId);
         logRecord.computeAndSetLogSize();
     }
 
     public static void formFlushLogRecord(LogRecord logRecord, int datasetId, PrimaryIndexOperationTracker opTracker,
             String nodeId, int numberOfIndexes) {
         logRecord.setLogType(LogType.FLUSH);
-        logRecord.setJobId(-1);
+        logRecord.setTxnId(-1);
         logRecord.setDatasetId(datasetId);
         logRecord.setOpTracker(opTracker);
         logRecord.setNumOfFlushedIndexes(numberOfIndexes);
@@ -60,7 +60,7 @@
             int PKHashValue, ITupleReference PKValue, int[] PKFields, int resourcePartition, byte entityCommitType) {
         logRecord.setTxnCtx(txnCtx);
         logRecord.setLogType(entityCommitType);
-        logRecord.setJobId(txnCtx.getJobId().getId());
+        logRecord.setTxnId(txnCtx.getTxnId().getId());
         logRecord.setDatasetId(datasetId);
         logRecord.setPKHashValue(PKHashValue);
         logRecord.setPKFieldCnt(PKFields.length);
@@ -76,7 +76,7 @@
         logRecord.setTxnCtx(txnCtx);
         logRecord.setLogSource(LogSource.LOCAL);
         logRecord.setLogType(LogType.MARKER);
-        logRecord.setJobId(txnCtx.getJobId().getId());
+        logRecord.setTxnId(txnCtx.getTxnId().getId());
         logRecord.setDatasetId(datasetId);
         logRecord.setResourcePartition(resourcePartition);
         marker.get(); // read the first byte since it is not part of the marker object
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
index 2e872fc..1384d26 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
@@ -32,7 +32,7 @@
 import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.common.functions.FunctionSignature;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.external.indexing.ExternalFile;
 import org.apache.asterix.metadata.api.IAsterixStateProxy;
 import org.apache.asterix.metadata.api.IExtensionMetadataEntity;
@@ -53,7 +53,7 @@
 import org.apache.asterix.metadata.entities.Node;
 import org.apache.asterix.metadata.entities.NodeGroup;
 import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.transaction.management.service.transaction.JobIdFactory;
+import org.apache.asterix.transaction.management.service.transaction.TxnIdFactory;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -123,36 +123,36 @@
 
     @Override
     public MetadataTransactionContext beginTransaction() throws RemoteException, ACIDException {
-        JobId jobId = JobIdFactory.generateJobId();
-        metadataNode.beginTransaction(jobId);
-        return new MetadataTransactionContext(jobId);
+        TxnId txnId = TxnIdFactory.create();
+        metadataNode.beginTransaction(txnId);
+        return new MetadataTransactionContext(txnId);
     }
 
     @Override
     public void commitTransaction(MetadataTransactionContext ctx) throws RemoteException, ACIDException {
-        metadataNode.commitTransaction(ctx.getJobId());
+        metadataNode.commitTransaction(ctx.getTxnId());
         cache.commit(ctx);
     }
 
     @Override
     public void abortTransaction(MetadataTransactionContext ctx) throws RemoteException, ACIDException {
-        metadataNode.abortTransaction(ctx.getJobId());
+        metadataNode.abortTransaction(ctx.getTxnId());
     }
 
     @Override
     public void lock(MetadataTransactionContext ctx, byte lockMode) throws RemoteException, ACIDException {
-        metadataNode.lock(ctx.getJobId(), lockMode);
+        metadataNode.lock(ctx.getTxnId(), lockMode);
     }
 
     @Override
     public void unlock(MetadataTransactionContext ctx, byte lockMode) throws RemoteException, ACIDException {
-        metadataNode.unlock(ctx.getJobId(), lockMode);
+        metadataNode.unlock(ctx.getTxnId(), lockMode);
     }
 
     @Override
     public void addDataverse(MetadataTransactionContext ctx, Dataverse dataverse) throws AlgebricksException {
         try {
-            metadataNode.addDataverse(ctx.getJobId(), dataverse);
+            metadataNode.addDataverse(ctx.getTxnId(), dataverse);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -162,7 +162,7 @@
     @Override
     public void dropDataverse(MetadataTransactionContext ctx, String dataverseName) throws AlgebricksException {
         try {
-            metadataNode.dropDataverse(ctx.getJobId(), dataverseName);
+            metadataNode.dropDataverse(ctx.getTxnId(), dataverseName);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -172,7 +172,7 @@
     @Override
     public List<Dataverse> getDataverses(MetadataTransactionContext ctx) throws AlgebricksException {
         try {
-            return metadataNode.getDataverses(ctx.getJobId());
+            return metadataNode.getDataverses(ctx.getTxnId());
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -199,7 +199,7 @@
             return dataverse;
         }
         try {
-            dataverse = metadataNode.getDataverse(ctx.getJobId(), dataverseName);
+            dataverse = metadataNode.getDataverse(ctx.getTxnId(), dataverseName);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -230,7 +230,7 @@
         try {
             // Assuming that the transaction can read its own writes on the
             // metadata node.
-            dataverseDatasets.addAll(metadataNode.getDataverseDatasets(ctx.getJobId(), dataverseName));
+            dataverseDatasets.addAll(metadataNode.getDataverseDatasets(ctx.getTxnId(), dataverseName));
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -244,7 +244,7 @@
         // add dataset into metadataNode
         if (!dataset.getDatasetDetails().isTemp()) {
             try {
-                metadataNode.addDataset(ctx.getJobId(), dataset);
+                metadataNode.addDataset(ctx.getTxnId(), dataset);
             } catch (RemoteException e) {
                 throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
             }
@@ -261,7 +261,7 @@
         // If a dataset is not in the cache, then it could not be a temp dataset
         if (dataset == null || !dataset.getDatasetDetails().isTemp()) {
             try {
-                metadataNode.dropDataset(ctx.getJobId(), dataverseName, datasetName);
+                metadataNode.dropDataset(ctx.getTxnId(), dataverseName, datasetName);
             } catch (RemoteException e) {
                 throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
             }
@@ -295,7 +295,7 @@
             return dataset;
         }
         try {
-            dataset = metadataNode.getDataset(ctx.getJobId(), dataverseName, datasetName);
+            dataset = metadataNode.getDataset(ctx.getTxnId(), dataverseName, datasetName);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -321,7 +321,7 @@
         } else {
             try {
                 // for persistent datasets
-                datasetIndexes = metadataNode.getDatasetIndexes(ctx.getJobId(), dataverseName, datasetName);
+                datasetIndexes = metadataNode.getDatasetIndexes(ctx.getTxnId(), dataverseName, datasetName);
             } catch (RemoteException e) {
                 throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
             }
@@ -333,7 +333,7 @@
     public void addCompactionPolicy(MetadataTransactionContext mdTxnCtx, CompactionPolicy compactionPolicy)
             throws AlgebricksException {
         try {
-            metadataNode.addCompactionPolicy(mdTxnCtx.getJobId(), compactionPolicy);
+            metadataNode.addCompactionPolicy(mdTxnCtx.getTxnId(), compactionPolicy);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -346,7 +346,7 @@
 
         CompactionPolicy compactionPolicy;
         try {
-            compactionPolicy = metadataNode.getCompactionPolicy(ctx.getJobId(), dataverse, policyName);
+            compactionPolicy = metadataNode.getCompactionPolicy(ctx.getTxnId(), dataverse, policyName);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -356,13 +356,13 @@
     @Override
     public void addDatatype(MetadataTransactionContext ctx, Datatype datatype) throws AlgebricksException {
         try {
-            metadataNode.addDatatype(ctx.getJobId(), datatype);
+            metadataNode.addDatatype(ctx.getTxnId(), datatype);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         try {
             ctx.addDatatype(
-                    metadataNode.getDatatype(ctx.getJobId(), datatype.getDataverseName(), datatype.getDatatypeName()));
+                    metadataNode.getDatatype(ctx.getTxnId(), datatype.getDataverseName(), datatype.getDatatypeName()));
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -372,7 +372,7 @@
     public void dropDatatype(MetadataTransactionContext ctx, String dataverseName, String datatypeName)
             throws AlgebricksException {
         try {
-            metadataNode.dropDatatype(ctx.getJobId(), dataverseName, datatypeName);
+            metadataNode.dropDatatype(ctx.getTxnId(), dataverseName, datatypeName);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -409,7 +409,7 @@
                     datatype.getIsAnonymous());
         }
         try {
-            datatype = metadataNode.getDatatype(ctx.getJobId(), dataverseName, datatypeName);
+            datatype = metadataNode.getDatatype(ctx.getTxnId(), dataverseName, datatypeName);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -428,7 +428,7 @@
         Dataset dataset = findDataset(ctx, dataverseName, datasetName);
         if (dataset == null || !dataset.getDatasetDetails().isTemp()) {
             try {
-                metadataNode.addIndex(ctx.getJobId(), index);
+                metadataNode.addIndex(ctx.getTxnId(), index);
             } catch (RemoteException e) {
                 throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
             }
@@ -439,7 +439,7 @@
     @Override
     public void addAdapter(MetadataTransactionContext mdTxnCtx, DatasourceAdapter adapter) throws AlgebricksException {
         try {
-            metadataNode.addAdapter(mdTxnCtx.getJobId(), adapter);
+            metadataNode.addAdapter(mdTxnCtx.getTxnId(), adapter);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -457,7 +457,7 @@
         // operations.
         if (dataset == null || !dataset.getDatasetDetails().isTemp()) {
             try {
-                metadataNode.dropIndex(ctx.getJobId(), dataverseName, datasetName, indexName);
+                metadataNode.dropIndex(ctx.getTxnId(), dataverseName, datasetName, indexName);
             } catch (RemoteException e) {
                 throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
             }
@@ -490,7 +490,7 @@
             return index;
         }
         try {
-            index = metadataNode.getIndex(ctx.getJobId(), dataverseName, datasetName, indexName);
+            index = metadataNode.getIndex(ctx.getTxnId(), dataverseName, datasetName, indexName);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -505,7 +505,7 @@
     @Override
     public void addNode(MetadataTransactionContext ctx, Node node) throws AlgebricksException {
         try {
-            metadataNode.addNode(ctx.getJobId(), node);
+            metadataNode.addNode(ctx.getTxnId(), node);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -514,7 +514,7 @@
     @Override
     public void addNodegroup(MetadataTransactionContext ctx, NodeGroup nodeGroup) throws AlgebricksException {
         try {
-            metadataNode.addNodeGroup(ctx.getJobId(), nodeGroup);
+            metadataNode.addNodeGroup(ctx.getTxnId(), nodeGroup);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -526,7 +526,7 @@
             throws AlgebricksException {
         boolean dropped;
         try {
-            dropped = metadataNode.dropNodegroup(ctx.getJobId(), nodeGroupName, failSilently);
+            dropped = metadataNode.dropNodegroup(ctx.getTxnId(), nodeGroupName, failSilently);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -556,7 +556,7 @@
             return nodeGroup;
         }
         try {
-            nodeGroup = metadataNode.getNodeGroup(ctx.getJobId(), nodeGroupName);
+            nodeGroup = metadataNode.getNodeGroup(ctx.getTxnId(), nodeGroupName);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -571,7 +571,7 @@
     @Override
     public void addFunction(MetadataTransactionContext mdTxnCtx, Function function) throws AlgebricksException {
         try {
-            metadataNode.addFunction(mdTxnCtx.getJobId(), function);
+            metadataNode.addFunction(mdTxnCtx.getTxnId(), function);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -582,7 +582,7 @@
     public void dropFunction(MetadataTransactionContext ctx, FunctionSignature functionSignature)
             throws AlgebricksException {
         try {
-            metadataNode.dropFunction(ctx.getJobId(), functionSignature);
+            metadataNode.dropFunction(ctx.getTxnId(), functionSignature);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -616,7 +616,7 @@
             return function;
         }
         try {
-            function = metadataNode.getFunction(ctx.getJobId(), functionSignature);
+            function = metadataNode.getFunction(ctx.getTxnId(), functionSignature);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -633,7 +633,7 @@
     public List<Function> getFunctions(MetadataTransactionContext ctx, String dataverseName)
             throws AlgebricksException {
         try {
-            return metadataNode.getFunctions(ctx.getJobId(), dataverseName);
+            return metadataNode.getFunctions(ctx.getTxnId(), dataverseName);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -643,7 +643,7 @@
     public void addFeedPolicy(MetadataTransactionContext mdTxnCtx, FeedPolicyEntity feedPolicy)
             throws AlgebricksException {
         try {
-            metadataNode.addFeedPolicy(mdTxnCtx.getJobId(), feedPolicy);
+            metadataNode.addFeedPolicy(mdTxnCtx.getTxnId(), feedPolicy);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -653,7 +653,7 @@
     @Override
     public void initializeDatasetIdFactory(MetadataTransactionContext ctx) throws AlgebricksException {
         try {
-            metadataNode.initializeDatasetIdFactory(ctx.getJobId());
+            metadataNode.initializeDatasetIdFactory(ctx.getTxnId());
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -675,7 +675,7 @@
         try {
             // Assuming that the transaction can read its own writes on the
             // metadata node.
-            dataverseFunctions = metadataNode.getDataverseFunctions(ctx.getJobId(), dataverseName);
+            dataverseFunctions = metadataNode.getDataverseFunctions(ctx.getTxnId(), dataverseName);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -688,7 +688,7 @@
     public void dropAdapter(MetadataTransactionContext ctx, String dataverseName, String name)
             throws AlgebricksException {
         try {
-            metadataNode.dropAdapter(ctx.getJobId(), dataverseName, name);
+            metadataNode.dropAdapter(ctx.getTxnId(), dataverseName, name);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -699,7 +699,7 @@
             throws AlgebricksException {
         DatasourceAdapter adapter;
         try {
-            adapter = metadataNode.getAdapter(ctx.getJobId(), dataverseName, name);
+            adapter = metadataNode.getAdapter(ctx.getTxnId(), dataverseName, name);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -710,7 +710,7 @@
     public void dropLibrary(MetadataTransactionContext ctx, String dataverseName, String libraryName)
             throws AlgebricksException {
         try {
-            metadataNode.dropLibrary(ctx.getJobId(), dataverseName, libraryName);
+            metadataNode.dropLibrary(ctx.getTxnId(), dataverseName, libraryName);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -724,7 +724,7 @@
         try {
             // Assuming that the transaction can read its own writes on the
             // metadata node.
-            dataverseLibaries = metadataNode.getDataverseLibraries(ctx.getJobId(), dataverseName);
+            dataverseLibaries = metadataNode.getDataverseLibraries(ctx.getTxnId(), dataverseName);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -736,7 +736,7 @@
     @Override
     public void addLibrary(MetadataTransactionContext ctx, Library library) throws AlgebricksException {
         try {
-            metadataNode.addLibrary(ctx.getJobId(), library);
+            metadataNode.addLibrary(ctx.getTxnId(), library);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -748,7 +748,7 @@
             throws AlgebricksException, RemoteException {
         Library library;
         try {
-            library = metadataNode.getLibrary(ctx.getJobId(), dataverseName, libraryName);
+            library = metadataNode.getLibrary(ctx.getTxnId(), dataverseName, libraryName);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -781,7 +781,7 @@
 
         FeedPolicyEntity feedPolicy;
         try {
-            feedPolicy = metadataNode.getFeedPolicy(ctx.getJobId(), dataverse, policyName);
+            feedPolicy = metadataNode.getFeedPolicy(ctx.getTxnId(), dataverse, policyName);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -792,7 +792,7 @@
     public Feed getFeed(MetadataTransactionContext ctx, String dataverse, String feedName) throws AlgebricksException {
         Feed feed;
         try {
-            feed = metadataNode.getFeed(ctx.getJobId(), dataverse, feedName);
+            feed = metadataNode.getFeed(ctx.getTxnId(), dataverse, feedName);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -803,7 +803,7 @@
     public List<Feed> getFeeds(MetadataTransactionContext ctx, String dataverse) throws AlgebricksException {
         List<Feed> feeds;
         try {
-            feeds = metadataNode.getFeeds(ctx.getJobId(), dataverse);
+            feeds = metadataNode.getFeeds(ctx.getTxnId(), dataverse);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -815,11 +815,11 @@
         Feed feed = null;
         List<FeedConnection> feedConnections = null;
         try {
-            feed = metadataNode.getFeed(ctx.getJobId(), dataverse, feedName);
-            feedConnections = metadataNode.getFeedConnections(ctx.getJobId(), dataverse, feedName);
-            metadataNode.dropFeed(ctx.getJobId(), dataverse, feedName);
+            feed = metadataNode.getFeed(ctx.getTxnId(), dataverse, feedName);
+            feedConnections = metadataNode.getFeedConnections(ctx.getTxnId(), dataverse, feedName);
+            metadataNode.dropFeed(ctx.getTxnId(), dataverse, feedName);
             for (FeedConnection feedConnection : feedConnections) {
-                metadataNode.dropFeedConnection(ctx.getJobId(), dataverse, feedName, feedConnection.getDatasetName());
+                metadataNode.dropFeedConnection(ctx.getTxnId(), dataverse, feedName, feedConnection.getDatasetName());
                 ctx.dropFeedConnection(dataverse, feedName, feedConnection.getDatasetName());
             }
         } catch (RemoteException e) {
@@ -831,7 +831,7 @@
     @Override
     public void addFeed(MetadataTransactionContext ctx, Feed feed) throws AlgebricksException {
         try {
-            metadataNode.addFeed(ctx.getJobId(), feed);
+            metadataNode.addFeed(ctx.getTxnId(), feed);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -842,7 +842,7 @@
     public void addFeedConnection(MetadataTransactionContext ctx, FeedConnection feedConnection)
             throws AlgebricksException {
         try {
-            metadataNode.addFeedConnection(ctx.getJobId(), feedConnection);
+            metadataNode.addFeedConnection(ctx.getTxnId(), feedConnection);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -853,7 +853,7 @@
     public void dropFeedConnection(MetadataTransactionContext ctx, String dataverseName, String feedName,
             String datasetName) throws AlgebricksException {
         try {
-            metadataNode.dropFeedConnection(ctx.getJobId(), dataverseName, feedName, datasetName);
+            metadataNode.dropFeedConnection(ctx.getTxnId(), dataverseName, feedName, datasetName);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -864,7 +864,7 @@
     public FeedConnection getFeedConnection(MetadataTransactionContext ctx, String dataverseName, String feedName,
             String datasetName) throws AlgebricksException {
         try {
-            return metadataNode.getFeedConnection(ctx.getJobId(), dataverseName, feedName, datasetName);
+            return metadataNode.getFeedConnection(ctx.getTxnId(), dataverseName, feedName, datasetName);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -874,7 +874,7 @@
     public List<FeedConnection> getFeedConections(MetadataTransactionContext ctx, String dataverseName, String feedName)
             throws AlgebricksException {
         try {
-            return metadataNode.getFeedConnections(ctx.getJobId(), dataverseName, feedName);
+            return metadataNode.getFeedConnections(ctx.getTxnId(), dataverseName, feedName);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -885,7 +885,7 @@
             throws AlgebricksException {
         List<DatasourceAdapter> dataverseAdapters;
         try {
-            dataverseAdapters = metadataNode.getDataverseAdapters(mdTxnCtx.getJobId(), dataverse);
+            dataverseAdapters = metadataNode.getDataverseAdapters(mdTxnCtx.getTxnId(), dataverse);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -897,8 +897,8 @@
             throws AlgebricksException {
         FeedPolicyEntity feedPolicy;
         try {
-            feedPolicy = metadataNode.getFeedPolicy(mdTxnCtx.getJobId(), dataverseName, policyName);
-            metadataNode.dropFeedPolicy(mdTxnCtx.getJobId(), dataverseName, policyName);
+            feedPolicy = metadataNode.getFeedPolicy(mdTxnCtx.getTxnId(), dataverseName, policyName);
+            metadataNode.dropFeedPolicy(mdTxnCtx.getTxnId(), dataverseName, policyName);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -909,7 +909,7 @@
             throws AlgebricksException {
         List<FeedPolicyEntity> dataverseFeedPolicies;
         try {
-            dataverseFeedPolicies = metadataNode.getDataversePolicies(mdTxnCtx.getJobId(), dataverse);
+            dataverseFeedPolicies = metadataNode.getDataversePolicies(mdTxnCtx.getTxnId(), dataverse);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -921,7 +921,7 @@
             throws AlgebricksException {
         List<ExternalFile> externalFiles;
         try {
-            externalFiles = metadataNode.getExternalFiles(mdTxnCtx.getJobId(), dataset);
+            externalFiles = metadataNode.getExternalFiles(mdTxnCtx.getTxnId(), dataset);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -931,7 +931,7 @@
     @Override
     public void addExternalFile(MetadataTransactionContext ctx, ExternalFile externalFile) throws AlgebricksException {
         try {
-            metadataNode.addExternalFile(ctx.getJobId(), externalFile);
+            metadataNode.addExternalFile(ctx.getTxnId(), externalFile);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -940,7 +940,7 @@
     @Override
     public void dropExternalFile(MetadataTransactionContext ctx, ExternalFile externalFile) throws AlgebricksException {
         try {
-            metadataNode.dropExternalFile(ctx.getJobId(), externalFile.getDataverseName(),
+            metadataNode.dropExternalFile(ctx.getTxnId(), externalFile.getDataverseName(),
                     externalFile.getDatasetName(), externalFile.getFileNumber());
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
@@ -952,7 +952,7 @@
             Integer fileNumber) throws AlgebricksException {
         ExternalFile file;
         try {
-            file = metadataNode.getExternalFile(ctx.getJobId(), dataverseName, datasetName, fileNumber);
+            file = metadataNode.getExternalFile(ctx.getTxnId(), dataverseName, datasetName, fileNumber);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -964,7 +964,7 @@
     public void dropDatasetExternalFiles(MetadataTransactionContext mdTxnCtx, Dataset dataset)
             throws AlgebricksException {
         try {
-            metadataNode.dropExternalFiles(mdTxnCtx.getJobId(), dataset);
+            metadataNode.dropExternalFiles(mdTxnCtx.getTxnId(), dataset);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -973,7 +973,7 @@
     @Override
     public void updateDataset(MetadataTransactionContext ctx, Dataset dataset) throws AlgebricksException {
         try {
-            metadataNode.updateDataset(ctx.getJobId(), dataset);
+            metadataNode.updateDataset(ctx.getTxnId(), dataset);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -999,7 +999,7 @@
     public <T extends IExtensionMetadataEntity> void addEntity(MetadataTransactionContext mdTxnCtx, T entity)
             throws AlgebricksException {
         try {
-            metadataNode.addEntity(mdTxnCtx.getJobId(), entity);
+            metadataNode.addEntity(mdTxnCtx.getTxnId(), entity);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -1009,7 +1009,7 @@
     public <T extends IExtensionMetadataEntity> void upsertEntity(MetadataTransactionContext mdTxnCtx, T entity)
             throws AlgebricksException {
         try {
-            metadataNode.upsertEntity(mdTxnCtx.getJobId(), entity);
+            metadataNode.upsertEntity(mdTxnCtx.getTxnId(), entity);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -1019,7 +1019,7 @@
     public <T extends IExtensionMetadataEntity> void deleteEntity(MetadataTransactionContext mdTxnCtx, T entity)
             throws AlgebricksException {
         try {
-            metadataNode.deleteEntity(mdTxnCtx.getJobId(), entity);
+            metadataNode.deleteEntity(mdTxnCtx.getTxnId(), entity);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
@@ -1029,7 +1029,7 @@
     public <T extends IExtensionMetadataEntity> List<T> getEntities(MetadataTransactionContext mdTxnCtx,
             IExtensionMetadataSearchKey searchKey) throws AlgebricksException {
         try {
-            return metadataNode.getEntities(mdTxnCtx.getJobId(), searchKey);
+            return metadataNode.getEntities(mdTxnCtx.getTxnId(), searchKey);
         } catch (RemoteException e) {
             throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
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 3b084c9..87a5272 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
@@ -42,7 +42,7 @@
 import org.apache.asterix.common.transactions.ITransactionContext;
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
 import org.apache.asterix.common.transactions.ImmutableDatasetId;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.external.indexing.ExternalFile;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.metadata.api.ExtensionMetadataDataset;
@@ -164,22 +164,22 @@
     }
 
     @Override
-    public void beginTransaction(JobId transactionId) throws ACIDException, RemoteException {
+    public void beginTransaction(TxnId transactionId) throws ACIDException, RemoteException {
         ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().beginTransaction(transactionId);
         txnCtx.setMetadataTransaction(true);
     }
 
     @Override
-    public void commitTransaction(JobId jobId) throws RemoteException, ACIDException {
-        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
+    public void commitTransaction(TxnId txnId) throws RemoteException, ACIDException {
+        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(txnId, false);
         transactionSubsystem.getTransactionManager().commitTransaction(txnCtx, DatasetId.NULL, -1);
     }
 
     @Override
-    public void abortTransaction(JobId jobId) throws RemoteException, ACIDException {
+    public void abortTransaction(TxnId txnId) throws RemoteException, ACIDException {
         try {
             ITransactionContext txnCtx =
-                    transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
+                    transactionSubsystem.getTransactionManager().getTransactionContext(txnId, false);
             transactionSubsystem.getTransactionManager().abortTransaction(txnCtx, DatasetId.NULL, -1);
         } catch (ACIDException e) {
             LOGGER.log(Level.WARNING, "Exception aborting transaction", e);
@@ -188,14 +188,14 @@
     }
 
     @Override
-    public void lock(JobId jobId, byte lockMode) throws ACIDException, RemoteException {
-        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
+    public void lock(TxnId txnId, byte lockMode) throws ACIDException, RemoteException {
+        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(txnId, false);
         transactionSubsystem.getLockManager().lock(METADATA_DATASET_ID, -1, lockMode, txnCtx);
     }
 
     @Override
-    public void unlock(JobId jobId, byte lockMode) throws ACIDException, RemoteException {
-        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
+    public void unlock(TxnId txnId, byte lockMode) throws ACIDException, RemoteException {
+        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(txnId, false);
         transactionSubsystem.getLockManager().unlock(METADATA_DATASET_ID, -1, lockMode, txnCtx);
     }
 
@@ -203,17 +203,17 @@
     /**
      * Add entity to index
      *
-     * @param jobId
+     * @param txnId
      * @param entity
      * @param tupleTranslator
      * @param index
      * @throws AlgebricksException
      */
-    private <T> void addEntity(JobId jobId, T entity, IMetadataEntityTupleTranslator<T> tupleTranslator,
+    private <T> void addEntity(TxnId txnId, T entity, IMetadataEntityTupleTranslator<T> tupleTranslator,
             IMetadataIndex index) throws AlgebricksException {
         try {
             ITupleReference tuple = tupleTranslator.getTupleFromMetadataEntity(entity);
-            insertTupleIntoIndex(jobId, index, tuple);
+            insertTupleIntoIndex(txnId, index, tuple);
         } catch (HyracksDataException | ACIDException e) {
             throw new AlgebricksException(e);
         }
@@ -222,17 +222,17 @@
     /**
      * Upsert entity to index
      *
-     * @param jobId
+     * @param txnId
      * @param entity
      * @param tupleTranslator
      * @param index
      * @throws AlgebricksException
      */
-    private <T> void upsertEntity(JobId jobId, T entity, IMetadataEntityTupleTranslator<T> tupleTranslator,
+    private <T> void upsertEntity(TxnId txnId, T entity, IMetadataEntityTupleTranslator<T> tupleTranslator,
             IMetadataIndex index) throws AlgebricksException {
         try {
             ITupleReference tuple = tupleTranslator.getTupleFromMetadataEntity(entity);
-            upsertTupleIntoIndex(jobId, index, tuple);
+            upsertTupleIntoIndex(txnId, index, tuple);
         } catch (HyracksDataException | ACIDException e) {
             throw new AlgebricksException(e);
         }
@@ -241,17 +241,17 @@
     /**
      * Delete entity from index
      *
-     * @param jobId
+     * @param txnId
      * @param entity
      * @param tupleTranslator
      * @param index
      * @throws AlgebricksException
      */
-    private <T> void deleteEntity(JobId jobId, T entity, IMetadataEntityTupleTranslator<T> tupleTranslator,
+    private <T> void deleteEntity(TxnId txnId, T entity, IMetadataEntityTupleTranslator<T> tupleTranslator,
             IMetadataIndex index) throws AlgebricksException {
         try {
             ITupleReference tuple = tupleTranslator.getTupleFromMetadataEntity(entity);
-            deleteTupleFromIndex(jobId, index, tuple);
+            deleteTupleFromIndex(txnId, index, tuple);
         } catch (HyracksDataException | ACIDException e) {
             throw new AlgebricksException(e);
         }
@@ -260,7 +260,7 @@
     /**
      * retrieve all entities that matches the searchKey
      *
-     * @param jobId
+     * @param txnId
      * @param searchKey
      * @param tupleTranslator
      * @param index
@@ -268,13 +268,13 @@
      * @throws AlgebricksException
      * @throws RemoteException
      */
-    private <T> List<T> getEntities(JobId jobId, ITupleReference searchKey,
+    private <T> List<T> getEntities(TxnId txnId, ITupleReference searchKey,
             IMetadataEntityTupleTranslator<T> tupleTranslator, IMetadataIndex index)
             throws AlgebricksException, RemoteException {
         try {
             IValueExtractor<T> valueExtractor = new MetadataEntityValueExtractor<>(tupleTranslator);
             List<T> results = new ArrayList<>();
-            searchIndex(jobId, index, searchKey, valueExtractor, results);
+            searchIndex(txnId, index, searchKey, valueExtractor, results);
             return results;
         } catch (HyracksDataException e) {
             throw new AlgebricksException(e);
@@ -283,43 +283,43 @@
 
     @SuppressWarnings("unchecked")
     @Override
-    public <T extends IExtensionMetadataEntity> void addEntity(JobId jobId, T entity)
+    public <T extends IExtensionMetadataEntity> void addEntity(TxnId txnId, T entity)
             throws AlgebricksException, RemoteException {
         ExtensionMetadataDataset<T> index = (ExtensionMetadataDataset<T>) extensionDatasets.get(entity.getDatasetId());
         if (index == null) {
             throw new AlgebricksException("Metadata Extension Index: " + entity.getDatasetId() + " was not found");
         }
         IMetadataEntityTupleTranslator<T> tupleTranslator = index.getTupleTranslator();
-        addEntity(jobId, entity, tupleTranslator, index);
+        addEntity(txnId, entity, tupleTranslator, index);
     }
 
     @SuppressWarnings("unchecked")
     @Override
-    public <T extends IExtensionMetadataEntity> void upsertEntity(JobId jobId, T entity)
+    public <T extends IExtensionMetadataEntity> void upsertEntity(TxnId txnId, T entity)
             throws AlgebricksException, RemoteException {
         ExtensionMetadataDataset<T> index = (ExtensionMetadataDataset<T>) extensionDatasets.get(entity.getDatasetId());
         if (index == null) {
             throw new AlgebricksException("Metadata Extension Index: " + entity.getDatasetId() + " was not found");
         }
         IMetadataEntityTupleTranslator<T> tupleTranslator = index.getTupleTranslator();
-        upsertEntity(jobId, entity, tupleTranslator, index);
+        upsertEntity(txnId, entity, tupleTranslator, index);
     }
 
     @SuppressWarnings("unchecked")
     @Override
-    public <T extends IExtensionMetadataEntity> void deleteEntity(JobId jobId, T entity)
+    public <T extends IExtensionMetadataEntity> void deleteEntity(TxnId txnId, T entity)
             throws AlgebricksException, RemoteException {
         ExtensionMetadataDataset<T> index = (ExtensionMetadataDataset<T>) extensionDatasets.get(entity.getDatasetId());
         if (index == null) {
             throw new AlgebricksException("Metadata Extension Index: " + entity.getDatasetId() + " was not found");
         }
         IMetadataEntityTupleTranslator<T> tupleTranslator = index.getTupleTranslator();
-        deleteEntity(jobId, entity, tupleTranslator, index);
+        deleteEntity(txnId, entity, tupleTranslator, index);
     }
 
     @SuppressWarnings("unchecked")
     @Override
-    public <T extends IExtensionMetadataEntity> List<T> getEntities(JobId jobId, IExtensionMetadataSearchKey searchKey)
+    public <T extends IExtensionMetadataEntity> List<T> getEntities(TxnId txnId, IExtensionMetadataSearchKey searchKey)
             throws AlgebricksException, RemoteException {
         ExtensionMetadataDataset<T> index =
                 (ExtensionMetadataDataset<T>) extensionDatasets.get(searchKey.getDatasetId());
@@ -327,15 +327,15 @@
             throw new AlgebricksException("Metadata Extension Index: " + searchKey.getDatasetId() + " was not found");
         }
         IMetadataEntityTupleTranslator<T> tupleTranslator = index.getTupleTranslator();
-        return getEntities(jobId, searchKey.getSearchKey(), tupleTranslator, index);
+        return getEntities(txnId, searchKey.getSearchKey(), tupleTranslator, index);
     }
 
     @Override
-    public void addDataverse(JobId jobId, Dataverse dataverse) throws AlgebricksException, RemoteException {
+    public void addDataverse(TxnId txnId, Dataverse dataverse) throws AlgebricksException, RemoteException {
         try {
             DataverseTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDataverseTupleTranslator(true);
             ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(dataverse);
-            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, tuple);
+            insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.DATAVERSE_DATASET, tuple);
         } catch (HyracksDataException e) {
             if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
                 throw new AlgebricksException(
@@ -349,12 +349,12 @@
     }
 
     @Override
-    public void addDataset(JobId jobId, Dataset dataset) throws AlgebricksException, RemoteException {
+    public void addDataset(TxnId txnId, Dataset dataset) throws AlgebricksException, RemoteException {
         try {
             // Insert into the 'dataset' dataset.
             DatasetTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDatasetTupleTranslator(true);
             ITupleReference datasetTuple = tupleReaderWriter.getTupleFromMetadataEntity(dataset);
-            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
+            insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
 
             if (dataset.getDatasetType() == DatasetType.INTERNAL) {
                 // Add the primary index for the dataset.
@@ -363,7 +363,7 @@
                         dataset.getDatasetName(), IndexType.BTREE, id.getPrimaryKey(), id.getKeySourceIndicator(),
                         id.getPrimaryKeyType(), false, false, true, dataset.getPendingOp());
 
-                addIndex(jobId, primaryIndex);
+                addIndex(txnId, primaryIndex);
             }
         } catch (HyracksDataException e) {
             if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
@@ -378,11 +378,11 @@
     }
 
     @Override
-    public void addIndex(JobId jobId, Index index) throws AlgebricksException, RemoteException {
+    public void addIndex(TxnId txnId, Index index) throws AlgebricksException, RemoteException {
         try {
-            IndexTupleTranslator tupleWriter = tupleTranslatorProvider.getIndexTupleTranslator(jobId, this, true);
+            IndexTupleTranslator tupleWriter = tupleTranslatorProvider.getIndexTupleTranslator(txnId, this, true);
             ITupleReference tuple = tupleWriter.getTupleFromMetadataEntity(index);
-            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, tuple);
+            insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.INDEX_DATASET, tuple);
         } catch (HyracksDataException e) {
             if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
                 throw new AlgebricksException("An index with name '" + index.getIndexName() + "' already exists.", e);
@@ -395,11 +395,11 @@
     }
 
     @Override
-    public void addNode(JobId jobId, Node node) throws AlgebricksException, RemoteException {
+    public void addNode(TxnId txnId, Node node) throws AlgebricksException, RemoteException {
         try {
             NodeTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getNodeTupleTranslator(true);
             ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(node);
-            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.NODE_DATASET, tuple);
+            insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.NODE_DATASET, tuple);
         } catch (HyracksDataException e) {
             if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
                 throw new AlgebricksException("A node with name '" + node.getNodeName() + "' already exists.", e);
@@ -412,11 +412,11 @@
     }
 
     @Override
-    public void addNodeGroup(JobId jobId, NodeGroup nodeGroup) throws AlgebricksException, RemoteException {
+    public void addNodeGroup(TxnId txnId, NodeGroup nodeGroup) throws AlgebricksException, RemoteException {
         try {
             NodeGroupTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getNodeGroupTupleTranslator(true);
             ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(nodeGroup);
-            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, tuple);
+            insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.NODEGROUP_DATASET, tuple);
         } catch (HyracksDataException e) {
             if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
                 throw new AlgebricksException(
@@ -430,12 +430,12 @@
     }
 
     @Override
-    public void addDatatype(JobId jobId, Datatype datatype) throws AlgebricksException, RemoteException {
+    public void addDatatype(TxnId txnId, Datatype datatype) throws AlgebricksException, RemoteException {
         try {
             DatatypeTupleTranslator tupleReaderWriter =
-                    tupleTranslatorProvider.getDataTypeTupleTranslator(jobId, this, true);
+                    tupleTranslatorProvider.getDataTypeTupleTranslator(txnId, this, true);
             ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(datatype);
-            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
+            insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
         } catch (HyracksDataException e) {
             if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
                 throw new AlgebricksException(
@@ -449,13 +449,13 @@
     }
 
     @Override
-    public void addFunction(JobId jobId, Function function) throws AlgebricksException, RemoteException {
+    public void addFunction(TxnId txnId, Function function) throws AlgebricksException, RemoteException {
         try {
             // Insert into the 'function' dataset.
             FunctionTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFunctionTupleTranslator(true);
 
             ITupleReference functionTuple = tupleReaderWriter.getTupleFromMetadataEntity(function);
-            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, functionTuple);
+            insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.FUNCTION_DATASET, functionTuple);
 
         } catch (HyracksDataException e) {
             if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
@@ -470,7 +470,7 @@
         }
     }
 
-    private void insertTupleIntoIndex(JobId jobId, IMetadataIndex metadataIndex, ITupleReference tuple)
+    private void insertTupleIntoIndex(TxnId txnId, IMetadataIndex metadataIndex, ITupleReference tuple)
             throws ACIDException, HyracksDataException {
         long resourceID = metadataIndex.getResourceId();
         String resourceName = metadataIndex.getFile().getRelativePath();
@@ -480,13 +480,13 @@
 
             // prepare a Callback for logging
             IModificationOperationCallback modCallback =
-                    createIndexModificationCallback(jobId, resourceID, metadataIndex, lsmIndex, Operation.INSERT);
+                    createIndexModificationCallback(txnId, resourceID, metadataIndex, lsmIndex, Operation.INSERT);
 
             IIndexAccessParameters iap = new IndexAccessParameters(modCallback, NoOpOperationCallback.INSTANCE);
             ILSMIndexAccessor indexAccessor = lsmIndex.createAccessor(iap);
 
             ITransactionContext txnCtx =
-                    transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
+                    transactionSubsystem.getTransactionManager().getTransactionContext(txnId, false);
             txnCtx.setWriteTxn(true);
             txnCtx.registerIndexAndCallback(resourceID, lsmIndex, (AbstractOperationCallback) modCallback,
                     metadataIndex.isPrimaryIndex());
@@ -511,7 +511,7 @@
         }
     }
 
-    private void upsertTupleIntoIndex(JobId jobId, IMetadataIndex metadataIndex, ITupleReference tuple)
+    private void upsertTupleIntoIndex(TxnId txnId, IMetadataIndex metadataIndex, ITupleReference tuple)
             throws ACIDException, HyracksDataException {
         long resourceId = metadataIndex.getResourceId();
         String resourceName = metadataIndex.getFile().getRelativePath();
@@ -520,7 +520,7 @@
         try {
             // prepare a Callback for logging
             ITransactionContext txnCtx =
-                    transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
+                    transactionSubsystem.getTransactionManager().getTransactionContext(txnId, false);
             IModificationOperationCallback modCallback =
                     new UpsertOperationCallback(metadataIndex.getDatasetId(), metadataIndex.getPrimaryKeyIndexes(),
                             txnCtx, transactionSubsystem.getLockManager(), transactionSubsystem, resourceId,
@@ -548,9 +548,9 @@
         }
     }
 
-    private IModificationOperationCallback createIndexModificationCallback(JobId jobId, long resourceId,
+    private IModificationOperationCallback createIndexModificationCallback(TxnId txnId, long resourceId,
             IMetadataIndex metadataIndex, ILSMIndex lsmIndex, Operation indexOp) throws ACIDException {
-        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
+        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(txnId, false);
 
         // Regardless of the index type (primary or secondary index), secondary index
         // modification callback is given
@@ -565,65 +565,65 @@
     }
 
     @Override
-    public void dropDataverse(JobId jobId, String dataverseName) throws AlgebricksException, RemoteException {
+    public void dropDataverse(TxnId txnId, String dataverseName) throws AlgebricksException, RemoteException {
         try {
 
-            confirmDataverseCanBeDeleted(jobId, dataverseName);
+            confirmDataverseCanBeDeleted(txnId, dataverseName);
 
             List<Dataset> dataverseDatasets;
             Dataset ds;
-            dataverseDatasets = getDataverseDatasets(jobId, dataverseName);
+            dataverseDatasets = getDataverseDatasets(txnId, dataverseName);
             // Drop all datasets in this dataverse.
             for (int i = 0; i < dataverseDatasets.size(); i++) {
                 ds = dataverseDatasets.get(i);
-                dropDataset(jobId, dataverseName, ds.getDatasetName());
+                dropDataset(txnId, dataverseName, ds.getDatasetName());
             }
 
             // After dropping datasets, drop datatypes
             List<Datatype> dataverseDatatypes;
             // As a side effect, acquires an S lock on the 'datatype' dataset
             // on behalf of txnId.
-            dataverseDatatypes = getDataverseDatatypes(jobId, dataverseName);
+            dataverseDatatypes = getDataverseDatatypes(txnId, dataverseName);
             // Drop all types in this dataverse.
             for (int i = 0; i < dataverseDatatypes.size(); i++) {
-                forceDropDatatype(jobId, dataverseName, dataverseDatatypes.get(i).getDatatypeName());
+                forceDropDatatype(txnId, dataverseName, dataverseDatatypes.get(i).getDatatypeName());
             }
 
             // As a side effect, acquires an S lock on the 'Function' dataset
             // on behalf of txnId.
-            List<Function> dataverseFunctions = getDataverseFunctions(jobId, dataverseName);
+            List<Function> dataverseFunctions = getDataverseFunctions(txnId, dataverseName);
             // Drop all functions in this dataverse.
             for (Function function : dataverseFunctions) {
-                dropFunction(jobId, new FunctionSignature(dataverseName, function.getName(), function.getArity()));
+                dropFunction(txnId, new FunctionSignature(dataverseName, function.getName(), function.getArity()));
             }
 
             // As a side effect, acquires an S lock on the 'Adapter' dataset
             // on behalf of txnId.
-            List<DatasourceAdapter> dataverseAdapters = getDataverseAdapters(jobId, dataverseName);
+            List<DatasourceAdapter> dataverseAdapters = getDataverseAdapters(txnId, dataverseName);
             // Drop all functions in this dataverse.
             for (DatasourceAdapter adapter : dataverseAdapters) {
-                dropAdapter(jobId, dataverseName, adapter.getAdapterIdentifier().getName());
+                dropAdapter(txnId, dataverseName, adapter.getAdapterIdentifier().getName());
             }
 
             List<Feed> dataverseFeeds;
             List<FeedConnection> feedConnections;
             Feed feed;
-            dataverseFeeds = getDataverseFeeds(jobId, dataverseName);
+            dataverseFeeds = getDataverseFeeds(txnId, dataverseName);
             // Drop all feeds&connections in this dataverse.
             for (int i = 0; i < dataverseFeeds.size(); i++) {
                 feed = dataverseFeeds.get(i);
-                feedConnections = getFeedConnections(jobId, dataverseName, feed.getFeedName());
+                feedConnections = getFeedConnections(txnId, dataverseName, feed.getFeedName());
                 for (FeedConnection feedConnection : feedConnections) {
-                    dropFeedConnection(jobId, dataverseName, feed.getFeedName(), feedConnection.getDatasetName());
+                    dropFeedConnection(txnId, dataverseName, feed.getFeedName(), feedConnection.getDatasetName());
                 }
-                dropFeed(jobId, dataverseName, feed.getFeedName());
+                dropFeed(txnId, dataverseName, feed.getFeedName());
             }
 
-            List<FeedPolicyEntity> feedPolicies = getDataversePolicies(jobId, dataverseName);
+            List<FeedPolicyEntity> feedPolicies = getDataversePolicies(txnId, dataverseName);
             if (feedPolicies != null && feedPolicies.size() > 0) {
                 // Drop all feed ingestion policies in this dataverse.
                 for (FeedPolicyEntity feedPolicy : feedPolicies) {
-                    dropFeedPolicy(jobId, dataverseName, feedPolicy.getPolicyName());
+                    dropFeedPolicy(txnId, dataverseName, feedPolicy.getPolicyName());
                 }
             }
 
@@ -631,8 +631,8 @@
             ITupleReference searchKey = createTuple(dataverseName);
             // As a side effect, acquires an S lock on the 'dataverse' dataset
             // on behalf of txnId.
-            ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, searchKey);
-            deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, tuple);
+            ITupleReference tuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.DATAVERSE_DATASET, searchKey);
+            deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.DATAVERSE_DATASET, tuple);
 
             // TODO: Change this to be a BTree specific exception, e.g.,
             // BTreeKeyDoesNotExistException.
@@ -650,9 +650,9 @@
     }
 
     @Override
-    public void dropDataset(JobId jobId, String dataverseName, String datasetName)
+    public void dropDataset(TxnId txnId, String dataverseName, String datasetName)
             throws AlgebricksException, RemoteException {
-        Dataset dataset = getDataset(jobId, dataverseName, datasetName);
+        Dataset dataset = getDataset(txnId, dataverseName, datasetName);
         if (dataset == null) {
             throw new AlgebricksException("Cannot drop dataset '" + datasetName + "' because it doesn't exist.");
         }
@@ -663,13 +663,13 @@
             // lock on the 'dataset' dataset.
             ITupleReference datasetTuple = null;
             try {
-                datasetTuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey);
+                datasetTuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey);
 
                 // Delete entry(s) from the 'indexes' dataset.
-                List<Index> datasetIndexes = getDatasetIndexes(jobId, dataverseName, datasetName);
+                List<Index> datasetIndexes = getDatasetIndexes(txnId, dataverseName, datasetName);
                 if (datasetIndexes != null) {
                     for (Index index : datasetIndexes) {
-                        dropIndex(jobId, dataverseName, datasetName, index.getIndexName());
+                        dropIndex(txnId, dataverseName, datasetName, index.getIndexName());
                     }
                 }
 
@@ -677,11 +677,11 @@
                     // Delete External Files
                     // As a side effect, acquires an S lock on the 'ExternalFile' dataset
                     // on behalf of txnId.
-                    List<ExternalFile> datasetFiles = getExternalFiles(jobId, dataset);
+                    List<ExternalFile> datasetFiles = getExternalFiles(txnId, dataset);
                     if (datasetFiles != null && datasetFiles.size() > 0) {
                         // Drop all external files in this dataset.
                         for (ExternalFile file : datasetFiles) {
-                            dropExternalFile(jobId, dataverseName, file.getDatasetName(), file.getFileNumber());
+                            dropExternalFile(txnId, dataverseName, file.getDatasetName(), file.getFileNumber());
                         }
                     }
                 }
@@ -693,7 +693,7 @@
                     throw new AlgebricksException(hde);
                 }
             } finally {
-                deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
+                deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
             }
         } catch (HyracksDataException | ACIDException e) {
             throw new AlgebricksException(e);
@@ -701,14 +701,14 @@
     }
 
     @Override
-    public void dropIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
+    public void dropIndex(TxnId txnId, String dataverseName, String datasetName, String indexName)
             throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName, datasetName, indexName);
             // Searches the index for the tuple to be deleted. Acquires an S
             // lock on the 'index' dataset.
-            ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey);
-            deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, tuple);
+            ITupleReference tuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey);
+            deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.INDEX_DATASET, tuple);
             // TODO: Change this to be a BTree specific exception, e.g.,
             // BTreeKeyDoesNotExistException.
         } catch (HyracksDataException e) {
@@ -725,9 +725,9 @@
     }
 
     @Override
-    public boolean dropNodegroup(JobId jobId, String nodeGroupName, boolean failSilently)
+    public boolean dropNodegroup(TxnId txnId, String nodeGroupName, boolean failSilently)
             throws AlgebricksException, RemoteException {
-        List<String> datasetNames = getDatasetNamesPartitionedOnThisNodeGroup(jobId, nodeGroupName);
+        List<String> datasetNames = getDatasetNamesPartitionedOnThisNodeGroup(txnId, nodeGroupName);
         if (!datasetNames.isEmpty()) {
             if (failSilently) {
                 return false;
@@ -744,8 +744,8 @@
             ITupleReference searchKey = createTuple(nodeGroupName);
             // Searches the index for the tuple to be deleted. Acquires an S
             // lock on the 'nodegroup' dataset.
-            ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, searchKey);
-            deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, tuple);
+            ITupleReference tuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.NODEGROUP_DATASET, searchKey);
+            deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.NODEGROUP_DATASET, tuple);
             // TODO: Change this to be a BTree specific exception, e.g.,
             // BTreeKeyDoesNotExistException.
             return true;
@@ -763,24 +763,24 @@
     }
 
     @Override
-    public void dropDatatype(JobId jobId, String dataverseName, String datatypeName)
+    public void dropDatatype(TxnId txnId, String dataverseName, String datatypeName)
             throws AlgebricksException, RemoteException {
 
-        confirmDatatypeIsUnused(jobId, dataverseName, datatypeName);
+        confirmDatatypeIsUnused(txnId, dataverseName, datatypeName);
 
         // Delete the datatype entry, including all it's nested anonymous types.
         try {
             ITupleReference searchKey = createTuple(dataverseName, datatypeName);
             // Searches the index for the tuple to be deleted. Acquires an S
             // lock on the 'datatype' dataset.
-            ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey);
+            ITupleReference tuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey);
             // Get nested types
-            List<String> nestedTypes = getNestedComplexDatatypeNamesForThisDatatype(jobId, dataverseName, datatypeName);
-            deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
+            List<String> nestedTypes = getNestedComplexDatatypeNamesForThisDatatype(txnId, dataverseName, datatypeName);
+            deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
             for (String nestedType : nestedTypes) {
-                Datatype dt = getDatatype(jobId, dataverseName, nestedType);
+                Datatype dt = getDatatype(txnId, dataverseName, nestedType);
                 if (dt != null && dt.getIsAnonymous()) {
-                    dropDatatype(jobId, dataverseName, dt.getDatatypeName());
+                    dropDatatype(txnId, dataverseName, dt.getDatatypeName());
                 }
             }
 
@@ -798,14 +798,14 @@
         }
     }
 
-    private void forceDropDatatype(JobId jobId, String dataverseName, String datatypeName)
+    private void forceDropDatatype(TxnId txnId, String dataverseName, String datatypeName)
             throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName, datatypeName);
             // Searches the index for the tuple to be deleted. Acquires an S
             // lock on the 'datatype' dataset.
-            ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey);
-            deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
+            ITupleReference tuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey);
+            deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
             // TODO: Change this to be a BTree specific exception, e.g.,
             // BTreeKeyDoesNotExistException.
         } catch (HyracksDataException e) {
@@ -820,7 +820,7 @@
         }
     }
 
-    private void deleteTupleFromIndex(JobId jobId, IMetadataIndex metadataIndex, ITupleReference tuple)
+    private void deleteTupleFromIndex(TxnId txnId, IMetadataIndex metadataIndex, ITupleReference tuple)
             throws ACIDException, HyracksDataException {
         long resourceID = metadataIndex.getResourceId();
         String resourceName = metadataIndex.getFile().getRelativePath();
@@ -829,12 +829,12 @@
             datasetLifecycleManager.open(resourceName);
             // prepare a Callback for logging
             IModificationOperationCallback modCallback =
-                    createIndexModificationCallback(jobId, resourceID, metadataIndex, lsmIndex, Operation.DELETE);
+                    createIndexModificationCallback(txnId, resourceID, metadataIndex, lsmIndex, Operation.DELETE);
             IIndexAccessParameters iap = new IndexAccessParameters(modCallback, NoOpOperationCallback.INSTANCE);
             ILSMIndexAccessor indexAccessor = lsmIndex.createAccessor(iap);
 
             ITransactionContext txnCtx =
-                    transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
+                    transactionSubsystem.getTransactionManager().getTransactionContext(txnId, false);
             txnCtx.setWriteTxn(true);
             txnCtx.registerIndexAndCallback(resourceID, lsmIndex, (AbstractOperationCallback) modCallback,
                     metadataIndex.isPrimaryIndex());
@@ -859,12 +859,12 @@
     }
 
     @Override
-    public List<Dataverse> getDataverses(JobId jobId) throws AlgebricksException, RemoteException {
+    public List<Dataverse> getDataverses(TxnId txnId) throws AlgebricksException, RemoteException {
         try {
             DataverseTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDataverseTupleTranslator(false);
             IValueExtractor<Dataverse> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
             List<Dataverse> results = new ArrayList<>();
-            searchIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, null, valueExtractor, results);
+            searchIndex(txnId, MetadataPrimaryIndexes.DATAVERSE_DATASET, null, valueExtractor, results);
             return results;
         } catch (HyracksDataException e) {
             throw new AlgebricksException(e);
@@ -872,13 +872,13 @@
     }
 
     @Override
-    public Dataverse getDataverse(JobId jobId, String dataverseName) throws AlgebricksException, RemoteException {
+    public Dataverse getDataverse(TxnId txnId, String dataverseName) throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName);
             DataverseTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDataverseTupleTranslator(false);
             IValueExtractor<Dataverse> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
             List<Dataverse> results = new ArrayList<>();
-            searchIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, searchKey, valueExtractor, results);
+            searchIndex(txnId, MetadataPrimaryIndexes.DATAVERSE_DATASET, searchKey, valueExtractor, results);
             if (results.isEmpty()) {
                 return null;
             }
@@ -889,14 +889,14 @@
     }
 
     @Override
-    public List<Dataset> getDataverseDatasets(JobId jobId, String dataverseName)
+    public List<Dataset> getDataverseDatasets(TxnId txnId, String dataverseName)
             throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName);
             DatasetTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDatasetTupleTranslator(false);
             IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
             List<Dataset> results = new ArrayList<>();
-            searchIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey, valueExtractor, results);
+            searchIndex(txnId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey, valueExtractor, results);
             return results;
         } catch (HyracksDataException e) {
             throw new AlgebricksException(e);
@@ -904,13 +904,13 @@
     }
 
     @Override
-    public List<Feed> getDataverseFeeds(JobId jobId, String dataverseName) throws AlgebricksException, RemoteException {
+    public List<Feed> getDataverseFeeds(TxnId txnId, String dataverseName) throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName);
             FeedTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFeedTupleTranslator(false);
             IValueExtractor<Feed> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
             List<Feed> results = new ArrayList<>();
-            searchIndex(jobId, MetadataPrimaryIndexes.FEED_DATASET, searchKey, valueExtractor, results);
+            searchIndex(txnId, MetadataPrimaryIndexes.FEED_DATASET, searchKey, valueExtractor, results);
             return results;
         } catch (HyracksDataException e) {
             throw new AlgebricksException(e);
@@ -918,29 +918,29 @@
     }
 
     @Override
-    public List<Library> getDataverseLibraries(JobId jobId, String dataverseName)
+    public List<Library> getDataverseLibraries(TxnId txnId, String dataverseName)
             throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName);
             LibraryTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getLibraryTupleTranslator(false);
             IValueExtractor<Library> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
             List<Library> results = new ArrayList<>();
-            searchIndex(jobId, MetadataPrimaryIndexes.LIBRARY_DATASET, searchKey, valueExtractor, results);
+            searchIndex(txnId, MetadataPrimaryIndexes.LIBRARY_DATASET, searchKey, valueExtractor, results);
             return results;
         } catch (HyracksDataException e) {
             throw new AlgebricksException(e);
         }
     }
 
-    private List<Datatype> getDataverseDatatypes(JobId jobId, String dataverseName)
+    private List<Datatype> getDataverseDatatypes(TxnId txnId, String dataverseName)
             throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName);
             DatatypeTupleTranslator tupleReaderWriter =
-                    tupleTranslatorProvider.getDataTypeTupleTranslator(jobId, this, false);
+                    tupleTranslatorProvider.getDataTypeTupleTranslator(txnId, this, false);
             IValueExtractor<Datatype> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
             List<Datatype> results = new ArrayList<>();
-            searchIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey, valueExtractor, results);
+            searchIndex(txnId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey, valueExtractor, results);
             return results;
         } catch (HyracksDataException e) {
             throw new AlgebricksException(e);
@@ -948,14 +948,14 @@
     }
 
     @Override
-    public Dataset getDataset(JobId jobId, String dataverseName, String datasetName)
+    public Dataset getDataset(TxnId txnId, String dataverseName, String datasetName)
             throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName, datasetName);
             DatasetTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDatasetTupleTranslator(false);
             List<Dataset> results = new ArrayList<>();
             IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
-            searchIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey, valueExtractor, results);
+            searchIndex(txnId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey, valueExtractor, results);
             if (results.isEmpty()) {
                 return null;
             }
@@ -965,39 +965,39 @@
         }
     }
 
-    public List<Dataset> getAllDatasets(JobId jobId) throws AlgebricksException, RemoteException {
+    public List<Dataset> getAllDatasets(TxnId txnId) throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = null;
             DatasetTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDatasetTupleTranslator(false);
             IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
             List<Dataset> results = new ArrayList<>();
-            searchIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey, valueExtractor, results);
+            searchIndex(txnId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey, valueExtractor, results);
             return results;
         } catch (HyracksDataException e) {
             throw new AlgebricksException(e);
         }
     }
 
-    public List<Datatype> getAllDatatypes(JobId jobId) throws AlgebricksException, RemoteException {
+    public List<Datatype> getAllDatatypes(TxnId txnId) throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = null;
             DatatypeTupleTranslator tupleReaderWriter =
-                    tupleTranslatorProvider.getDataTypeTupleTranslator(jobId, this, false);
+                    tupleTranslatorProvider.getDataTypeTupleTranslator(txnId, this, false);
             IValueExtractor<Datatype> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
             List<Datatype> results = new ArrayList<>();
-            searchIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey, valueExtractor, results);
+            searchIndex(txnId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey, valueExtractor, results);
             return results;
         } catch (HyracksDataException e) {
             throw new AlgebricksException(e);
         }
     }
 
-    private void confirmDataverseCanBeDeleted(JobId jobId, String dataverseName)
+    private void confirmDataverseCanBeDeleted(TxnId txnId, String dataverseName)
             throws AlgebricksException, RemoteException {
         // If a dataset from a DIFFERENT dataverse
         // uses a type from this dataverse
         // throw an error
-        List<Dataset> datasets = getAllDatasets(jobId);
+        List<Dataset> datasets = getAllDatasets(txnId);
         for (Dataset set : datasets) {
             if (set.getDataverseName().equals(dataverseName)) {
                 continue;
@@ -1010,16 +1010,16 @@
         }
     }
 
-    private void confirmDatatypeIsUnused(JobId jobId, String dataverseName, String datatypeName)
+    private void confirmDatatypeIsUnused(TxnId txnId, String dataverseName, String datatypeName)
             throws AlgebricksException, RemoteException {
-        confirmDatatypeIsUnusedByDatatypes(jobId, dataverseName, datatypeName);
-        confirmDatatypeIsUnusedByDatasets(jobId, dataverseName, datatypeName);
+        confirmDatatypeIsUnusedByDatatypes(txnId, dataverseName, datatypeName);
+        confirmDatatypeIsUnusedByDatasets(txnId, dataverseName, datatypeName);
     }
 
-    private void confirmDatatypeIsUnusedByDatasets(JobId jobId, String dataverseName, String datatypeName)
+    private void confirmDatatypeIsUnusedByDatasets(TxnId txnId, String dataverseName, String datatypeName)
             throws AlgebricksException, RemoteException {
         // If any dataset uses this type, throw an error
-        List<Dataset> datasets = getAllDatasets(jobId);
+        List<Dataset> datasets = getAllDatasets(txnId);
         for (Dataset set : datasets) {
             if (set.getItemTypeName().equals(datatypeName) && set.getItemTypeDataverseName().equals(dataverseName)) {
                 throw new AlgebricksException("Cannot drop type " + dataverseName + "." + datatypeName
@@ -1028,15 +1028,15 @@
         }
     }
 
-    private void confirmDatatypeIsUnusedByDatatypes(JobId jobId, String dataverseName, String datatypeName)
+    private void confirmDatatypeIsUnusedByDatatypes(TxnId txnId, String dataverseName, String datatypeName)
             throws AlgebricksException, RemoteException {
         // If any datatype uses this type, throw an error
         // TODO: Currently this loads all types into memory. This will need to be fixed
         // for large numbers of types
-        Datatype dataTypeToBeDropped = getDatatype(jobId, dataverseName, datatypeName);
+        Datatype dataTypeToBeDropped = getDatatype(txnId, dataverseName, datatypeName);
         assert dataTypeToBeDropped != null;
         IAType typeToBeDropped = dataTypeToBeDropped.getDatatype();
-        List<Datatype> datatypes = getAllDatatypes(jobId);
+        List<Datatype> datatypes = getAllDatatypes(txnId);
         for (Datatype dataType : datatypes) {
             // skip types in different dataverses as well as the type to be dropped itself
             if (!dataType.getDataverseName().equals(dataverseName)
@@ -1052,10 +1052,10 @@
         }
     }
 
-    private List<String> getNestedComplexDatatypeNamesForThisDatatype(JobId jobId, String dataverseName,
+    private List<String> getNestedComplexDatatypeNamesForThisDatatype(TxnId txnId, String dataverseName,
             String datatypeName) throws AlgebricksException, RemoteException {
         // Return all field types that aren't builtin types
-        Datatype parentType = getDatatype(jobId, dataverseName, datatypeName);
+        Datatype parentType = getDatatype(txnId, dataverseName, datatypeName);
 
         List<IAType> subTypes = null;
         if (parentType.getDatatype().getTypeTag() == ATypeTag.OBJECT) {
@@ -1077,12 +1077,12 @@
         return nestedTypes;
     }
 
-    public List<String> getDatasetNamesPartitionedOnThisNodeGroup(JobId jobId, String nodegroup)
+    public List<String> getDatasetNamesPartitionedOnThisNodeGroup(TxnId txnId, String nodegroup)
             throws AlgebricksException, RemoteException {
         // this needs to scan the datasets and return the datasets that use this
         // nodegroup
         List<String> nodeGroupDatasets = new ArrayList<>();
-        List<Dataset> datasets = getAllDatasets(jobId);
+        List<Dataset> datasets = getAllDatasets(txnId);
         for (Dataset set : datasets) {
             if (set.getNodeGroupName().equals(nodegroup)) {
                 nodeGroupDatasets.add(set.getDatasetName());
@@ -1093,15 +1093,15 @@
     }
 
     @Override
-    public Index getIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
+    public Index getIndex(TxnId txnId, String dataverseName, String datasetName, String indexName)
             throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName, datasetName, indexName);
             IndexTupleTranslator tupleReaderWriter =
-                    tupleTranslatorProvider.getIndexTupleTranslator(jobId, this, false);
+                    tupleTranslatorProvider.getIndexTupleTranslator(txnId, this, false);
             IValueExtractor<Index> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
             List<Index> results = new ArrayList<>();
-            searchIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey, valueExtractor, results);
+            searchIndex(txnId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey, valueExtractor, results);
             if (results.isEmpty()) {
                 return null;
             }
@@ -1112,15 +1112,15 @@
     }
 
     @Override
-    public List<Index> getDatasetIndexes(JobId jobId, String dataverseName, String datasetName)
+    public List<Index> getDatasetIndexes(TxnId txnId, String dataverseName, String datasetName)
             throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName, datasetName);
             IndexTupleTranslator tupleReaderWriter =
-                    tupleTranslatorProvider.getIndexTupleTranslator(jobId, this, false);
+                    tupleTranslatorProvider.getIndexTupleTranslator(txnId, this, false);
             IValueExtractor<Index> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
             List<Index> results = new ArrayList<>();
-            searchIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey, valueExtractor, results);
+            searchIndex(txnId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey, valueExtractor, results);
             return results;
         } catch (HyracksDataException e) {
             throw new AlgebricksException(e);
@@ -1128,15 +1128,15 @@
     }
 
     @Override
-    public Datatype getDatatype(JobId jobId, String dataverseName, String datatypeName)
+    public Datatype getDatatype(TxnId txnId, String dataverseName, String datatypeName)
             throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName, datatypeName);
             DatatypeTupleTranslator tupleReaderWriter =
-                    tupleTranslatorProvider.getDataTypeTupleTranslator(jobId, this, false);
+                    tupleTranslatorProvider.getDataTypeTupleTranslator(txnId, this, false);
             IValueExtractor<Datatype> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
             List<Datatype> results = new ArrayList<>();
-            searchIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey, valueExtractor, results);
+            searchIndex(txnId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey, valueExtractor, results);
             if (results.isEmpty()) {
                 return null;
             }
@@ -1147,13 +1147,13 @@
     }
 
     @Override
-    public NodeGroup getNodeGroup(JobId jobId, String nodeGroupName) throws AlgebricksException, RemoteException {
+    public NodeGroup getNodeGroup(TxnId txnId, String nodeGroupName) throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(nodeGroupName);
             NodeGroupTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getNodeGroupTupleTranslator(false);
             IValueExtractor<NodeGroup> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
             List<NodeGroup> results = new ArrayList<>();
-            searchIndex(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, searchKey, valueExtractor, results);
+            searchIndex(txnId, MetadataPrimaryIndexes.NODEGROUP_DATASET, searchKey, valueExtractor, results);
             if (results.isEmpty()) {
                 return null;
             }
@@ -1164,7 +1164,7 @@
     }
 
     @Override
-    public Function getFunction(JobId jobId, FunctionSignature functionSignature)
+    public Function getFunction(TxnId txnId, FunctionSignature functionSignature)
             throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(functionSignature.getNamespace(), functionSignature.getName(),
@@ -1172,7 +1172,7 @@
             FunctionTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFunctionTupleTranslator(false);
             List<Function> results = new ArrayList<>();
             IValueExtractor<Function> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
-            searchIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey, valueExtractor, results);
+            searchIndex(txnId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey, valueExtractor, results);
             if (results.isEmpty()) {
                 return null;
             }
@@ -1183,13 +1183,13 @@
     }
 
     @Override
-    public List<Function> getFunctions(JobId jobId, String dataverseName) throws AlgebricksException, RemoteException {
+    public List<Function> getFunctions(TxnId txnId, String dataverseName) throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName);
             FunctionTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFunctionTupleTranslator(false);
             List<Function> results = new ArrayList<>();
             IValueExtractor<Function> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
-            searchIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey, valueExtractor, results);
+            searchIndex(txnId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey, valueExtractor, results);
             return results;
         } catch (HyracksDataException e) {
             throw new AlgebricksException(e);
@@ -1197,10 +1197,10 @@
     }
 
     @Override
-    public void dropFunction(JobId jobId, FunctionSignature functionSignature)
+    public void dropFunction(TxnId txnId, FunctionSignature functionSignature)
             throws AlgebricksException, RemoteException {
 
-        Function function = getFunction(jobId, functionSignature);
+        Function function = getFunction(txnId, functionSignature);
 
         if (function == null) {
             throw new AlgebricksException(
@@ -1213,8 +1213,8 @@
             // Searches the index for the tuple to be deleted. Acquires an S
             // lock on the 'function' dataset.
             ITupleReference functionTuple =
-                    getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey);
-            deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, functionTuple);
+                    getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey);
+            deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.FUNCTION_DATASET, functionTuple);
 
             // TODO: Change this to be a BTree specific exception, e.g.,
             // BTreeKeyDoesNotExistException.
@@ -1231,11 +1231,11 @@
         }
     }
 
-    private ITupleReference getTupleToBeDeleted(JobId jobId, IMetadataIndex metadataIndex, ITupleReference searchKey)
+    private ITupleReference getTupleToBeDeleted(TxnId txnId, IMetadataIndex metadataIndex, ITupleReference searchKey)
             throws AlgebricksException, HyracksDataException, RemoteException {
         IValueExtractor<ITupleReference> valueExtractor = new TupleCopyValueExtractor(metadataIndex.getTypeTraits());
         List<ITupleReference> results = new ArrayList<>();
-        searchIndex(jobId, metadataIndex, searchKey, valueExtractor, results);
+        searchIndex(txnId, metadataIndex, searchKey, valueExtractor, results);
         if (results.isEmpty()) {
             // TODO: Temporarily a TreeIndexException to make it get caught by
             // caller in the appropriate catch block.
@@ -1320,7 +1320,7 @@
         return sb.toString();
     }
 
-    private <ResultType> void searchIndex(JobId jobId, IMetadataIndex index, ITupleReference searchKey,
+    private <ResultType> void searchIndex(TxnId txnId, IMetadataIndex index, ITupleReference searchKey,
             IValueExtractor<ResultType> valueExtractor, List<ResultType> results)
             throws AlgebricksException, HyracksDataException, RemoteException {
         IBinaryComparatorFactory[] comparatorFactories = index.getKeyBinaryComparatorFactory();
@@ -1349,7 +1349,7 @@
         try {
             while (rangeCursor.hasNext()) {
                 rangeCursor.next();
-                ResultType result = valueExtractor.getValue(jobId, rangeCursor.getTuple());
+                ResultType result = valueExtractor.getValue(txnId, rangeCursor.getTuple());
                 if (result != null) {
                     results.add(result);
                 }
@@ -1361,7 +1361,7 @@
     }
 
     @Override
-    public void initializeDatasetIdFactory(JobId jobId) throws AlgebricksException, RemoteException {
+    public void initializeDatasetIdFactory(TxnId txnId) throws AlgebricksException, RemoteException {
         int mostRecentDatasetId = MetadataIndexImmutableProperties.FIRST_AVAILABLE_USER_DATASET_ID;
         try {
             String resourceName = MetadataPrimaryIndexes.DATASET_DATASET.getFile().getRelativePath();
@@ -1382,7 +1382,7 @@
                     while (rangeCursor.hasNext()) {
                         rangeCursor.next();
                         final ITupleReference ref = rangeCursor.getTuple();
-                        final Dataset ds = valueExtractor.getValue(jobId, ref);
+                        final Dataset ds = valueExtractor.getValue(txnId, ref);
                         datasetId = ds.getDatasetId();
                         if (mostRecentDatasetId < datasetId) {
                             mostRecentDatasetId = datasetId;
@@ -1426,14 +1426,14 @@
     }
 
     @Override
-    public List<Function> getDataverseFunctions(JobId jobId, String dataverseName)
+    public List<Function> getDataverseFunctions(TxnId txnId, String dataverseName)
             throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName);
             FunctionTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFunctionTupleTranslator(false);
             IValueExtractor<Function> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
             List<Function> results = new ArrayList<>();
-            searchIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey, valueExtractor, results);
+            searchIndex(txnId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey, valueExtractor, results);
             return results;
         } catch (HyracksDataException e) {
             throw new AlgebricksException(e);
@@ -1441,13 +1441,13 @@
     }
 
     @Override
-    public void addAdapter(JobId jobId, DatasourceAdapter adapter) throws AlgebricksException, RemoteException {
+    public void addAdapter(TxnId txnId, DatasourceAdapter adapter) throws AlgebricksException, RemoteException {
         try {
             // Insert into the 'Adapter' dataset.
             DatasourceAdapterTupleTranslator tupleReaderWriter =
                     tupleTranslatorProvider.getAdapterTupleTranslator(true);
             ITupleReference adapterTuple = tupleReaderWriter.getTupleFromMetadataEntity(adapter);
-            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, adapterTuple);
+            insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, adapterTuple);
         } catch (HyracksDataException e) {
             if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
                 throw new AlgebricksException("A adapter with this name " + adapter.getAdapterIdentifier().getName()
@@ -1461,9 +1461,9 @@
     }
 
     @Override
-    public void dropAdapter(JobId jobId, String dataverseName, String adapterName)
+    public void dropAdapter(TxnId txnId, String dataverseName, String adapterName)
             throws AlgebricksException, RemoteException {
-        DatasourceAdapter adapter = getAdapter(jobId, dataverseName, adapterName);
+        DatasourceAdapter adapter = getAdapter(txnId, dataverseName, adapterName);
         if (adapter == null) {
             throw new AlgebricksException("Cannot drop adapter '" + adapter + "' because it doesn't exist.");
         }
@@ -1473,8 +1473,8 @@
             // Searches the index for the tuple to be deleted. Acquires an S
             // lock on the 'Adapter' dataset.
             ITupleReference datasetTuple =
-                    getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey);
-            deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, datasetTuple);
+                    getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey);
+            deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, datasetTuple);
 
             // TODO: Change this to be a BTree specific exception, e.g.,
             // BTreeKeyDoesNotExistException.
@@ -1492,7 +1492,7 @@
     }
 
     @Override
-    public DatasourceAdapter getAdapter(JobId jobId, String dataverseName, String adapterName)
+    public DatasourceAdapter getAdapter(TxnId txnId, String dataverseName, String adapterName)
             throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName, adapterName);
@@ -1500,7 +1500,7 @@
                     tupleTranslatorProvider.getAdapterTupleTranslator(false);
             List<DatasourceAdapter> results = new ArrayList<>();
             IValueExtractor<DatasourceAdapter> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
-            searchIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey, valueExtractor, results);
+            searchIndex(txnId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey, valueExtractor, results);
             if (results.isEmpty()) {
                 return null;
             }
@@ -1511,14 +1511,14 @@
     }
 
     @Override
-    public void addCompactionPolicy(JobId jobId, CompactionPolicy compactionPolicy)
+    public void addCompactionPolicy(TxnId txnId, CompactionPolicy compactionPolicy)
             throws AlgebricksException, RemoteException {
         try {
             // Insert into the 'CompactionPolicy' dataset.
             CompactionPolicyTupleTranslator tupleReaderWriter =
                     tupleTranslatorProvider.getCompactionPolicyTupleTranslator(true);
             ITupleReference compactionPolicyTuple = tupleReaderWriter.getTupleFromMetadataEntity(compactionPolicy);
-            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.COMPACTION_POLICY_DATASET, compactionPolicyTuple);
+            insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.COMPACTION_POLICY_DATASET, compactionPolicyTuple);
         } catch (HyracksDataException e) {
             if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
                 throw new AlgebricksException("A compcation policy with this name " + compactionPolicy.getPolicyName()
@@ -1532,7 +1532,7 @@
     }
 
     @Override
-    public CompactionPolicy getCompactionPolicy(JobId jobId, String dataverse, String policyName)
+    public CompactionPolicy getCompactionPolicy(TxnId txnId, String dataverse, String policyName)
             throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverse, policyName);
@@ -1540,7 +1540,7 @@
                     tupleTranslatorProvider.getCompactionPolicyTupleTranslator(false);
             List<CompactionPolicy> results = new ArrayList<>();
             IValueExtractor<CompactionPolicy> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
-            searchIndex(jobId, MetadataPrimaryIndexes.COMPACTION_POLICY_DATASET, searchKey, valueExtractor, results);
+            searchIndex(txnId, MetadataPrimaryIndexes.COMPACTION_POLICY_DATASET, searchKey, valueExtractor, results);
             if (!results.isEmpty()) {
                 return results.get(0);
             }
@@ -1551,7 +1551,7 @@
     }
 
     @Override
-    public List<DatasourceAdapter> getDataverseAdapters(JobId jobId, String dataverseName)
+    public List<DatasourceAdapter> getDataverseAdapters(TxnId txnId, String dataverseName)
             throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName);
@@ -1559,7 +1559,7 @@
                     tupleTranslatorProvider.getAdapterTupleTranslator(false);
             IValueExtractor<DatasourceAdapter> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
             List<DatasourceAdapter> results = new ArrayList<>();
-            searchIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey, valueExtractor, results);
+            searchIndex(txnId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey, valueExtractor, results);
             return results;
         } catch (HyracksDataException e) {
             throw new AlgebricksException(e);
@@ -1567,12 +1567,12 @@
     }
 
     @Override
-    public void addLibrary(JobId jobId, Library library) throws AlgebricksException, RemoteException {
+    public void addLibrary(TxnId txnId, Library library) throws AlgebricksException, RemoteException {
         try {
             // Insert into the 'Library' dataset.
             LibraryTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getLibraryTupleTranslator(true);
             ITupleReference libraryTuple = tupleReaderWriter.getTupleFromMetadataEntity(library);
-            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.LIBRARY_DATASET, libraryTuple);
+            insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.LIBRARY_DATASET, libraryTuple);
 
         } catch (HyracksDataException e) {
             if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
@@ -1587,9 +1587,9 @@
     }
 
     @Override
-    public void dropLibrary(JobId jobId, String dataverseName, String libraryName)
+    public void dropLibrary(TxnId txnId, String dataverseName, String libraryName)
             throws AlgebricksException, RemoteException {
-        Library library = getLibrary(jobId, dataverseName, libraryName);
+        Library library = getLibrary(txnId, dataverseName, libraryName);
         if (library == null) {
             throw new AlgebricksException("Cannot drop library '" + library + "' because it doesn't exist.");
         }
@@ -1599,8 +1599,8 @@
             // Searches the index for the tuple to be deleted. Acquires an S
             // lock on the 'Adapter' dataset.
             ITupleReference datasetTuple =
-                    getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.LIBRARY_DATASET, searchKey);
-            deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.LIBRARY_DATASET, datasetTuple);
+                    getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.LIBRARY_DATASET, searchKey);
+            deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.LIBRARY_DATASET, datasetTuple);
 
             // TODO: Change this to be a BTree specific exception, e.g.,
             // BTreeKeyDoesNotExistException.
@@ -1618,14 +1618,14 @@
     }
 
     @Override
-    public Library getLibrary(JobId jobId, String dataverseName, String libraryName)
+    public Library getLibrary(TxnId txnId, String dataverseName, String libraryName)
             throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName, libraryName);
             LibraryTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getLibraryTupleTranslator(false);
             List<Library> results = new ArrayList<>();
             IValueExtractor<Library> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
-            searchIndex(jobId, MetadataPrimaryIndexes.LIBRARY_DATASET, searchKey, valueExtractor, results);
+            searchIndex(txnId, MetadataPrimaryIndexes.LIBRARY_DATASET, searchKey, valueExtractor, results);
             if (results.isEmpty()) {
                 return null;
             }
@@ -1641,12 +1641,12 @@
     }
 
     @Override
-    public void addFeedPolicy(JobId jobId, FeedPolicyEntity feedPolicy) throws AlgebricksException, RemoteException {
+    public void addFeedPolicy(TxnId txnId, FeedPolicyEntity feedPolicy) throws AlgebricksException, RemoteException {
         try {
             // Insert into the 'FeedPolicy' dataset.
             FeedPolicyTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFeedPolicyTupleTranslator(true);
             ITupleReference feedPolicyTuple = tupleReaderWriter.getTupleFromMetadataEntity(feedPolicy);
-            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.FEED_POLICY_DATASET, feedPolicyTuple);
+            insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.FEED_POLICY_DATASET, feedPolicyTuple);
         } catch (HyracksDataException e) {
             if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
                 throw new AlgebricksException("A feed policy with this name " + feedPolicy.getPolicyName()
@@ -1660,14 +1660,14 @@
     }
 
     @Override
-    public FeedPolicyEntity getFeedPolicy(JobId jobId, String dataverse, String policyName)
+    public FeedPolicyEntity getFeedPolicy(TxnId txnId, String dataverse, String policyName)
             throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverse, policyName);
             FeedPolicyTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFeedPolicyTupleTranslator(false);
             List<FeedPolicyEntity> results = new ArrayList<>();
             IValueExtractor<FeedPolicyEntity> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
-            searchIndex(jobId, MetadataPrimaryIndexes.FEED_POLICY_DATASET, searchKey, valueExtractor, results);
+            searchIndex(txnId, MetadataPrimaryIndexes.FEED_POLICY_DATASET, searchKey, valueExtractor, results);
             if (!results.isEmpty()) {
                 return results.get(0);
             }
@@ -1678,25 +1678,25 @@
     }
 
     @Override
-    public void addFeedConnection(JobId jobId, FeedConnection feedConnection) throws AlgebricksException {
+    public void addFeedConnection(TxnId txnId, FeedConnection feedConnection) throws AlgebricksException {
         try {
             FeedConnectionTupleTranslator tupleReaderWriter = new FeedConnectionTupleTranslator(true);
             ITupleReference feedConnTuple = tupleReaderWriter.getTupleFromMetadataEntity(feedConnection);
-            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.FEED_CONNECTION_DATASET, feedConnTuple);
+            insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.FEED_CONNECTION_DATASET, feedConnTuple);
         } catch (HyracksDataException | ACIDException e) {
             throw new AlgebricksException(e);
         }
     }
 
     @Override
-    public List<FeedConnection> getFeedConnections(JobId jobId, String dataverseName, String feedName)
+    public List<FeedConnection> getFeedConnections(TxnId txnId, String dataverseName, String feedName)
             throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName, feedName);
             FeedConnectionTupleTranslator tupleReaderWriter = new FeedConnectionTupleTranslator(false);
             List<FeedConnection> results = new ArrayList<>();
             IValueExtractor<FeedConnection> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
-            searchIndex(jobId, MetadataPrimaryIndexes.FEED_CONNECTION_DATASET, searchKey, valueExtractor, results);
+            searchIndex(txnId, MetadataPrimaryIndexes.FEED_CONNECTION_DATASET, searchKey, valueExtractor, results);
             return results;
         } catch (HyracksDataException e) {
             throw new AlgebricksException(e);
@@ -1704,14 +1704,14 @@
     }
 
     @Override
-    public FeedConnection getFeedConnection(JobId jobId, String dataverseName, String feedName, String datasetName)
+    public FeedConnection getFeedConnection(TxnId txnId, String dataverseName, String feedName, String datasetName)
             throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName, feedName, datasetName);
             FeedConnectionTupleTranslator tupleReaderWriter = new FeedConnectionTupleTranslator(false);
             List<FeedConnection> results = new ArrayList<>();
             IValueExtractor<FeedConnection> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
-            searchIndex(jobId, MetadataPrimaryIndexes.FEED_CONNECTION_DATASET, searchKey, valueExtractor, results);
+            searchIndex(txnId, MetadataPrimaryIndexes.FEED_CONNECTION_DATASET, searchKey, valueExtractor, results);
             if (!results.isEmpty()) {
                 return results.get(0);
             }
@@ -1722,25 +1722,25 @@
     }
 
     @Override
-    public void dropFeedConnection(JobId jobId, String dataverseName, String feedName, String datasetName)
+    public void dropFeedConnection(TxnId txnId, String dataverseName, String feedName, String datasetName)
             throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName, feedName, datasetName);
             ITupleReference tuple =
-                    getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.FEED_CONNECTION_DATASET, searchKey);
-            deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.FEED_CONNECTION_DATASET, tuple);
+                    getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.FEED_CONNECTION_DATASET, searchKey);
+            deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.FEED_CONNECTION_DATASET, tuple);
         } catch (HyracksDataException | ACIDException e) {
             throw new AlgebricksException(e);
         }
     }
 
     @Override
-    public void addFeed(JobId jobId, Feed feed) throws AlgebricksException, RemoteException {
+    public void addFeed(TxnId txnId, Feed feed) throws AlgebricksException, RemoteException {
         try {
             // Insert into the 'Feed' dataset.
             FeedTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFeedTupleTranslator(true);
             ITupleReference feedTuple = tupleReaderWriter.getTupleFromMetadataEntity(feed);
-            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.FEED_DATASET, feedTuple);
+            insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.FEED_DATASET, feedTuple);
         } catch (HyracksDataException e) {
             if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
                 throw new AlgebricksException("A feed with this name " + feed.getFeedName()
@@ -1754,13 +1754,13 @@
     }
 
     @Override
-    public Feed getFeed(JobId jobId, String dataverse, String feedName) throws AlgebricksException, RemoteException {
+    public Feed getFeed(TxnId txnId, String dataverse, String feedName) throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverse, feedName);
             FeedTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFeedTupleTranslator(false);
             List<Feed> results = new ArrayList<>();
             IValueExtractor<Feed> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
-            searchIndex(jobId, MetadataPrimaryIndexes.FEED_DATASET, searchKey, valueExtractor, results);
+            searchIndex(txnId, MetadataPrimaryIndexes.FEED_DATASET, searchKey, valueExtractor, results);
             if (!results.isEmpty()) {
                 return results.get(0);
             }
@@ -1771,13 +1771,13 @@
     }
 
     @Override
-    public List<Feed> getFeeds(JobId jobId, String dataverse) throws AlgebricksException, RemoteException {
+    public List<Feed> getFeeds(TxnId txnId, String dataverse) throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverse);
             FeedTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFeedTupleTranslator(false);
             List<Feed> results = new ArrayList<>();
             IValueExtractor<Feed> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
-            searchIndex(jobId, MetadataPrimaryIndexes.FEED_DATASET, searchKey, valueExtractor, results);
+            searchIndex(txnId, MetadataPrimaryIndexes.FEED_DATASET, searchKey, valueExtractor, results);
             return results;
         } catch (HyracksDataException e) {
             throw new AlgebricksException(e);
@@ -1785,13 +1785,13 @@
     }
 
     @Override
-    public void dropFeed(JobId jobId, String dataverse, String feedName) throws AlgebricksException, RemoteException {
+    public void dropFeed(TxnId txnId, String dataverse, String feedName) throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverse, feedName);
             // Searches the index for the tuple to be deleted. Acquires an S
             // lock on the 'nodegroup' dataset.
-            ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.FEED_DATASET, searchKey);
-            deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.FEED_DATASET, tuple);
+            ITupleReference tuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.FEED_DATASET, searchKey);
+            deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.FEED_DATASET, tuple);
             // TODO: Change this to be a BTree specific exception, e.g.,
             // BTreeKeyDoesNotExistException.
         } catch (HyracksDataException e) {
@@ -1807,12 +1807,12 @@
     }
 
     @Override
-    public void dropFeedPolicy(JobId jobId, String dataverseName, String policyName)
+    public void dropFeedPolicy(TxnId txnId, String dataverseName, String policyName)
             throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName, policyName);
-            ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.FEED_POLICY_DATASET, searchKey);
-            deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.FEED_POLICY_DATASET, tuple);
+            ITupleReference tuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.FEED_POLICY_DATASET, searchKey);
+            deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.FEED_POLICY_DATASET, tuple);
         } catch (HyracksDataException e) {
             if (e.getComponent().equals(ErrorCode.HYRACKS)
                     && e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
@@ -1826,14 +1826,14 @@
     }
 
     @Override
-    public List<FeedPolicyEntity> getDataversePolicies(JobId jobId, String dataverse)
+    public List<FeedPolicyEntity> getDataversePolicies(TxnId txnId, String dataverse)
             throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverse);
             FeedPolicyTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFeedPolicyTupleTranslator(false);
             IValueExtractor<FeedPolicyEntity> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
             List<FeedPolicyEntity> results = new ArrayList<>();
-            searchIndex(jobId, MetadataPrimaryIndexes.FEED_POLICY_DATASET, searchKey, valueExtractor, results);
+            searchIndex(txnId, MetadataPrimaryIndexes.FEED_POLICY_DATASET, searchKey, valueExtractor, results);
             return results;
         } catch (HyracksDataException e) {
             throw new AlgebricksException(e);
@@ -1841,13 +1841,13 @@
     }
 
     @Override
-    public void addExternalFile(JobId jobId, ExternalFile externalFile) throws AlgebricksException, RemoteException {
+    public void addExternalFile(TxnId txnId, ExternalFile externalFile) throws AlgebricksException, RemoteException {
         try {
             // Insert into the 'externalFiles' dataset.
             ExternalFileTupleTranslator tupleReaderWriter =
                     tupleTranslatorProvider.getExternalFileTupleTranslator(true);
             ITupleReference externalFileTuple = tupleReaderWriter.getTupleFromMetadataEntity(externalFile);
-            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, externalFileTuple);
+            insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, externalFileTuple);
         } catch (HyracksDataException e) {
             if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
                 throw new AlgebricksException("An externalFile with this number " + externalFile.getFileNumber()
@@ -1862,7 +1862,7 @@
     }
 
     @Override
-    public List<ExternalFile> getExternalFiles(JobId jobId, Dataset dataset)
+    public List<ExternalFile> getExternalFiles(TxnId txnId, Dataset dataset)
             throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataset.getDataverseName(), dataset.getDatasetName());
@@ -1870,7 +1870,7 @@
                     tupleTranslatorProvider.getExternalFileTupleTranslator(false);
             IValueExtractor<ExternalFile> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
             List<ExternalFile> results = new ArrayList<>();
-            searchIndex(jobId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, searchKey, valueExtractor, results);
+            searchIndex(txnId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, searchKey, valueExtractor, results);
             return results;
         } catch (HyracksDataException e) {
             throw new AlgebricksException(e);
@@ -1878,7 +1878,7 @@
     }
 
     @Override
-    public void dropExternalFile(JobId jobId, String dataverseName, String datasetName, int fileNumber)
+    public void dropExternalFile(TxnId txnId, String dataverseName, String datasetName, int fileNumber)
             throws AlgebricksException, RemoteException {
         try {
             // Delete entry from the 'ExternalFile' dataset.
@@ -1886,8 +1886,8 @@
             // Searches the index for the tuple to be deleted. Acquires an S
             // lock on the 'ExternalFile' dataset.
             ITupleReference datasetTuple =
-                    getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, searchKey);
-            deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, datasetTuple);
+                    getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, searchKey);
+            deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, datasetTuple);
         } catch (HyracksDataException e) {
             if (e.getComponent().equals(ErrorCode.HYRACKS)
                     && e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
@@ -1901,11 +1901,11 @@
     }
 
     @Override
-    public void dropExternalFiles(JobId jobId, Dataset dataset) throws AlgebricksException, RemoteException {
-        List<ExternalFile> files = getExternalFiles(jobId, dataset);
+    public void dropExternalFiles(TxnId txnId, Dataset dataset) throws AlgebricksException, RemoteException {
+        List<ExternalFile> files = getExternalFiles(txnId, dataset);
         // loop through files and delete them
         for (int i = 0; i < files.size(); i++) {
-            dropExternalFile(jobId, files.get(i).getDataverseName(), files.get(i).getDatasetName(),
+            dropExternalFile(txnId, files.get(i).getDataverseName(), files.get(i).getDatasetName(),
                     files.get(i).getFileNumber());
         }
     }
@@ -1943,7 +1943,7 @@
     }
 
     @Override
-    public ExternalFile getExternalFile(JobId jobId, String dataverseName, String datasetName, Integer fileNumber)
+    public ExternalFile getExternalFile(TxnId txnId, String dataverseName, String datasetName, Integer fileNumber)
             throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = createExternalFileSearchTuple(dataverseName, datasetName, fileNumber);
@@ -1951,7 +1951,7 @@
                     tupleTranslatorProvider.getExternalFileTupleTranslator(false);
             IValueExtractor<ExternalFile> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
             List<ExternalFile> results = new ArrayList<>();
-            searchIndex(jobId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, searchKey, valueExtractor, results);
+            searchIndex(txnId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, searchKey, valueExtractor, results);
             if (results.isEmpty()) {
                 return null;
             }
@@ -1962,7 +1962,7 @@
     }
 
     @Override
-    public void updateDataset(JobId jobId, Dataset dataset) throws AlgebricksException, RemoteException {
+    public void updateDataset(TxnId txnId, Dataset dataset) throws AlgebricksException, RemoteException {
         try {
             // This method will delete previous entry of the dataset and insert the new one
             // Delete entry from the 'datasets' dataset.
@@ -1971,13 +1971,13 @@
             // Searches the index for the tuple to be deleted. Acquires an S
             // lock on the 'dataset' dataset.
             ITupleReference datasetTuple =
-                    getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey);
-            deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
+                    getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey);
+            deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
             // Previous tuple was deleted
             // Insert into the 'dataset' dataset.
             DatasetTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDatasetTupleTranslator(true);
             datasetTuple = tupleReaderWriter.getTupleFromMetadataEntity(dataset);
-            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
+            insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
         } catch (HyracksDataException | ACIDException e) {
             throw new AlgebricksException(e);
         }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
index 72285d0..406b3d6 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
@@ -22,7 +22,7 @@
 import java.util.ArrayList;
 
 import org.apache.asterix.common.functions.FunctionSignature;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.external.dataset.adapter.AdapterIdentifier;
 import org.apache.asterix.metadata.entities.CompactionPolicy;
 import org.apache.asterix.metadata.entities.Dataset;
@@ -68,14 +68,14 @@
     protected MetadataCache droppedCache = new MetadataCache();
 
     protected ArrayList<MetadataLogicalOperation> opLog = new ArrayList<>();
-    private final JobId jobId;
+    private final TxnId txnId;
 
-    public MetadataTransactionContext(JobId jobId) {
-        this.jobId = jobId;
+    public MetadataTransactionContext(TxnId txnId) {
+        this.txnId = txnId;
     }
 
-    public JobId getJobId() {
-        return jobId;
+    public TxnId getTxnId() {
+        return txnId;
     }
 
     public void addDataverse(Dataverse dataverse) {
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 8c4920f..cdb27d7 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
@@ -26,7 +26,7 @@
 
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.functions.FunctionSignature;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.external.indexing.ExternalFile;
 import org.apache.asterix.metadata.entities.CompactionPolicy;
 import org.apache.asterix.metadata.entities.Dataset;
@@ -59,7 +59,7 @@
      * @throws ACIDException
      * @throws RemoteException
      */
-    void beginTransaction(JobId jobId) throws ACIDException, RemoteException;
+    void beginTransaction(TxnId txnId) throws ACIDException, RemoteException;
 
     /**
      * Commits a local transaction against the metadata.
@@ -67,7 +67,7 @@
      * @throws ACIDException
      * @throws RemoteException
      */
-    void commitTransaction(JobId jobId) throws ACIDException, RemoteException;
+    void commitTransaction(TxnId txnId) throws ACIDException, RemoteException;
 
     /**
      * Aborts a local transaction against the metadata.
@@ -75,7 +75,7 @@
      * @throws ACIDException
      * @throws RemoteException
      */
-    void abortTransaction(JobId jobId) throws ACIDException, RemoteException;
+    void abortTransaction(TxnId txnId) throws ACIDException, RemoteException;
 
     /**
      * Locally locks the entire metadata in given mode on behalf of given
@@ -84,7 +84,7 @@
      * @throws ACIDException
      * @throws RemoteException
      */
-    void lock(JobId jobId, byte lockMode) throws ACIDException, RemoteException;
+    void lock(TxnId txnId, byte lockMode) throws ACIDException, RemoteException;
 
     /**
      * Releases all local locks of given transaction id.
@@ -92,13 +92,13 @@
      * @throws ACIDException
      * @throws RemoteException
      */
-    void unlock(JobId jobId, byte lockMode) throws ACIDException, RemoteException;
+    void unlock(TxnId txnId, byte lockMode) throws ACIDException, RemoteException;
 
     /**
      * Inserts a new dataverse into the metadata, acquiring local locks on behalf of
      * the given transaction id.
      *
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @param dataverse
      *            Dataverse instance to be inserted.
@@ -106,26 +106,26 @@
      *             For example, if the dataverse already exists.
      * @throws RemoteException
      */
-    void addDataverse(JobId jobId, Dataverse dataverse) throws AlgebricksException, RemoteException;
+    void addDataverse(TxnId txnId, Dataverse dataverse) throws AlgebricksException, RemoteException;
 
     /**
      * Retrieves all dataverses, acquiring local locks on behalf of the given
      * transaction id.
      *
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @return A list of dataverse instances.
      * @throws AlgebricksException
      *             For example, if the dataverse does not exist.
      * @throws RemoteException
      */
-    List<Dataverse> getDataverses(JobId jobId) throws AlgebricksException, RemoteException;
+    List<Dataverse> getDataverses(TxnId txnId) throws AlgebricksException, RemoteException;
 
     /**
      * Retrieves a dataverse with given name, acquiring local locks on behalf of the
      * given transaction id.
      *
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @param dataverseName
      *            Name of the dataverse to retrieve.
@@ -134,13 +134,13 @@
      *             For example, if the dataverse does not exist.
      * @throws RemoteException
      */
-    Dataverse getDataverse(JobId jobId, String dataverseName) throws AlgebricksException, RemoteException;
+    Dataverse getDataverse(TxnId txnId, String dataverseName) throws AlgebricksException, RemoteException;
 
     /**
      * Retrieves all datasets belonging to the given dataverse, acquiring local
      * locks on behalf of the given transaction id.
      *
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @param dataverseName
      *            Name of the dataverse of which to find all datasets.
@@ -148,27 +148,27 @@
      * @throws AlgebricksException
      *             For example, if the dataverse does not exist. RemoteException
      */
-    List<Dataset> getDataverseDatasets(JobId jobId, String dataverseName) throws AlgebricksException, RemoteException;
+    List<Dataset> getDataverseDatasets(TxnId txnId, String dataverseName) throws AlgebricksException, RemoteException;
 
     /**
      * Deletes the dataverse with given name, and all it's associated datasets,
      * indexes, and types, acquiring local locks on behalf of the given transaction
      * id.
      *
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @return A list of dataset instances.
      * @throws AlgebricksException
      *             For example, if the dataverse does not exist.
      * @throws RemoteException
      */
-    void dropDataverse(JobId jobId, String dataverseName) throws AlgebricksException, RemoteException;
+    void dropDataverse(TxnId txnId, String dataverseName) throws AlgebricksException, RemoteException;
 
     /**
      * Inserts a new dataset into the metadata, acquiring local locks on behalf of
      * the given transaction id.
      *
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @param dataset
      *            Dataset instance to be inserted.
@@ -176,13 +176,13 @@
      *             For example, if the dataset already exists.
      * @throws RemoteException
      */
-    void addDataset(JobId jobId, Dataset dataset) throws AlgebricksException, RemoteException;
+    void addDataset(TxnId txnId, Dataset dataset) throws AlgebricksException, RemoteException;
 
     /**
      * Retrieves a dataset within a given dataverse, acquiring local locks on behalf
      * of the given transaction id.
      *
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @param dataverseName
      *            Dataverse name to look for the dataset.
@@ -193,14 +193,14 @@
      *             For example, if the dataset does not exist.
      * @throws RemoteException
      */
-    Dataset getDataset(JobId jobId, String dataverseName, String datasetName)
+    Dataset getDataset(TxnId txnId, String dataverseName, String datasetName)
             throws AlgebricksException, RemoteException;
 
     /**
      * Retrieves all indexes of a dataset, acquiring local locks on behalf of the
      * given transaction id.
      *
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @param dataverseName
      *            Name of dataverse which holds the given dataset.
@@ -211,14 +211,14 @@
      *             For example, if the dataset and/or dataverse does not exist.
      * @throws RemoteException
      */
-    List<Index> getDatasetIndexes(JobId jobId, String dataverseName, String datasetName)
+    List<Index> getDatasetIndexes(TxnId txnId, String dataverseName, String datasetName)
             throws AlgebricksException, RemoteException;
 
     /**
      * Deletes the dataset with given name, and all it's associated indexes,
      * acquiring local locks on behalf of the given transaction id.
      *
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @param dataverseName
      *            Name of dataverse which holds the given dataset.
@@ -228,14 +228,14 @@
      *             For example, if the dataset and/or dataverse does not exist.
      * @throws RemoteException
      */
-    void dropDataset(JobId jobId, String dataverseName, String datasetName) throws AlgebricksException, RemoteException;
+    void dropDataset(TxnId txnId, String dataverseName, String datasetName) throws AlgebricksException, RemoteException;
 
     /**
      * Inserts an index into the metadata, acquiring local locks on behalf of the
      * given transaction id. The index itself knows its name, and which dataset it
      * belongs to.
      *
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @param index
      *            Index instance to be inserted.
@@ -243,13 +243,13 @@
      *             For example, if the index already exists.
      * @throws RemoteException
      */
-    void addIndex(JobId jobId, Index index) throws AlgebricksException, RemoteException;
+    void addIndex(TxnId txnId, Index index) throws AlgebricksException, RemoteException;
 
     /**
      * Retrieves the index with given name, in given dataverse and dataset,
      * acquiring local locks on behalf of the given transaction id.
      *
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @param dataverseName
      *            Name of the datavers holding the given dataset.
@@ -261,14 +261,14 @@
      *             For example, if the index does not exist.
      * @throws RemoteException
      */
-    Index getIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
+    Index getIndex(TxnId txnId, String dataverseName, String datasetName, String indexName)
             throws AlgebricksException, RemoteException;
 
     /**
      * Deletes the index with given name, in given dataverse and dataset, acquiring
      * local locks on behalf of the given transaction id.
      *
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @param dataverseName
      *            Name of the datavers holding the given dataset.
@@ -279,14 +279,14 @@
      *             For example, if the index does not exist.
      * @throws RemoteException
      */
-    void dropIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
+    void dropIndex(TxnId txnId, String dataverseName, String datasetName, String indexName)
             throws AlgebricksException, RemoteException;
 
     /**
      * Inserts a datatype, acquiring local locks on behalf of the given transaction
      * id.
      *
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @param datatype
      *            Datatype instance to be inserted.
@@ -294,13 +294,13 @@
      *             For example, if the datatype already exists.
      * @throws RemoteException
      */
-    void addDatatype(JobId jobId, Datatype datatype) throws AlgebricksException, RemoteException;
+    void addDatatype(TxnId txnId, Datatype datatype) throws AlgebricksException, RemoteException;
 
     /**
      * Retrieves the datatype with given name in given dataverse, acquiring local
      * locks on behalf of the given transaction id.
      *
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @param dataverseName
      *            Name of dataverse holding the datatype.
@@ -311,14 +311,14 @@
      *             For example, if the datatype does not exist.
      * @throws RemoteException
      */
-    Datatype getDatatype(JobId jobId, String dataverseName, String datatypeName)
+    Datatype getDatatype(TxnId txnId, String dataverseName, String datatypeName)
             throws AlgebricksException, RemoteException;
 
     /**
      * Deletes the given datatype in given dataverse, acquiring local locks on
      * behalf of the given transaction id.
      *
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @param dataverseName
      *            Name of dataverse holding the datatype.
@@ -329,14 +329,14 @@
      *             deleted.
      * @throws RemoteException
      */
-    void dropDatatype(JobId jobId, String dataverseName, String datatypeName)
+    void dropDatatype(TxnId txnId, String dataverseName, String datatypeName)
             throws AlgebricksException, RemoteException;
 
     /**
      * Inserts a node group, acquiring local locks on behalf of the given
      * transaction id.
      *
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @param nodeGroup
      *            Node group instance to insert.
@@ -344,13 +344,13 @@
      *             For example, if the node group already exists.
      * @throws RemoteException
      */
-    void addNodeGroup(JobId jobId, NodeGroup nodeGroup) throws AlgebricksException, RemoteException;
+    void addNodeGroup(TxnId txnId, NodeGroup nodeGroup) throws AlgebricksException, RemoteException;
 
     /**
      * Retrieves a node group, acquiring local locks on behalf of the given
      * transaction id.
      *
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @param nodeGroupName
      *            Name of node group to be retrieved.
@@ -358,13 +358,13 @@
      *             For example, if the node group does not exist.
      * @throws RemoteException
      */
-    NodeGroup getNodeGroup(JobId jobId, String nodeGroupName) throws AlgebricksException, RemoteException;
+    NodeGroup getNodeGroup(TxnId txnId, String nodeGroupName) throws AlgebricksException, RemoteException;
 
     /**
      * Deletes a node group, acquiring local locks on behalf of the given
      * transaction id.
      *
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @param nodeGroupName
      *            Name of node group to be deleted.
@@ -377,14 +377,14 @@
      *             group to be deleted.
      * @throws RemoteException
      */
-    boolean dropNodegroup(JobId jobId, String nodeGroupName, boolean failSilently)
+    boolean dropNodegroup(TxnId txnId, String nodeGroupName, boolean failSilently)
             throws AlgebricksException, RemoteException;
 
     /**
      * Inserts a node (compute node), acquiring local locks on behalf of the given
      * transaction id.
      *
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @param node
      *            Node instance to be inserted.
@@ -392,10 +392,10 @@
      *             For example, if the node already exists.
      * @throws RemoteException
      */
-    void addNode(JobId jobId, Node node) throws AlgebricksException, RemoteException;
+    void addNode(TxnId txnId, Node node) throws AlgebricksException, RemoteException;
 
     /**
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @param functionSignature
      *            An instance of functionSignature representing the function
@@ -403,15 +403,15 @@
      * @throws AlgebricksException
      * @throws RemoteException
      */
-    Function getFunction(JobId jobId, FunctionSignature functionSignature) throws AlgebricksException, RemoteException;
+    Function getFunction(TxnId txnId, FunctionSignature functionSignature) throws AlgebricksException, RemoteException;
 
-    List<Function> getFunctions(JobId jobId, String dataverseName) throws AlgebricksException, RemoteException;
+    List<Function> getFunctions(TxnId txnId, String dataverseName) throws AlgebricksException, RemoteException;
 
     /**
      * Deletes a function, acquiring local locks on behalf of the given transaction
      * id.
      *
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @param functionSignature
      *            An instance of functionSignature representing the function
@@ -420,10 +420,10 @@
      *             group to be deleted.
      * @throws RemoteException
      */
-    void dropFunction(JobId jobId, FunctionSignature functionSignature) throws AlgebricksException, RemoteException;
+    void dropFunction(TxnId txnId, FunctionSignature functionSignature) throws AlgebricksException, RemoteException;
 
     /**
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @param function
      *            Function to be inserted
@@ -432,45 +432,45 @@
      *             unknown function
      * @throws RemoteException
      */
-    void addFunction(JobId jobId, Function function) throws AlgebricksException, RemoteException;
+    void addFunction(TxnId txnId, Function function) throws AlgebricksException, RemoteException;
 
     /**
-     * @param jobId
+     * @param txnId
      * @param dataverseName
      * @return List<Function> A list containing the functions in the specified
      *         dataverse
      * @throws AlgebricksException
      * @throws RemoteException
      */
-    List<Function> getDataverseFunctions(JobId jobId, String dataverseName) throws AlgebricksException, RemoteException;
+    List<Function> getDataverseFunctions(TxnId txnId, String dataverseName) throws AlgebricksException, RemoteException;
 
     /**
-     * @param jobId
+     * @param txnId
      * @param dataverseName
      * @return List<Adapter> A list containing the adapters in the specified
      *         dataverse
      * @throws AlgebricksException
      * @throws RemoteException
      */
-    List<DatasourceAdapter> getDataverseAdapters(JobId jobId, String dataverseName)
+    List<DatasourceAdapter> getDataverseAdapters(TxnId txnId, String dataverseName)
             throws AlgebricksException, RemoteException;
 
     /**
-     * @param jobId
+     * @param txnId
      * @param dataverseName
      * @param adapterName
      * @return
      * @throws AlgebricksException
      * @throws RemoteException
      */
-    DatasourceAdapter getAdapter(JobId jobId, String dataverseName, String adapterName)
+    DatasourceAdapter getAdapter(TxnId txnId, String dataverseName, String adapterName)
             throws AlgebricksException, RemoteException;
 
     /**
      * Deletes a adapter , acquiring local locks on behalf of the given transaction
      * id.
      *
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @param dataverseName
      *            dataverse asociated with the adapter that is to be deleted.
@@ -480,10 +480,10 @@
      * @throws AlgebricksException
      * @throws RemoteException
      */
-    void dropAdapter(JobId jobId, String dataverseName, String adapterName) throws AlgebricksException, RemoteException;
+    void dropAdapter(TxnId txnId, String dataverseName, String adapterName) throws AlgebricksException, RemoteException;
 
     /**
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @param adapter
      *            Adapter to be inserted
@@ -491,34 +491,34 @@
      *             for example, if the adapter already exists.
      * @throws RemoteException
      */
-    void addAdapter(JobId jobId, DatasourceAdapter adapter) throws AlgebricksException, RemoteException;
+    void addAdapter(TxnId txnId, DatasourceAdapter adapter) throws AlgebricksException, RemoteException;
 
     /**
-     * @param jobId
+     * @param txnId
      * @param compactionPolicy
      * @throws AlgebricksException
      * @throws RemoteException
      */
-    void addCompactionPolicy(JobId jobId, CompactionPolicy compactionPolicy)
+    void addCompactionPolicy(TxnId txnId, CompactionPolicy compactionPolicy)
             throws AlgebricksException, RemoteException;
 
     /**
-     * @param jobId
+     * @param txnId
      * @param dataverse
      * @param policy
      * @return
      * @throws AlgebricksException
      * @throws RemoteException
      */
-    CompactionPolicy getCompactionPolicy(JobId jobId, String dataverse, String policy)
+    CompactionPolicy getCompactionPolicy(TxnId txnId, String dataverse, String policy)
             throws AlgebricksException, RemoteException;
 
     /**
-     * @param jobId
+     * @param txnId
      * @throws AlgebricksException
      * @throws RemoteException
      */
-    void initializeDatasetIdFactory(JobId jobId) throws AlgebricksException, RemoteException;
+    void initializeDatasetIdFactory(TxnId txnId) throws AlgebricksException, RemoteException;
 
     /**
      * @return
@@ -528,58 +528,58 @@
     int getMostRecentDatasetId() throws AlgebricksException, RemoteException;
 
     /**
-     * @param jobId
+     * @param txnId
      * @param feed
      * @throws AlgebricksException
      * @throws RemoteException
      */
-    void addFeed(JobId jobId, Feed feed) throws AlgebricksException, RemoteException;
+    void addFeed(TxnId txnId, Feed feed) throws AlgebricksException, RemoteException;
 
     /**
-     * @param jobId
+     * @param txnId
      * @param dataverse
      * @param feedName
      * @return
      * @throws AlgebricksException
      * @throws RemoteException
      */
-    Feed getFeed(JobId jobId, String dataverse, String feedName) throws AlgebricksException, RemoteException;
+    Feed getFeed(TxnId txnId, String dataverse, String feedName) throws AlgebricksException, RemoteException;
 
-    List<Feed> getFeeds(JobId jobId, String dataverse) throws AlgebricksException, RemoteException;
+    List<Feed> getFeeds(TxnId txnId, String dataverse) throws AlgebricksException, RemoteException;
 
     /**
-     * @param jobId
+     * @param txnId
      * @param dataverse
      * @param feedName
      * @throws AlgebricksException
      * @throws RemoteException
      */
-    void dropFeed(JobId jobId, String dataverse, String feedName) throws AlgebricksException, RemoteException;
+    void dropFeed(TxnId txnId, String dataverse, String feedName) throws AlgebricksException, RemoteException;
 
     /**
-     * @param jobId
+     * @param txnId
      * @param feedPolicy
      * @throws AlgebricksException
      * @throws RemoteException
      */
-    void addFeedPolicy(JobId jobId, FeedPolicyEntity feedPolicy) throws AlgebricksException, RemoteException;
+    void addFeedPolicy(TxnId txnId, FeedPolicyEntity feedPolicy) throws AlgebricksException, RemoteException;
 
     /**
-     * @param jobId
+     * @param txnId
      * @param dataverse
      * @param policy
      * @return
      * @throws AlgebricksException
      * @throws RemoteException
      */
-    FeedPolicyEntity getFeedPolicy(JobId jobId, String dataverse, String policy)
+    FeedPolicyEntity getFeedPolicy(TxnId txnId, String dataverse, String policy)
             throws AlgebricksException, RemoteException;
 
     /**
      * Removes a library , acquiring local locks on behalf of the given transaction
      * id.
      *
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @param dataverseName
      *            dataverse asociated with the adapter that is to be deleted.
@@ -589,12 +589,12 @@
      * @throws AlgebricksException
      * @throws RemoteException
      */
-    void dropLibrary(JobId jobId, String dataverseName, String libraryName) throws AlgebricksException, RemoteException;
+    void dropLibrary(TxnId txnId, String dataverseName, String libraryName) throws AlgebricksException, RemoteException;
 
     /**
      * Adds a library, acquiring local locks on behalf of the given transaction id.
      *
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @param library
      *            Library to be added
@@ -602,10 +602,10 @@
      *             for example, if the library is already added.
      * @throws RemoteException
      */
-    void addLibrary(JobId jobId, Library library) throws AlgebricksException, RemoteException;
+    void addLibrary(TxnId txnId, Library library) throws AlgebricksException, RemoteException;
 
     /**
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @param dataverseName
      *            dataverse asociated with the library that is to be retrieved.
@@ -615,7 +615,7 @@
      * @throws AlgebricksException
      * @throws RemoteException
      */
-    Library getLibrary(JobId jobId, String dataverseName, String libraryName)
+    Library getLibrary(TxnId txnId, String dataverseName, String libraryName)
             throws AlgebricksException, RemoteException;
 
     /**
@@ -629,42 +629,42 @@
      * @throws AlgebricksException
      * @throws RemoteException
      */
-    List<Library> getDataverseLibraries(JobId jobId, String dataverseName) throws AlgebricksException, RemoteException;
+    List<Library> getDataverseLibraries(TxnId txnId, String dataverseName) throws AlgebricksException, RemoteException;
 
     /**
-     * @param jobId
+     * @param txnId
      * @param dataverseName
      * @return
      * @throws AlgebricksException
      * @throws RemoteException
      */
-    List<Feed> getDataverseFeeds(JobId jobId, String dataverseName) throws AlgebricksException, RemoteException;
+    List<Feed> getDataverseFeeds(TxnId txnId, String dataverseName) throws AlgebricksException, RemoteException;
 
     /**
      * delete a give feed (ingestion) policy
      *
-     * @param jobId
+     * @param txnId
      * @param dataverseName
      * @param policyName
      * @return
      * @throws RemoteException
      * @throws AlgebricksException
      */
-    void dropFeedPolicy(JobId jobId, String dataverseName, String policyName)
+    void dropFeedPolicy(TxnId txnId, String dataverseName, String policyName)
             throws AlgebricksException, RemoteException;
 
     /**
-     * @param jobId
+     * @param txnId
      * @param dataverse
      * @return
      * @throws AlgebricksException
      * @throws RemoteException
      */
-    List<FeedPolicyEntity> getDataversePolicies(JobId jobId, String dataverse)
+    List<FeedPolicyEntity> getDataversePolicies(TxnId txnId, String dataverse)
             throws AlgebricksException, RemoteException;
 
     /**
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @param externalFile
      *            An object representing the external file entity
@@ -672,23 +672,23 @@
      *             for example, if the file already exists.
      * @throws RemoteException
      */
-    void addExternalFile(JobId jobId, ExternalFile externalFile) throws AlgebricksException, RemoteException;
+    void addExternalFile(TxnId txnId, ExternalFile externalFile) throws AlgebricksException, RemoteException;
 
     /**
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @param dataset
      *            A dataset the files belongs to.
      * @throws AlgebricksException
      * @throws RemoteException
      */
-    List<ExternalFile> getExternalFiles(JobId jobId, Dataset dataset) throws AlgebricksException, RemoteException;
+    List<ExternalFile> getExternalFiles(TxnId txnId, Dataset dataset) throws AlgebricksException, RemoteException;
 
     /**
      * Deletes an externalFile , acquiring local locks on behalf of the given
      * transaction id.
      *
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @param dataverseName
      *            dataverse asociated with the external dataset that owns the file
@@ -700,27 +700,27 @@
      * @throws AlgebricksException
      * @throws RemoteException
      */
-    void dropExternalFile(JobId jobId, String dataverseName, String datasetName, int fileNumber)
+    void dropExternalFile(TxnId txnId, String dataverseName, String datasetName, int fileNumber)
             throws AlgebricksException, RemoteException;
 
     /**
      * Deletes all external files belonging to a dataset, acquiring local locks on
      * behalf of the given transaction id.
      *
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @param dataset
      *            An external dataset the files belong to.
      * @throws AlgebricksException
      * @throws RemoteException
      */
-    void dropExternalFiles(JobId jobId, Dataset dataset) throws AlgebricksException, RemoteException;
+    void dropExternalFiles(TxnId txnId, Dataset dataset) throws AlgebricksException, RemoteException;
 
     /**
      * Retrieves the file with given number, in given dataverse and dataset,
      * acquiring local locks on behalf of the given transaction id.
      *
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @param dataverseName
      *            Name of the datavers holding the given dataset.
@@ -733,14 +733,14 @@
      *             For example, if the index does not exist.
      * @throws RemoteException
      */
-    ExternalFile getExternalFile(JobId jobId, String dataverseName, String datasetName, Integer fileNumber)
+    ExternalFile getExternalFile(TxnId txnId, String dataverseName, String datasetName, Integer fileNumber)
             throws AlgebricksException, RemoteException;
 
     /**
      * update an existing dataset in the metadata, acquiring local locks on behalf
      * of the given transaction id.
      *
-     * @param jobId
+     * @param txnId
      *            A globally unique id for an active metadata transaction.
      * @param dataset
      *            updated Dataset instance.
@@ -748,63 +748,63 @@
      *             For example, if the dataset already exists.
      * @throws RemoteException
      */
-    void updateDataset(JobId jobId, Dataset dataset) throws AlgebricksException, RemoteException;
+    void updateDataset(TxnId txnId, Dataset dataset) throws AlgebricksException, RemoteException;
 
     /**
      * Adds an extension entity under the ongoing transaction job id
      *
-     * @param jobId
+     * @param txnId
      * @param entity
      * @throws AlgebricksException
      * @throws RemoteException
      */
-    <T extends IExtensionMetadataEntity> void addEntity(JobId jobId, T entity)
+    <T extends IExtensionMetadataEntity> void addEntity(TxnId txnId, T entity)
             throws AlgebricksException, RemoteException;
 
     /**
      * Upserts an extension entity under the ongoing transaction job id
      *
-     * @param jobId
+     * @param txnId
      * @param entity
      * @throws AlgebricksException
      * @throws RemoteException
      */
-    <T extends IExtensionMetadataEntity> void upsertEntity(JobId jobId, T entity)
+    <T extends IExtensionMetadataEntity> void upsertEntity(TxnId txnId, T entity)
             throws AlgebricksException, RemoteException;
 
     /**
      * Deletes an extension entity under the ongoing transaction job id
      *
-     * @param jobId
+     * @param txnId
      * @param entity
      * @throws AlgebricksException
      * @throws RemoteException
      */
-    <T extends IExtensionMetadataEntity> void deleteEntity(JobId jobId, T entity)
+    <T extends IExtensionMetadataEntity> void deleteEntity(TxnId txnId, T entity)
             throws AlgebricksException, RemoteException;
 
     /**
      * Gets a list of extension entities matching a search key under the ongoing
      * transaction
      *
-     * @param jobId
+     * @param txnId
      * @param searchKey
      * @return
      * @throws AlgebricksException
      * @throws RemoteException
      */
-    <T extends IExtensionMetadataEntity> List<T> getEntities(JobId jobId, IExtensionMetadataSearchKey searchKey)
+    <T extends IExtensionMetadataEntity> List<T> getEntities(TxnId txnId, IExtensionMetadataSearchKey searchKey)
             throws AlgebricksException, RemoteException;
 
-    void addFeedConnection(JobId jobId, FeedConnection feedConnection) throws AlgebricksException, RemoteException;
+    void addFeedConnection(TxnId txnId, FeedConnection feedConnection) throws AlgebricksException, RemoteException;
 
-    FeedConnection getFeedConnection(JobId jobId, String dataverseName, String feedName, String datasetName)
+    FeedConnection getFeedConnection(TxnId txnId, String dataverseName, String feedName, String datasetName)
             throws AlgebricksException, RemoteException;
 
-    void dropFeedConnection(JobId jobId, String dataverseName, String feedName, String datasetName)
+    void dropFeedConnection(TxnId txnId, String dataverseName, String feedName, String datasetName)
             throws AlgebricksException, RemoteException;
 
-    List<FeedConnection> getFeedConnections(JobId jobId, String dataverseName, String feedName)
+    List<FeedConnection> getFeedConnections(TxnId txnId, String dataverseName, String feedName)
             throws AlgebricksException, RemoteException;
 
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IValueExtractor.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IValueExtractor.java
index 7d19b20..4cc7719 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IValueExtractor.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IValueExtractor.java
@@ -22,7 +22,7 @@
 import java.io.IOException;
 import java.rmi.RemoteException;
 
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -38,7 +38,7 @@
     /**
      * Extracts an object of type T from a given tuple.
      *
-     * @param jobId
+     * @param txnId
      *            A globally unique transaction id.
      * @param tuple
      *            Tuple from which an object shall be extracted.
@@ -47,5 +47,5 @@
      * @throws HyracksDataException
      * @throws IOException
      */
-    T getValue(JobId jobId, ITupleReference tuple) throws AlgebricksException, HyracksDataException, RemoteException;
+    T getValue(TxnId txnId, ITupleReference tuple) throws AlgebricksException, HyracksDataException, RemoteException;
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
index 463c96b..71ed913 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
@@ -37,7 +37,7 @@
 import org.apache.asterix.common.dataflow.LSMTreeInsertDeleteOperatorDescriptor;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.metadata.LockList;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.asterix.dataflow.data.nontagged.MissingWriterFactory;
 import org.apache.asterix.external.adapter.factory.LookupAdapterFactory;
@@ -156,7 +156,7 @@
     private boolean asyncResults;
     private ResultSetId resultSetId;
     private IResultSerializerFactoryProvider resultSerializerFactoryProvider;
-    private JobId jobId;
+    private TxnId txnId;
     private Map<String, Integer> externalDataLocks;
     private boolean isTemporaryDatasetWriteJob = true;
     private boolean blockingOperatorDisabled = false;
@@ -188,8 +188,8 @@
         return config;
     }
 
-    public void setJobId(JobId jobId) {
-        this.jobId = jobId;
+    public void setTxnId(TxnId txnId) {
+        this.txnId = txnId;
     }
 
     public Dataverse getDefaultDataverse() {
@@ -452,7 +452,7 @@
         }
 
         ISearchOperationCallbackFactory searchCallbackFactory = dataset.getSearchCallbackFactory(
-                storageComponentProvider, theIndex, jobId, IndexOperation.SEARCH, primaryKeyFields);
+                storageComponentProvider, theIndex, txnId, IndexOperation.SEARCH, primaryKeyFields);
         IStorageManager storageManager = getStorageComponentProvider().getStorageManager();
         IIndexDataflowHelperFactory indexHelperFactory = new IndexDataflowHelperFactory(storageManager, spPc.first);
         BTreeSearchOperatorDescriptor btreeSearchOp;
@@ -491,7 +491,7 @@
         }
 
         ISearchOperationCallbackFactory searchCallbackFactory = dataset.getSearchCallbackFactory(
-                storageComponentProvider, secondaryIndex, jobId, IndexOperation.SEARCH, primaryKeyFields);
+                storageComponentProvider, secondaryIndex, txnId, IndexOperation.SEARCH, primaryKeyFields);
         RTreeSearchOperatorDescriptor rtreeSearchOp;
         IIndexDataflowHelperFactory indexDataflowHelperFactory =
                 new IndexDataflowHelperFactory(storageComponentProvider.getStorageManager(), spPc.first);
@@ -735,8 +735,8 @@
         }
     }
 
-    public JobId getJobId() {
-        return jobId;
+    public TxnId getTxnId() {
+        return txnId;
     }
 
     public static ILinearizeComparatorFactory proposeLinearizer(ATypeTag keyType, int numKeyFields)
@@ -797,7 +797,7 @@
             // files index
             RecordDescriptor outRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
             ISearchOperationCallbackFactory searchOpCallbackFactory = dataset
-                    .getSearchCallbackFactory(storageComponentProvider, fileIndex, jobId, IndexOperation.SEARCH, null);
+                    .getSearchCallbackFactory(storageComponentProvider, fileIndex, txnId, IndexOperation.SEARCH, null);
             // Create the operator
             ExternalLookupOperatorDescriptor op = new ExternalLookupOperatorDescriptor(jobSpec, adapterFactory,
                     outRecDesc, indexDataflowHelperFactory, searchOpCallbackFactory,
@@ -972,7 +972,7 @@
             primaryKeyFields[i] = i;
         }
         IModificationOperationCallbackFactory modificationCallbackFactory = dataset.getModificationCallbackFactory(
-                storageComponentProvider, primaryIndex, jobId, indexOp, primaryKeyFields);
+                storageComponentProvider, primaryIndex, txnId, indexOp, primaryKeyFields);
         IIndexDataflowHelperFactory idfh =
                 new IndexDataflowHelperFactory(storageComponentProvider.getStorageManager(), splitsAndConstraint.first);
         IOperatorDescriptor op;
@@ -1097,9 +1097,9 @@
             Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
                     getSplitProviderAndConstraints(dataset, secondaryIndex.getIndexName());
             // prepare callback
-            JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
+            TxnId txnId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getTxnId();
             IModificationOperationCallbackFactory modificationCallbackFactory = dataset.getModificationCallbackFactory(
-                    storageComponentProvider, secondaryIndex, jobId, indexOp, modificationCallbackPrimaryKeyFields);
+                    storageComponentProvider, secondaryIndex, txnId, indexOp, modificationCallbackPrimaryKeyFields);
             IIndexDataflowHelperFactory idfh = new IndexDataflowHelperFactory(
                     storageComponentProvider.getStorageManager(), splitsAndConstraint.first);
             IOperatorDescriptor op;
@@ -1198,9 +1198,9 @@
                 getSplitProviderAndConstraints(dataset, secondaryIndex.getIndexName());
 
         // prepare callback
-        JobId planJobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
+        TxnId planTxnId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getTxnId();
         IModificationOperationCallbackFactory modificationCallbackFactory = dataset.getModificationCallbackFactory(
-                storageComponentProvider, secondaryIndex, planJobId, indexOp, modificationCallbackPrimaryKeyFields);
+                storageComponentProvider, secondaryIndex, planTxnId, indexOp, modificationCallbackPrimaryKeyFields);
         IIndexDataflowHelperFactory indexDataflowHelperFactory =
                 new IndexDataflowHelperFactory(storageComponentProvider.getStorageManager(), splitsAndConstraint.first);
         IOperatorDescriptor op;
@@ -1311,9 +1311,9 @@
                     getSplitProviderAndConstraints(dataset, secondaryIndex.getIndexName());
 
             // prepare callback
-            JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
+            TxnId txnId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getTxnId();
             IModificationOperationCallbackFactory modificationCallbackFactory = dataset.getModificationCallbackFactory(
-                    storageComponentProvider, secondaryIndex, jobId, indexOp, modificationCallbackPrimaryKeyFields);
+                    storageComponentProvider, secondaryIndex, txnId, indexOp, modificationCallbackPrimaryKeyFields);
             IIndexDataflowHelperFactory indexDataFlowFactory = new IndexDataflowHelperFactory(
                     storageComponentProvider.getStorageManager(), splitsAndConstraint.first);
             IOperatorDescriptor op;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/BuiltinTypeMap.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/BuiltinTypeMap.java
index 22732d3..a531add 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/BuiltinTypeMap.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/BuiltinTypeMap.java
@@ -26,7 +26,7 @@
 import java.util.Set;
 
 import org.apache.asterix.common.exceptions.MetadataException;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.metadata.MetadataNode;
 import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.BuiltinType;
@@ -93,12 +93,12 @@
         return new HashSet<>(_builtinTypeMap.values());
     }
 
-    public static IAType getTypeFromTypeName(MetadataNode metadataNode, JobId jobId, String dataverseName,
+    public static IAType getTypeFromTypeName(MetadataNode metadataNode, TxnId txnId, String dataverseName,
             String typeName, boolean optional) throws AlgebricksException {
         IAType type = _builtinTypeMap.get(typeName);
         if (type == null) {
             try {
-                Datatype dt = metadataNode.getDatatype(jobId, dataverseName, typeName);
+                Datatype dt = metadataNode.getDatatype(txnId, dataverseName, typeName);
                 type = dt.getDatatype();
             } catch (RemoteException e) {
                 throw new MetadataException(e);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
index e5f97f0..48a6e6a 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
@@ -42,7 +42,7 @@
 import org.apache.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
 import org.apache.asterix.common.metadata.IDataset;
 import org.apache.asterix.common.transactions.IRecoveryManager.ResourceType;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.common.utils.JobUtils;
 import org.apache.asterix.common.utils.JobUtils.ProgressState;
 import org.apache.asterix.external.feed.management.FeedConnectionId;
@@ -543,7 +543,7 @@
      *
      * @param index
      *            the index
-     * @param jobId
+     * @param txnId
      *            the job id being compiled
      * @param op
      *            the operation this search is part of
@@ -555,7 +555,7 @@
      *             if the callback factory could not be created
      */
     public ISearchOperationCallbackFactory getSearchCallbackFactory(IStorageComponentProvider storageComponentProvider,
-            Index index, JobId jobId, IndexOperation op, int[] primaryKeyFields) throws AlgebricksException {
+            Index index, TxnId txnId, IndexOperation op, int[] primaryKeyFields) throws AlgebricksException {
         if (getDatasetDetails().isTemp()) {
             return NoOpOperationCallbackFactory.INSTANCE;
         } else if (index.isPrimaryIndex()) {
@@ -564,14 +564,14 @@
              * we may acquire very short duration lock(i.e., instant lock) for readers.
              */
             return (op == IndexOperation.UPSERT)
-                    ? new LockThenSearchOperationCallbackFactory(jobId, getDatasetId(), primaryKeyFields,
+                    ? new LockThenSearchOperationCallbackFactory(txnId, getDatasetId(), primaryKeyFields,
                             storageComponentProvider.getTransactionSubsystemProvider(), ResourceType.LSM_BTREE)
-                    : new PrimaryIndexInstantSearchOperationCallbackFactory(jobId, getDatasetId(), primaryKeyFields,
+                    : new PrimaryIndexInstantSearchOperationCallbackFactory(txnId, getDatasetId(), primaryKeyFields,
                             storageComponentProvider.getTransactionSubsystemProvider(), ResourceType.LSM_BTREE);
         } else if (index.getKeyFieldNames().isEmpty()) {
             // this is the case where the index is secondary primary index and locking is required
             // since the secondary primary index replaces the dataset index (which locks)
-            return new PrimaryIndexInstantSearchOperationCallbackFactory(jobId, getDatasetId(), primaryKeyFields,
+            return new PrimaryIndexInstantSearchOperationCallbackFactory(txnId, getDatasetId(), primaryKeyFields,
                     storageComponentProvider.getTransactionSubsystemProvider(), ResourceType.LSM_BTREE);
         }
         return new SecondaryIndexSearchOperationCallbackFactory();
@@ -582,7 +582,7 @@
      *
      * @param index
      *            the index
-     * @param jobId
+     * @param txnId
      *            the job id of the job being compiled
      * @param op
      *            the operation performed for this callback
@@ -594,31 +594,31 @@
      *             If the callback factory could not be created
      */
     public IModificationOperationCallbackFactory getModificationCallbackFactory(
-            IStorageComponentProvider componentProvider, Index index, JobId jobId, IndexOperation op,
+            IStorageComponentProvider componentProvider, Index index, TxnId txnId, IndexOperation op,
             int[] primaryKeyFields) throws AlgebricksException {
         if (getDatasetDetails().isTemp()) {
             return op == IndexOperation.DELETE || op == IndexOperation.INSERT || op == IndexOperation.UPSERT
                     ? index.isPrimaryIndex()
-                            ? new TempDatasetPrimaryIndexModificationOperationCallbackFactory(jobId, datasetId,
+                            ? new TempDatasetPrimaryIndexModificationOperationCallbackFactory(txnId, datasetId,
                                     primaryKeyFields, componentProvider.getTransactionSubsystemProvider(),
                                     Operation.get(op), index.resourceType())
-                            : new TempDatasetSecondaryIndexModificationOperationCallbackFactory(jobId, getDatasetId(),
+                            : new TempDatasetSecondaryIndexModificationOperationCallbackFactory(txnId, getDatasetId(),
                                     primaryKeyFields, componentProvider.getTransactionSubsystemProvider(),
                                     Operation.get(op), index.resourceType())
                     : NoOpOperationCallbackFactory.INSTANCE;
         } else if (index.isPrimaryIndex()) {
             return op == IndexOperation.UPSERT
-                    ? new UpsertOperationCallbackFactory(jobId, getDatasetId(), primaryKeyFields,
+                    ? new UpsertOperationCallbackFactory(txnId, getDatasetId(), primaryKeyFields,
                             componentProvider.getTransactionSubsystemProvider(), Operation.get(op),
                             index.resourceType())
                     : op == IndexOperation.DELETE || op == IndexOperation.INSERT
-                            ? new PrimaryIndexModificationOperationCallbackFactory(jobId, getDatasetId(),
+                            ? new PrimaryIndexModificationOperationCallbackFactory(txnId, getDatasetId(),
                                     primaryKeyFields, componentProvider.getTransactionSubsystemProvider(),
                                     Operation.get(op), index.resourceType())
                             : NoOpOperationCallbackFactory.INSTANCE;
         } else {
             return op == IndexOperation.DELETE || op == IndexOperation.INSERT || op == IndexOperation.UPSERT
-                    ? new SecondaryIndexModificationOperationCallbackFactory(jobId, getDatasetId(), primaryKeyFields,
+                    ? new SecondaryIndexModificationOperationCallbackFactory(txnId, getDatasetId(), primaryKeyFields,
                             componentProvider.getTransactionSubsystemProvider(), Operation.get(op),
                             index.resourceType())
                     : NoOpOperationCallbackFactory.INSTANCE;
@@ -665,7 +665,7 @@
      *
      * @param metadataProvider,
      *            the metadata provider.
-     * @param jobId,
+     * @param txnId,
      *            the AsterixDB job id for transaction management.
      * @param primaryKeyFieldPermutation,
      *            the primary key field permutation according to the input.
@@ -674,10 +674,10 @@
      * @return the commit runtime factory for inserting/upserting/deleting operations on this dataset.
      * @throws AlgebricksException
      */
-    public IPushRuntimeFactory getCommitRuntimeFactory(MetadataProvider metadataProvider, JobId jobId,
+    public IPushRuntimeFactory getCommitRuntimeFactory(MetadataProvider metadataProvider, TxnId txnId,
             int[] primaryKeyFieldPermutation, boolean isSink) throws AlgebricksException {
         int[] datasetPartitions = getDatasetPartitions(metadataProvider);
-        return new CommitRuntimeFactory(jobId, datasetId, primaryKeyFieldPermutation,
+        return new CommitRuntimeFactory(txnId, datasetId, primaryKeyFieldPermutation,
                 metadataProvider.isTemporaryDatasetWriteJob(), metadataProvider.isWriteTransaction(), datasetPartitions,
                 isSink);
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
index f75ea8f..6d7e25f 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
@@ -29,7 +29,7 @@
 import org.apache.asterix.builders.IARecordBuilder;
 import org.apache.asterix.builders.OrderedListBuilder;
 import org.apache.asterix.builders.RecordBuilder;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.metadata.MetadataNode;
 import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
@@ -79,11 +79,11 @@
     private ISerializerDeserializer<ARecord> recordSerDes =
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(MetadataRecordTypes.DATATYPE_RECORDTYPE);
     private final MetadataNode metadataNode;
-    private final JobId jobId;
+    private final TxnId txnId;
 
-    protected DatatypeTupleTranslator(JobId jobId, MetadataNode metadataNode, boolean getTuple) {
+    protected DatatypeTupleTranslator(TxnId txnId, MetadataNode metadataNode, boolean getTuple) {
         super(getTuple, MetadataPrimaryIndexes.DATATYPE_DATASET.getFieldCount());
-        this.jobId = jobId;
+        this.txnId = txnId;
         this.metadataNode = metadataNode;
     }
 
@@ -142,7 +142,7 @@
                         boolean isNullable = ((ABoolean) field
                                 .getValueByPos(MetadataRecordTypes.FIELD_ARECORD_ISNULLABLE_FIELD_INDEX)).getBoolean()
                                         .booleanValue();
-                        fieldTypes[fieldId] = BuiltinTypeMap.getTypeFromTypeName(metadataNode, jobId, dataverseName,
+                        fieldTypes[fieldId] = BuiltinTypeMap.getTypeFromTypeName(metadataNode, txnId, dataverseName,
                                 fieldTypeName, isNullable);
                         fieldId++;
                     }
@@ -154,7 +154,7 @@
                             .getValueByPos(MetadataRecordTypes.DERIVEDTYPE_ARECORD_UNORDEREDLIST_FIELD_INDEX))
                                     .getStringValue();
                     return new Datatype(dataverseName, datatypeName,
-                            new AUnorderedListType(BuiltinTypeMap.getTypeFromTypeName(metadataNode, jobId,
+                            new AUnorderedListType(BuiltinTypeMap.getTypeFromTypeName(metadataNode, txnId,
                                     dataverseName, unorderedlistTypeName, false), datatypeName),
                             isAnonymous);
                 }
@@ -163,7 +163,7 @@
                             .getValueByPos(MetadataRecordTypes.DERIVEDTYPE_ARECORD_ORDEREDLIST_FIELD_INDEX))
                                     .getStringValue();
                     return new Datatype(dataverseName, datatypeName,
-                            new AOrderedListType(BuiltinTypeMap.getTypeFromTypeName(metadataNode, jobId, dataverseName,
+                            new AOrderedListType(BuiltinTypeMap.getTypeFromTypeName(metadataNode, txnId, dataverseName,
                                     orderedlistTypeName, false), datatypeName),
                             isAnonymous);
                 }
@@ -365,7 +365,7 @@
     private String handleNestedDerivedType(String typeName, AbstractComplexType nestedType, Datatype topLevelType,
             String dataverseName, String datatypeName) throws HyracksDataException {
         try {
-            metadataNode.addDatatype(jobId, new Datatype(dataverseName, typeName, nestedType, true));
+            metadataNode.addDatatype(txnId, new Datatype(dataverseName, typeName, nestedType, true));
         } catch (AlgebricksException e) {
             // The nested record type may have been inserted by a previous DDL statement or
             // by
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
index 8989cdc..a154d7f 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
@@ -29,7 +29,7 @@
 
 import org.apache.asterix.builders.OrderedListBuilder;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.metadata.MetadataNode;
 import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
@@ -94,11 +94,11 @@
     private ISerializerDeserializer<ARecord> recordSerde =
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(MetadataRecordTypes.INDEX_RECORDTYPE);
     private final MetadataNode metadataNode;
-    private final JobId jobId;
+    private final TxnId txnId;
 
-    protected IndexTupleTranslator(JobId jobId, MetadataNode metadataNode, boolean getTuple) {
+    protected IndexTupleTranslator(TxnId txnId, MetadataNode metadataNode, boolean getTuple) {
         super(getTuple, MetadataPrimaryIndexes.INDEX_DATASET.getFieldCount());
-        this.jobId = jobId;
+        this.txnId = txnId;
         this.metadataNode = metadataNode;
     }
 
@@ -141,7 +141,7 @@
         List<IAType> searchKeyType = new ArrayList<>(searchKey.size());
         while (fieldTypeCursor.next()) {
             String typeName = ((AString) fieldTypeCursor.get()).getStringValue();
-            IAType fieldType = BuiltinTypeMap.getTypeFromTypeName(metadataNode, jobId, dvName, typeName, false);
+            IAType fieldType = BuiltinTypeMap.getTypeFromTypeName(metadataNode, txnId, dvName, typeName, false);
             searchKeyType.add(fieldType);
         }
         boolean isOverridingKeyTypes = !searchKeyType.isEmpty();
@@ -180,16 +180,16 @@
         // from the record metadata
         if (searchKeyType.isEmpty()) {
             try {
-                Dataset dSet = metadataNode.getDataset(jobId, dvName, dsName);
+                Dataset dSet = metadataNode.getDataset(txnId, dvName, dsName);
                 String datatypeName = dSet.getItemTypeName();
                 String datatypeDataverseName = dSet.getItemTypeDataverseName();
                 ARecordType recordDt = (ARecordType) metadataNode
-                        .getDatatype(jobId, datatypeDataverseName, datatypeName).getDatatype();
+                        .getDatatype(txnId, datatypeDataverseName, datatypeName).getDatatype();
                 String metatypeName = dSet.getMetaItemTypeName();
                 String metatypeDataverseName = dSet.getMetaItemTypeDataverseName();
                 ARecordType metaDt = null;
                 if (metatypeName != null && metatypeDataverseName != null) {
-                    metaDt = (ARecordType) metadataNode.getDatatype(jobId, metatypeDataverseName, metatypeName)
+                    metaDt = (ARecordType) metadataNode.getDatatype(txnId, metatypeDataverseName, metatypeName)
                             .getDatatype();
                 }
                 searchKeyType = KeyFieldTypeUtil.getKeyTypes(recordDt, metaDt, searchKey, keyFieldSourceIndicator);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/MetadataTupleTranslatorProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/MetadataTupleTranslatorProvider.java
index 20f04c4..0625fc4 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/MetadataTupleTranslatorProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/MetadataTupleTranslatorProvider.java
@@ -18,7 +18,7 @@
  */
 package org.apache.asterix.metadata.entitytupletranslators;
 
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.metadata.MetadataNode;
 
 public class MetadataTupleTranslatorProvider {
@@ -35,9 +35,9 @@
         return new DatasourceAdapterTupleTranslator(getTuple);
     }
 
-    public DatatypeTupleTranslator getDataTypeTupleTranslator(JobId jobId, MetadataNode metadataNode,
+    public DatatypeTupleTranslator getDataTypeTupleTranslator(TxnId txnId, MetadataNode metadataNode,
             boolean getTuple) {
-        return new DatatypeTupleTranslator(jobId, metadataNode, getTuple);
+        return new DatatypeTupleTranslator(txnId, metadataNode, getTuple);
     }
 
     public DataverseTupleTranslator getDataverseTupleTranslator(boolean getTuple) {
@@ -60,8 +60,8 @@
         return new FunctionTupleTranslator(getTuple);
     }
 
-    public IndexTupleTranslator getIndexTupleTranslator(JobId jobId, MetadataNode metadataNode, boolean getTuple) {
-        return new IndexTupleTranslator(jobId, metadataNode, getTuple);
+    public IndexTupleTranslator getIndexTupleTranslator(TxnId txnId, MetadataNode metadataNode, boolean getTuple) {
+        return new IndexTupleTranslator(txnId, metadataNode, getTuple);
     }
 
     public LibraryTupleTranslator getLibraryTupleTranslator(boolean getTuple) {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
index 2c457a9..07d3c69 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
@@ -38,7 +38,7 @@
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.transactions.IRecoveryManager;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.external.indexing.IndexingConstants;
 import org.apache.asterix.formats.base.IDataFormat;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
@@ -337,13 +337,13 @@
      *            the metadata provider.
      * @param dataset,
      *            the dataset to scan.
-     * @param jobId,
+     * @param txnId,
      *            the AsterixDB job id for transaction management.
      * @return a primary index scan operator.
      * @throws AlgebricksException
      */
     public static IOperatorDescriptor createPrimaryIndexScanOp(JobSpecification spec, MetadataProvider metadataProvider,
-            Dataset dataset, JobId jobId) throws AlgebricksException {
+            Dataset dataset, TxnId txnId) throws AlgebricksException {
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> primarySplitsAndConstraint =
                 metadataProvider.getSplitProviderAndConstraints(dataset);
         IFileSplitProvider primaryFileSplitProvider = primarySplitsAndConstraint.first;
@@ -355,7 +355,7 @@
         ITransactionSubsystemProvider txnSubsystemProvider = TransactionSubsystemProvider.INSTANCE;
         boolean temp = dataset.getDatasetDetails().isTemp();
         ISearchOperationCallbackFactory searchCallbackFactory = temp ? NoOpOperationCallbackFactory.INSTANCE
-                : new PrimaryIndexInstantSearchOperationCallbackFactory(jobId, dataset.getDatasetId(),
+                : new PrimaryIndexInstantSearchOperationCallbackFactory(txnId, dataset.getDatasetId(),
                         dataset.getPrimaryBloomFilterFields(), txnSubsystemProvider,
                         IRecoveryManager.ResourceType.LSM_BTREE);
         IndexDataflowHelperFactory indexHelperFactory = new IndexDataflowHelperFactory(
@@ -399,7 +399,7 @@
                 metadataProvider.getSplitProviderAndConstraints(dataset);
 
         // prepare callback
-        JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
+        TxnId txnId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getTxnId();
         int[] primaryKeyFields = new int[numKeys];
         for (int i = 0; i < numKeys; i++) {
             primaryKeyFields[i] = i;
@@ -408,9 +408,9 @@
                 metadataProvider.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName()).size() > 1;
         IStorageComponentProvider storageComponentProvider = metadataProvider.getStorageComponentProvider();
         IModificationOperationCallbackFactory modificationCallbackFactory = dataset.getModificationCallbackFactory(
-                storageComponentProvider, primaryIndex, jobId, IndexOperation.UPSERT, primaryKeyFields);
+                storageComponentProvider, primaryIndex, txnId, IndexOperation.UPSERT, primaryKeyFields);
         ISearchOperationCallbackFactory searchCallbackFactory = dataset.getSearchCallbackFactory(
-                storageComponentProvider, primaryIndex, jobId, IndexOperation.UPSERT, primaryKeyFields);
+                storageComponentProvider, primaryIndex, txnId, IndexOperation.UPSERT, primaryKeyFields);
         IIndexDataflowHelperFactory idfh =
                 new IndexDataflowHelperFactory(storageComponentProvider.getStorageManager(), splitsAndConstraint.first);
         LSMPrimaryUpsertOperatorDescriptor op;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
index cc6923e..e6a24e3 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
@@ -28,20 +28,19 @@
 import org.apache.asterix.common.config.OptimizationConfUtil;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.external.indexing.ExternalFile;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
 import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
-import org.apache.asterix.transaction.management.service.transaction.JobIdFactory;
+import org.apache.asterix.transaction.management.service.transaction.TxnIdFactory;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.job.IJobletEventListenerFactory;
 import org.apache.hyracks.api.job.JobSpecification;
-import org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor;
 
 public class IndexUtil {
 
@@ -162,13 +161,13 @@
      *            the metadata provider.
      * @return the AsterixDB job id for transaction management.
      */
-    public static JobId bindJobEventListener(JobSpecification spec, MetadataProvider metadataProvider) {
-        JobId jobId = JobIdFactory.generateJobId();
-        metadataProvider.setJobId(jobId);
+    public static TxnId bindJobEventListener(JobSpecification spec, MetadataProvider metadataProvider) {
+        TxnId txnId = TxnIdFactory.create();
+        metadataProvider.setTxnId(txnId);
         boolean isWriteTransaction = metadataProvider.isWriteTransaction();
-        IJobletEventListenerFactory jobEventListenerFactory = new JobEventListenerFactory(jobId, isWriteTransaction);
+        IJobletEventListenerFactory jobEventListenerFactory = new JobEventListenerFactory(txnId, isWriteTransaction);
         spec.setJobletEventListenerFactory(jobEventListenerFactory);
-        return jobId;
+        return txnId;
     }
 
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
index 78d9c19..8f70f21 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
@@ -22,7 +22,7 @@
 
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.GlobalConfig;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.external.indexing.IndexingConstants;
 import org.apache.asterix.external.operators.ExternalScanOperatorDescriptor;
 import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -129,11 +129,11 @@
             // Create dummy key provider for feeding the primary index scan.
             IOperatorDescriptor keyProviderOp = DatasetUtil.createDummyKeyProviderOp(spec, dataset,
                     metadataProvider);
-            JobId jobId = IndexUtil.bindJobEventListener(spec, metadataProvider);
+            TxnId txnId = IndexUtil.bindJobEventListener(spec, metadataProvider);
 
             // Create primary index scan op.
             IOperatorDescriptor primaryScanOp = DatasetUtil.createPrimaryIndexScanOp(spec, metadataProvider, dataset,
-                    jobId);
+                    txnId);
 
             // Assign op.
             IOperatorDescriptor sourceOp = primaryScanOp;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
index 8ef5f34..89bd4b1 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
@@ -21,7 +21,7 @@
 import java.util.List;
 
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
@@ -72,14 +72,14 @@
 
         // only handle internal datasets
         // Create dummy key provider for feeding the primary index scan.
-        JobId jobId = IndexUtil.bindJobEventListener(spec, metadataProvider);
+        TxnId txnId = IndexUtil.bindJobEventListener(spec, metadataProvider);
 
         // Create dummy key provider for feeding the primary index scan.
         IOperatorDescriptor keyProviderOp = DatasetUtil.createDummyKeyProviderOp(spec, dataset, metadataProvider);
 
         // Create primary index scan op.
         IOperatorDescriptor primaryScanOp = createPrimaryIndexScanDiskComponentsOp(spec, metadataProvider,
-                getTaggedRecordDescriptor(dataset.getPrimaryRecordDescriptor(metadataProvider)), jobId);
+                getTaggedRecordDescriptor(dataset.getPrimaryRecordDescriptor(metadataProvider)), txnId);
 
         // Assign op.
         IOperatorDescriptor sourceOp = primaryScanOp;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
index 262b259..93cc11d 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
@@ -21,7 +21,7 @@
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.dataflow.data.nontagged.MissingWriterFactory;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
@@ -206,14 +206,14 @@
     @Override
     public JobSpecification buildLoadingJobSpec() throws AlgebricksException {
         JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext());
-        JobId jobId = IndexUtil.bindJobEventListener(spec, metadataProvider);
+        TxnId txnId = IndexUtil.bindJobEventListener(spec, metadataProvider);
 
         // Create dummy key provider for feeding the primary index scan.
         IOperatorDescriptor keyProviderOp = DatasetUtil.createDummyKeyProviderOp(spec, dataset, metadataProvider);
 
         // Create primary index scan op.
         IOperatorDescriptor primaryScanOp = createPrimaryIndexScanDiskComponentsOp(spec, metadataProvider,
-                getTaggedRecordDescriptor(dataset.getPrimaryRecordDescriptor(metadataProvider)), jobId);
+                getTaggedRecordDescriptor(dataset.getPrimaryRecordDescriptor(metadataProvider)), txnId);
 
         IOperatorDescriptor sourceOp = primaryScanOp;
         boolean isOverridingKeyFieldTypes = index.isOverridingKeyFieldTypes();
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
index 9106193..1333493 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
@@ -23,7 +23,7 @@
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.formats.nontagged.TypeTraitProvider;
@@ -184,11 +184,11 @@
 
         // Create dummy key provider for feeding the primary index scan.
         IOperatorDescriptor keyProviderOp = DatasetUtil.createDummyKeyProviderOp(spec, dataset, metadataProvider);
-        JobId jobId = IndexUtil.bindJobEventListener(spec, metadataProvider);
+        TxnId txnId = IndexUtil.bindJobEventListener(spec, metadataProvider);
 
         // Create primary index scan op.
         IOperatorDescriptor primaryScanOp = createPrimaryIndexScanDiskComponentsOp(spec, metadataProvider,
-                getTaggedRecordDescriptor(dataset.getPrimaryRecordDescriptor(metadataProvider)), jobId);
+                getTaggedRecordDescriptor(dataset.getPrimaryRecordDescriptor(metadataProvider)), txnId);
 
         // Assign op.
         IOperatorDescriptor sourceOp = primaryScanOp;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
index b99ae2f..994370c 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
@@ -24,14 +24,12 @@
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.transactions.IRecoveryManager;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.dataflow.data.nontagged.MissingWriterFactory;
 import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
-import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.runtime.operators.LSMSecondaryIndexBulkLoadOperatorDescriptor;
 import org.apache.asterix.runtime.operators.LSMSecondaryIndexCreationTupleProcessorOperatorDescriptor;
 import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexInstantSearchOperationCallbackFactory;
@@ -276,11 +274,11 @@
     }
 
     protected IOperatorDescriptor createPrimaryIndexScanDiskComponentsOp(JobSpecification spec,
-            MetadataProvider metadataProvider, RecordDescriptor outRecDesc, JobId jobId) throws AlgebricksException {
+            MetadataProvider metadataProvider, RecordDescriptor outRecDesc, TxnId txnId) throws AlgebricksException {
         ITransactionSubsystemProvider txnSubsystemProvider = TransactionSubsystemProvider.INSTANCE;
         boolean temp = dataset.getDatasetDetails().isTemp();
         ISearchOperationCallbackFactory searchCallbackFactory = temp ? NoOpOperationCallbackFactory.INSTANCE
-                : new PrimaryIndexInstantSearchOperationCallbackFactory(jobId, dataset.getDatasetId(),
+                : new PrimaryIndexInstantSearchOperationCallbackFactory(txnId, dataset.getDatasetId(),
                         dataset.getPrimaryBloomFilterFields(), txnSubsystemProvider,
                         IRecoveryManager.ResourceType.LSM_BTREE);
         IndexDataflowHelperFactory indexHelperFactory = new IndexDataflowHelperFactory(
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
index 077e431..3626f16 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
@@ -22,7 +22,7 @@
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.dataflow.data.nontagged.MissingWriterFactory;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
@@ -208,14 +208,14 @@
     @Override
     public JobSpecification buildLoadingJobSpec() throws AlgebricksException {
         JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext());
-        JobId jobId = IndexUtil.bindJobEventListener(spec, metadataProvider);
+        TxnId txnId = IndexUtil.bindJobEventListener(spec, metadataProvider);
 
         // Create dummy key provider for feeding the primary index scan.
         IOperatorDescriptor keyProviderOp = DatasetUtil.createDummyKeyProviderOp(spec, dataset, metadataProvider);
 
         // Create primary index scan op.
         IOperatorDescriptor primaryScanOp =
-                DatasetUtil.createPrimaryIndexScanOp(spec, metadataProvider, dataset, jobId);
+                DatasetUtil.createPrimaryIndexScanOp(spec, metadataProvider, dataset, txnId);
 
         IOperatorDescriptor sourceOp = primaryScanOp;
         boolean isOverridingKeyFieldTypes = index.isOverridingKeyFieldTypes();
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
index b11b527..613df21 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
@@ -23,7 +23,7 @@
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.external.indexing.IndexingConstants;
 import org.apache.asterix.external.operators.ExternalScanOperatorDescriptor;
 import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
@@ -201,11 +201,11 @@
         if (dataset.getDatasetType() == DatasetType.INTERNAL) {
             // Create dummy key provider for feeding the primary index scan.
             IOperatorDescriptor keyProviderOp = DatasetUtil.createDummyKeyProviderOp(spec, dataset, metadataProvider);
-            JobId jobId = IndexUtil.bindJobEventListener(spec, metadataProvider);
+            TxnId txnId = IndexUtil.bindJobEventListener(spec, metadataProvider);
 
             // Create primary index scan op.
             IOperatorDescriptor primaryScanOp = DatasetUtil.createPrimaryIndexScanOp(spec, metadataProvider, dataset,
-                    jobId);
+                    txnId);
 
             // Assign op.
             IOperatorDescriptor sourceOp = primaryScanOp;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/MetadataEntityValueExtractor.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/MetadataEntityValueExtractor.java
index 40623bd..54d94a9 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/MetadataEntityValueExtractor.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/MetadataEntityValueExtractor.java
@@ -21,8 +21,7 @@
 
 import java.rmi.RemoteException;
 
-import org.apache.asterix.common.exceptions.MetadataException;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.metadata.api.IMetadataEntityTupleTranslator;
 import org.apache.asterix.metadata.api.IValueExtractor;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -40,7 +39,7 @@
     }
 
     @Override
-    public T getValue(JobId jobId, ITupleReference tuple)
+    public T getValue(TxnId txnId, ITupleReference tuple)
             throws AlgebricksException, HyracksDataException, RemoteException {
         return tupleReaderWriter.getMetadataEntityFromTuple(tuple);
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
index 22aea26..cf02930 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
@@ -24,7 +24,7 @@
 import java.io.DataInputStream;
 import java.io.IOException;
 
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.metadata.api.IValueExtractor;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -49,7 +49,7 @@
     private final UTF8StringReader reader = new UTF8StringReader();
 
     @Override
-    public String getValue(JobId jobId, ITupleReference tuple) throws AlgebricksException, HyracksDataException {
+    public String getValue(TxnId txnId, ITupleReference tuple) throws AlgebricksException, HyracksDataException {
         byte[] serRecord = tuple.getFieldData(2);
         int recordStartOffset = tuple.getFieldStart(2);
         int recordLength = tuple.getFieldLength(2);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/TupleCopyValueExtractor.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/TupleCopyValueExtractor.java
index 5f16543..8960ba6 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/TupleCopyValueExtractor.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/TupleCopyValueExtractor.java
@@ -21,7 +21,7 @@
 
 import java.nio.ByteBuffer;
 
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.metadata.api.IValueExtractor;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
@@ -48,7 +48,7 @@
     }
 
     @Override
-    public ITupleReference getValue(JobId jobId, ITupleReference tuple)
+    public ITupleReference getValue(TxnId txnId, ITupleReference tuple)
             throws AlgebricksException, HyracksDataException {
         int numBytes = tupleWriter.bytesRequired(tuple);
         tupleBytes = new byte[numBytes];
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationChannel.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationChannel.java
index 5e58802..9d8c351 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationChannel.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationChannel.java
@@ -470,7 +470,7 @@
                     case LogType.JOB_COMMIT:
                     case LogType.ABORT:
                         LogRecord jobTerminationLog = new LogRecord();
-                        TransactionUtil.formJobTerminateLogRecord(jobTerminationLog, remoteLog.getJobId(),
+                        TransactionUtil.formJobTerminateLogRecord(jobTerminationLog, remoteLog.getTxnId(),
                                 remoteLog.getLogType() == LogType.JOB_COMMIT);
                         jobTerminationLog.setReplicationThread(this);
                         jobTerminationLog.setLogSource(LogSource.REMOTE);
@@ -523,7 +523,7 @@
                     LogRecord logRecord = pendingNotificationRemoteLogsQ.take();
                     //send ACK to requester
                     logRecord.getReplicationThread().getReplicationClientSocket().socket().getOutputStream()
-                            .write((localNodeID + ReplicationProtocol.JOB_REPLICATION_ACK + logRecord.getJobId()
+                            .write((localNodeID + ReplicationProtocol.JOB_REPLICATION_ACK + logRecord.getTxnId()
                                     + System.lineSeparator()).getBytes());
                 } catch (InterruptedException e) {
                     Thread.currentThread().interrupt();
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationManager.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationManager.java
index 126114b..b0aa0fb 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationManager.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationManager.java
@@ -99,8 +99,8 @@
     private static final int MAX_JOB_COMMIT_ACK_WAIT = 10000;
     private final String nodeId;
     private ExecutorService replicationListenerThreads;
-    private final Map<Integer, Set<String>> jobCommitAcks;
-    private final Map<Integer, ILogRecord> replicationJobsPendingAcks;
+    private final Map<Long, Set<String>> txnCommitAcks;
+    private final Map<Long, ILogRecord> replicationTxnsPendingAcks;
     private ByteBuffer dataBuffer;
     private final LinkedBlockingQueue<IReplicationJob> replicationJobsQ;
     private final LinkedBlockingQueue<ReplicaEvent> replicaEventsQ;
@@ -157,8 +157,8 @@
         terminateJobsReplication = new AtomicBoolean(false);
         jobsReplicationSuspended = new AtomicBoolean(true);
         replicationSuspended = new AtomicBoolean(true);
-        jobCommitAcks = new ConcurrentHashMap<>();
-        replicationJobsPendingAcks = new ConcurrentHashMap<>();
+        txnCommitAcks = new ConcurrentHashMap<>();
+        replicationTxnsPendingAcks = new ConcurrentHashMap<>();
         shuttingDownReplicaIds = new HashSet<>();
         dataBuffer = ByteBuffer.allocate(INITIAL_BUFFER_SIZE);
         replicationMonitor = new ReplicasEventsMonitor();
@@ -228,7 +228,7 @@
             }
             Set<String> replicaIds = Collections.synchronizedSet(new HashSet<String>());
             replicaIds.add(nodeId);
-            jobCommitAcks.put(logRecord.getJobId(), replicaIds);
+            txnCommitAcks.put(logRecord.getTxnId(), replicaIds);
         }
 
         appendToLogBuffer(logRecord);
@@ -580,16 +580,16 @@
 
         //wait for any ACK to arrive before closing sockets.
         if (logsRepSockets != null) {
-            synchronized (jobCommitAcks) {
+            synchronized (txnCommitAcks) {
                 try {
                     long waitStartTime = System.currentTimeMillis();
-                    while (!jobCommitAcks.isEmpty()) {
-                        jobCommitAcks.wait(1000);
+                    while (!txnCommitAcks.isEmpty()) {
+                        txnCommitAcks.wait(1000);
                         long waitDuration = System.currentTimeMillis() - waitStartTime;
                         if (waitDuration > MAX_JOB_COMMIT_ACK_WAIT) {
                             LOGGER.log(Level.SEVERE,
-                                    "Timeout before receving all job ACKs from replicas. Pending jobs ("
-                                            + jobCommitAcks.keySet().toString() + ")");
+                                    "Timeout before receving all job ACKs from replicas. Pending txns ("
+                                            + txnCommitAcks.keySet().toString() + ")");
                             break;
                         }
                     }
@@ -747,9 +747,9 @@
 
             if (newState == ReplicaState.DEAD) {
                 //assume the dead replica ACK has been received for all pending jobs
-                synchronized (jobCommitAcks) {
-                    for (Integer jobId : jobCommitAcks.keySet()) {
-                        addAckToJob(jobId, replicaId);
+                synchronized (txnCommitAcks) {
+                    for (Long txnId : txnCommitAcks.keySet()) {
+                        addAckToJob(txnId, replicaId);
                     }
                 }
             }
@@ -777,27 +777,27 @@
     /**
      * When an ACK for a JOB_COMMIT is received, it is added to the corresponding job.
      *
-     * @param jobId
+     * @param txnId
      * @param replicaId
      *            The remote replica id the ACK received from.
      */
-    private void addAckToJob(int jobId, String replicaId) {
-        synchronized (jobCommitAcks) {
+    private void addAckToJob(long txnId, String replicaId) {
+        synchronized (txnCommitAcks) {
             //add ACK to the job
-            if (jobCommitAcks.containsKey(jobId)) {
-                Set<String> replicaIds = jobCommitAcks.get(jobId);
+            if (txnCommitAcks.containsKey(txnId)) {
+                Set<String> replicaIds = txnCommitAcks.get(txnId);
                 replicaIds.add(replicaId);
             } else {
                 if (LOGGER.isLoggable(Level.WARNING)) {
-                    LOGGER.warning("Invalid job replication ACK received for jobId(" + jobId + ")");
+                    LOGGER.warning("Invalid job replication ACK received for txnId(" + txnId + ")");
                 }
                 return;
             }
 
             //if got ACKs from all remote replicas, notify pending jobs if any
 
-            if (jobCommitAcks.get(jobId).size() == replicationFactor && replicationJobsPendingAcks.containsKey(jobId)) {
-                ILogRecord pendingLog = replicationJobsPendingAcks.get(jobId);
+            if (txnCommitAcks.get(txnId).size() == replicationFactor && replicationTxnsPendingAcks.containsKey(txnId)) {
+                ILogRecord pendingLog = replicationTxnsPendingAcks.get(txnId);
                 synchronized (pendingLog) {
                     pendingLog.notifyAll();
                 }
@@ -807,23 +807,23 @@
 
     @Override
     public boolean hasBeenReplicated(ILogRecord logRecord) {
-        int jobId = logRecord.getJobId();
-        if (jobCommitAcks.containsKey(jobId)) {
-            synchronized (jobCommitAcks) {
+        long txnId = logRecord.getTxnId();
+        if (txnCommitAcks.containsKey(txnId)) {
+            synchronized (txnCommitAcks) {
                 //check if all ACKs have been received
-                if (jobCommitAcks.get(jobId).size() == replicationFactor) {
-                    jobCommitAcks.remove(jobId);
+                if (txnCommitAcks.get(txnId).size() == replicationFactor) {
+                    txnCommitAcks.remove(txnId);
 
                     //remove from pending jobs if exists
-                    replicationJobsPendingAcks.remove(jobId);
+                    replicationTxnsPendingAcks.remove(txnId);
 
                     //notify any threads waiting for all jobs to finish
-                    if (jobCommitAcks.size() == 0) {
-                        jobCommitAcks.notifyAll();
+                    if (txnCommitAcks.size() == 0) {
+                        txnCommitAcks.notifyAll();
                     }
                     return true;
                 } else {
-                    replicationJobsPendingAcks.putIfAbsent(jobId, logRecord);
+                    replicationTxnsPendingAcks.putIfAbsent(txnId, logRecord);
                     return false;
                 }
             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/job/listener/JobEventListenerFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/job/listener/JobEventListenerFactory.java
index 2749b5a..1422e42 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/job/listener/JobEventListenerFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/job/listener/JobEventListenerFactory.java
@@ -23,7 +23,7 @@
 import org.apache.asterix.common.transactions.DatasetId;
 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.TxnId;
 import org.apache.hyracks.api.context.IHyracksJobletContext;
 import org.apache.hyracks.api.job.IJobletEventListener;
 import org.apache.hyracks.api.job.IJobletEventListenerFactory;
@@ -32,16 +32,16 @@
 public class JobEventListenerFactory implements IJobletEventListenerFactory {
 
     private static final long serialVersionUID = 1L;
-    private final JobId jobId;
+    private final TxnId txnId;
     private final boolean transactionalWrite;
 
-    public JobEventListenerFactory(JobId jobId, boolean transactionalWrite) {
-        this.jobId = jobId;
+    public JobEventListenerFactory(TxnId txnId, boolean transactionalWrite) {
+        this.txnId = txnId;
         this.transactionalWrite = transactionalWrite;
     }
 
-    public JobId getJobId() {
-        return jobId;
+    public TxnId getTxnId() {
+        return txnId;
     }
 
     @Override
@@ -53,7 +53,7 @@
                 try {
                     ITransactionManager txnManager = ((INcApplicationContext) jobletContext.getServiceContext()
                             .getApplicationContext()).getTransactionSubsystem().getTransactionManager();
-                    ITransactionContext txnContext = txnManager.getTransactionContext(jobId, false);
+                    ITransactionContext txnContext = txnManager.getTransactionContext(txnId, false);
                     txnContext.setWriteTxn(transactionalWrite);
                     txnManager.completedTransaction(txnContext, DatasetId.NULL, -1,
                             !(jobStatus == JobStatus.FAILURE));
@@ -66,7 +66,7 @@
             public void jobletStart() {
                 try {
                     ((INcApplicationContext) jobletContext.getServiceContext().getApplicationContext())
-                            .getTransactionSubsystem().getTransactionManager().getTransactionContext(jobId, true);
+                            .getTransactionSubsystem().getTransactionManager().getTransactionContext(txnId, true);
                 } catch (ACIDException e) {
                     throw new Error(e);
                 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/job/listener/MultiTransactionJobletEventListenerFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/job/listener/MultiTransactionJobletEventListenerFactory.java
index f41f326..a63f3ca 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/job/listener/MultiTransactionJobletEventListenerFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/job/listener/MultiTransactionJobletEventListenerFactory.java
@@ -25,7 +25,7 @@
 import org.apache.asterix.common.transactions.DatasetId;
 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.TxnId;
 import org.apache.hyracks.api.context.IHyracksJobletContext;
 import org.apache.hyracks.api.job.IJobletEventListener;
 import org.apache.hyracks.api.job.IJobletEventListenerFactory;
@@ -38,11 +38,11 @@
 public class MultiTransactionJobletEventListenerFactory implements IJobletEventListenerFactory {
 
     private static final long serialVersionUID = 1L;
-    private final List<JobId> jobIds;
+    private final List<TxnId> txnIds;
     private final boolean transactionalWrite;
 
-    public MultiTransactionJobletEventListenerFactory(List<JobId> jobIds, boolean transactionalWrite) {
-        this.jobIds = jobIds;
+    public MultiTransactionJobletEventListenerFactory(List<TxnId> txnIds, boolean transactionalWrite) {
+        this.txnIds = txnIds;
         this.transactionalWrite = transactionalWrite;
     }
 
@@ -56,8 +56,8 @@
                     ITransactionManager txnManager =
                             ((INcApplicationContext) jobletContext.getServiceContext().getApplicationContext())
                                     .getTransactionSubsystem().getTransactionManager();
-                    for (JobId jobId : jobIds) {
-                        ITransactionContext txnContext = txnManager.getTransactionContext(jobId, false);
+                    for (TxnId txnId : txnIds) {
+                        ITransactionContext txnContext = txnManager.getTransactionContext(txnId, false);
                         txnContext.setWriteTxn(transactionalWrite);
                         txnManager.completedTransaction(txnContext, DatasetId.NULL, -1,
                                 !(jobStatus == JobStatus.FAILURE));
@@ -70,9 +70,9 @@
             @Override
             public void jobletStart() {
                 try {
-                    for (JobId jobId : jobIds) {
+                    for (TxnId txnId : txnIds) {
                         ((INcApplicationContext) jobletContext.getServiceContext().getApplicationContext())
-                                .getTransactionSubsystem().getTransactionManager().getTransactionContext(jobId, true);
+                                .getTransactionSubsystem().getTransactionManager().getTransactionContext(txnId, true);
                     }
                 } catch (ACIDException e) {
                     throw new Error(e);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/std/FlushDatasetOperatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/std/FlushDatasetOperatorDescriptor.java
index f2deb74..6f7287b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/std/FlushDatasetOperatorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/std/FlushDatasetOperatorDescriptor.java
@@ -28,7 +28,7 @@
 import org.apache.asterix.common.transactions.ITransactionContext;
 import org.apache.asterix.common.transactions.ITransactionManager;
 import org.apache.asterix.common.transactions.ImmutableDatasetId;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
@@ -40,12 +40,12 @@
 
 public class FlushDatasetOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
     private static final long serialVersionUID = 1L;
-    private final JobId jobId;
+    private final TxnId txnId;
     private final DatasetId datasetId;
 
-    public FlushDatasetOperatorDescriptor(IOperatorDescriptorRegistry spec, JobId jobId, int datasetId) {
+    public FlushDatasetOperatorDescriptor(IOperatorDescriptorRegistry spec, TxnId txnId, int datasetId) {
         super(spec, 1, 0);
-        this.jobId = jobId;
+        this.txnId = txnId;
         this.datasetId = new ImmutableDatasetId(datasetId);
     }
 
@@ -78,7 +78,7 @@
                     ILockManager lockManager = appCtx.getTransactionSubsystem().getLockManager();
                     ITransactionManager txnManager = appCtx.getTransactionSubsystem().getTransactionManager();
                     // get the local transaction
-                    ITransactionContext txnCtx = txnManager.getTransactionContext(jobId, false);
+                    ITransactionContext txnCtx = txnManager.getTransactionContext(txnId, false);
                     // lock the dataset granule
                     lockManager.lock(datasetId, -1, LockMode.S, txnCtx);
                     // flush the dataset synchronously
diff --git a/asterixdb/asterix-tools/src/test/java/org/apache/asterix/tools/datagen/AdmDataGen.java b/asterixdb/asterix-tools/src/test/java/org/apache/asterix/tools/datagen/AdmDataGen.java
index 818444e..09a898a 100644
--- a/asterixdb/asterix-tools/src/test/java/org/apache/asterix/tools/datagen/AdmDataGen.java
+++ b/asterixdb/asterix-tools/src/test/java/org/apache/asterix/tools/datagen/AdmDataGen.java
@@ -46,7 +46,7 @@
 import org.apache.asterix.common.annotations.TypeDataGen;
 import org.apache.asterix.common.annotations.UndeclaredFieldsDataGen;
 import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.lang.aql.parser.AQLParserFactory;
 import org.apache.asterix.lang.aql.parser.ParseException;
 import org.apache.asterix.lang.common.base.IParser;
@@ -941,7 +941,7 @@
         List<Statement> statements = parser.parse();
         aql.close();
         // TODO: Need to fix how to use transactions here.
-        MetadataTransactionContext mdTxnCtx = new MetadataTransactionContext(new JobId(-1));
+        MetadataTransactionContext mdTxnCtx = new MetadataTransactionContext(new TxnId(-1));
         ADGenDmlTranslator dmlt = new ADGenDmlTranslator(mdTxnCtx, statements);
         dmlt.translate();
         typeMap = dmlt.getTypeMap();
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/AbstractIndexModificationOperationCallback.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/AbstractIndexModificationOperationCallback.java
index 1acc235..3a2f195 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/AbstractIndexModificationOperationCallback.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/AbstractIndexModificationOperationCallback.java
@@ -85,7 +85,7 @@
         logRecord = new LogRecord();
         logRecord.setTxnCtx(txnCtx);
         logRecord.setLogType(LogType.UPDATE);
-        logRecord.setJobId(txnCtx.getJobId().getId());
+        logRecord.setTxnId(txnCtx.getTxnId().getId());
         logRecord.setDatasetId(datasetId.getId());
         logRecord.setResourceId(resourceId);
         logRecord.setResourcePartition(resourcePartition);
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 f76cb89..c97fb1b 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
@@ -55,7 +55,7 @@
         logRecord.setTxnCtx(txnCtx);
         logRecord.setLogSource(LogSource.LOCAL);
         logRecord.setLogType(LogType.WAIT);
-        logRecord.setJobId(txnCtx.getJobId().getId());
+        logRecord.setTxnId(txnCtx.getTxnId().getId());
         logRecord.computeAndSetLogSize();
     }
 
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 b3d4f03..73b9b41 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
@@ -24,7 +24,7 @@
 import org.apache.asterix.common.transactions.DatasetId;
 import org.apache.asterix.common.transactions.ITransactionContext;
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -35,9 +35,9 @@
 
     private static final long serialVersionUID = 1L;
 
-    public LockThenSearchOperationCallbackFactory(JobId jobId, int datasetId, int[] entityIdFields,
+    public LockThenSearchOperationCallbackFactory(TxnId txnId, int datasetId, int[] entityIdFields,
             ITransactionSubsystemProvider txnSubsystemProvider, byte resourceType) {
-        super(jobId, datasetId, entityIdFields, txnSubsystemProvider, resourceType);
+        super(txnId, datasetId, entityIdFields, txnSubsystemProvider, resourceType);
     }
 
     @Override
@@ -45,7 +45,7 @@
             IOperatorNodePushable operatorNodePushable) throws HyracksDataException {
         ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         try {
-            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
+            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(txnId, false);
             return new LockThenSearchOperationCallback(new DatasetId(datasetId), primaryKeyFields, txnSubsystem, txnCtx,
                     operatorNodePushable);
         } catch (ACIDException 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 0d447a9..dbf58e4 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
@@ -25,7 +25,7 @@
 import org.apache.asterix.common.transactions.DatasetId;
 import org.apache.asterix.common.transactions.ITransactionContext;
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -37,9 +37,9 @@
 
     private static final long serialVersionUID = 1L;
 
-    public PrimaryIndexInstantSearchOperationCallbackFactory(JobId jobId, int datasetId, int[] entityIdFields,
+    public PrimaryIndexInstantSearchOperationCallbackFactory(TxnId txnId, int datasetId, int[] entityIdFields,
             ITransactionSubsystemProvider txnSubsystemProvider, byte resourceType) {
-        super(jobId, datasetId, entityIdFields, txnSubsystemProvider, resourceType);
+        super(txnId, datasetId, entityIdFields, txnSubsystemProvider, resourceType);
     }
 
     @Override
@@ -47,7 +47,7 @@
             throws HyracksDataException {
         ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         try {
-            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
+            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(txnId, false);
             return new PrimaryIndexInstantSearchOperationCallback(new DatasetId(datasetId), primaryKeyFields,
                     txnSubsystem.getLockManager(), txnCtx);
         } catch (ACIDException e) {
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 932c925..b8c6084 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,7 +27,7 @@
 import org.apache.asterix.common.transactions.DatasetId;
 import org.apache.asterix.common.transactions.ITransactionContext;
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.transaction.management.opcallbacks.AbstractIndexModificationOperationCallback.Operation;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
@@ -48,9 +48,9 @@
     private static final long serialVersionUID = 1L;
     private final Operation indexOp;
 
-    public PrimaryIndexModificationOperationCallbackFactory(JobId jobId, int datasetId, int[] primaryKeyFields,
+    public PrimaryIndexModificationOperationCallbackFactory(TxnId txnId, int datasetId, int[] primaryKeyFields,
             ITransactionSubsystemProvider txnSubsystemProvider, Operation indexOp, byte resourceType) {
-        super(jobId, datasetId, primaryKeyFields, txnSubsystemProvider, resourceType);
+        super(txnId, datasetId, primaryKeyFields, txnSubsystemProvider, resourceType);
         this.indexOp = indexOp;
     }
 
@@ -66,7 +66,7 @@
         }
 
         try {
-            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
+            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(txnId, false);
             DatasetLocalResource aResource = (DatasetLocalResource) resource.getResource();
             IModificationOperationCallback modCallback = new PrimaryIndexModificationOperationCallback(
                     new DatasetId(datasetId), primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem,
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 8bdbb9e..49490a1 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
@@ -25,7 +25,7 @@
 import org.apache.asterix.common.transactions.DatasetId;
 import org.apache.asterix.common.transactions.ITransactionContext;
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -37,9 +37,9 @@
 
     private static final long serialVersionUID = 1L;
 
-    public PrimaryIndexSearchOperationCallbackFactory(JobId jobId, int datasetId, int[] entityIdFields,
+    public PrimaryIndexSearchOperationCallbackFactory(TxnId txnId, int datasetId, int[] entityIdFields,
             ITransactionSubsystemProvider txnSubsystemProvider, byte resourceType) {
-        super(jobId, datasetId, entityIdFields, txnSubsystemProvider, resourceType);
+        super(txnId, datasetId, entityIdFields, txnSubsystemProvider, resourceType);
     }
 
     @Override
@@ -47,7 +47,7 @@
             IOperatorNodePushable operatorNodePushable) throws HyracksDataException {
         ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         try {
-            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
+            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(txnId, false);
             return new PrimaryIndexSearchOperationCallback(new DatasetId(datasetId), primaryKeyFields,
                     txnSubsystem.getLockManager(), txnCtx);
         } catch (ACIDException e) {
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 b339d27..1847e80 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,7 +27,7 @@
 import org.apache.asterix.common.transactions.DatasetId;
 import org.apache.asterix.common.transactions.ITransactionContext;
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.transaction.management.opcallbacks.AbstractIndexModificationOperationCallback.Operation;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
@@ -44,9 +44,9 @@
     private static final long serialVersionUID = 1L;
     private final Operation indexOp;
 
-    public SecondaryIndexModificationOperationCallbackFactory(JobId jobId, int datasetId, int[] primaryKeyFields,
+    public SecondaryIndexModificationOperationCallbackFactory(TxnId txnId, int datasetId, int[] primaryKeyFields,
             ITransactionSubsystemProvider txnSubsystemProvider, Operation indexOp, byte resourceType) {
-        super(jobId, datasetId, primaryKeyFields, txnSubsystemProvider, resourceType);
+        super(txnId, datasetId, primaryKeyFields, txnSubsystemProvider, resourceType);
         this.indexOp = indexOp;
     }
 
@@ -62,7 +62,7 @@
         }
 
         try {
-            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
+            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(txnId, false);
             DatasetLocalResource aResource = (DatasetLocalResource) resource.getResource();
             IModificationOperationCallback modCallback = new SecondaryIndexModificationOperationCallback(
                     new DatasetId(datasetId), primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem,
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 7b7eff6..38adf2b 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,7 +27,7 @@
 import org.apache.asterix.common.transactions.DatasetId;
 import org.apache.asterix.common.transactions.ITransactionContext;
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.transaction.management.opcallbacks.AbstractIndexModificationOperationCallback.Operation;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
@@ -44,10 +44,10 @@
     private static final long serialVersionUID = 1L;
     private final Operation indexOp;
 
-    public TempDatasetPrimaryIndexModificationOperationCallbackFactory(JobId jobId, int datasetId,
+    public TempDatasetPrimaryIndexModificationOperationCallbackFactory(TxnId txnId, int datasetId,
             int[] primaryKeyFields, ITransactionSubsystemProvider txnSubsystemProvider, Operation indexOp,
             byte resourceType) {
-        super(jobId, datasetId, primaryKeyFields, txnSubsystemProvider, resourceType);
+        super(txnId, datasetId, primaryKeyFields, txnSubsystemProvider, resourceType);
         this.indexOp = indexOp;
     }
 
@@ -63,7 +63,7 @@
         }
 
         try {
-            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
+            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(txnId, false);
             DatasetLocalResource aResource = (DatasetLocalResource) resource.getResource();
             IModificationOperationCallback modCallback = new TempDatasetIndexModificationOperationCallback(
                     new DatasetId(datasetId), primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem,
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 1dc1c4e..5e499fc 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,7 +27,7 @@
 import org.apache.asterix.common.transactions.DatasetId;
 import org.apache.asterix.common.transactions.ITransactionContext;
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.transaction.management.opcallbacks.AbstractIndexModificationOperationCallback.Operation;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
@@ -45,10 +45,10 @@
     private static final long serialVersionUID = 1L;
     private final Operation indexOp;
 
-    public TempDatasetSecondaryIndexModificationOperationCallbackFactory(JobId jobId, int datasetId,
+    public TempDatasetSecondaryIndexModificationOperationCallbackFactory(TxnId txnId, int datasetId,
             int[] primaryKeyFields, ITransactionSubsystemProvider txnSubsystemProvider, Operation indexOp,
             byte resourceType) {
-        super(jobId, datasetId, primaryKeyFields, txnSubsystemProvider, resourceType);
+        super(txnId, datasetId, primaryKeyFields, txnSubsystemProvider, resourceType);
         this.indexOp = indexOp;
     }
 
@@ -65,7 +65,7 @@
         }
 
         try {
-            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
+            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(txnId, false);
             IModificationOperationCallback modCallback = new TempDatasetIndexModificationOperationCallback(
                     new DatasetId(datasetId), primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem,
                     resource.getId(), aResource.getPartition(), resourceType, indexOp);
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 15b4344..be75ab9 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,7 +26,7 @@
 import org.apache.asterix.common.transactions.DatasetId;
 import org.apache.asterix.common.transactions.ITransactionContext;
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.transaction.management.opcallbacks.AbstractIndexModificationOperationCallback.Operation;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
@@ -43,9 +43,9 @@
     private static final long serialVersionUID = 1L;
     protected final Operation indexOp;
 
-    public UpsertOperationCallbackFactory(JobId jobId, int datasetId, int[] primaryKeyFields,
+    public UpsertOperationCallbackFactory(TxnId txnId, int datasetId, int[] primaryKeyFields,
             ITransactionSubsystemProvider txnSubsystemProvider, Operation indexOp, byte resourceType) {
-        super(jobId, datasetId, primaryKeyFields, txnSubsystemProvider, resourceType);
+        super(txnId, datasetId, primaryKeyFields, txnSubsystemProvider, resourceType);
         this.indexOp = indexOp;
     }
 
@@ -62,7 +62,7 @@
         }
 
         try {
-            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
+            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(txnId, false);
             IModificationOperationCallback modCallback = new UpsertOperationCallback(new DatasetId(datasetId),
                     primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resource.getId(),
                     aResource.getPartition(), resourceType, indexOp);
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/runtime/CommitRuntime.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/runtime/CommitRuntime.java
index 1e22458..fe758e1 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/runtime/CommitRuntime.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/runtime/CommitRuntime.java
@@ -27,7 +27,7 @@
 import org.apache.asterix.common.transactions.ILogMarkerCallback;
 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.TxnId;
 import org.apache.asterix.common.transactions.LogRecord;
 import org.apache.asterix.common.transactions.LogType;
 import org.apache.asterix.common.utils.TransactionUtil;
@@ -50,7 +50,7 @@
 
     protected final ITransactionManager transactionManager;
     protected final ILogManager logMgr;
-    protected final JobId jobId;
+    protected final TxnId txnId;
     protected final int datasetId;
     protected final int[] primaryKeyFields;
     protected final boolean isTemporaryDatasetWriteJob;
@@ -62,14 +62,14 @@
     protected LogRecord logRecord;
     protected final boolean isSink;
 
-    public CommitRuntime(IHyracksTaskContext ctx, JobId jobId, int datasetId, int[] primaryKeyFields,
+    public CommitRuntime(IHyracksTaskContext ctx, TxnId txnId, int datasetId, int[] primaryKeyFields,
             boolean isTemporaryDatasetWriteJob, boolean isWriteTransaction, int resourcePartition, boolean isSink) {
         this.ctx = ctx;
         INcApplicationContext appCtx =
                 (INcApplicationContext) ctx.getJobletContext().getServiceContext().getApplicationContext();
         this.transactionManager = appCtx.getTransactionSubsystem().getTransactionManager();
         this.logMgr = appCtx.getTransactionSubsystem().getLogManager();
-        this.jobId = jobId;
+        this.txnId = txnId;
         this.datasetId = datasetId;
         this.primaryKeyFields = primaryKeyFields;
         this.tRef = new FrameTupleReference();
@@ -83,7 +83,7 @@
     @Override
     public void open() throws HyracksDataException {
         try {
-            transactionContext = transactionManager.getTransactionContext(jobId, false);
+            transactionContext = transactionManager.getTransactionContext(txnId, false);
             transactionContext.setWriteTxn(isWriteTransaction);
             ILogMarkerCallback callback = TaskUtil.get(ILogMarkerCallback.KEY_MARKER_CALLBACK, ctx);
             logRecord = new LogRecord(callback);
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/runtime/CommitRuntimeFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/runtime/CommitRuntimeFactory.java
index 1b32d89..2e43957 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/runtime/CommitRuntimeFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/runtime/CommitRuntimeFactory.java
@@ -19,7 +19,7 @@
 
 package org.apache.asterix.transaction.management.runtime;
 
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -29,7 +29,7 @@
 
     private static final long serialVersionUID = 1L;
 
-    protected final JobId jobId;
+    protected final TxnId txnId;
     protected final int datasetId;
     protected final int[] primaryKeyFields;
     protected final boolean isTemporaryDatasetWriteJob;
@@ -37,9 +37,9 @@
     protected int[] datasetPartitions;
     protected final boolean isSink;
 
-    public CommitRuntimeFactory(JobId jobId, int datasetId, int[] primaryKeyFields, boolean isTemporaryDatasetWriteJob,
+    public CommitRuntimeFactory(TxnId txnId, int datasetId, int[] primaryKeyFields, boolean isTemporaryDatasetWriteJob,
             boolean isWriteTransaction, int[] datasetPartitions, boolean isSink) {
-        this.jobId = jobId;
+        this.txnId = txnId;
         this.datasetId = datasetId;
         this.primaryKeyFields = primaryKeyFields;
         this.isTemporaryDatasetWriteJob = isTemporaryDatasetWriteJob;
@@ -55,7 +55,7 @@
 
     @Override
     public IPushRuntime createPushRuntime(IHyracksTaskContext ctx) throws HyracksDataException {
-            return new CommitRuntime(ctx, jobId, datasetId, primaryKeyFields, isTemporaryDatasetWriteJob,
+            return new CommitRuntime(ctx, txnId, datasetId, primaryKeyFields, isTemporaryDatasetWriteJob,
                     isWriteTransaction, datasetPartitions[ctx.getTaskAttemptId().getTaskId().getPartition()], isSink);
     }
 }
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 f386b39..3aa2578 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
@@ -57,7 +57,7 @@
     private ResourceArenaManager resArenaMgr;
     private RequestArenaManager reqArenaMgr;
     private JobArenaManager jobArenaMgr;
-    private ConcurrentHashMap<Integer, Long> jobId2JobSlotMap;
+    private ConcurrentHashMap<Long, Long> txnId2TxnSlotMap;
     private LockManagerStats stats = new LockManagerStats(10000);
 
     enum LockAction {
@@ -96,7 +96,7 @@
         resArenaMgr = new ResourceArenaManager(noArenas, lockManagerShrinkTimer);
         reqArenaMgr = new RequestArenaManager(noArenas, lockManagerShrinkTimer);
         jobArenaMgr = new JobArenaManager(noArenas, lockManagerShrinkTimer);
-        jobId2JobSlotMap = new ConcurrentHashMap<>();
+        txnId2TxnSlotMap = new ConcurrentHashMap<>();
     }
 
     @Override
@@ -105,8 +105,8 @@
         log("lock", datasetId.getId(), entityHashValue, lockMode, txnContext);
         stats.lock();
 
-        final int jobId = txnContext.getJobId().getId();
-        final long jobSlot = findOrAllocJobSlot(jobId);
+        final long txnId = txnContext.getTxnId().getId();
+        final long jobSlot = findOrAllocJobSlot(txnId);
         final ResourceGroup group = table.get(datasetId.getId(), entityHashValue);
         group.getLatch();
         try {
@@ -328,7 +328,7 @@
         log("instantLock", datasetId.getId(), entityHashValue, lockMode, txnContext);
         stats.instantLock();
 
-        final int jobId = txnContext.getJobId().getId();
+        final long txnId = txnContext.getTxnId().getId();
         final ResourceGroup group = table.get(datasetId.getId(), entityHashValue);
         if (group.firstResourceIndex.get() == NILL) {
             validateJob(txnContext);
@@ -349,7 +349,7 @@
                 return;
             }
 
-            final long jobSlot = findOrAllocJobSlot(jobId);
+            final long jobSlot = findOrAllocJobSlot(txnId);
 
             while (true) {
                 final LockAction act = determineLockAction(resSlot, jobSlot, lockMode);
@@ -389,8 +389,8 @@
         log("tryLock", datasetId.getId(), entityHashValue, lockMode, txnContext);
         stats.tryLock();
 
-        final int jobId = txnContext.getJobId().getId();
-        final long jobSlot = findOrAllocJobSlot(jobId);
+        final long txnId = txnContext.getTxnId().getId();
+        final long jobSlot = findOrAllocJobSlot(txnId);
         final ResourceGroup group = table.get(datasetId.getId(), entityHashValue);
         group.getLatch();
 
@@ -425,7 +425,7 @@
         log("instantTryLock", datasetId.getId(), entityHashValue, lockMode, txnContext);
         stats.instantTryLock();
 
-        final int jobId = txnContext.getJobId().getId();
+        final long txnId = txnContext.getTxnId().getId();
         final ResourceGroup group = table.get(datasetId.getId(), entityHashValue);
         if (group.firstResourceIndex.get() == NILL) {
             validateJob(txnContext);
@@ -444,7 +444,7 @@
                 return true;
             }
 
-            final long jobSlot = findOrAllocJobSlot(jobId);
+            final long jobSlot = findOrAllocJobSlot(txnId);
 
             LockAction act = determineLockAction(resSlot, jobSlot, lockMode);
             switch (act) {
@@ -467,8 +467,8 @@
     public void unlock(DatasetId datasetId, int entityHashValue, byte lockMode, ITransactionContext txnContext)
             throws ACIDException {
         log("unlock", datasetId.getId(), entityHashValue, lockMode, txnContext);
-        final int jobId = txnContext.getJobId().getId();
-        final long jobSlot = jobId2JobSlotMap.get(jobId);
+        final long txnId = txnContext.getTxnId().getId();
+        final long jobSlot = txnId2TxnSlotMap.get(txnId);
 
         unlock(datasetId.getId(), entityHashValue, lockMode, jobSlot);
     }
@@ -528,8 +528,8 @@
         log("releaseLocks", NIL, NIL, LockMode.ANY, txnContext);
         stats.releaseLocks();
 
-        int jobId = txnContext.getJobId().getId();
-        Long jobSlot = jobId2JobSlotMap.get(jobId);
+        long txnId = txnContext.getTxnId().getId();
+        Long jobSlot = txnId2TxnSlotMap.get(txnId);
         if (jobSlot == null) {
             // we don't know the job, so there are no locks for it - we're done
             return;
@@ -557,19 +557,19 @@
             LOGGER.finer("del job slot " + TypeUtil.Global.toString(jobSlot));
         }
         jobArenaMgr.deallocate(jobSlot);
-        jobId2JobSlotMap.remove(jobId);
+        txnId2TxnSlotMap.remove(txnId);
         stats.logCounters(LOGGER, Level.FINE, true);
     }
 
-    private long findOrAllocJobSlot(int jobId) {
-        Long jobSlot = jobId2JobSlotMap.get(jobId);
+    private long findOrAllocJobSlot(long txnId) {
+        Long jobSlot = txnId2TxnSlotMap.get(txnId);
         if (jobSlot == null) {
             jobSlot = new Long(jobArenaMgr.allocate());
             if (DEBUG_MODE) {
-                LOGGER.finer("new job slot " + TypeUtil.Global.toString(jobSlot) + " (" + jobId + ")");
+                LOGGER.finer("new job slot " + TypeUtil.Global.toString(jobSlot) + " (" + txnId + ")");
             }
-            jobArenaMgr.setJobId(jobSlot, jobId);
-            Long oldSlot = jobId2JobSlotMap.putIfAbsent(jobId, jobSlot);
+            jobArenaMgr.setTxnId(jobSlot, txnId);
+            Long oldSlot = txnId2TxnSlotMap.putIfAbsent(txnId, jobSlot);
             if (oldSlot != null) {
                 // if another thread allocated a slot for this jobThreadId between
                 // get(..) and putIfAbsent(..), we'll use that slot and
@@ -917,7 +917,7 @@
 
     private void validateJob(ITransactionContext txnContext) throws ACIDException {
         if (txnContext.getTxnState() == ITransactionManager.ABORTED) {
-            throw new ACIDException("" + txnContext.getJobId() + " is in ABORTED state.");
+            throw new ACIDException("" + txnContext.getTxnId() + " is in ABORTED state.");
         } else if (txnContext.isTimeout()) {
             requestAbort(txnContext, "timeout");
         }
@@ -926,7 +926,7 @@
     private void requestAbort(ITransactionContext txnContext, String msg) throws ACIDException {
         txnContext.setTimeout(true);
         throw new ACIDException(
-                "Transaction " + txnContext.getJobId() + " should abort (requested by the Lock Manager)" + ":\n" + msg);
+                "Transaction " + txnContext.getTxnId() + " should abort (requested by the Lock Manager)" + ":\n" + msg);
     }
 
     /*
@@ -949,7 +949,7 @@
             sb.append(" , mode : ").append(LockMode.toString(lockMode));
         }
         if (txnContext != null) {
-            sb.append(" , jobId : ").append(txnContext.getJobId());
+            sb.append(" , txnId : ").append(txnContext.getTxnId());
         }
         sb.append(" , thread : ").append(Thread.currentThread().getName());
         sb.append(" }");
@@ -970,8 +970,8 @@
                             while (reqSlot != NILL) {
                                 byte lockMode = (byte) reqArenaMgr.getLockMode(reqSlot);
                                 long jobSlot = reqArenaMgr.getJobSlot(reqSlot);
-                                int jobId = jobArenaMgr.getJobId(jobSlot);
-                                assertLockCanBeFoundInJobQueue(dsId, entityHashValue, lockMode, jobId);
+                                long txnId = jobArenaMgr.getTxnId(jobSlot);
+                                assertLockCanBeFoundInJobQueue(dsId, entityHashValue, lockMode, txnId);
                                 reqSlot = reqArenaMgr.getNextRequest(reqSlot);
                             }
                             resSlot = resArenaMgr.getNext(resSlot);
@@ -988,10 +988,10 @@
         }
     }
 
-    private void assertLockCanBeFoundInJobQueue(int dsId, int entityHashValue, byte lockMode, int jobId) {
-        if (findLockInJobQueue(dsId, entityHashValue, jobId, lockMode) == NILL) {
+    private void assertLockCanBeFoundInJobQueue(int dsId, int entityHashValue, byte lockMode, long txnId) {
+        if (findLockInJobQueue(dsId, entityHashValue, txnId, lockMode) == NILL) {
             String msg = "request for " + LockMode.toString(lockMode) + " lock on dataset " + dsId + " entity "
-                    + entityHashValue + " not found for job " + jobId + " in thread "
+                    + entityHashValue + " not found for txn " + txnId + " in thread "
                     + Thread.currentThread().getName();
             LOGGER.severe(msg);
             throw new IllegalStateException(msg);
@@ -1005,14 +1005,14 @@
      *            dataset id
      * @param entityHashValue
      *            primary key hash value
-     * @param jobId
+     * @param txnId
      *            job id
      * @param lockMode
      *            lock mode
      * @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 = jobId2JobSlotMap.get(jobId);
+    private long findLockInJobQueue(final int dsId, final int entityHashValue, final long txnId, byte lockMode) {
+        Long jobSlot = txnId2TxnSlotMap.get(txnId);
         if (jobSlot == null) {
             return NILL;
         }
@@ -1040,7 +1040,7 @@
     }
 
     private TablePrinter getDumpTablePrinter() {
-        return new DumpTablePrinter(table, resArenaMgr, reqArenaMgr, jobArenaMgr, jobId2JobSlotMap);
+        return new DumpTablePrinter(table, resArenaMgr, reqArenaMgr, jobArenaMgr, txnId2TxnSlotMap);
     }
 
     public String printByResource() {
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 a75f756..26261c2 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,15 +26,15 @@
     private ResourceArenaManager resArenaMgr;
     private RequestArenaManager reqArenaMgr;
     private JobArenaManager jobArenaMgr;
-    private ConcurrentHashMap<Integer, Long> jobId2JobSlotMap;
+    private ConcurrentHashMap<Long, Long> txnIdToJobSlotMap;
 
     DumpTablePrinter(ResourceGroupTable table, ResourceArenaManager resArenaMgr, RequestArenaManager reqArenaMgr,
-            JobArenaManager jobArenaMgr, ConcurrentHashMap<Integer, Long> jobId2JobSlotMap) {
+            JobArenaManager jobArenaMgr, ConcurrentHashMap<Long, Long> txnIdToJobSlotMap) {
         this.table = table;
         this.resArenaMgr = resArenaMgr;
         this.reqArenaMgr = reqArenaMgr;
         this.jobArenaMgr = jobArenaMgr;
-        this.jobId2JobSlotMap = jobId2JobSlotMap;
+        this.txnIdToJobSlotMap = txnIdToJobSlotMap;
     }
 
     public StringBuilder append(StringBuilder sb) {
@@ -52,10 +52,10 @@
             reqArenaMgr.append(sb);
             sb.append(">>dump_end\t>>----- [reqArenaMgr] -----\n");
 
-            sb.append(">>dump_begin\t>>----- [jobIdSlotMap] -----\n");
-            for (Integer i : jobId2JobSlotMap.keySet()) {
+            sb.append(">>dump_begin\t>>----- [txnIdSlotMap] -----\n");
+            for (Long i : txnIdToJobSlotMap.keySet()) {
                 sb.append(i).append(" : ");
-                TypeUtil.Global.append(sb, jobId2JobSlotMap.get(i));
+                TypeUtil.Global.append(sb, txnIdToJobSlotMap.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/Job.json b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/Job.json
index a649b7c..5e0f588 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/Job.json
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/Job.json
@@ -17,8 +17,8 @@
             "initial" : "-1"
         },
         {
-            "name" : "job id",
-            "type" : "INT"
+            "name" : "txn id",
+            "type" : "GLOBAL"
         }
     ]
 }
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ResourceTablePrinter.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ResourceTablePrinter.java
index eff9e21..e8ef2c8 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ResourceTablePrinter.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ResourceTablePrinter.java
@@ -99,7 +99,7 @@
 
     StringBuilder appendRequest(StringBuilder sb, long req) {
         long job = reqArenaMgr.getJobSlot(req);
-        sb.append("{ \"job\": ").append(jobArenaMgr.getJobId(job));
+        sb.append("{ \"job\": ").append(jobArenaMgr.getTxnId(job));
         sb.append(", \"mode\": \"").append(string(reqArenaMgr.getLockMode(req)));
         return sb.append("\" }");
     }
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 668eab1..3d78ad9 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
@@ -33,11 +33,11 @@
 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.JobId;
 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.common.transactions.MutableLong;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -61,7 +61,7 @@
     protected final LinkedBlockingQueue<ILogRecord> remoteJobsQ;
     private FileChannel fileChannel;
     private boolean stop;
-    private final JobId reusableJobId;
+    private final MutableTxnId reusableTxnId;
     private final DatasetId reusableDatasetId;
 
     public LogBuffer(ITransactionSubsystem txnSubsystem, int logPageSize, MutableLong flushLSN) {
@@ -79,7 +79,7 @@
         syncCommitQ = new LinkedBlockingQueue<>(logPageSize / ILogRecord.JOB_TERMINATE_LOG_SIZE);
         flushQ = new LinkedBlockingQueue<>();
         remoteJobsQ = new LinkedBlockingQueue<>();
-        reusableJobId = new JobId(-1);
+        reusableTxnId = new MutableTxnId(-1);
         reusableDatasetId = new DatasetId(-1);
     }
 
@@ -241,9 +241,9 @@
             while (logRecord != null) {
                 if (logRecord.getLogSource() == LogSource.LOCAL) {
                     if (logRecord.getLogType() == LogType.ENTITY_COMMIT) {
-                        reusableJobId.setId(logRecord.getJobId());
+                        reusableTxnId.setId(logRecord.getTxnId());
                         reusableDatasetId.setId(logRecord.getDatasetId());
-                        txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(reusableJobId, false);
+                        txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(reusableTxnId, false);
                         txnSubsystem.getLockManager().unlock(reusableDatasetId, logRecord.getPKHashValue(),
                                 LockMode.ANY, txnCtx);
                         txnCtx.notifyOptracker(false);
@@ -252,8 +252,8 @@
                         }
                     } else if (logRecord.getLogType() == LogType.JOB_COMMIT
                             || logRecord.getLogType() == LogType.ABORT) {
-                        reusableJobId.setId(logRecord.getJobId());
-                        txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(reusableJobId, false);
+                        reusableTxnId.setId(logRecord.getTxnId());
+                        txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(reusableTxnId, false);
                         txnCtx.notifyOptracker(true);
                         notifyJobTermination();
                     } else if (logRecord.getLogType() == LogType.FLUSH) {
@@ -341,4 +341,15 @@
     public int getLogPageSize() {
         return logPageSize;
     }
+
+    private class MutableTxnId extends TxnId {
+
+        public MutableTxnId(long id) {
+            super(id);
+        }
+
+        public void setId(long id) {
+            this.id = id;
+        }
+    }
 }
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 5f9369d..4d671f3 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
@@ -159,7 +159,7 @@
             ITransactionContext txnCtx = logRecord.getTxnCtx();
             if (txnCtx.getTxnState() == ITransactionManager.ABORTED && logRecord.getLogType() != LogType.ABORT) {
                 throw new ACIDException(
-                        "Aborted job(" + txnCtx.getJobId() + ") tried to write non-abort type log record.");
+                        "Aborted txn(" + txnCtx.getTxnId() + ") tried to write non-abort type log record.");
             }
         }
 
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 dd8fb6e..3f94749 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
@@ -35,7 +35,7 @@
 
     private IReplicationManager replicationManager;
     private final IReplicationStrategy replicationStrategy;
-    private final Set<Integer> replicatedJob = ConcurrentHashMap.newKeySet();
+    private final Set<Long> replicatedTxn = ConcurrentHashMap.newKeySet();
 
     public LogManagerWithReplication(ITransactionSubsystem txnSubsystem, IReplicationStrategy replicationStrategy) {
         super(txnSubsystem);
@@ -51,13 +51,13 @@
                 case LogType.UPDATE:
                 case LogType.FLUSH:
                     shouldReplicate = replicationStrategy.isMatch(logRecord.getDatasetId());
-                    if (shouldReplicate && !replicatedJob.contains(logRecord.getJobId())) {
-                        replicatedJob.add(logRecord.getJobId());
+                    if (shouldReplicate && !replicatedTxn.contains(logRecord.getTxnId())) {
+                        replicatedTxn.add(logRecord.getTxnId());
                     }
                     break;
                 case LogType.JOB_COMMIT:
                 case LogType.ABORT:
-                    shouldReplicate = replicatedJob.remove(logRecord.getJobId());
+                    shouldReplicate = replicatedTxn.remove(logRecord.getTxnId());
                     break;
                 default:
                     shouldReplicate = false;
@@ -120,7 +120,7 @@
             ITransactionContext txnCtx = logRecord.getTxnCtx();
             if (txnCtx.getTxnState() == ITransactionManager.ABORTED && logRecord.getLogType() != LogType.ABORT) {
                 throw new ACIDException(
-                        "Aborted job(" + txnCtx.getJobId() + ") tried to write non-abort type log record.");
+                        "Aborted txn(" + txnCtx.getTxnId() + ") tried to write non-abort type log record.");
             }
         }
 
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/AbstractCheckpointManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/AbstractCheckpointManager.java
index 36a91dc..aad2a19 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/AbstractCheckpointManager.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/AbstractCheckpointManager.java
@@ -174,7 +174,7 @@
     protected void capture(long minMCTFirstLSN, boolean sharp) throws HyracksDataException {
         ILogManager logMgr = txnSubsystem.getLogManager();
         ITransactionManager txnMgr = txnSubsystem.getTransactionManager();
-        Checkpoint checkpointObject = new Checkpoint(logMgr.getAppendLSN(), minMCTFirstLSN, txnMgr.getMaxJobId(),
+        Checkpoint checkpointObject = new Checkpoint(logMgr.getAppendLSN(), minMCTFirstLSN, txnMgr.getMaxTxnId(),
                 System.currentTimeMillis(), sharp, StorageConstants.VERSION);
         persist(checkpointObject);
         cleanup();
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/TxnId.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/TxnEntityId.java
similarity index 66%
rename from asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/TxnId.java
rename to asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/TxnEntityId.java
index 9cb54af..af74b13 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/TxnId.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/TxnEntityId.java
@@ -21,22 +21,22 @@
 import java.nio.ByteBuffer;
 
 import org.apache.asterix.common.transactions.ILogRecord;
-import org.apache.asterix.common.transactions.JobId;
 import org.apache.asterix.common.transactions.LogRecord;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 
-public class TxnId {
+public class TxnEntityId {
     public boolean isByteArrayPKValue;
-    public int jobId;
+    public long txnId;
     public int datasetId;
     public int pkHashValue;
     public int pkSize;
     public byte[] byteArrayPKValue;
     public ITupleReference tupleReferencePKValue;
 
-    public TxnId(int jobId, int datasetId, int pkHashValue, ITupleReference pkValue, int pkSize,
+    public TxnEntityId(long txnId, int datasetId, int pkHashValue, ITupleReference pkValue, int pkSize,
             boolean isByteArrayPKValue) {
-        this.jobId = jobId;
+        this.txnId = txnId;
         this.datasetId = datasetId;
         this.pkHashValue = pkHashValue;
         this.pkSize = pkSize;
@@ -49,7 +49,7 @@
         }
     }
 
-    public TxnId() {
+    public TxnEntityId() {
     }
 
     private static void readPKValueIntoByteArray(ITupleReference pkValue, int pkSize, byte[] byteArrayPKValue) {
@@ -60,8 +60,8 @@
         }
     }
 
-    public void setTxnId(int jobId, int datasetId, int pkHashValue, ITupleReference pkValue, int pkSize) {
-        this.jobId = jobId;
+    public void setTxnId(long txnId, int datasetId, int pkHashValue, ITupleReference pkValue, int pkSize) {
+        this.txnId = txnId;
         this.datasetId = datasetId;
         this.pkHashValue = pkHashValue;
         this.tupleReferencePKValue = pkValue;
@@ -71,7 +71,7 @@
 
     @Override
     public String toString() {
-        return "[" + jobId + "," + datasetId + "," + pkHashValue + "," + pkSize + "]";
+        return "[" + txnId + "," + datasetId + "," + pkHashValue + "," + pkSize + "]";
     }
 
     @Override
@@ -84,23 +84,23 @@
         if (o == this) {
             return true;
         }
-        if (!(o instanceof TxnId)) {
+        if (!(o instanceof TxnEntityId)) {
             return false;
         }
-        TxnId txnId = (TxnId) o;
-        return (txnId.pkHashValue == pkHashValue && txnId.datasetId == datasetId && txnId.jobId == jobId
-                && pkSize == txnId.pkSize && isEqualTo(txnId));
+        TxnEntityId txnEntityId = (TxnEntityId) o;
+        return (txnEntityId.pkHashValue == pkHashValue && txnEntityId.datasetId == datasetId
+                && txnEntityId.txnId == txnId && pkSize == txnEntityId.pkSize && isEqualTo(txnEntityId));
     }
 
-    private boolean isEqualTo(TxnId txnId) {
-        if (isByteArrayPKValue && txnId.isByteArrayPKValue) {
-            return isEqual(byteArrayPKValue, txnId.byteArrayPKValue, pkSize);
-        } else if (isByteArrayPKValue && (!txnId.isByteArrayPKValue)) {
-            return isEqual(byteArrayPKValue, txnId.tupleReferencePKValue, pkSize);
-        } else if ((!isByteArrayPKValue) && txnId.isByteArrayPKValue) {
-            return isEqual(txnId.byteArrayPKValue, tupleReferencePKValue, pkSize);
+    private boolean isEqualTo(TxnEntityId txnEntityId) {
+        if (isByteArrayPKValue && txnEntityId.isByteArrayPKValue) {
+            return isEqual(byteArrayPKValue, txnEntityId.byteArrayPKValue, pkSize);
+        } else if (isByteArrayPKValue && (!txnEntityId.isByteArrayPKValue)) {
+            return isEqual(byteArrayPKValue, txnEntityId.tupleReferencePKValue, pkSize);
+        } else if ((!isByteArrayPKValue) && txnEntityId.isByteArrayPKValue) {
+            return isEqual(txnEntityId.byteArrayPKValue, tupleReferencePKValue, pkSize);
         } else {
-            return isEqual(tupleReferencePKValue, txnId.tupleReferencePKValue, pkSize);
+            return isEqual(tupleReferencePKValue, txnEntityId.tupleReferencePKValue, pkSize);
         }
     }
 
@@ -138,7 +138,7 @@
     }
 
     public void serialize(ByteBuffer buffer) {
-        buffer.putInt(jobId);
+        buffer.putLong(txnId);
         buffer.putInt(datasetId);
         buffer.putInt(pkHashValue);
         buffer.putInt(pkSize);
@@ -148,24 +148,24 @@
         }
     }
 
-    public static TxnId deserialize(ByteBuffer buffer) {
-        TxnId txnId = new TxnId();
-        txnId.jobId = buffer.getInt();
-        txnId.datasetId = buffer.getInt();
-        txnId.pkHashValue = buffer.getInt();
-        txnId.pkSize = buffer.getInt();
-        txnId.isByteArrayPKValue = (buffer.get() == 1);
-        if (txnId.isByteArrayPKValue) {
-            byte[] byteArrayPKValue = new byte[txnId.pkSize];
+    public static TxnEntityId deserialize(ByteBuffer buffer) {
+        TxnEntityId txnEntityId = new TxnEntityId();
+        txnEntityId.txnId = buffer.getLong();
+        txnEntityId.datasetId = buffer.getInt();
+        txnEntityId.pkHashValue = buffer.getInt();
+        txnEntityId.pkSize = buffer.getInt();
+        txnEntityId.isByteArrayPKValue = (buffer.get() == 1);
+        if (txnEntityId.isByteArrayPKValue) {
+            byte[] byteArrayPKValue = new byte[txnEntityId.pkSize];
             buffer.get(byteArrayPKValue);
-            txnId.byteArrayPKValue = byteArrayPKValue;
+            txnEntityId.byteArrayPKValue = byteArrayPKValue;
         }
-        return txnId;
+        return txnEntityId;
     }
 
     public int getCurrentSize() {
-        //job id, dataset id, pkHashValue, arraySize, isByteArrayPKValue
-        int size = JobId.BYTES + ILogRecord.DS_LEN + LogRecord.PKHASH_LEN + LogRecord.PKSZ_LEN + Byte.BYTES;
+        //txn id, dataset id, pkHashValue, arraySize, isByteArrayPKValue
+        int size = TxnId.BYTES + ILogRecord.DS_LEN + LogRecord.PKHASH_LEN + LogRecord.PKSZ_LEN + Byte.BYTES;
         //byte arraySize
         if (isByteArrayPKValue && byteArrayPKValue != null) {
             size += byteArrayPKValue.length;
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionContext.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionContext.java
index 3159e6b..1681a27 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionContext.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionContext.java
@@ -30,8 +30,8 @@
 import org.apache.asterix.common.transactions.AbstractOperationCallback;
 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.TxnId;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
@@ -39,7 +39,7 @@
 
 /*
  * An object of TransactionContext is created and accessed(read/written) by multiple threads which work for
- * a single job identified by a jobId. Thus, the member variables in the object can be read/written
+ * a single job identified by a txnId. Thus, the member variables in the object can be read/written
  * concurrently. Please see each variable declaration to know which one is accessed concurrently and
  * which one is not.
  */
@@ -47,8 +47,8 @@
 
     private static final long serialVersionUID = -6105616785783310111L;
 
-    // jobId is set once and read concurrently.
-    private final JobId jobId;
+    // txnId is set once and read concurrently.
+    private final TxnId txnId;
 
     // There are no concurrent writers on both firstLSN and lastLSN
     // since both values are updated by serialized log appenders.
@@ -95,8 +95,8 @@
     // creations.
     // also, the pool can throttle the number of concurrent active jobs at every
     // moment.
-    public TransactionContext(JobId jobId) throws ACIDException {
-        this.jobId = jobId;
+    public TransactionContext(TxnId txnId) throws ACIDException {
+        this.txnId = txnId;
         firstLSN = new AtomicLong(-1);
         lastLSN = new AtomicLong(-1);
         txnState = new AtomicInteger(ITransactionManager.ACTIVE);
@@ -180,8 +180,8 @@
     }
 
     @Override
-    public JobId getJobId() {
-        return jobId;
+    public TxnId getTxnId() {
+        return txnId;
     }
 
     @Override
@@ -206,7 +206,7 @@
 
     @Override
     public int hashCode() {
-        return jobId.getId();
+        return Long.hashCode(txnId.getId());
     }
 
     @Override
@@ -227,7 +227,7 @@
     @Override
     public String prettyPrint() {
         StringBuilder sb = new StringBuilder();
-        sb.append("\n" + jobId + "\n");
+        sb.append("\n" + txnId + "\n");
         sb.append("isWriteTxn: " + isWriteTxn + "\n");
         sb.append("firstLSN: " + firstLSN.get() + "\n");
         sb.append("lastLSN: " + lastLSN.get() + "\n");
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 c9a1bad..1799ea1 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
@@ -22,7 +22,7 @@
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
@@ -31,7 +31,7 @@
 import org.apache.asterix.common.transactions.ITransactionContext;
 import org.apache.asterix.common.transactions.ITransactionManager;
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.common.transactions.LogRecord;
 import org.apache.asterix.common.utils.TransactionUtil;
 import org.apache.hyracks.api.lifecycle.ILifeCycleComponent;
@@ -45,8 +45,8 @@
     public static final boolean IS_DEBUG_MODE = false;//true
     private static final Logger LOGGER = Logger.getLogger(TransactionManager.class.getName());
     private final ITransactionSubsystem txnSubsystem;
-    private Map<JobId, ITransactionContext> transactionContextRepository = new ConcurrentHashMap<>();
-    private AtomicInteger maxJobId = new AtomicInteger(0);
+    private Map<TxnId, ITransactionContext> transactionContextRepository = new ConcurrentHashMap<>();
+    private AtomicLong maxTxnId = new AtomicLong(0);
 
     public TransactionManager(ITransactionSubsystem provider) {
         this.txnSubsystem = provider;
@@ -74,30 +74,30 @@
         } finally {
             txnCtx.complete();
             txnSubsystem.getLockManager().releaseLocks(txnCtx);
-            transactionContextRepository.remove(txnCtx.getJobId());
+            transactionContextRepository.remove(txnCtx.getTxnId());
         }
     }
 
     @Override
-    public ITransactionContext beginTransaction(JobId jobId) throws ACIDException {
-        return getTransactionContext(jobId, true);
+    public ITransactionContext beginTransaction(TxnId txnId) throws ACIDException {
+        return getTransactionContext(txnId, true);
     }
 
     @Override
-    public ITransactionContext getTransactionContext(JobId jobId, boolean createIfNotExist) throws ACIDException {
-        setMaxJobId(jobId.getId());
-        ITransactionContext txnCtx = transactionContextRepository.get(jobId);
+    public ITransactionContext getTransactionContext(TxnId txnId, boolean createIfNotExist) throws ACIDException {
+        setMaxTxnId(txnId.getId());
+        ITransactionContext txnCtx = transactionContextRepository.get(txnId);
         if (txnCtx == null) {
             if (createIfNotExist) {
                 synchronized (this) {
-                    txnCtx = transactionContextRepository.get(jobId);
+                    txnCtx = transactionContextRepository.get(txnId);
                     if (txnCtx == null) {
-                        txnCtx = new TransactionContext(jobId);
-                        transactionContextRepository.put(jobId, txnCtx);
+                        txnCtx = new TransactionContext(txnId);
+                        transactionContextRepository.put(txnId, txnCtx);
                     }
                 }
             } else {
-                throw new ACIDException("TransactionContext of " + jobId + " doesn't exist.");
+                throw new ACIDException("TransactionContext of " + txnId + " doesn't exist.");
             }
         }
         return txnCtx;
@@ -115,13 +115,13 @@
             }
         } catch (Exception ae) {
             if (LOGGER.isLoggable(Level.SEVERE)) {
-                LOGGER.severe(" caused exception in commit !" + txnCtx.getJobId());
+                LOGGER.severe(" caused exception in commit !" + txnCtx.getTxnId());
             }
             throw ae;
         } finally {
             txnCtx.complete();
             txnSubsystem.getLockManager().releaseLocks(txnCtx);
-            transactionContextRepository.remove(txnCtx.getJobId());
+            transactionContextRepository.remove(txnCtx.getTxnId());
             txnCtx.setTxnState(ITransactionManager.COMMITTED);
         }
     }
@@ -141,16 +141,16 @@
         return txnSubsystem;
     }
 
-    public void setMaxJobId(int jobId) {
-        int maxId = maxJobId.get();
-        if (jobId > maxId) {
-            maxJobId.compareAndSet(maxId, jobId);
+    public void setMaxTxnId(long txnId) {
+        long maxId = maxTxnId.get();
+        if (txnId > maxId) {
+            maxTxnId.compareAndSet(maxId, txnId);
         }
     }
 
     @Override
-    public int getMaxJobId() {
-        return maxJobId.get();
+    public long getMaxTxnId() {
+        return maxTxnId.get();
     }
 
     @Override
@@ -172,19 +172,19 @@
     }
 
     private void dumpTxnContext(OutputStream os) {
-        JobId jobId;
+        TxnId txnId;
         ITransactionContext txnCtx;
         StringBuilder sb = new StringBuilder();
 
         try {
             sb.append("\n>>dump_begin\t>>----- [ConfVars] -----");
-            Set<Map.Entry<JobId, ITransactionContext>> entrySet = transactionContextRepository.entrySet();
+            Set<Map.Entry<TxnId, ITransactionContext>> entrySet = transactionContextRepository.entrySet();
             if (entrySet != null) {
-                for (Map.Entry<JobId, ITransactionContext> entry : entrySet) {
+                for (Map.Entry<TxnId, ITransactionContext> entry : entrySet) {
                     if (entry != null) {
-                        jobId = entry.getKey();
-                        if (jobId != null) {
-                            sb.append("\n" + jobId);
+                        txnId = entry.getKey();
+                        if (txnId != null) {
+                            sb.append("\n" + txnId);
                         } else {
                             sb.append("\nJID:null");
                         }
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/JobIdFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TxnIdFactory.java
similarity index 71%
rename from asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/JobIdFactory.java
rename to asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TxnIdFactory.java
index 6e0af1c..71d7f56 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/JobIdFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TxnIdFactory.java
@@ -18,21 +18,25 @@
  */
 package org.apache.asterix.transaction.management.service.transaction;
 
-import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
 
-import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.TxnId;
 
 /**
  * Represents a factory to generate unique transaction IDs.
  */
-public class JobIdFactory {
-    private static final AtomicInteger Id = new AtomicInteger();
+public class TxnIdFactory {
 
-    public static JobId generateJobId() {
-        return new JobId(Id.incrementAndGet());
+    private static final AtomicLong id = new AtomicLong();
+
+    private TxnIdFactory() {
     }
 
-    public static void initJobId(int id) {
-        Id.set(id);
+    public static TxnId create() {
+        return new TxnId(id.incrementAndGet());
     }
-}
+
+    public static void ensureMinimumId(long id) {
+        TxnIdFactory.id.set(id);
+    }
+}
\ No newline at end of file
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 14e4020..64ac3cb 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
@@ -36,7 +36,7 @@
 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.common.transactions.TxnId;
 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;
@@ -300,7 +300,7 @@
      * @return throwable for said error
      */
     private static Throwable getError(Map<String, Throwable> errors, ITransactionContext txnCtx) {
-        return errors.get(txnCtx.getJobId().toString());
+        return errors.get(txnCtx.getTxnId().toString());
     }
 
     /**
@@ -318,7 +318,7 @@
         Throwable error = getError(errors, txnCtx);
         if (error == null) {
             throw new AssertionError(
-                    "expected " + clazz.getSimpleName() + " for " + txnCtx.getJobId() + ", got no " + "exception");
+                    "expected " + clazz.getSimpleName() + " for " + txnCtx.getTxnId() + ", got no " + "exception");
         }
         if (!clazz.isInstance(error)) {
             throw new AssertionError(error);
@@ -354,7 +354,7 @@
     private ITransactionContext j(int jId) {
         if (!jobId2TxnCtxMap.containsKey(jId)) {
             ITransactionContext mockTxnContext = mock(ITransactionContext.class);
-            when(mockTxnContext.getJobId()).thenReturn(new JobId(jId));
+            when(mockTxnContext.getTxnId()).thenReturn(new TxnId(jId));
             jobId2TxnCtxMap.put(jId, mockTxnContext);
         }
         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 97b4f8a..ef7d40e 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
@@ -71,7 +71,7 @@
      */
     Locker(ILockManager lockMgr, ITransactionContext txnCtx, List<Request> allRequests, AtomicInteger time,
             PrintStream err) {
-        this.name = txnCtx == null ? "admin" : txnCtx.getJobId().toString();
+        this.name = txnCtx == null ? "admin" : txnCtx.getTxnId().toString();
         this.lockMgr = lockMgr;
 
         this.requests = new LinkedList<>();
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 fd4dae5..112dc5f 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
@@ -58,7 +58,7 @@
 
     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 + ":"
+        return txnCtx.getTxnId() + ":" + kind.name() + ":" + dsId.getId() + ":" + hashValue + ":"
                 + TransactionManagementConstants.LockManagerConstants.LockMode.toString(lockMode);
     }
 
@@ -147,7 +147,7 @@
 
                 @Override
                 public String toString() {
-                    return txnCtx.getJobId().toString() + ":" + kind.name();
+                    return txnCtx.getTxnId().toString() + ":" + kind.name();
                 }
             };
         }