Merge branch 'master' into zheilbron/asterix_issue470

Conflicts:
	asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
	asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
	asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
	asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java
	asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java
	asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixStorageProperties.java
	asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
	asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java
	asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java
	asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
index d17d77c..da2e838 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
@@ -4,6 +4,7 @@
 import java.util.List;
 
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
+import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import edu.uci.ics.asterix.common.dataflow.IAsterixApplicationContextInfo;
 import edu.uci.ics.asterix.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.MetadataManager;
@@ -201,19 +202,19 @@
             AsterixStorageProperties storageProperties = AsterixAppContextInfo.getInstance().getStorageProperties();
             if (!isPartitioned) {
                 dataflowHelperFactory = new LSMInvertedIndexDataflowHelperFactory(
+                        new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
                         AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                        storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
                         storageProperties.getBloomFilterFalsePositiveRate());
             } else {
                 dataflowHelperFactory = new PartitionedLSMInvertedIndexDataflowHelperFactory(
+                        new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
                         AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                        storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
                         storageProperties.getBloomFilterFalsePositiveRate());
             }
             LSMInvertedIndexSearchOperatorDescriptor invIndexSearchOp = new LSMInvertedIndexSearchOperatorDescriptor(
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
index e64f68f..60404ca 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
@@ -13,6 +13,7 @@
 import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
 import edu.uci.ics.asterix.common.context.AsterixFileMapManager;
 import edu.uci.ics.asterix.common.context.ConstantMergePolicy;
+import edu.uci.ics.asterix.common.context.DatasetLifecycleManager;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
@@ -29,10 +30,10 @@
 import edu.uci.ics.hyracks.api.io.IIOManager;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexLifecycleManager;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import edu.uci.ics.hyracks.storage.am.lsm.common.impls.SynchronousScheduler;
 import edu.uci.ics.hyracks.storage.common.buffercache.BufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.ClockPageReplacementStrategy;
@@ -50,9 +51,6 @@
 import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactoryProvider;
 
 public class AsterixAppRuntimeContext implements IAsterixAppRuntimeContext, IAsterixPropertiesProvider {
-    private static final int DEFAULT_BUFFER_CACHE_PAGE_SIZE = 32768;
-    private static final int DEFAULT_LIFECYCLEMANAGER_MEMORY_BUDGET = 1024 * 1024 * 1024; // 1GB
-    private static final int DEFAULT_MAX_OPEN_FILES = Integer.MAX_VALUE;
     private static final int METADATA_IO_DEVICE_ID = 0;
 
     private final INCApplicationContext ncApplicationContext;
@@ -63,7 +61,7 @@
     private AsterixStorageProperties storageProperties;
     private AsterixTransactionProperties txnProperties;
 
-    private IIndexLifecycleManager indexLifecycleManager;
+    private DatasetLifecycleManager indexLifecycleManager;
     private IFileMapManager fileMapManager;
     private IBufferCache bufferCache;
     private ITransactionSubsystem txnSubsystem;
@@ -101,8 +99,6 @@
                 storageProperties.getBufferCachePageSize(), storageProperties.getBufferCacheNumPages(),
                 storageProperties.getBufferCacheMaxOpenFiles());
 
-        indexLifecycleManager = new IndexLifecycleManager(storageProperties.getMemoryComponentGlobalBudget());
-
         lsmIOScheduler = SynchronousScheduler.INSTANCE;
         mergePolicy = new ConstantMergePolicy(storageProperties.getLSMIndexMergeThreshold(), this);
         lsmBTreeOpTrackerFactory = new IndexOperationTrackerFactory(LSMBTreeIOOperationCallbackFactory.INSTANCE);
@@ -115,7 +111,7 @@
         localResourceRepository = (PersistentLocalResourceRepository) persistentLocalResourceRepositoryFactory
                 .createRepository();
         resourceIdFactory = (new ResourceIdFactoryProvider(localResourceRepository)).createResourceIdFactory();
-
+        indexLifecycleManager = new DatasetLifecycleManager(storageProperties, localResourceRepository);
         IAsterixAppRuntimeContextProvider asterixAppRuntimeContextProvider = new AsterixAppRuntimeContextProviderForRecovery(
                 this);
         txnSubsystem = new TransactionSubsystem(ncApplicationContext.getNodeId(), asterixAppRuntimeContextProvider);
@@ -217,4 +213,9 @@
     public AsterixExternalProperties getExternalProperties() {
         return externalProperties;
     }
+
+    @Override
+    public IVirtualBufferCache getVirtualBufferCache(int datasetID) {
+        return indexLifecycleManager.getVirtualBufferCache(datasetID);
+    }
 }
\ No newline at end of file
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContextProviderForRecovery.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContextProviderForRecovery.java
index 8c73a63..c3dae04 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContextProviderForRecovery.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContextProviderForRecovery.java
@@ -8,6 +8,7 @@
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
@@ -87,6 +88,17 @@
     }
 
     @Override
+    public IVirtualBufferCache getVirtualBufferCache(int datasetID) {
+        return asterixAppRuntimeContext.getVirtualBufferCache(datasetID);
+    }
+
+    @Override
+    public ILSMIOOperationCallbackProvider getNoOpIOOperationCallbackProvider() {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
     public ILSMIOOperationCallbackProvider getLSMBTreeIOOperationCallbackProvider() {
         // TODO Auto-generated method stub
         return null;
@@ -103,11 +115,4 @@
         // TODO Auto-generated method stub
         return null;
     }
-
-    @Override
-    public ILSMIOOperationCallbackProvider getNoOpIOOperationCallbackProvider() {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
 }
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
index 911f6fd..8ba7f9f 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
@@ -547,12 +547,13 @@
         String datasetName = null;
         String indexName = null;
         JobSpecification spec = null;
+        Dataset ds = null;
         try {
             CreateIndexStatement stmtCreateIndex = (CreateIndexStatement) stmt;
             dataverseName = getActiveDataverseName(stmtCreateIndex.getDataverseName());
             datasetName = stmtCreateIndex.getDatasetName().getValue();
 
-            Dataset ds = MetadataManager.INSTANCE.getDataset(metadataProvider.getMetadataTxnContext(), dataverseName,
+            ds = MetadataManager.INSTANCE.getDataset(metadataProvider.getMetadataTxnContext(), dataverseName,
                     datasetName);
             if (ds == null) {
                 throw new AlgebricksException("There is no dataset with this name " + datasetName + " in dataverse "
@@ -638,7 +639,8 @@
                 metadataProvider.setMetadataTxnContext(mdTxnCtx);
                 CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName, indexName);
                 try {
-                    JobSpecification jobSpec = IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider);
+                    JobSpecification jobSpec = IndexOperations
+                            .buildDropSecondaryIndexJobSpec(cds, metadataProvider, ds);
                     MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                     bActiveTxn = false;
 
@@ -747,7 +749,8 @@
                         if (indexes.get(k).isSecondaryIndex()) {
                             CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName,
                                     indexes.get(k).getIndexName());
-                            jobsToExecute.add(IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider));
+                            jobsToExecute.add(IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider,
+                                    datasets.get(j)));
                         }
                     }
 
@@ -859,7 +862,7 @@
                     if (indexes.get(j).isSecondaryIndex()) {
                         CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName,
                                 indexes.get(j).getIndexName());
-                        jobsToExecute.add(IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider));
+                        jobsToExecute.add(IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider, ds));
                     }
                 }
                 CompiledDatasetDropStatement cds = new CompiledDatasetDropStatement(dataverseName, datasetName);
@@ -965,7 +968,7 @@
                 }
                 //#. prepare a job to drop the index in NC.
                 CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName, indexName);
-                jobsToExecute.add(IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider));
+                jobsToExecute.add(IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider, ds));
 
                 //#. mark PendingDropOp on the existing index
                 MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
index e22c215..8b3323e 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
@@ -22,10 +22,12 @@
 import java.util.logging.Logger;
 
 import edu.uci.ics.asterix.api.common.Job;
+import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.config.GlobalConfig;
 import edu.uci.ics.asterix.common.config.OptimizationConfUtil;
+import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.formats.base.IDataFormat;
@@ -40,7 +42,6 @@
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
-import edu.uci.ics.asterix.transaction.management.resource.ILocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.resource.LSMBTreeLocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
 import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
@@ -123,12 +124,11 @@
         AsterixStorageProperties storageProperties = AsterixAppContextInfo.getInstance().getStorageProperties();
         IndexDropOperatorDescriptor primaryBtreeDrop = new IndexDropOperatorDescriptor(specPrimary,
                 AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
-                splitsAndConstraint.first, new LSMBTreeDataflowHelperFactory(
+                splitsAndConstraint.first, new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
+                        dataset.getDatasetId()), AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
                         storageProperties.getBloomFilterFalsePositiveRate()));
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(specPrimary, primaryBtreeDrop,
                 splitsAndConstraint.second);
@@ -170,20 +170,19 @@
         AsterixStorageProperties storageProperties = AsterixAppContextInfo.getInstance().getStorageProperties();
         //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
         ILocalResourceMetadata localResourceMetadata = new LSMBTreeLocalResourceMetadata(typeTraits,
-                comparatorFactories, blooFilterKeyFields, true, storageProperties.getMemoryComponentPageSize(),
-                storageProperties.getMemoryComponentNumPages(), fs);
+                comparatorFactories, blooFilterKeyFields, fs, dataset.getDatasetId());
         ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
                 localResourceMetadata, LocalResource.LSMBTreeResource);
 
         TreeIndexCreateOperatorDescriptor indexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
                 AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
                 splitsAndConstraint.first, typeTraits, comparatorFactories, blooFilterKeyFields,
-                new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, storageProperties
-                                .getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
-                        storageProperties.getBloomFilterFalsePositiveRate()), localResourceFactoryProvider,
+                                .getBloomFilterFalsePositiveRate()), localResourceFactoryProvider,
                 NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, indexCreateOp,
                 splitsAndConstraint.second);
@@ -269,11 +268,11 @@
                 AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
                 splitsAndConstraint.first, typeTraits, comparatorFactories, blooFilterKeyFields, fieldPermutation,
                 GlobalConfig.DEFAULT_BTREE_FILL_FACTOR, false, numElementsHint, new LSMBTreeDataflowHelperFactory(
+                        new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
                         storageProperties.getBloomFilterFalsePositiveRate()), NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, btreeBulkLoad,
                 splitsAndConstraint.second);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java
index 90a1ef0..4a17a0f 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java
@@ -2,9 +2,11 @@
 
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.OptimizationConfUtil;
+import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
 import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
 import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.asterix.translator.CompiledStatements.CompiledCreateIndexStatement;
@@ -39,12 +41,11 @@
     }
 
     public static JobSpecification buildDropSecondaryIndexJobSpec(CompiledIndexDropStatement indexDropStmt,
-            AqlMetadataProvider metadataProvider) throws AlgebricksException, MetadataException {
+            AqlMetadataProvider metadataProvider, Dataset dataset) throws AlgebricksException, MetadataException {
         String dataverseName = indexDropStmt.getDataverseName() == null ? metadataProvider.getDefaultDataverseName()
                 : indexDropStmt.getDataverseName();
         String datasetName = indexDropStmt.getDatasetName();
         String indexName = indexDropStmt.getIndexName();
-
         JobSpecification spec = new JobSpecification();
 
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadataProvider
@@ -52,12 +53,11 @@
         AsterixStorageProperties storageProperties = AsterixAppContextInfo.getInstance().getStorageProperties();
         IndexDropOperatorDescriptor btreeDrop = new IndexDropOperatorDescriptor(spec,
                 AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
-                splitsAndConstraint.first, new LSMBTreeDataflowHelperFactory(
+                splitsAndConstraint.first, new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
+                        dataset.getDatasetId()), AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
                         storageProperties.getBloomFilterFalsePositiveRate()));
         AlgebricksPartitionConstraintHelper
                 .setPartitionConstraintInJobSpec(spec, btreeDrop, splitsAndConstraint.second);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
index ebd1a9c..12b0470 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
@@ -1,9 +1,10 @@
 package edu.uci.ics.asterix.file;
 
+import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
+import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.transaction.management.resource.ILocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.resource.LSMBTreeLocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
 import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
@@ -39,21 +40,19 @@
         AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
         //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
         ILocalResourceMetadata localResourceMetadata = new LSMBTreeLocalResourceMetadata(
-                secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields, false,
-                storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
-                secondaryFileSplitProvider.getFileSplits());
+                secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields,
+                secondaryFileSplitProvider.getFileSplits(), dataset.getDatasetId());
         ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
                 localResourceMetadata, LocalResource.LSMBTreeResource);
 
         TreeIndexCreateOperatorDescriptor secondaryIndexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
                 AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
                 secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
-                secondaryBloomFilterKeyFields, new LSMBTreeDataflowHelperFactory(
+                secondaryBloomFilterKeyFields, new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
+                        dataset.getDatasetId()), AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
                         storageProperties.getBloomFilterFalsePositiveRate()), localResourceFactoryProvider,
                 NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,
@@ -90,12 +89,12 @@
         TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(
                 spec,
                 numSecondaryKeys,
-                new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, storageProperties
-                                .getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
-                        storageProperties.getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
+                                .getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
 
         // Connect the operators.
         spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java
index b79102b..9329735 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java
@@ -22,6 +22,7 @@
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
+import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.formats.nontagged.AqlBinaryBooleanInspectorImpl;
 import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
@@ -279,12 +280,12 @@
                 AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
                 primaryFileSplitProvider, primaryRecDesc.getTypeTraits(), primaryComparatorFactories,
                 primaryBloomFilterKeyFields, lowKeyFields, highKeyFields, true, true,
-                new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, storageProperties
-                                .getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
-                        storageProperties.getBloomFilterFalsePositiveRate()), false,
+                                .getBloomFilterFalsePositiveRate()), false,
                 NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, primarySearchOp,
                 primaryPartitionConstraint);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java
index ef94ec2..5e68053 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java
@@ -2,16 +2,17 @@
 
 import java.util.List;
 
+import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
 import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
+import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
 import edu.uci.ics.asterix.runtime.formats.FormatUtils;
-import edu.uci.ics.asterix.transaction.management.resource.ILocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.resource.LSMInvertedIndexLocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
 import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
@@ -152,12 +153,10 @@
     public JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException {
         JobSpecification spec = new JobSpecification();
 
-        AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
         //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
         ILocalResourceMetadata localResourceMetadata = new LSMInvertedIndexLocalResourceMetadata(invListsTypeTraits,
-                primaryComparatorFactories, tokenTypeTraits, tokenComparatorFactories, tokenizerFactory,
-                storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
-                isPartitioned, secondaryFileSplitProvider.getFileSplits());
+                primaryComparatorFactories, tokenTypeTraits, tokenComparatorFactories, tokenizerFactory, isPartitioned,
+                secondaryFileSplitProvider.getFileSplits(), dataset.getDatasetId());
         ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
                 localResourceMetadata, LocalResource.LSMInvertedIndexResource);
 
@@ -265,20 +264,18 @@
     private IIndexDataflowHelperFactory createDataflowHelperFactory() {
         AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
         if (!isPartitioned) {
-            return new LSMInvertedIndexDataflowHelperFactory(
+            return new LSMInvertedIndexDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
+                    dataset.getDatasetId()), AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                     AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                     AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                     AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                    AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                    storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
                     storageProperties.getBloomFilterFalsePositiveRate());
         } else {
-            return new PartitionedLSMInvertedIndexDataflowHelperFactory(
+            return new PartitionedLSMInvertedIndexDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
+                    dataset.getDatasetId()), AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                     AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                     AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                     AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                    AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                    storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
                     storageProperties.getBloomFilterFalsePositiveRate());
         }
     }
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java
index b7e4886..b27de9f 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java
@@ -2,8 +2,10 @@
 
 import java.util.List;
 
+import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
+import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.dataflow.data.nontagged.valueproviders.AqlPrimitiveValueProviderFactory;
 import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
@@ -14,7 +16,6 @@
 import edu.uci.ics.asterix.om.types.ATypeTag;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
-import edu.uci.ics.asterix.transaction.management.resource.ILocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.resource.LSMRTreeLocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
 import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
@@ -64,8 +65,8 @@
         ILocalResourceMetadata localResourceMetadata = new LSMRTreeLocalResourceMetadata(
                 secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, primaryComparatorFactories,
                 valueProviderFactories, RTreePolicyType.RTREE, AqlMetadataProvider.proposeLinearizer(keyType,
-                        secondaryComparatorFactories.length), storageProperties.getMemoryComponentPageSize(),
-                storageProperties.getMemoryComponentNumPages(), secondaryFileSplitProvider.getFileSplits());
+                        secondaryComparatorFactories.length), secondaryFileSplitProvider.getFileSplits(),
+                dataset.getDatasetId());
         ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
                 localResourceMetadata, LocalResource.LSMRTreeResource);
 
@@ -73,13 +74,13 @@
                 AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
                 secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, null,
                 new LSMRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
-                        primaryComparatorFactories, AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
+                        primaryComparatorFactories, new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
+                        AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, AqlMetadataProvider.proposeLinearizer(
                                 keyType, secondaryComparatorFactories.length), storageProperties
-                                .getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
-                        storageProperties.getBloomFilterFalsePositiveRate()), localResourceFactoryProvider,
+                                .getBloomFilterFalsePositiveRate()), localResourceFactoryProvider,
                 NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,
                 secondaryPartitionConstraint);
@@ -159,13 +160,13 @@
                 spec,
                 numNestedSecondaryKeyFields,
                 new LSMRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
-                        primaryComparatorFactories, AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
+                        primaryComparatorFactories, new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
+                        AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, AqlMetadataProvider.proposeLinearizer(
                                 keyType, secondaryComparatorFactories.length), storageProperties
-                                .getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
-                        storageProperties.getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
+                                .getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
 
         // Connect the operators.
         spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixAppRuntimeContext.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixAppRuntimeContext.java
index e053989..7d52252 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixAppRuntimeContext.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixAppRuntimeContext.java
@@ -10,6 +10,7 @@
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
@@ -47,4 +48,5 @@
 
     public double getBloomFilterFalsePositiveRate();
 
+    public IVirtualBufferCache getVirtualBufferCache(int datasetID);
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ILocalResourceMetadata.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/ILocalResourceMetadata.java
similarity index 86%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ILocalResourceMetadata.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/api/ILocalResourceMetadata.java
index 3b32dc2..9fdfe5f 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ILocalResourceMetadata.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/ILocalResourceMetadata.java
@@ -1,4 +1,4 @@
-package edu.uci.ics.asterix.transaction.management.resource;
+package edu.uci.ics.asterix.common.api;
 
 import java.io.Serializable;
 
@@ -10,5 +10,6 @@
 
     public ILSMIndex createIndexInstance(IAsterixAppRuntimeContextProvider runtimeContextProvider, String filePath,
             int partition) throws HyracksDataException;
-    
+
+    public int getDatasetID();
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixExternalProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixExternalProperties.java
index cf38932..6dd2ced 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixExternalProperties.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixExternalProperties.java
@@ -8,7 +8,7 @@
     private static int EXTERNAL_WEBPORT_DEFAULT = 19001;
 
     private static final String EXTERNAL_LOGLEVEL_KEY = "log.level";
-    private static Level EXTERNAL_LOGLEVEL_DEFAULT = Level.INFO;
+    private static Level EXTERNAL_LOGLEVEL_DEFAULT = Level.WARNING;
 
     private static final String EXTERNAL_APISERVER_KEY = "api.port";
     private static int EXTERNAL_APISERVER_DEFAULT = 19101;
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixStorageProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixStorageProperties.java
index 91bdca6..8ea64b8 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixStorageProperties.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixStorageProperties.java
@@ -15,10 +15,10 @@
     private static final int STORAGE_MEMORYCOMPONENT_PAGESIZE_DEFAULT = (32 << 10); // 32KB
 
     private static final String STORAGE_MEMORYCOMPONENT_NUMPAGES_KEY = "storage.memorycomponent.numpages";
-    private static final int STORAGE_MEMORYCOMPONENT_NUMPAGES_DEFAULT = 2048; // ... so 64MB components
+    private static final int STORAGE_MEMORYCOMPONENT_NUMPAGES_DEFAULT = 1024; // ... so 32MB components
 
     private static final String STORAGE_MEMORYCOMPONENT_GLOBALBUDGET_KEY = "storage.memorycomponent.globalbudget";
-    private static final long STORAGE_MEMORYCOMPONENT_GLOBALBUDGET_DEFAULT = (1 << 30); // 1GB
+    private static final long STORAGE_MEMORYCOMPONENT_GLOBALBUDGET_DEFAULT = 536870912; // 512MB
 
     private static final String STORAGE_LSM_MERGETHRESHOLD_KEY = "storage.lsm.mergethreshold";
     private static int STORAGE_LSM_MERGETHRESHOLD_DEFAULT = 3;
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixVirtualBufferCacheProvider.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixVirtualBufferCacheProvider.java
new file mode 100644
index 0000000..c046e42
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixVirtualBufferCacheProvider.java
@@ -0,0 +1,23 @@
+package edu.uci.ics.asterix.common.context;
+
+import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
+
+public class AsterixVirtualBufferCacheProvider implements IVirtualBufferCacheProvider {
+
+    private static final long serialVersionUID = 1L;
+    private final int datasetID;
+
+    public AsterixVirtualBufferCacheProvider(int datasetID) {
+        this.datasetID = datasetID;
+    }
+
+    @Override
+    public IVirtualBufferCache getVirtualBufferCache(IHyracksTaskContext ctx) {
+        return ((IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
+                .getVirtualBufferCache(datasetID);
+    }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java
new file mode 100644
index 0000000..5808d3a
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java
@@ -0,0 +1,288 @@
+package edu.uci.ics.asterix.common.context;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
+import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.MultitenantVirtualBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.VirtualBufferCache;
+import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
+import edu.uci.ics.hyracks.storage.common.file.LocalResource;
+
+public class DatasetLifecycleManager implements IIndexLifecycleManager {
+    private final AsterixStorageProperties storageProperties;
+    private final Map<Integer, MultitenantVirtualBufferCache> datasetVirtualBufferCaches;
+    private final Map<Integer, DatasetInfo> datasetInfos;
+    private final ILocalResourceRepository resourceRepository;
+    private final long capacity;
+    private long used;
+
+    public DatasetLifecycleManager(AsterixStorageProperties storageProperties,
+            ILocalResourceRepository resourceRepository) {
+        this.storageProperties = storageProperties;
+        this.resourceRepository = resourceRepository;
+        datasetVirtualBufferCaches = new HashMap<Integer, MultitenantVirtualBufferCache>();
+        datasetInfos = new HashMap<Integer, DatasetInfo>();
+        capacity = storageProperties.getMemoryComponentGlobalBudget();
+        used = 0;
+    }
+
+    @Override
+    public synchronized IIndex getIndex(long resourceID) throws HyracksDataException {
+        DatasetInfo dsInfo = datasetInfos.get(getDIDfromRID(resourceID));
+        if (dsInfo == null) {
+            return null;
+        }
+        IndexInfo iInfo = dsInfo.indexes.get(resourceID);
+        if (iInfo == null) {
+            return null;
+        }
+        return iInfo.index;
+    }
+
+    @Override
+    public synchronized void register(long resourceID, IIndex index) throws HyracksDataException {
+        int did = getDIDfromRID(resourceID);
+        DatasetInfo dsInfo = datasetInfos.get(did);
+        if (dsInfo == null) {
+            dsInfo = new DatasetInfo(did);
+        } else if (dsInfo.indexes.containsKey(resourceID)) {
+            throw new HyracksDataException("Index with resource ID " + resourceID + " already exists.");
+        }
+        datasetInfos.put(did, dsInfo);
+        dsInfo.indexes.put(resourceID, new IndexInfo((ILSMIndex) index));
+    }
+
+    private int getDIDfromRID(long resourceID) throws HyracksDataException {
+        LocalResource lr = resourceRepository.getResourceById(resourceID);
+        if (lr == null) {
+            return -1;
+        }
+        return ((ILocalResourceMetadata) lr.getResourceObject()).getDatasetID();
+    }
+
+    @Override
+    public synchronized void unregister(long resourceID) throws HyracksDataException {
+        int did = getDIDfromRID(resourceID);
+        DatasetInfo dsInfo = datasetInfos.get(did);
+        IndexInfo iInfo = dsInfo.indexes.remove(resourceID);
+        if (dsInfo == null || iInfo == null) {
+            throw new HyracksDataException("Index with resource ID " + resourceID + " does not exist.");
+        }
+
+        if (iInfo.referenceCount != 0) {
+            dsInfo.indexes.put(resourceID, iInfo);
+            throw new HyracksDataException("Cannot remove index while it is open.");
+        }
+
+        if (iInfo.isOpen) {
+            iInfo.index.deactivate(true);
+        }
+
+        if (dsInfo.referenceCount == 0 && dsInfo.isOpen) {
+            for (IndexInfo i : dsInfo.indexes.values()) {
+                i.index.deactivate(true);
+            }
+            IVirtualBufferCache vbc = getVirtualBufferCache(did);
+            assert vbc != null;
+            used -= (vbc.getNumPages() * vbc.getPageSize());
+        }
+        datasetInfos.remove(did);
+    }
+
+    @Override
+    public synchronized void open(long resourceID) throws HyracksDataException {
+        int did = getDIDfromRID(resourceID);
+        DatasetInfo dsInfo = datasetInfos.get(did);
+        if (dsInfo == null) {
+            throw new HyracksDataException("Failed to open index with resource ID " + resourceID
+                    + " since it does not exist.");
+        }
+
+        IndexInfo iInfo = dsInfo.indexes.get(resourceID);
+        if (iInfo == null) {
+            throw new HyracksDataException("Failed to open index with resource ID " + resourceID
+                    + " since it does not exist.");
+        }
+
+        if (!dsInfo.isOpen) {
+            IVirtualBufferCache vbc = getVirtualBufferCache(did);
+            assert vbc != null;
+            long additionalSize = vbc.getNumPages() * vbc.getPageSize();
+            while (used + additionalSize > capacity) {
+                if (!evictCandidateDataset()) {
+                    throw new HyracksDataException("Cannot activate index since memory budget would be exceeded.");
+                }
+            }
+            used += additionalSize;
+        }
+
+        dsInfo.isOpen = true;
+        dsInfo.touch();
+        if (!iInfo.isOpen) {
+            iInfo.index.activate();
+            iInfo.isOpen = true;
+        }
+        iInfo.touch();
+    }
+
+    private boolean evictCandidateDataset() throws HyracksDataException {
+        // Why min()? As a heuristic for eviction, we will take an open index (an index consuming memory) 
+        // that is not being used (refcount == 0) and has been least recently used. The sort order defined 
+        // for IndexInfo maintains this. See IndexInfo.compareTo().
+        DatasetInfo dsInfo = Collections.min(datasetInfos.values());
+        if (dsInfo.referenceCount == 0 && dsInfo.isOpen) {
+            for (IndexInfo iInfo : dsInfo.indexes.values()) {
+                if (iInfo.isOpen) {
+                    iInfo.index.deactivate(true);
+                    iInfo.isOpen = false;
+                }
+                assert iInfo.referenceCount == 0;
+            }
+
+            IVirtualBufferCache vbc = getVirtualBufferCache(dsInfo.datasetID);
+            used -= vbc.getNumPages() * vbc.getPageSize();
+            dsInfo.isOpen = false;
+            return true;
+        }
+        return false;
+    }
+
+    @Override
+    public synchronized void close(long resourceID) throws HyracksDataException {
+        int did = getDIDfromRID(resourceID);
+        DatasetInfo dsInfo = datasetInfos.get(did);
+        if (dsInfo == null) {
+            throw new HyracksDataException("No index found with resourceID " + resourceID);
+        }
+        IndexInfo iInfo = dsInfo.indexes.get(resourceID);
+        if (iInfo == null) {
+            throw new HyracksDataException("No index found with resourceID " + resourceID);
+        }
+        iInfo.untouch();
+        dsInfo.untouch();
+    }
+
+    @Override
+    public synchronized List<IIndex> getOpenIndexes() {
+        List<IIndex> openIndexes = new ArrayList<IIndex>();
+        for (DatasetInfo dsInfo : datasetInfos.values()) {
+            for (IndexInfo iInfo : dsInfo.indexes.values()) {
+                if (iInfo.isOpen) {
+                    openIndexes.add(iInfo.index);
+                }
+            }
+        }
+        return openIndexes;
+    }
+
+    public IVirtualBufferCache getVirtualBufferCache(int datasetID) {
+        synchronized (datasetVirtualBufferCaches) {
+            MultitenantVirtualBufferCache vbc = datasetVirtualBufferCaches.get(datasetID);
+            if (vbc == null) {
+                vbc = new MultitenantVirtualBufferCache(new VirtualBufferCache(new HeapBufferAllocator(),
+                        storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages()));
+                datasetVirtualBufferCaches.put(datasetID, vbc);
+            }
+            return vbc;
+        }
+    }
+
+    private static abstract class Info {
+        protected int referenceCount;
+        protected boolean isOpen;
+
+        public Info() {
+            referenceCount = 0;
+            isOpen = false;
+        }
+
+        public void touch() {
+            ++referenceCount;
+        }
+
+        public void untouch() {
+            --referenceCount;
+        }
+    }
+
+    private static class IndexInfo extends Info {
+        private ILSMIndex index;
+
+        public IndexInfo(ILSMIndex index) {
+            this.index = index;
+        }
+    }
+
+    private static class DatasetInfo extends Info implements Comparable<DatasetInfo> {
+        private final Map<Long, IndexInfo> indexes;
+        private final int datasetID;
+        private long lastAccess;
+
+        public DatasetInfo(int datasetID) {
+            this.indexes = new HashMap<Long, IndexInfo>();
+            this.lastAccess = -1;
+            this.datasetID = datasetID;
+        }
+
+        public void touch() {
+            super.touch();
+            lastAccess = System.currentTimeMillis();
+        }
+
+        public void untouch() {
+            super.untouch();
+            lastAccess = System.currentTimeMillis();
+        }
+
+        @Override
+        public int compareTo(DatasetInfo i) {
+            // sort by (isOpen, referenceCount, lastAccess) ascending, where true < false
+            //
+            // Example sort order:
+            // -------------------
+            // (F, 0, 70)       <-- largest
+            // (F, 0, 60)
+            // (T, 10, 80)
+            // (T, 10, 70)
+            // (T, 9, 90)
+            // (T, 0, 100)      <-- smallest
+            if (isOpen && !i.isOpen) {
+                return -1;
+            } else if (!isOpen && i.isOpen) {
+                return 1;
+            } else {
+                if (referenceCount < i.referenceCount) {
+                    return -1;
+                } else if (referenceCount > i.referenceCount) {
+                    return 1;
+                } else {
+                    if (lastAccess < i.lastAccess) {
+                        return -1;
+                    } else if (lastAccess > i.lastAccess) {
+                        return 1;
+                    } else {
+                        return 0;
+                    }
+                }
+            }
+
+        }
+
+        public String toString() {
+            return "DatasetID: " + datasetID + ", isOpen: " + isOpen + ", refCount: " + referenceCount
+                    + ", lastAccess: " + lastAccess + "}";
+        }
+    }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IAsterixAppRuntimeContextProvider.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IAsterixAppRuntimeContextProvider.java
index f762b16..1b91b0c 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IAsterixAppRuntimeContextProvider.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IAsterixAppRuntimeContextProvider.java
@@ -6,6 +6,7 @@
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
@@ -46,4 +47,6 @@
     public ResourceIdFactory getResourceIdFactory();
 
     public IIOManager getIOManager();
+
+    public IVirtualBufferCache getVirtualBufferCache(int datasetID);
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILoggerRepository.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILoggerRepository.java
index 036d66c..323b9ce 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILoggerRepository.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILoggerRepository.java
@@ -1,6 +1,8 @@
 package edu.uci.ics.asterix.common.transactions;
 
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+
 public interface ILoggerRepository {
 
-    public  ILogger getIndexLogger(long resourceId, byte resourceType);
+    public ILogger getIndexLogger(long resourceId, byte resourceType) throws ACIDException;
 }
diff --git a/asterix-installer/src/main/resources/conf/asterix-configuration.xml b/asterix-installer/src/main/resources/conf/asterix-configuration.xml
index bfec2db..6c2b757 100644
--- a/asterix-installer/src/main/resources/conf/asterix-configuration.xml
+++ b/asterix-installer/src/main/resources/conf/asterix-configuration.xml
@@ -47,9 +47,9 @@
 
 	<property>
 		<name>storage.memorycomponent.numpages</name>
-		<value>4096</value>
+		<value>1024</value>
 		<description>The number of pages to allocate for a memory component.
-			(Default = 4096)
+			(Default = 1024)
 		</description>
 	</property>
 
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
index 5aca7c4..9963ec7 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
@@ -974,8 +974,8 @@
     public void initializeDatasetIdFactory(JobId jobId) throws MetadataException, RemoteException {
         int mostRecentDatasetId = MetadataPrimaryIndexes.FIRST_AVAILABLE_USER_DATASET_ID;
         long resourceID = MetadataPrimaryIndexes.DATASET_DATASET.getResourceID();
-        IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
         try {
+            IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
             indexLifecycleManager.open(resourceID);
             IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
                     NoOpOperationCallback.INSTANCE);
@@ -991,6 +991,8 @@
             try {
                 while (rangeCursor.hasNext()) {
                     rangeCursor.next();
+                    ITupleReference ref = rangeCursor.getTuple();
+                    Dataset ds = valueExtractor.getValue(jobId, rangeCursor.getTuple());
                     datasetId = ((Dataset) valueExtractor.getValue(jobId, rangeCursor.getTuple())).getDatasetId();
                     if (mostRecentDatasetId < datasetId) {
                         mostRecentDatasetId = datasetId;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
index 422fdfd..4dc0e59 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -27,8 +27,8 @@
 import java.util.logging.Logger;
 
 import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
+import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
 import edu.uci.ics.asterix.common.config.AsterixMetadataProperties;
-import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
 import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
@@ -57,7 +57,6 @@
 import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat;
-import edu.uci.ics.asterix.transaction.management.resource.ILocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.resource.LSMBTreeLocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
 import edu.uci.ics.asterix.transaction.management.service.logging.IndexResourceManager;
@@ -69,24 +68,17 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.io.IIOManager;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.util.IndexFileNameUtil;
 import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTree;
 import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryBufferCache;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryFreePageManager;
-import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactory;
 import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactoryProvider;
 import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
 import edu.uci.ics.hyracks.storage.common.file.LocalResource;
-import edu.uci.ics.hyracks.storage.common.file.TransientFileMapManager;
 
 /**
  * Initializes the remote metadata storage facilities ("universe") using a
@@ -99,8 +91,6 @@
  */
 public class MetadataBootstrap {
     private static final Logger LOGGER = Logger.getLogger(MetadataBootstrap.class.getName());
-    private static final int DEFAULT_MEM_PAGE_SIZE = 32768;
-    private static final int DEFAULT_MEM_NUM_PAGES = 100;
 
     private static IAsterixAppRuntimeContext runtimeContext;
 
@@ -215,6 +205,7 @@
                 //change the exception type to AbortFailureException
                 throw new MetadataException(e);
             }
+            throw e;
         }
     }
 
@@ -338,43 +329,34 @@
                 metadataStore + File.separator + index.getFileNameRelativePath(),
                 runtimeContext.getMetaDataIODeviceId());
         FileReference file = new FileReference(new File(filePath));
-        IInMemoryBufferCache memBufferCache = new InMemoryBufferCache(new HeapBufferAllocator(), DEFAULT_MEM_PAGE_SIZE,
-                DEFAULT_MEM_NUM_PAGES, new TransientFileMapManager());
+        IVirtualBufferCache virtualBufferCache = runtimeContext.getVirtualBufferCache(index.getDatasetId().getId());
         ITypeTraits[] typeTraits = index.getTypeTraits();
         IBinaryComparatorFactory[] comparatorFactories = index.getKeyBinaryComparatorFactory();
         int[] bloomFilterKeyFields = index.getBloomFilterKeyFields();
-        ITreeIndexMetaDataFrameFactory metaDataFrameFactory = new LIFOMetaDataFrameFactory();
-        IInMemoryFreePageManager memFreePageManager = new InMemoryFreePageManager(DEFAULT_MEM_NUM_PAGES,
-                metaDataFrameFactory);
         LSMBTree lsmBtree = null;
         long resourceID = -1;
         if (create) {
-            lsmBtree = LSMBTreeUtils.createLSMTree(memBufferCache, memFreePageManager, ioManager, file, bufferCache,
-                    fileMapProvider, typeTraits, comparatorFactories, bloomFilterKeyFields,
+            lsmBtree = LSMBTreeUtils.createLSMTree(virtualBufferCache, ioManager, file, bufferCache, fileMapProvider,
+                    typeTraits, comparatorFactories, bloomFilterKeyFields,
                     runtimeContext.getBloomFilterFalsePositiveRate(), runtimeContext.getLSMMergePolicy(),
                     runtimeContext.getLSMBTreeOperationTrackerFactory(), runtimeContext.getLSMIOScheduler(),
                     AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, runtimeContext.getMetaDataIODeviceId());
             lsmBtree.create();
             resourceID = runtimeContext.getResourceIdFactory().createId();
-            indexLifecycleManager.register(resourceID, lsmBtree);
-
-            AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
             ILocalResourceMetadata localResourceMetadata = new LSMBTreeLocalResourceMetadata(typeTraits,
-                    comparatorFactories, bloomFilterKeyFields, index.isPrimaryIndex(),
-                    storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
-                    runtimeContext.getMetaDataIODeviceId());
+                    comparatorFactories, bloomFilterKeyFields, runtimeContext.getMetaDataIODeviceId(), index.getDatasetId().getId());
             ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
                     localResourceMetadata, LocalResource.LSMBTreeResource);
             ILocalResourceFactory localResourceFactory = localResourceFactoryProvider.getLocalResourceFactory();
-            localResourceRepository.insert(
-                    localResourceFactory.createLocalResource(resourceID, file.getFile().getPath(), 0),
-                    runtimeContext.getMetaDataIODeviceId());
+            localResourceRepository.insert(localResourceFactory.createLocalResource(resourceID, file.getFile()
+                    .getPath(), 0), runtimeContext.getMetaDataIODeviceId());
+            indexLifecycleManager.register(resourceID, lsmBtree);
         } else {
             resourceID = localResourceRepository.getResourceByName(file.getFile().getPath()).getResourceId();
             lsmBtree = (LSMBTree) indexLifecycleManager.getIndex(resourceID);
             if (lsmBtree == null) {
-                lsmBtree = LSMBTreeUtils.createLSMTree(memBufferCache, memFreePageManager, ioManager, file,
-                        bufferCache, fileMapProvider, typeTraits, comparatorFactories, bloomFilterKeyFields,
+                lsmBtree = LSMBTreeUtils.createLSMTree(virtualBufferCache, ioManager, file, bufferCache,
+                        fileMapProvider, typeTraits, comparatorFactories, bloomFilterKeyFields,
                         runtimeContext.getBloomFilterFalsePositiveRate(), runtimeContext.getLSMMergePolicy(),
                         runtimeContext.getLSMBTreeOperationTrackerFactory(), runtimeContext.getLSMIOScheduler(),
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, runtimeContext.getMetaDataIODeviceId());
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
index 117f492..b0e2d7c 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
@@ -27,6 +27,7 @@
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
 import edu.uci.ics.asterix.common.config.GlobalConfig;
+import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import edu.uci.ics.asterix.common.context.ITransactionSubsystemProvider;
 import edu.uci.ics.asterix.common.context.TransactionSubsystemProvider;
 import edu.uci.ics.asterix.common.dataflow.IAsterixApplicationContextInfo;
@@ -546,12 +547,11 @@
                     appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(), spPc.first,
                     typeTraits, comparatorFactories, bloomFilterKeyFields, lowKeyFields, highKeyFields,
                     lowKeyInclusive, highKeyInclusive, new LSMBTreeDataflowHelperFactory(
+                            new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
                             AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                            storageProperties.getMemoryComponentPageSize(),
-                            storageProperties.getMemoryComponentNumPages(),
                             storageProperties.getBloomFilterFalsePositiveRate()), retainInput, searchCallbackFactory);
 
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeSearchOp, spPc.second);
@@ -616,13 +616,12 @@
                     appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(), spPc.first,
                     typeTraits, comparatorFactories, keyFields, new LSMRTreeDataflowHelperFactory(
                             valueProviderFactories, RTreePolicyType.RTREE, primaryComparatorFactories,
+                            new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
                             AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, proposeLinearizer(
                                     nestedKeyType.getTypeTag(), comparatorFactories.length),
-                            storageProperties.getMemoryComponentPageSize(),
-                            storageProperties.getMemoryComponentNumPages(),
                             storageProperties.getBloomFilterFalsePositiveRate()), retainInput, searchCallbackFactory);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(rtreeSearchOp, spPc.second);
 
@@ -780,12 +779,11 @@
                     appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(),
                     splitsAndConstraint.first, typeTraits, comparatorFactories, bloomFilterKeyFields, fieldPermutation,
                     GlobalConfig.DEFAULT_BTREE_FILL_FACTOR, false, numElementsHint, new LSMBTreeDataflowHelperFactory(
+                            new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
                             AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                            storageProperties.getMemoryComponentPageSize(),
-                            storageProperties.getMemoryComponentNumPages(),
                             storageProperties.getBloomFilterFalsePositiveRate()), NoOpOperationCallbackFactory.INSTANCE);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad,
                     splitsAndConstraint.second);
@@ -849,12 +847,12 @@
                     spec, recordDesc, appContext.getStorageManagerInterface(),
                     appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
                     comparatorFactories, bloomFilterKeyFields, fieldPermutation, indexOp,
-                    new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                    new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(datasetId),
+                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, storageProperties
-                                    .getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
-                            storageProperties.getBloomFilterFalsePositiveRate()), null, modificationCallbackFactory);
+                                    .getBloomFilterFalsePositiveRate()), null, modificationCallbackFactory);
 
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad,
                     splitsAndConstraint.second);
@@ -1045,13 +1043,12 @@
                     spec, recordDesc, appContext.getStorageManagerInterface(),
                     appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
                     comparatorFactories, bloomFilterKeyFields, fieldPermutation, indexOp,
-                    new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                    new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(datasetId),
+                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, storageProperties
-                                    .getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
-                            storageProperties.getBloomFilterFalsePositiveRate()), filterFactory,
-                    modificationCallbackFactory);
+                                    .getBloomFilterFalsePositiveRate()), filterFactory, modificationCallbackFactory);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad,
                     splitsAndConstraint.second);
         } catch (MetadataException e) {
@@ -1172,14 +1169,12 @@
                     spec, recordDesc, appContext.getStorageManagerInterface(), splitsAndConstraint.first,
                     appContext.getIndexLifecycleManagerProvider(), tokenTypeTraits, tokenComparatorFactories,
                     invListsTypeTraits, invListComparatorFactories, tokenizerFactory, fieldPermutation, indexOp,
-                    new LSMInvertedIndexDataflowHelperFactory(
+                    new LSMInvertedIndexDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(datasetId),
                             AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER, storageProperties
-                                    .getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
-                            storageProperties.getBloomFilterFalsePositiveRate()), filterFactory,
-                    modificationCallbackFactory);
+                                    .getBloomFilterFalsePositiveRate()), filterFactory, modificationCallbackFactory);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(insertDeleteOp,
                     splitsAndConstraint.second);
         } catch (MetadataException e) {
@@ -1268,13 +1263,12 @@
                     appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
                     comparatorFactories, null, fieldPermutation, indexOp, new LSMRTreeDataflowHelperFactory(
                             valueProviderFactories, RTreePolicyType.RTREE, primaryComparatorFactories,
+                            new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
                             AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, proposeLinearizer(
                                     nestedKeyType.getTypeTag(), comparatorFactories.length),
-                            storageProperties.getMemoryComponentPageSize(),
-                            storageProperties.getMemoryComponentNumPages(),
                             storageProperties.getBloomFilterFalsePositiveRate()), filterFactory,
                     modificationCallbackFactory);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(rtreeUpdate, splitsAndConstraint.second);
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java
index 15ccde4..9097570 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java
@@ -41,9 +41,9 @@
     protected final IndexOperation indexOp;
     protected final ITransactionSubsystem txnSubsystem;
 
-    public PrimaryIndexModificationOperationCallback(int datasetId, int[] primaryKeyFields,
-            ITransactionContext txnCtx, ILockManager lockManager,
-            ITransactionSubsystem txnSubsystem, long resourceId, byte resourceType, IndexOperation indexOp) {
+    public PrimaryIndexModificationOperationCallback(int datasetId, int[] primaryKeyFields, ITransactionContext txnCtx,
+            ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId, byte resourceType,
+            IndexOperation indexOp) {
         super(datasetId, primaryKeyFields, txnCtx, lockManager);
         this.resourceId = resourceId;
         this.resourceType = resourceType;
@@ -63,19 +63,19 @@
 
     @Override
     public void found(ITupleReference before, ITupleReference after) throws HyracksDataException {
-        ILogger logger = txnSubsystem.getTreeLoggerRepository().getIndexLogger(resourceId, resourceType);
-        int pkHash = computePrimaryKeyHashValue(after, primaryKeyFields);
-        LSMBTreeTupleReference lsmBTreeTuple = (LSMBTreeTupleReference) before;
-        IndexOperation oldOp = IndexOperation.INSERT;
-        if (before == null) {
-            oldOp = IndexOperation.NOOP;
-        }
-        if (lsmBTreeTuple != null && lsmBTreeTuple.isAntimatter()) {
-            oldOp = IndexOperation.DELETE;
-        }
         try {
-            ((IndexLogger)logger).generateLogRecord(txnSubsystem, txnCtx, datasetId.getId(), pkHash, resourceId, indexOp, after,
-                    oldOp, before);
+            ILogger logger = txnSubsystem.getTreeLoggerRepository().getIndexLogger(resourceId, resourceType);
+            int pkHash = computePrimaryKeyHashValue(after, primaryKeyFields);
+            LSMBTreeTupleReference lsmBTreeTuple = (LSMBTreeTupleReference) before;
+            IndexOperation oldOp = IndexOperation.INSERT;
+            if (before == null) {
+                oldOp = IndexOperation.NOOP;
+            }
+            if (lsmBTreeTuple != null && lsmBTreeTuple.isAntimatter()) {
+                oldOp = IndexOperation.DELETE;
+            }
+            ((IndexLogger) logger).generateLogRecord(txnSubsystem, txnCtx, datasetId.getId(), pkHash, resourceId,
+                    indexOp, after, oldOp, before);
         } catch (ACIDException e) {
             throw new HyracksDataException(e);
         }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallback.java
index 0e66b32..2287ed7 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallback.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallback.java
@@ -61,9 +61,10 @@
 
     @Override
     public void found(ITupleReference before, ITupleReference after) throws HyracksDataException {
-        ILogger logger = txnSubsystem.getTreeLoggerRepository().getIndexLogger(resourceId, resourceType);
-        int pkHash = computePrimaryKeyHashValue(after, primaryKeyFields);
+
         try {
+            ILogger logger = txnSubsystem.getTreeLoggerRepository().getIndexLogger(resourceId, resourceType);
+            int pkHash = computePrimaryKeyHashValue(after, primaryKeyFields);
             IndexOperation effectiveOldOp;
             if (resourceType == ResourceType.LSM_BTREE) {
                 LSMBTreeTupleReference lsmBTreeTuple = (LSMBTreeTupleReference) before;
@@ -77,8 +78,8 @@
             } else {
                 effectiveOldOp = oldOp;
             }
-            ((IndexLogger)logger).generateLogRecord(txnSubsystem, txnCtx, datasetId.getId(), pkHash, resourceId, indexOp, after,
-                    effectiveOldOp, before);
+            ((IndexLogger) logger).generateLogRecord(txnSubsystem, txnCtx, datasetId.getId(), pkHash, resourceId,
+                    indexOp, after, effectiveOldOp, before);
         } catch (ACIDException e) {
             throw new HyracksDataException(e);
         }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/AbstractLSMLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/AbstractLSMLocalResourceMetadata.java
new file mode 100644
index 0000000..a95b677
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/AbstractLSMLocalResourceMetadata.java
@@ -0,0 +1,18 @@
+package edu.uci.ics.asterix.transaction.management.resource;
+
+import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
+
+public abstract class AbstractLSMLocalResourceMetadata implements ILocalResourceMetadata {
+
+    private static final long serialVersionUID = 1L;
+
+    protected final int datasetID;
+
+    public AbstractLSMLocalResourceMetadata(int datasetID) {
+        this.datasetID = datasetID;
+    }
+
+    public int getDatasetID() {
+        return datasetID;
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java
index c3f9747..a00fbb6 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java
@@ -7,47 +7,36 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTree;
 import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryBufferCache;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryFreePageManager;
-import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
-import edu.uci.ics.hyracks.storage.common.file.TransientFileMapManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 
-public class LSMBTreeLocalResourceMetadata implements ILocalResourceMetadata {
+public class LSMBTreeLocalResourceMetadata extends AbstractLSMLocalResourceMetadata {
 
     private static final long serialVersionUID = 1L;
 
     private final ITypeTraits[] typeTraits;
     private final IBinaryComparatorFactory[] cmpFactories;
     private final int[] bloomFilterKeyFields;
-    private final int memPageSize;
-    private final int memNumPages;
     private FileSplit[] fileSplits;
     private int ioDeviceID;
 
     public LSMBTreeLocalResourceMetadata(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
-            int[] bloomFilterKeyFields, boolean isPrimary, int memPageSize, int memNumPages, FileSplit[] fileSplits) {
+            int[] bloomFilterKeyFields, FileSplit[] fileSplits, int datasetID) {
+        super(datasetID);
         this.typeTraits = typeTraits;
         this.cmpFactories = cmpFactories;
         this.bloomFilterKeyFields = bloomFilterKeyFields;
-        this.memPageSize = memPageSize;
-        this.memNumPages = memNumPages;
         this.fileSplits = fileSplits;
     }
 
     public LSMBTreeLocalResourceMetadata(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
-            int[] bloomFilterKeyFields, boolean isPrimary, int memPageSize, int memNumPages, int ioDeviceID) {
+            int[] bloomFilterKeyFields, int ioDeviceID, int datasetID) {
+        super(datasetID);
         this.typeTraits = typeTraits;
         this.cmpFactories = cmpFactories;
         this.bloomFilterKeyFields = bloomFilterKeyFields;
-        this.memPageSize = memPageSize;
-        this.memNumPages = memNumPages;
         this.ioDeviceID = ioDeviceID;
     }
 
@@ -55,16 +44,14 @@
     public ILSMIndex createIndexInstance(IAsterixAppRuntimeContextProvider runtimeContextProvider, String filePath,
             int partition) {
         FileReference file = new FileReference(new File(filePath));
-        IInMemoryBufferCache memBufferCache = new InMemoryBufferCache(new HeapBufferAllocator(), memPageSize,
-                memNumPages, new TransientFileMapManager());
-        ITreeIndexMetaDataFrameFactory metaDataFrameFactory = new LIFOMetaDataFrameFactory();
-        IInMemoryFreePageManager memFreePageManager = new InMemoryFreePageManager(memNumPages, metaDataFrameFactory);
-        LSMBTree lsmBTree = LSMBTreeUtils.createLSMTree(memBufferCache, memFreePageManager, runtimeContextProvider
-                .getIOManager(), file, runtimeContextProvider.getBufferCache(), runtimeContextProvider
-                .getFileMapManager(), typeTraits, cmpFactories, bloomFilterKeyFields, runtimeContextProvider
-                .getBloomFilterFalsePositiveRate(), runtimeContextProvider.getLSMMergePolicy(), runtimeContextProvider
-                .getLSMBTreeOperationTrackerFactory(), runtimeContextProvider.getLSMIOScheduler(),
-                runtimeContextProvider.getLSMBTreeIOOperationCallbackProvider(), fileSplits == null ? ioDeviceID
+        IVirtualBufferCache virtualBufferCache = runtimeContextProvider.getVirtualBufferCache(datasetID);
+        LSMBTree lsmBTree = LSMBTreeUtils.createLSMTree(virtualBufferCache, runtimeContextProvider.getIOManager(),
+                file, runtimeContextProvider.getBufferCache(), runtimeContextProvider.getFileMapManager(), typeTraits,
+                cmpFactories, bloomFilterKeyFields, runtimeContextProvider.getBloomFilterFalsePositiveRate(),
+                runtimeContextProvider.getLSMMergePolicy(),
+                runtimeContextProvider.getLSMBTreeOperationTrackerFactory(),
+                runtimeContextProvider.getLSMIOScheduler(), runtimeContextProvider
+                        .getLSMBTreeIOOperationCallbackProvider(), fileSplits == null ? ioDeviceID
                         : fileSplits[partition].getIODeviceId());
         return lsmBTree;
     }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java
index 89a32c5..743e047 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java
@@ -5,19 +5,13 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
-import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryBufferCache;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.InvertedIndexUtils;
-import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
 
-public class LSMInvertedIndexLocalResourceMetadata implements ILocalResourceMetadata {
+public class LSMInvertedIndexLocalResourceMetadata extends AbstractLSMLocalResourceMetadata {
 
     private static final long serialVersionUID = 1L;
 
@@ -26,22 +20,19 @@
     private final ITypeTraits[] tokenTypeTraits;
     private final IBinaryComparatorFactory[] tokenCmpFactories;
     private final IBinaryTokenizerFactory tokenizerFactory;
-    private final int memPageSize;
-    private final int memNumPages;
     private final boolean isPartitioned;
     private final FileSplit[] fileSplits;
 
     public LSMInvertedIndexLocalResourceMetadata(ITypeTraits[] invListTypeTraits,
             IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
-            IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory, int memPageSize,
-            int memNumPages, boolean isPartitioned, FileSplit[] fileSplits) {
+            IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
+            boolean isPartitioned, FileSplit[] fileSplits, int datasetID) {
+        super(datasetID);
         this.invListTypeTraits = invListTypeTraits;
         this.invListCmpFactories = invListCmpFactories;
         this.tokenTypeTraits = tokenTypeTraits;
         this.tokenCmpFactories = tokenCmpFactories;
         this.tokenizerFactory = tokenizerFactory;
-        this.memPageSize = memPageSize;
-        this.memNumPages = memNumPages;
         this.isPartitioned = isPartitioned;
         this.fileSplits = fileSplits;
     }
@@ -49,15 +40,10 @@
     @Override
     public ILSMIndex createIndexInstance(IAsterixAppRuntimeContextProvider runtimeContextProvider, String filePath,
             int partition) throws HyracksDataException {
-
-        ITreeIndexMetaDataFrameFactory metaDataFrameFactory = new LIFOMetaDataFrameFactory();
-        IInMemoryBufferCache memBufferCache = new DualIndexInMemoryBufferCache(new HeapBufferAllocator(), memPageSize,
-                memNumPages);
-        IInMemoryFreePageManager memFreePageManager = new DualIndexInMemoryFreePageManager(memNumPages,
-                metaDataFrameFactory);
+        IVirtualBufferCache virtualBufferCache = runtimeContextProvider.getVirtualBufferCache(datasetID);
         try {
             if (isPartitioned) {
-                return InvertedIndexUtils.createPartitionedLSMInvertedIndex(memBufferCache, memFreePageManager,
+                return InvertedIndexUtils.createPartitionedLSMInvertedIndex(virtualBufferCache,
                         runtimeContextProvider.getFileMapManager(), invListTypeTraits, invListCmpFactories,
                         tokenTypeTraits, tokenCmpFactories, tokenizerFactory, runtimeContextProvider.getBufferCache(),
                         runtimeContextProvider.getIOManager(), filePath,
@@ -68,7 +54,7 @@
                         runtimeContextProvider.getLSMInvertedIndexIOOperationCallbackProvider(),
                         fileSplits[partition].getIODeviceId());
             } else {
-                return InvertedIndexUtils.createLSMInvertedIndex(memBufferCache, memFreePageManager,
+                return InvertedIndexUtils.createLSMInvertedIndex(virtualBufferCache,
                         runtimeContextProvider.getFileMapManager(), invListTypeTraits, invListCmpFactories,
                         tokenTypeTraits, tokenCmpFactories, tokenizerFactory, runtimeContextProvider.getBufferCache(),
                         runtimeContextProvider.getIOManager(), filePath,
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java
index 2506697..3e45b77 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java
@@ -9,20 +9,14 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
-import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryBufferCache;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import edu.uci.ics.hyracks.storage.am.lsm.rtree.utils.LSMRTreeUtils;
 import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
-import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
 
-public class LSMRTreeLocalResourceMetadata implements ILocalResourceMetadata {
+public class LSMRTreeLocalResourceMetadata extends AbstractLSMLocalResourceMetadata {
 
     private static final long serialVersionUID = 1L;
 
@@ -32,22 +26,18 @@
     private final IPrimitiveValueProviderFactory[] valueProviderFactories;
     private final RTreePolicyType rtreePolicyType;
     private final ILinearizeComparatorFactory linearizeCmpFactory;
-    private final int memPageSize;
-    private final int memNumPages;
     private final FileSplit[] fileSplits;
 
     public LSMRTreeLocalResourceMetadata(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
             IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
-            RTreePolicyType rtreePolicyType, ILinearizeComparatorFactory linearizeCmpFactory, int memPageSize,
-            int memNumPages, FileSplit[] fileSplits) {
+            RTreePolicyType rtreePolicyType, ILinearizeComparatorFactory linearizeCmpFactory, FileSplit[] fileSplits, int datasetID) {
+        super(datasetID);
         this.typeTraits = typeTraits;
         this.rtreeCmpFactories = rtreeCmpFactories;
         this.btreeCmpFactories = btreeCmpFactories;
         this.valueProviderFactories = valueProviderFactories;
         this.rtreePolicyType = rtreePolicyType;
         this.linearizeCmpFactory = linearizeCmpFactory;
-        this.memPageSize = memPageSize;
-        this.memNumPages = memNumPages;
         this.fileSplits = fileSplits;
     }
 
@@ -55,17 +45,12 @@
     public ILSMIndex createIndexInstance(IAsterixAppRuntimeContextProvider runtimeContextProvider, String filePath,
             int partition) throws HyracksDataException {
         FileReference file = new FileReference(new File(filePath));
-        ITreeIndexMetaDataFrameFactory metaDataFrameFactory = new LIFOMetaDataFrameFactory();
-        IInMemoryBufferCache memBufferCache = new DualIndexInMemoryBufferCache(new HeapBufferAllocator(), memPageSize,
-                memNumPages);
-        IInMemoryFreePageManager memFreePageManager = new DualIndexInMemoryFreePageManager(memNumPages,
-                metaDataFrameFactory);
-
+        IVirtualBufferCache virtualBufferCache = runtimeContextProvider.getVirtualBufferCache(datasetID);
         try {
-            return LSMRTreeUtils.createLSMTree(memBufferCache, memFreePageManager,
-                    runtimeContextProvider.getIOManager(), file, runtimeContextProvider.getBufferCache(),
-                    runtimeContextProvider.getFileMapManager(), typeTraits, rtreeCmpFactories, btreeCmpFactories,
-                    valueProviderFactories, rtreePolicyType, runtimeContextProvider.getBloomFilterFalsePositiveRate(),
+            return LSMRTreeUtils.createLSMTree(virtualBufferCache, runtimeContextProvider.getIOManager(), file,
+                    runtimeContextProvider.getBufferCache(), runtimeContextProvider.getFileMapManager(), typeTraits,
+                    rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
+                    runtimeContextProvider.getBloomFilterFalsePositiveRate(),
                     runtimeContextProvider.getLSMMergePolicy(),
                     runtimeContextProvider.getLSMRTreeOperationTrackerFactory(),
                     runtimeContextProvider.getLSMIOScheduler(),
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceFactory.java
index ed0f79f..707e02e 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceFactory.java
@@ -1,5 +1,6 @@
 package edu.uci.ics.asterix.transaction.management.resource;
 
+import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
 import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactory;
 import edu.uci.ics.hyracks.storage.common.file.LocalResource;
 
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceFactoryProvider.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceFactoryProvider.java
index 4157c32..6afb9a9 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceFactoryProvider.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceFactoryProvider.java
@@ -1,5 +1,6 @@
 package edu.uci.ics.asterix.transaction.management.resource;
 
+import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
 import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactory;
 import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactoryProvider;
 
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLoggerRepository.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLoggerRepository.java
index c51bb11..bd7d159 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLoggerRepository.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLoggerRepository.java
@@ -17,10 +17,12 @@
 import java.util.HashMap;
 import java.util.Map;
 
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.transactions.ILogger;
 import edu.uci.ics.asterix.common.transactions.ILoggerRepository;
 import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
 import edu.uci.ics.asterix.common.transactions.MutableResourceId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
 
 public class IndexLoggerRepository implements ILoggerRepository {
@@ -35,13 +37,18 @@
     }
 
     @Override
-    public synchronized ILogger getIndexLogger(long resourceId, byte resourceType) {
+    public synchronized ILogger getIndexLogger(long resourceId, byte resourceType) throws ACIDException {
         mutableResourceId.setId(resourceId);
         ILogger logger = loggers.get(mutableResourceId);
         if (logger == null) {
             MutableResourceId newMutableResourceId = new MutableResourceId(resourceId);
-            IIndex index = (IIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
-                    .getIndex(resourceId);
+            IIndex index;
+            try {
+                index = (IIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
+                        .getIndex(resourceId);
+            } catch (HyracksDataException e) {
+                throw new ACIDException(e);
+            }
             logger = new IndexLogger(resourceId, resourceType, index);
             loggers.put(newMutableResourceId, logger);
         }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexResourceManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexResourceManager.java
index ce3401c..8506df7 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexResourceManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexResourceManager.java
@@ -19,6 +19,7 @@
 import edu.uci.ics.asterix.common.transactions.IResourceManager;
 import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
 import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
@@ -45,11 +46,13 @@
         long resourceId = logRecordHelper.getResourceId(logLocator);
         int offset = logRecordHelper.getLogContentBeginPos(logLocator);
 
-        //TODO
-        //replace TransactionResourceRepository with IndexLifeCycleManager
-        // look up the repository to obtain the resource object
-        IIndex index = (IIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
-                .getIndex(resourceId);
+        IIndex index;
+        try {
+            index = (IIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
+                    .getIndex(resourceId);
+        } catch (HyracksDataException e1) {
+            throw new ACIDException("Cannot undo: unable to find index");
+        }
 
         /* field count */
         int fieldCount = logLocator.getBuffer().readInt(offset);
@@ -115,8 +118,13 @@
         long resourceId = logRecordHelper.getResourceId(logLocator);
         int offset = logRecordHelper.getLogContentBeginPos(logLocator);
 
-        IIndex index = (IIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
-                .getIndex(resourceId);
+        IIndex index;
+        try {
+            index = (IIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
+                    .getIndex(resourceId);
+        } catch (HyracksDataException e1) {
+            throw new ACIDException("Cannot redo: unable to find index");
+        }
 
         /* field count */
         int fieldCount = logLocator.getBuffer().readInt(offset);
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
index 50d4625..91356a3 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
@@ -34,6 +34,7 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
 import edu.uci.ics.asterix.common.transactions.IBuffer;
@@ -48,7 +49,6 @@
 import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
 import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
 import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
-import edu.uci.ics.asterix.transaction.management.resource.ILocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.service.logging.IndexResourceManager;
 import edu.uci.ics.asterix.transaction.management.service.logging.LogManager;
 import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
diff --git a/pom.xml b/pom.xml
index aa0bc82..c4e50c1 100644
--- a/pom.xml
+++ b/pom.xml
@@ -50,7 +50,7 @@
             </property>
           </activation>
           <properties>
-            <test.heap.size>2047</test.heap.size>
+            <test.heap.size>2048</test.heap.size>
           </properties>
         </profile>