[NO ISSUE][OTR] Remove AppRuntimeContextProviderForRecovery

- user model changes: no
- storage format changes: no
- interface changes: yes
    - Remove IAppRuntimeContextProvider

Details:
AppRuntimeContextProviderForRecovery is not needed since it has
reference to ApplicationContext. ApplicationContext itself has all
required references by other classes.

Change-Id: I264b86b1bfff37c137936f620745025f0fb837ad
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2265
Sonar-Qube: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: abdullah alamoudi <bamousaa@gmail.com>
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AppRuntimeContextProviderForRecovery.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AppRuntimeContextProviderForRecovery.java
deleted file mode 100644
index 18ef143..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AppRuntimeContextProviderForRecovery.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.api.common;
-
-import java.util.concurrent.ExecutorService;
-
-import org.apache.asterix.app.nc.NCAppRuntimeContext;
-import org.apache.asterix.common.api.IDatasetLifecycleManager;
-import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.transactions.IAppRuntimeContextProvider;
-import org.apache.asterix.common.transactions.ITransactionSubsystem;
-import org.apache.hyracks.api.io.IIOManager;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
-import org.apache.hyracks.storage.common.ILocalResourceRepository;
-import org.apache.hyracks.storage.common.buffercache.IBufferCache;
-
-public class AppRuntimeContextProviderForRecovery implements IAppRuntimeContextProvider {
-
-    private final NCAppRuntimeContext asterixAppRuntimeContext;
-
-    public AppRuntimeContextProviderForRecovery(NCAppRuntimeContext asterixAppRuntimeContext) {
-        this.asterixAppRuntimeContext = asterixAppRuntimeContext;
-    }
-
-    @Override
-    public IBufferCache getBufferCache() {
-        return asterixAppRuntimeContext.getBufferCache();
-    }
-
-    @Override
-    public ITransactionSubsystem getTransactionSubsystem() {
-        return asterixAppRuntimeContext.getTransactionSubsystem();
-    }
-
-    @Override
-    public IDatasetLifecycleManager getDatasetLifecycleManager() {
-        return asterixAppRuntimeContext.getDatasetLifecycleManager();
-    }
-
-    @Override
-    public double getBloomFilterFalsePositiveRate() {
-        return asterixAppRuntimeContext.getBloomFilterFalsePositiveRate();
-    }
-
-    @Override
-    public ILSMIOOperationScheduler getLSMIOScheduler() {
-        return asterixAppRuntimeContext.getLSMIOScheduler();
-    }
-
-    @Override
-    public ILocalResourceRepository getLocalResourceRepository() {
-        return asterixAppRuntimeContext.getLocalResourceRepository();
-    }
-
-    @Override
-    public IIOManager getIOManager() {
-        return asterixAppRuntimeContext.getIoManager();
-    }
-
-    @Override
-    public ILSMOperationTracker getLSMBTreeOperationTracker(int datasetID) {
-        return asterixAppRuntimeContext.getLSMBTreeOperationTracker(datasetID);
-    }
-
-    @Override
-    public INcApplicationContext getAppContext() {
-        return asterixAppRuntimeContext;
-    }
-
-    @Override
-    public ExecutorService getThreadExecutor() {
-        return asterixAppRuntimeContext.getThreadExecutor();
-    }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
index 9c53c18..cb8c161 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
@@ -30,7 +30,6 @@
 import java.util.stream.Collectors;
 
 import org.apache.asterix.active.ActiveManager;
-import org.apache.asterix.api.common.AppRuntimeContextProviderForRecovery;
 import org.apache.asterix.common.api.ICoordinationService;
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.api.IDatasetMemoryManager;
@@ -58,7 +57,6 @@
 import org.apache.asterix.common.replication.IReplicationManager;
 import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
 import org.apache.asterix.common.storage.IReplicaManager;
-import org.apache.asterix.common.transactions.IAppRuntimeContextProvider;
 import org.apache.asterix.common.transactions.IRecoveryManager;
 import org.apache.asterix.common.transactions.IRecoveryManager.SystemState;
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
@@ -69,8 +67,8 @@
 import org.apache.asterix.metadata.api.IAsterixStateProxy;
 import org.apache.asterix.metadata.api.IMetadataNode;
 import org.apache.asterix.metadata.bootstrap.MetadataBootstrap;
-import org.apache.asterix.replication.management.ReplicationManager;
 import org.apache.asterix.replication.management.ReplicationChannel;
+import org.apache.asterix.replication.management.ReplicationManager;
 import org.apache.asterix.runtime.transaction.GlobalResourceIdFactoryProvider;
 import org.apache.asterix.runtime.utils.NoOpCoordinationService;
 import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
@@ -188,9 +186,7 @@
         localResourceRepository =
                 (PersistentLocalResourceRepository) persistentLocalResourceRepositoryFactory.createRepository();
 
-        IAppRuntimeContextProvider asterixAppRuntimeContextProvider = new AppRuntimeContextProviderForRecovery(this);
-        txnSubsystem = new TransactionSubsystem(getServiceContext(), getServiceContext().getNodeId(),
-                asterixAppRuntimeContextProvider, txnProperties);
+        txnSubsystem = new TransactionSubsystem(this);
         IRecoveryManager recoveryMgr = txnSubsystem.getRecoveryManager();
         SystemState systemState = recoveryMgr.getSystemState();
         if (initialRun || systemState == SystemState.PERMANENT_DATA_LOSS) {
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 080ad48..c189983 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
@@ -50,7 +50,6 @@
 import org.apache.asterix.common.storage.DatasetResourceReference;
 import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
 import org.apache.asterix.common.transactions.Checkpoint;
-import org.apache.asterix.common.transactions.IAppRuntimeContextProvider;
 import org.apache.asterix.common.transactions.ICheckpointManager;
 import org.apache.asterix.common.transactions.ILogReader;
 import org.apache.asterix.common.transactions.ILogRecord;
@@ -102,7 +101,7 @@
     public RecoveryManager(ITransactionSubsystem txnSubsystem, INCServiceContext serviceCtx) {
         this.serviceCtx = serviceCtx;
         this.txnSubsystem = txnSubsystem;
-        this.appCtx = txnSubsystem.getAsterixAppRuntimeContextProvider().getAppContext();
+        this.appCtx = txnSubsystem.getApplicationContext();
         logMgr = (LogManager) txnSubsystem.getLogManager();
         ReplicationProperties repProperties = appCtx.getReplicationProperties();
         replicationEnabled = repProperties.isReplicationEnabled();
@@ -277,8 +276,7 @@
         boolean foundWinner = false;
         JobEntityCommits jobEntityWinners = null;
 
-        IAppRuntimeContextProvider appRuntimeContext = txnSubsystem.getAsterixAppRuntimeContextProvider();
-        IDatasetLifecycleManager datasetLifecycleManager = appRuntimeContext.getDatasetLifecycleManager();
+        IDatasetLifecycleManager datasetLifecycleManager = appCtx.getDatasetLifecycleManager();
         final IIndexCheckpointManagerProvider indexCheckpointManagerProvider =
                 ((INcApplicationContext) (serviceCtx.getApplicationContext())).getIndexCheckpointManagerProvider();
 
@@ -409,8 +407,7 @@
 
     @Override
     public long getLocalMinFirstLSN() throws HyracksDataException {
-        IDatasetLifecycleManager datasetLifecycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
-                .getDatasetLifecycleManager();
+        final IDatasetLifecycleManager datasetLifecycleManager = appCtx.getDatasetLifecycleManager();
         List<IIndex> openIndexList = datasetLifecycleManager.getOpenResources();
         long firstLSN;
         //the min first lsn can only be the current append or smaller
@@ -431,8 +428,7 @@
     private long getRemoteMinFirstLSN() throws HyracksDataException {
         // find the min first lsn of partitions that are replicated on this node
         final Set<Integer> allPartitions = localResourceRepository.getAllPartitions();
-        final INcApplicationContext appContext = txnSubsystem.getAsterixAppRuntimeContextProvider().getAppContext();
-        final Set<Integer> masterPartitions = appContext.getReplicaManager().getPartitions();
+        final Set<Integer> masterPartitions = appCtx.getReplicaManager().getPartitions();
         allPartitions.removeAll(masterPartitions);
         return getPartitionsMinLSN(allPartitions);
     }
@@ -632,8 +628,7 @@
             //undo loserTxn's effect
             LOGGER.log(Level.INFO, "undoing loser transaction's effect");
 
-            IDatasetLifecycleManager datasetLifecycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
-                    .getDatasetLifecycleManager();
+            final IDatasetLifecycleManager datasetLifecycleManager = appCtx.getDatasetLifecycleManager();
             //TODO sort loser entities by smallest LSN to undo in one pass.
             Iterator<Entry<TxnEntityId, List<Long>>> iter = jobLoserEntity2LSNsMap.entrySet().iterator();
             int undoCount = 0;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/TransactionSubsystem.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/TransactionSubsystem.java
index fae0413..8158096 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/TransactionSubsystem.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/TransactionSubsystem.java
@@ -19,13 +19,13 @@
 package org.apache.asterix.app.nc;
 
 import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
 
+import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.asterix.common.config.ReplicationProperties;
 import org.apache.asterix.common.config.TransactionProperties;
-import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.transactions.Checkpoint;
 import org.apache.asterix.common.transactions.CheckpointProperties;
-import org.apache.asterix.common.transactions.IAppRuntimeContextProvider;
 import org.apache.asterix.common.transactions.ICheckpointManager;
 import org.apache.asterix.common.transactions.ILockManager;
 import org.apache.asterix.common.transactions.ILogManager;
@@ -37,7 +37,6 @@
 import org.apache.asterix.transaction.management.service.logging.LogManagerWithReplication;
 import org.apache.asterix.transaction.management.service.recovery.CheckpointManagerFactory;
 import org.apache.asterix.transaction.management.service.transaction.TransactionManager;
-import org.apache.hyracks.api.application.INCServiceContext;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.Logger;
 
@@ -52,26 +51,22 @@
     private final ILockManager lockManager;
     private final ITransactionManager transactionManager;
     private final IRecoveryManager recoveryManager;
-    private final IAppRuntimeContextProvider asterixAppRuntimeContextProvider;
     private final TransactionProperties txnProperties;
     private final ICheckpointManager checkpointManager;
+    private final INcApplicationContext appCtx;
 
     //for profiling purpose
     private long profilerEntityCommitLogCount = 0;
     private EntityCommitProfiler ecp;
 
-    public TransactionSubsystem(INCServiceContext serviceCtx, String id,
-            IAppRuntimeContextProvider asterixAppRuntimeContextProvider, TransactionProperties txnProperties)
-            throws ACIDException {
-        this.asterixAppRuntimeContextProvider = asterixAppRuntimeContextProvider;
-        this.id = id;
-        this.txnProperties = txnProperties;
+    public TransactionSubsystem(INcApplicationContext appCtx) {
+        this.appCtx = appCtx;
+        this.id = appCtx.getServiceContext().getNodeId();
+        this.txnProperties = appCtx.getTransactionProperties();
         this.transactionManager = new TransactionManager(this);
         this.lockManager = new ConcurrentLockManager(txnProperties.getLockManagerShrinkTimer());
-        ReplicationProperties repProperties =
-                asterixAppRuntimeContextProvider.getAppContext().getReplicationProperties();
+        final ReplicationProperties repProperties = appCtx.getReplicationProperties();
         final boolean replicationEnabled = repProperties.isReplicationEnabled();
-
         final CheckpointProperties checkpointProperties = new CheckpointProperties(txnProperties, id);
         if (LOGGER.isInfoEnabled()) {
             LOGGER.log(Level.INFO, "Checkpoint Properties: " + checkpointProperties);
@@ -83,10 +78,10 @@
         }
 
         this.logManager = replicationEnabled ? new LogManagerWithReplication(this) : new LogManager(this);
-        this.recoveryManager = new RecoveryManager(this, serviceCtx);
-        if (this.txnProperties.isCommitProfilerEnabled()) {
+        this.recoveryManager = new RecoveryManager(this, appCtx.getServiceContext());
+        if (txnProperties.isCommitProfilerEnabled()) {
             ecp = new EntityCommitProfiler(this, this.txnProperties.getCommitProfilerReportInterval());
-            getAsterixAppRuntimeContextProvider().getThreadExecutor().submit(ecp);
+            ((ExecutorService) appCtx.getThreadExecutor()).submit(ecp);
         }
     }
 
@@ -111,8 +106,8 @@
     }
 
     @Override
-    public IAppRuntimeContextProvider getAsterixAppRuntimeContextProvider() {
-        return asterixAppRuntimeContextProvider;
+    public INcApplicationContext getApplicationContext() {
+        return appCtx;
     }
 
     @Override
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IAppRuntimeContextProvider.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IAppRuntimeContextProvider.java
deleted file mode 100644
index 229fb6d..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IAppRuntimeContextProvider.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.common.transactions;
-
-import java.util.concurrent.ExecutorService;
-
-import org.apache.asterix.common.api.IDatasetLifecycleManager;
-import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.hyracks.api.io.IIOManager;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
-import org.apache.hyracks.storage.common.ILocalResourceRepository;
-import org.apache.hyracks.storage.common.buffercache.IBufferCache;
-
-public interface IAppRuntimeContextProvider {
-
-    ExecutorService getThreadExecutor();
-
-    IBufferCache getBufferCache();
-
-    ITransactionSubsystem getTransactionSubsystem();
-
-    IDatasetLifecycleManager getDatasetLifecycleManager();
-
-    double getBloomFilterFalsePositiveRate();
-
-    ILSMOperationTracker getLSMBTreeOperationTracker(int datasetID);
-
-    ILSMIOOperationScheduler getLSMIOScheduler();
-
-    ILocalResourceRepository getLocalResourceRepository();
-
-    IIOManager getIOManager();
-
-    INcApplicationContext getAppContext();
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITransactionSubsystem.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITransactionSubsystem.java
index 7bd55e8..3642a71 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITransactionSubsystem.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITransactionSubsystem.java
@@ -18,6 +18,7 @@
  */
 package org.apache.asterix.common.transactions;
 
+import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.asterix.common.config.TransactionProperties;
 
 /**
@@ -34,7 +35,7 @@
 
     IRecoveryManager getRecoveryManager();
 
-    IAppRuntimeContextProvider getAsterixAppRuntimeContextProvider();
+    INcApplicationContext getApplicationContext();
 
     String getId();
 
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 97fd7ce..d057c50 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
@@ -59,7 +59,7 @@
             IHyracksTaskContext ctx, IOperatorNodePushable operatorNodePushable) throws HyracksDataException {
         ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         IResourceLifecycleManager<IIndex> indexLifeCycleManager =
-                txnSubsystem.getAsterixAppRuntimeContextProvider().getDatasetLifecycleManager();
+                txnSubsystem.getApplicationContext().getDatasetLifecycleManager();
         ILSMIndex index = (ILSMIndex) indexLifeCycleManager.get(resource.getPath());
         if (index == null) {
             throw new HyracksDataException("Index(id:" + resource.getId() + ") is not registered.");
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 0c20ee9..26e1b22 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
@@ -55,7 +55,7 @@
             IHyracksTaskContext ctx, IOperatorNodePushable operatorNodePushable) throws HyracksDataException {
         ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         IResourceLifecycleManager indexLifeCycleManager =
-                txnSubsystem.getAsterixAppRuntimeContextProvider().getDatasetLifecycleManager();
+                txnSubsystem.getApplicationContext().getDatasetLifecycleManager();
         ILSMIndex index = (ILSMIndex) indexLifeCycleManager.get(resource.getPath());
         if (index == null) {
             throw new HyracksDataException("Index(id:" + resource.getId() + ") is not registered.");
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 c2f512f..1449a1b 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
@@ -55,7 +55,7 @@
         DatasetLocalResource aResource = (DatasetLocalResource) resource.getResource();
         ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         IResourceLifecycleManager indexLifeCycleManager =
-                txnSubsystem.getAsterixAppRuntimeContextProvider().getDatasetLifecycleManager();
+                txnSubsystem.getApplicationContext().getDatasetLifecycleManager();
         ILSMIndex index = (ILSMIndex) indexLifeCycleManager.get(resource.getPath());
         if (index == null) {
             throw new HyracksDataException("Index(id:" + resource.getId() + ") is not registered.");
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 833f8f6..96d0539 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
@@ -34,6 +34,7 @@
 import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.Semaphore;
@@ -77,7 +78,6 @@
     private final String logFilePrefix;
     private final MutableLong flushLSN;
     private final String nodeId;
-    private final FlushLogsLogger flushLogsLogger;
     private final HashMap<Long, Integer> txnLogFileId2ReaderCount = new HashMap<>();
     private final long logFileSize;
     private final int logPageSize;
@@ -107,7 +107,7 @@
         appendLSN = new AtomicLong();
         nodeId = txnSubsystem.getId();
         flushLogsQ = new LinkedBlockingQueue<>();
-        flushLogsLogger = new FlushLogsLogger();
+        txnSubsystem.getApplicationContext().getThreadExecutor().execute(new FlushLogsLogger());
         initializeLogManager(SMALLEST_LOG_FILE_ID);
     }
 
@@ -130,10 +130,8 @@
         }
         initNewPage(INITIAL_LOG_SIZE);
         logFlusher = new LogFlusher(this, emptyQ, flushQ, stashQ);
-        futureLogFlusher = txnSubsystem.getAsterixAppRuntimeContextProvider().getThreadExecutor().submit(logFlusher);
-        if (!flushLogsLogger.isAlive()) {
-            txnSubsystem.getAsterixAppRuntimeContextProvider().getThreadExecutor().execute(flushLogsLogger);
-        }
+        futureLogFlusher =
+                ((ExecutorService) txnSubsystem.getApplicationContext().getThreadExecutor()).submit(logFlusher);
     }
 
     @Override
@@ -633,20 +631,20 @@
 
     /**
      * This class is used to log FLUSH logs.
-     * FLUSH logs are flushed on a different thread to avoid a possible deadlock in LogBuffer batchUnlock which calls PrimaryIndexOpeartionTracker.completeOperation
-     * The deadlock happens when PrimaryIndexOpeartionTracker.completeOperation results in generating a FLUSH log and there are no empty log buffers available to log it.
+     * FLUSH logs are flushed on a different thread to avoid a possible deadlock in {@link LogBuffer} batchUnlock
+     * which calls {@link org.apache.asterix.common.context.PrimaryIndexOperationTracker} completeOperation. The
+     * deadlock happens when completeOperation generates a FLUSH log and there are no empty log buffers available
+     * to log it.
      */
-    private class FlushLogsLogger extends Thread {
+    private class FlushLogsLogger implements Runnable {
         @Override
         public void run() {
-            while (true) {
+            while (!Thread.currentThread().isInterrupted()) {
                 try {
-                    ILogRecord logRecord = flushLogsQ.take();
+                    final ILogRecord logRecord = flushLogsQ.take();
                     appendToLogTail(logRecord);
-                } catch (ACIDException e) {
-                    e.printStackTrace();
                 } catch (InterruptedException e) {
-                    //ignore
+                    Thread.currentThread().interrupt();
                 }
             }
         }
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/CheckpointManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/CheckpointManager.java
index 3cb91ff..487bc84 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/CheckpointManager.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/CheckpointManager.java
@@ -45,7 +45,7 @@
     @Override
     public synchronized void doSharpCheckpoint() throws HyracksDataException {
         LOGGER.info("Starting sharp checkpoint...");
-        final IDatasetLifecycleManager datasetLifecycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
+        final IDatasetLifecycleManager datasetLifecycleManager = txnSubsystem.getApplicationContext()
                 .getDatasetLifecycleManager();
         datasetLifecycleManager.flushAllDatasets();
         capture(SHARP_CHECKPOINT_LSN, true);
@@ -66,7 +66,7 @@
         boolean checkpointSucceeded = minFirstLSN >= checkpointTargetLSN;
         if (!checkpointSucceeded) {
             // Flush datasets with indexes behind target checkpoint LSN
-            IDatasetLifecycleManager datasetLifecycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
+            IDatasetLifecycleManager datasetLifecycleManager = txnSubsystem.getApplicationContext()
                     .getDatasetLifecycleManager();
             datasetLifecycleManager.scheduleAsyncFlushForLaggingDatasets(checkpointTargetLSN);
         }
diff --git a/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/TestRuntimeContextProvider.java b/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/TestRuntimeContextProvider.java
deleted file mode 100644
index f897aca..0000000
--- a/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/TestRuntimeContextProvider.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.transaction.management.service.locking;
-
-import static org.mockito.Mockito.mock;
-
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-
-import org.apache.asterix.common.api.IDatasetLifecycleManager;
-import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.transactions.IAppRuntimeContextProvider;
-import org.apache.asterix.common.transactions.ITransactionSubsystem;
-import org.apache.hyracks.api.io.IIOManager;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
-import org.apache.hyracks.storage.common.ILocalResourceRepository;
-import org.apache.hyracks.storage.common.buffercache.IBufferCache;
-
-class TestRuntimeContextProvider implements IAppRuntimeContextProvider {
-
-    ExecutorService ate = Executors.newCachedThreadPool(Executors.defaultThreadFactory());
-    IDatasetLifecycleManager dlcm = mock(IDatasetLifecycleManager.class);
-
-    @Override
-    public ExecutorService getThreadExecutor() {
-        return ate;
-    }
-
-    @Override
-    public IBufferCache getBufferCache() {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public ITransactionSubsystem getTransactionSubsystem() {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public IDatasetLifecycleManager getDatasetLifecycleManager() {
-        return dlcm;
-    }
-
-    @Override
-    public double getBloomFilterFalsePositiveRate() {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public ILSMOperationTracker getLSMBTreeOperationTracker(int datasetID) {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public ILSMIOOperationScheduler getLSMIOScheduler() {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public ILocalResourceRepository getLocalResourceRepository() {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public IIOManager getIOManager() {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public INcApplicationContext getAppContext() {
-        throw new UnsupportedOperationException();
-    }
-}