You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@asterixdb.apache.org by "Murtadha Hubail (Code Review)" <do...@asterix-gerrit.ics.uci.edu> on 2015/08/07 18:59:02 UTC

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Murtadha Hubail has uploaded a new change for review.

  https://asterix-gerrit.ics.uci.edu/344

Change subject: Allow lazy loading for persistent local resources
......................................................................

Allow lazy loading for persistent local resources

- Allow indexes to be accessed by name.
- Allow lazy loading for persistent local resources.
- Caching for local resources.
- Fix server start timing issue.

Change-Id: I48b9260a3280750145f6ddb3783673a299055910
---
M asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
M asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
M asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java
M asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
M asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
M asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
M asterix-transactions/pom.xml
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/TestRuntimeContextProvider.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
15 files changed, 435 insertions(+), 301 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/44/344/1

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 6d7f2a4..6bd9fff 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
@@ -92,7 +92,6 @@
     private AsterixTransactionProperties txnProperties;
     private AsterixFeedProperties feedProperties;
 
-
     private AsterixThreadExecutor threadExecutor;
     private DatasetLifecycleManager indexLifecycleManager;
     private IFileMapManager fileMapManager;
@@ -136,7 +135,7 @@
         metadataMergePolicyFactory = new PrefixMergePolicyFactory();
 
         ILocalResourceRepositoryFactory persistentLocalResourceRepositoryFactory = new PersistentLocalResourceRepositoryFactory(
-                ioManager);
+                ioManager, ncApplicationContext.getNodeId());
         localResourceRepository = (PersistentLocalResourceRepository) persistentLocalResourceRepositoryFactory
                 .createRepository();
         resourceIdFactory = (new ResourceIdFactoryProvider(localResourceRepository)).createResourceIdFactory();
@@ -145,10 +144,10 @@
                 this);
         txnSubsystem = new TransactionSubsystem(ncApplicationContext.getNodeId(), asterixAppRuntimeContextProvider,
                 txnProperties);
-        
+
         indexLifecycleManager = new DatasetLifecycleManager(storageProperties, localResourceRepository,
-                MetadataPrimaryIndexes.FIRST_AVAILABLE_USER_DATASET_ID,(LogManager)txnSubsystem.getLogManager());
-        
+                MetadataPrimaryIndexes.FIRST_AVAILABLE_USER_DATASET_ID, (LogManager) txnSubsystem.getLogManager());
+
         isShuttingdown = false;
 
         feedManager = new FeedManager(ncApplicationContext.getNodeId(), feedProperties,
@@ -239,7 +238,7 @@
     public AsterixExternalProperties getExternalProperties() {
         return externalProperties;
     }
-    
+
     @Override
     public AsterixFeedProperties getFeedProperties() {
         return feedProperties;
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
index 99d883c..9072983 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
@@ -80,7 +80,7 @@
         MetadataManager.INSTANCE = new MetadataManager(proxy, metadataProperties);
 
         AsterixAppContextInfo.getInstance().getCCApplicationContext()
-        .addJobLifecycleListener(FeedLifecycleListener.INSTANCE);
+                .addJobLifecycleListener(FeedLifecycleListener.INSTANCE);
 
         AsterixExternalProperties externalProperties = AsterixAppContextInfo.getInstance().getExternalProperties();
         setupWebServer(externalProperties);
@@ -88,16 +88,17 @@
 
         setupJSONAPIServer(externalProperties);
         jsonAPIServer.start();
-        ExternalLibraryBootstrap.setUpExternaLibraries(false);
 
         setupFeedServer(externalProperties);
         feedServer.start();
-        centralFeedManager = CentralFeedManager.getInstance(); 
-        centralFeedManager.start();
 
         waitUntilServerStart(webServer);
         waitUntilServerStart(jsonAPIServer);
         waitUntilServerStart(feedServer);
+
+        ExternalLibraryBootstrap.setUpExternaLibraries(false);
+        centralFeedManager = CentralFeedManager.getInstance();
+        centralFeedManager.start();
 
         AsterixGlobalRecoveryManager.INSTANCE = new AsterixGlobalRecoveryManager(
                 (HyracksConnection) getNewHyracksClientConnection());
@@ -178,7 +179,7 @@
 
         feedServer.setHandler(context);
         context.addServlet(new ServletHolder(new FeedServlet()), "/");
-   
+
         // add paths here
     }
 }
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java
index 3f02a1a..b6da731 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java
@@ -73,8 +73,22 @@
     }
 
     @Override
-    public synchronized IIndex getIndex(long resourceID) throws HyracksDataException {
-        DatasetInfo dsInfo = datasetInfos.get(getDIDfromRID(resourceID));
+    public synchronized IIndex getIndex(String resourceName) throws HyracksDataException {
+        DatasetInfo dsInfo = datasetInfos.get(getDIDfromResourceName(resourceName));
+        if (dsInfo == null) {
+            return null;
+        }
+        long resourceID = getResourceIDfromResourceName(resourceName);
+        IndexInfo iInfo = dsInfo.indexes.get(resourceID);
+        if (iInfo == null) {
+            return null;
+        }
+        return iInfo.index;
+    }
+
+    @Override
+    public synchronized IIndex getIndex(int datasetID, long resourceID) throws HyracksDataException {
+        DatasetInfo dsInfo = datasetInfos.get(datasetID);
         if (dsInfo == null) {
             return null;
         }
@@ -86,8 +100,9 @@
     }
 
     @Override
-    public synchronized void register(long resourceID, IIndex index) throws HyracksDataException {
-        int did = getDIDfromRID(resourceID);
+    public synchronized void register(String resourceName, IIndex index) throws HyracksDataException {
+        int did = getDIDfromResourceName(resourceName);
+        long resourceID = getResourceIDfromResourceName(resourceName);
         DatasetInfo dsInfo = datasetInfos.get(did);
         if (dsInfo == null) {
             dsInfo = new DatasetInfo(did, !index.hasMemoryComponents());
@@ -103,17 +118,27 @@
         dsInfo.indexes.put(resourceID, new IndexInfo((ILSMIndex) index));
     }
 
-    private int getDIDfromRID(long resourceID) throws HyracksDataException {
-        LocalResource lr = resourceRepository.getResourceById(resourceID);
+    private int getDIDfromResourceName(String resourceName) throws HyracksDataException {
+        LocalResource lr = resourceRepository.getResourceByName(resourceName);
         if (lr == null) {
             return -1;
         }
         return ((ILocalResourceMetadata) lr.getResourceObject()).getDatasetID();
     }
 
+    private long getResourceIDfromResourceName(String resourceName) throws HyracksDataException {
+        LocalResource lr = resourceRepository.getResourceByName(resourceName);
+        if (lr == null) {
+            return -1;
+        }
+        return lr.getResourceId();
+    }
+
     @Override
-    public synchronized void unregister(long resourceID) throws HyracksDataException {
-        int did = getDIDfromRID(resourceID);
+    public synchronized void unregister(String resourceName) throws HyracksDataException {
+        int did = getDIDfromResourceName(resourceName);
+        long resourceID = getResourceIDfromResourceName(resourceName);
+
         DatasetInfo dsInfo = datasetInfos.get(did);
         IndexInfo iInfo = dsInfo.indexes.get(resourceID);
 
@@ -153,20 +178,15 @@
 
         dsInfo.indexes.remove(resourceID);
         if (dsInfo.referenceCount == 0 && dsInfo.isOpen && dsInfo.indexes.isEmpty() && !dsInfo.isExternal) {
-            List<IVirtualBufferCache> vbcs = getVirtualBufferCaches(did);
-            assert vbcs != null;
-            for (IVirtualBufferCache vbc : vbcs) {
-                used -= (vbc.getNumPages() * vbc.getPageSize());
-            }
-            datasetInfos.remove(did);
-            datasetVirtualBufferCaches.remove(did);
-            datasetOpTrackers.remove(did);
+            removeDatasetFromCache(dsInfo.datasetID);
         }
     }
 
     @Override
-    public synchronized void open(long resourceID) throws HyracksDataException {
-        int did = getDIDfromRID(resourceID);
+    public synchronized void open(String resourceName) throws HyracksDataException {
+        int did = getDIDfromResourceName(resourceName);
+        long resourceID = getResourceIDfromResourceName(resourceName);
+
         DatasetInfo dsInfo = datasetInfos.get(did);
         if (dsInfo == null) {
             throw new HyracksDataException("Failed to open index with resource ID " + resourceID
@@ -247,15 +267,16 @@
     }
 
     public DatasetInfo getDatasetInfo(int datasetID) {
-
         synchronized (datasetInfos) {
             return datasetInfos.get(datasetID);
         }
     }
 
     @Override
-    public synchronized void close(long resourceID) throws HyracksDataException {
-        int did = getDIDfromRID(resourceID);
+    public synchronized void close(String resourceName) throws HyracksDataException {
+        int did = getDIDfromResourceName(resourceName);
+        long resourceID = getResourceIDfromResourceName(resourceName);
+
         DatasetInfo dsInfo = datasetInfos.get(did);
         if (dsInfo == null) {
             throw new HyracksDataException("No index found with resourceID " + resourceID);
@@ -298,6 +319,17 @@
             }
             return vbcs;
         }
+    }
+
+    private void removeDatasetFromCache(int datasetID) {
+        List<IVirtualBufferCache> vbcs = getVirtualBufferCaches(datasetID);
+        assert vbcs != null;
+        for (IVirtualBufferCache vbc : vbcs) {
+            used -= (vbc.getNumPages() * vbc.getPageSize());
+        }
+        datasetInfos.remove(datasetID);
+        datasetVirtualBufferCaches.remove(datasetID);
+        datasetOpTrackers.remove(datasetID);
     }
 
     public ILSMOperationTracker getOperationTracker(int datasetID) {
@@ -498,7 +530,6 @@
         }
 
         if (asyncFlush) {
-
             for (IndexInfo iInfo : dsInfo.indexes.values()) {
                 ILSMIndexAccessor accessor = iInfo.index.createAccessor(NoOpOperationCallback.INSTANCE,
                         NoOpOperationCallback.INSTANCE);
@@ -546,10 +577,7 @@
         }
         dsInfo.isOpen = false;
 
-        List<IVirtualBufferCache> vbcs = getVirtualBufferCaches(dsInfo.datasetID);
-        for (IVirtualBufferCache vbc : vbcs) {
-            used -= vbc.getNumPages() * vbc.getPageSize();
-        }
+        removeDatasetFromCache(dsInfo.datasetID);
     }
 
     @Override
@@ -600,4 +628,34 @@
 
         outputStream.write(sb.toString().getBytes());
     }
+
+    @Override
+    public boolean supportsPersistentLocalResources() {
+        return true;
+    }
+
+    @Override
+    public void register(long resourceID, IIndex index) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void open(long resourceID) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public synchronized void close(long resourceID) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public synchronized void unregister(long resourceID) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public synchronized IIndex getIndex(long resourceID) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
index 93d9fbc..9e9d2f9 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
@@ -53,7 +53,7 @@
         indexHelper.open();
         AbstractLSMIndex lsmIndex = (AbstractLSMIndex) indexHelper.getIndexInstance();
         try {
-            modCallback = opDesc.getModificationOpCallbackFactory().createModificationOperationCallback(
+            modCallback = opDesc.getModificationOpCallbackFactory().createModificationOperationCallback(indexHelper.getFileReference().getFile().getPath(),
                     indexHelper.getResourceID(), lsmIndex, ctx);
             indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
             ITupleFilterFactory tupleFilterFactory = opDesc.getTupleFilterFactory();
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 b61f410..b16220c 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
@@ -287,9 +287,9 @@
     private void insertTupleIntoIndex(JobId jobId, IMetadataIndex metadataIndex, ITupleReference tuple)
             throws Exception {
         long resourceID = metadataIndex.getResourceID();
-        ILSMIndex lsmIndex = (ILSMIndex) indexLifecycleManager.getIndex(resourceID);
+        ILSMIndex lsmIndex = (ILSMIndex) indexLifecycleManager.getIndex(metadataIndex.getFile().toString());
         try {
-            indexLifecycleManager.open(resourceID);
+            indexLifecycleManager.open(metadataIndex.getFile().toString());
 
             // prepare a Callback for logging
             IModificationOperationCallback modCallback = createIndexModificationCallback(jobId, resourceID,
@@ -310,7 +310,7 @@
         } catch (Exception e) {
             throw e;
         } finally {
-            indexLifecycleManager.close(resourceID);
+            indexLifecycleManager.close(metadataIndex.getFile().toString());
         }
     }
 
@@ -636,9 +636,9 @@
     private void deleteTupleFromIndex(JobId jobId, IMetadataIndex metadataIndex, ITupleReference tuple)
             throws Exception {
         long resourceID = metadataIndex.getResourceID();
-        ILSMIndex lsmIndex = (ILSMIndex) indexLifecycleManager.getIndex(resourceID);
+        ILSMIndex lsmIndex = (ILSMIndex) indexLifecycleManager.getIndex(metadataIndex.getFile().toString());
         try {
-            indexLifecycleManager.open(resourceID);
+            indexLifecycleManager.open(metadataIndex.getFile().toString());
             // prepare a Callback for logging
             IModificationOperationCallback modCallback = createIndexModificationCallback(jobId, resourceID,
                     metadataIndex, lsmIndex, IndexOperation.DELETE);
@@ -656,7 +656,7 @@
         } catch (Exception e) {
             throw e;
         } finally {
-            indexLifecycleManager.close(resourceID);
+            indexLifecycleManager.close(metadataIndex.getFile().toString());
         }
     }
 
@@ -970,8 +970,8 @@
         try {
             IMetadataIndex index = MetadataPrimaryIndexes.DATAVERSE_DATASET;
             long resourceID = index.getResourceID();
-            IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
-            indexLifecycleManager.open(resourceID);
+            IIndex indexInstance = indexLifecycleManager.getIndex(index.getFile().toString());
+            indexLifecycleManager.open(index.getFile().toString());
             IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
                     NoOpOperationCallback.INSTANCE);
             ITreeIndexCursor rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor(false);
@@ -989,12 +989,12 @@
             } finally {
                 rangeCursor.close();
             }
-            indexLifecycleManager.close(resourceID);
+            indexLifecycleManager.close(index.getFile().toString());
 
             index = MetadataPrimaryIndexes.DATASET_DATASET;
             resourceID = index.getResourceID();
-            indexInstance = indexLifecycleManager.getIndex(resourceID);
-            indexLifecycleManager.open(resourceID);
+            indexInstance = indexLifecycleManager.getIndex(index.getFile().toString());
+            indexLifecycleManager.open(index.getFile().toString());
             indexAccessor = indexInstance
                     .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
             rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor(false);
@@ -1012,12 +1012,12 @@
             } finally {
                 rangeCursor.close();
             }
-            indexLifecycleManager.close(resourceID);
+            indexLifecycleManager.close(index.getFile().toString());
 
             index = MetadataPrimaryIndexes.INDEX_DATASET;
             resourceID = index.getResourceID();
-            indexInstance = indexLifecycleManager.getIndex(resourceID);
-            indexLifecycleManager.open(resourceID);
+            indexInstance = indexLifecycleManager.getIndex(index.getFile().toString());
+            indexLifecycleManager.open(index.getFile().toString());
             indexAccessor = indexInstance
                     .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
             rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor(false);
@@ -1036,7 +1036,7 @@
             } finally {
                 rangeCursor.close();
             }
-            indexLifecycleManager.close(resourceID);
+            indexLifecycleManager.close(index.getFile().toString());
         } catch (Exception e) {
             e.printStackTrace();
         }
@@ -1046,9 +1046,8 @@
     private <ResultType> void searchIndex(JobId jobId, IMetadataIndex index, ITupleReference searchKey,
             IValueExtractor<ResultType> valueExtractor, List<ResultType> results) throws Exception {
         IBinaryComparatorFactory[] comparatorFactories = index.getKeyBinaryComparatorFactory();
-        long resourceID = index.getResourceID();
-        IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
-        indexLifecycleManager.open(resourceID);
+        IIndex indexInstance = indexLifecycleManager.getIndex(index.getFile().toString());
+        indexLifecycleManager.open(index.getFile().toString());
         IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
                 NoOpOperationCallback.INSTANCE);
         ITreeIndexCursor rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor(false);
@@ -1077,7 +1076,7 @@
         } finally {
             rangeCursor.close();
         }
-        indexLifecycleManager.close(resourceID);
+        indexLifecycleManager.close(index.getFile().toString());
     }
 
     @Override
@@ -1085,8 +1084,9 @@
         int mostRecentDatasetId = MetadataPrimaryIndexes.FIRST_AVAILABLE_USER_DATASET_ID;
         long resourceID = MetadataPrimaryIndexes.DATASET_DATASET.getResourceID();
         try {
-            IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
-            indexLifecycleManager.open(resourceID);
+            IIndex indexInstance = indexLifecycleManager.getIndex(MetadataPrimaryIndexes.DATASET_DATASET.getFile()
+                    .toString());
+            indexLifecycleManager.open(MetadataPrimaryIndexes.DATASET_DATASET.getFile().toString());
             try {
                 IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
                         NoOpOperationCallback.INSTANCE);
@@ -1113,7 +1113,7 @@
                     rangeCursor.close();
                 }
             } finally {
-                indexLifecycleManager.close(resourceID);
+                indexLifecycleManager.close(MetadataPrimaryIndexes.DATASET_DATASET.getFile().toString());
             }
 
         } catch (Exception e) {
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 97c90f8..ec1179d 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
@@ -123,9 +123,8 @@
                 MetadataPrimaryIndexes.INDEX_DATASET, MetadataPrimaryIndexes.NODE_DATASET,
                 MetadataPrimaryIndexes.NODEGROUP_DATASET, MetadataPrimaryIndexes.FUNCTION_DATASET,
                 MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, MetadataPrimaryIndexes.FEED_DATASET,
-                MetadataPrimaryIndexes.FEED_POLICY_DATASET,
-                MetadataPrimaryIndexes.LIBRARY_DATASET, MetadataPrimaryIndexes.COMPACTION_POLICY_DATASET,
-                MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET };
+                MetadataPrimaryIndexes.FEED_POLICY_DATASET, MetadataPrimaryIndexes.LIBRARY_DATASET,
+                MetadataPrimaryIndexes.COMPACTION_POLICY_DATASET, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET };
 
         secondaryIndexes = new IMetadataIndex[] { MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX,
                 MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX,
@@ -417,11 +416,11 @@
                     localResourceMetadata, LocalResource.LSMBTreeResource);
             ILocalResourceFactory localResourceFactory = localResourceFactoryProvider.getLocalResourceFactory();
             localResourceRepository.insert(localResourceFactory.createLocalResource(resourceID, path, 0));
-            indexLifecycleManager.register(resourceID, lsmBtree);
+            indexLifecycleManager.register(path, lsmBtree);
         } else {
             final LocalResource resource = localResourceRepository.getResourceByName(path);
             resourceID = resource.getResourceId();
-            lsmBtree = (LSMBTree) indexLifecycleManager.getIndex(resourceID);
+            lsmBtree = (LSMBTree) indexLifecycleManager.getIndex(resource.getResourceName());
             if (lsmBtree == null) {
                 lsmBtree = LSMBTreeUtils.createLSMTree(
                         virtualBufferCaches,
@@ -436,7 +435,7 @@
                                 GlobalConfig.DEFAULT_COMPACTION_POLICY_PROPERTIES, indexLifecycleManager), opTracker,
                         runtimeContext.getLSMIOScheduler(), LSMBTreeIOOperationCallbackFactory.INSTANCE
                                 .createIOOperationCallback(), index.isPrimaryIndex(), null, null, null, null, true);
-                indexLifecycleManager.register(resourceID, lsmBtree);
+                indexLifecycleManager.register(path, lsmBtree);
             }
         }
 
diff --git a/asterix-transactions/pom.xml b/asterix-transactions/pom.xml
index fbec3b8..c3a7555 100644
--- a/asterix-transactions/pom.xml
+++ b/asterix-transactions/pom.xml
@@ -1,18 +1,14 @@
-<!--
- ! Copyright 2009-2013 by The Regents of the University of California
- ! Licensed under the Apache License, Version 2.0 (the "License");
- ! you may not use this file except in compliance with the License.
- ! you may obtain a copy of the License from
- ! 
- !     http://www.apache.org/licenses/LICENSE-2.0
- ! 
- ! Unless required by applicable law or agreed to in writing, software
- ! distributed under the License is distributed on an "AS IS" BASIS,
- ! WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- ! See the License for the specific language governing permissions and
- ! limitations under the License.
- !-->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<!-- ! Copyright 2009-2013 by The Regents of the University of California 
+	! Licensed under the Apache License, Version 2.0 (the "License"); ! you may 
+	not use this file except in compliance with the License. ! you may obtain 
+	a copy of the License from ! ! http://www.apache.org/licenses/LICENSE-2.0 
+	! ! Unless required by applicable law or agreed to in writing, software ! 
+	distributed under the License is distributed on an "AS IS" BASIS, ! WITHOUT 
+	WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. ! See the 
+	License for the specific language governing permissions and ! limitations 
+	under the License. ! -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
 	<modelVersion>4.0.0</modelVersion>
 	<parent>
 		<artifactId>asterix</artifactId>
@@ -21,14 +17,14 @@
 	</parent>
 	<artifactId>asterix-transactions</artifactId>
 
-  <licenses>
-    <license>
-      <name>Apache License, Version 2.0</name>
-      <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
-      <distribution>repo</distribution>
-      <comments>A business-friendly OSS license</comments>
-    </license>
-  </licenses>
+	<licenses>
+		<license>
+			<name>Apache License, Version 2.0</name>
+			<url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+			<distribution>repo</distribution>
+			<comments>A business-friendly OSS license</comments>
+		</license>
+	</licenses>
 
 	<build>
 		<plugins>
@@ -42,52 +38,50 @@
 					<fork>true</fork>
 				</configuration>
 			</plugin>
-            <plugin>
-                <groupId>edu.uci.ics.asterix</groupId>
-                <artifactId>record-manager-generator-maven-plugin</artifactId>
-                <version>0.8.7-SNAPSHOT</version>
-                <configuration>
-                    <debug>false</debug>
-                    <inputFiles>
-                        <param>src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/Job.json</param>
-                        <param>src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/Resource.json</param>
-                        <param>src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/Request.json</param>
-                    </inputFiles>
-                    <packageName>edu.uci.ics.asterix.transaction.management.service.locking</packageName>
-                </configuration>
-                <executions>
-                    <execution>
-                        <id>generate-record-manager</id>
-                        <phase>generate-sources</phase>
-                        <goals>
-                            <goal>generate-record-manager</goal>
-                        </goals>
-                    </execution>
-                </executions>
-            </plugin>
-            <plugin>
-                <groupId>org.codehaus.mojo</groupId>
-                <artifactId>build-helper-maven-plugin</artifactId>
-                <version>1.9</version>
-                <executions>
-                    <execution>
-                        <id>add-source</id>
-                        <phase>generate-sources</phase>
-                        <goals>
-                            <goal>add-source</goal>
-                        </goals>
-                        <configuration>
-                            <sources>
-                                <source>${project.build.directory}/generated-sources/java/</source>
-                            </sources>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
+			<plugin>
+				<groupId>edu.uci.ics.asterix</groupId>
+				<artifactId>record-manager-generator-maven-plugin</artifactId>
+				<version>0.8.7-SNAPSHOT</version>
+				<configuration>
+					<debug>false</debug>
+					<inputFiles>
+						<param>src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/Job.json</param>
+						<param>src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/Resource.json</param>
+						<param>src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/Request.json</param>
+					</inputFiles>
+					<packageName>edu.uci.ics.asterix.transaction.management.service.locking</packageName>
+				</configuration>
+				<executions>
+					<execution>
+						<id>generate-record-manager</id>
+						<phase>generate-sources</phase>
+						<goals>
+							<goal>generate-record-manager</goal>
+						</goals>
+					</execution>
+				</executions>
+			</plugin>
+			<plugin>
+				<groupId>org.codehaus.mojo</groupId>
+				<artifactId>build-helper-maven-plugin</artifactId>
+				<version>1.9</version>
+				<executions>
+					<execution>
+						<id>add-source</id>
+						<phase>generate-sources</phase>
+						<goals>
+							<goal>add-source</goal>
+						</goals>
+						<configuration>
+							<sources>
+								<source>${project.build.directory}/generated-sources/java/</source>
+							</sources>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
 		</plugins>
-
 	</build>
-
 	<dependencies>
 		<dependency>
 			<groupId>edu.uci.ics.hyracks</groupId>
@@ -105,13 +99,17 @@
 			<groupId>edu.uci.ics.hyracks</groupId>
 			<artifactId>hyracks-storage-am-lsm-invertedindex</artifactId>
 		</dependency>
-                <dependency>
-                        <groupId>edu.uci.ics.asterix</groupId>
-                        <artifactId>asterix-common</artifactId>
-                        <version>0.8.7-SNAPSHOT</version>
-                        <type>jar</type>
-                        <scope>compile</scope>
-                </dependency>
+		<dependency>
+			<groupId>edu.uci.ics.asterix</groupId>
+			<artifactId>asterix-common</artifactId>
+			<version>0.8.7-SNAPSHOT</version>
+			<type>jar</type>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
+			<groupId>com.google.guava</groupId>
+			<artifactId>guava</artifactId>
+			<version>18.0</version>
+		</dependency>
 	</dependencies>
-
 </project>
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
index 652e55a..a94c068 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
@@ -46,13 +46,13 @@
     }
 
     @Override
-    public IModificationOperationCallback createModificationOperationCallback(long resourceId, Object resource,
-            IHyracksTaskContext ctx) throws HyracksDataException {
+    public IModificationOperationCallback createModificationOperationCallback(String resourceName, long resourceId,
+            Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
 
         ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
                 .getIndexLifecycleManager();
-        ILSMIndex index = (ILSMIndex) indexLifeCycleManager.getIndex(resourceId);
+        ILSMIndex index = (ILSMIndex) indexLifeCycleManager.getIndex(resourceName);
         if (index == null) {
             throw new HyracksDataException("Index(id:" + resourceId + ") is not registered.");
         }
@@ -68,4 +68,10 @@
             throw new HyracksDataException(e);
         }
     }
+
+    @Override
+    public IModificationOperationCallback createModificationOperationCallback(long resourceId, Object resource,
+            IHyracksTaskContext ctx) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
index a04eb62..c92c760 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
@@ -43,13 +43,12 @@
     }
 
     @Override
-    public IModificationOperationCallback createModificationOperationCallback(long resourceId, Object resource,
-            IHyracksTaskContext ctx) throws HyracksDataException {
-
+    public IModificationOperationCallback createModificationOperationCallback(String resourceName, long resourceId,
+            Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
         ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
                 .getIndexLifecycleManager();
-        ILSMIndex index = (ILSMIndex) indexLifeCycleManager.getIndex(resourceId);
+        ILSMIndex index = (ILSMIndex) indexLifeCycleManager.getIndex(resourceName);
         if (index == null) {
             throw new HyracksDataException("Index(id:" + resourceId + ") is not registered.");
         }
@@ -65,4 +64,10 @@
             throw new HyracksDataException(e);
         }
     }
+
+    @Override
+    public IModificationOperationCallback createModificationOperationCallback(long resourceId, Object resource,
+            IHyracksTaskContext ctx) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
index 4081101..17a0cde 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
@@ -30,25 +30,26 @@
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
 
-public class TempDatasetPrimaryIndexModificationOperationCallbackFactory extends AbstractOperationCallbackFactory implements
-        IModificationOperationCallbackFactory {
+public class TempDatasetPrimaryIndexModificationOperationCallbackFactory extends AbstractOperationCallbackFactory
+        implements IModificationOperationCallbackFactory {
 
     private static final long serialVersionUID = 1L;
     private final IndexOperation indexOp;
 
-    public TempDatasetPrimaryIndexModificationOperationCallbackFactory(JobId jobId, int datasetId, int[] primaryKeyFields,
-            ITransactionSubsystemProvider txnSubsystemProvider, IndexOperation indexOp, byte resourceType) {
+    public TempDatasetPrimaryIndexModificationOperationCallbackFactory(JobId jobId, int datasetId,
+            int[] primaryKeyFields, ITransactionSubsystemProvider txnSubsystemProvider, IndexOperation indexOp,
+            byte resourceType) {
         super(jobId, datasetId, primaryKeyFields, txnSubsystemProvider, resourceType);
         this.indexOp = indexOp;
     }
 
     @Override
-    public IModificationOperationCallback createModificationOperationCallback(long resourceId, Object resource,
-            IHyracksTaskContext ctx) throws HyracksDataException {
+    public IModificationOperationCallback createModificationOperationCallback(String resourceName, long resourceId,
+            Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
         ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
                 .getIndexLifecycleManager();
-        ILSMIndex index = (ILSMIndex) indexLifeCycleManager.getIndex(resourceId);
+        ILSMIndex index = (ILSMIndex) indexLifeCycleManager.getIndex(resourceName);
         if (index == null) {
             throw new HyracksDataException("Index(id:" + resourceId + ") is not registered.");
         }
@@ -64,4 +65,10 @@
             throw new HyracksDataException(e);
         }
     }
+    
+    @Override
+    public IModificationOperationCallback createModificationOperationCallback(long resourceId, Object resource,
+            IHyracksTaskContext ctx) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
index ce10bd7..27934a2 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
@@ -30,25 +30,26 @@
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
 
-public class TempDatasetSecondaryIndexModificationOperationCallbackFactory extends AbstractOperationCallbackFactory implements
-        IModificationOperationCallbackFactory {
+public class TempDatasetSecondaryIndexModificationOperationCallbackFactory extends AbstractOperationCallbackFactory
+        implements IModificationOperationCallbackFactory {
 
     private static final long serialVersionUID = 1L;
     private final IndexOperation indexOp;
 
-    public TempDatasetSecondaryIndexModificationOperationCallbackFactory(JobId jobId, int datasetId, int[] primaryKeyFields,
-            ITransactionSubsystemProvider txnSubsystemProvider, IndexOperation indexOp, byte resourceType) {
+    public TempDatasetSecondaryIndexModificationOperationCallbackFactory(JobId jobId, int datasetId,
+            int[] primaryKeyFields, ITransactionSubsystemProvider txnSubsystemProvider, IndexOperation indexOp,
+            byte resourceType) {
         super(jobId, datasetId, primaryKeyFields, txnSubsystemProvider, resourceType);
         this.indexOp = indexOp;
     }
 
     @Override
-    public IModificationOperationCallback createModificationOperationCallback(long resourceId, Object resource,
-            IHyracksTaskContext ctx) throws HyracksDataException {
+    public IModificationOperationCallback createModificationOperationCallback(String resourceName, long resourceId,
+            Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
         ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
                 .getIndexLifecycleManager();
-        ILSMIndex index = (ILSMIndex) indexLifeCycleManager.getIndex(resourceId);
+        ILSMIndex index = (ILSMIndex) indexLifeCycleManager.getIndex(resourceName);
         if (index == null) {
             throw new HyracksDataException("Index(id:" + resourceId + ") is not registered.");
         }
@@ -64,4 +65,10 @@
             throw new HyracksDataException(e);
         }
     }
+
+    @Override
+    public IModificationOperationCallback createModificationOperationCallback(long resourceId, Object resource,
+            IHyracksTaskContext ctx) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
index 903ddc8..6d92534 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
@@ -25,8 +25,12 @@
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 import java.util.logging.Level;
 import java.util.logging.Logger;
+
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
 
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.IODeviceHandle;
@@ -42,14 +46,14 @@
     private static final String ROOT_METADATA_FILE_NAME_PREFIX = ".asterix_root_metadata";
     private static final long ROOT_LOCAL_RESOURCE_ID = -4321;
     private static final String METADATA_FILE_NAME = ".metadata";
-    private Map<String, LocalResource> name2ResourceMap = new HashMap<String, LocalResource>();
-    private Map<Long, LocalResource> id2ResourceMap = new HashMap<Long, LocalResource>();
-    private final int numIODevices;
+    private final Cache<String, LocalResource> resourcesCache;
+    private final String nodeId;
+    private static final int MAX_CACHED_RESOURCES = 100;
 
-    public PersistentLocalResourceRepository(List<IODeviceHandle> devices) throws HyracksDataException {
-        numIODevices = devices.size();
-        this.mountPoints = new String[numIODevices];
-        for (int i = 0; i < numIODevices; i++) {
+    public PersistentLocalResourceRepository(List<IODeviceHandle> devices, String nodeId) throws HyracksDataException {
+        mountPoints = new String[devices.size()];
+        this.nodeId = nodeId;
+        for (int i = 0; i < mountPoints.length; i++) {
             String mountPoint = devices.get(i).getPath().getPath();
             File mountPointDir = new File(mountPoint);
             if (!mountPointDir.exists()) {
@@ -61,6 +65,8 @@
                 mountPoints[i] = new String(mountPoint);
             }
         }
+
+        resourcesCache = CacheBuilder.newBuilder().maximumSize(MAX_CACHED_RESOURCES).build();
     }
 
     private String prepareRootMetaDataFileName(String mountPoint, String nodeId, int ioDeviceId) {
@@ -75,7 +81,7 @@
 
         if (isNewUniverse) {
             //#. if the rootMetadataFile doesn't exist, create it and return.
-            for (int i = 0; i < numIODevices; i++) {
+            for (int i = 0; i < mountPoints.length; i++) {
                 String rootMetadataFileName = prepareRootMetaDataFileName(mountPoints[i], nodeId, i) + File.separator
                         + ROOT_METADATA_FILE_NAME_PREFIX;
                 File rootMetadataFile = new File(rootMetadataFileName);
@@ -115,78 +121,13 @@
             return;
         }
 
-        FilenameFilter filter = new FilenameFilter() {
-            public boolean accept(File dir, String name) {
-                if (name.equalsIgnoreCase(METADATA_FILE_NAME)) {
-                    return true;
-                } else {
-                    return false;
-                }
-            }
-        };
-
         long maxResourceId = 0;
-        for (int i = 0; i < numIODevices; i++) {
-            String rootMetadataFileName = prepareRootMetaDataFileName(mountPoints[i], nodeId, i) + File.separator
-                    + ROOT_METADATA_FILE_NAME_PREFIX;
-            File rootMetadataFile = new File(rootMetadataFileName);
-            //#. if the rootMetadataFile exists, read it and set this.rootDir.
-            LocalResource rootLocalResource = readLocalResource(rootMetadataFile);
-            String mountedRootDir = (String) rootLocalResource.getResourceObject();
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("The root directory of the local resource repository is " + mountedRootDir);
-            }
-
-            //#. load all local resources. 
-            File rootDirFile = new File(mountedRootDir);
-            if (!rootDirFile.exists()) {
-                //rootDir may not exist if this node is not the metadata node and doesn't have any user data.
-                if (LOGGER.isLoggable(Level.INFO)) {
-                    LOGGER.info("The root directory of the local resource repository doesn't exist: there is no local resource.");
-                    LOGGER.info("Completed the initialization of the local resource repository");
-                }
-                continue;
-            }
-
-            File[] dataverseFileList = rootDirFile.listFiles();
-            if (dataverseFileList == null) {
-                throw new HyracksDataException("Metadata dataverse doesn't exist.");
-            }
-            for (File dataverseFile : dataverseFileList) {
-                if (dataverseFile.isDirectory()) {
-                    File[] indexFileList = dataverseFile.listFiles();
-                    if (indexFileList != null) {
-                        for (File indexFile : indexFileList) {
-                            if (indexFile.isDirectory()) {
-                                File[] ioDevicesList = indexFile.listFiles();
-                                if (ioDevicesList != null) {
-                                    for (File ioDeviceFile : ioDevicesList) {
-                                        if (ioDeviceFile.isDirectory()) {
-                                            File[] metadataFiles = ioDeviceFile.listFiles(filter);
-                                            if (metadataFiles != null) {
-                                                for (File metadataFile : metadataFiles) {
-                                                    LocalResource localResource = readLocalResource(metadataFile);
-                                                    id2ResourceMap.put(localResource.getResourceId(), localResource);
-                                                    name2ResourceMap
-                                                            .put(localResource.getResourceName(), localResource);
-                                                    maxResourceId = Math.max(localResource.getResourceId(),
-                                                            maxResourceId);
-                                                    if (LOGGER.isLoggable(Level.INFO)) {
-                                                        LOGGER.info("loaded local resource - [id: "
-                                                                + localResource.getResourceId() + ", name: "
-                                                                + localResource.getResourceName() + "]");
-                                                    }
-                                                }
-                                            }
-                                        }
-                                    }
-                                }
-                            }
-                        }
-                    }
-                }
-            }
+        //TODO this could be optimized by storing the current max resource id in the checkpoint file
+        Map<Long, LocalResource> resourceMap = loadAndGetAllResources();
+        for (Long resourceId : resourceMap.keySet()) {
+            maxResourceId = Math.max(maxResourceId, resourceId);
         }
+
         resourceIdFactory.initId(maxResourceId + 1);
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("The resource id factory is intialized with the value: " + (maxResourceId + 1));
@@ -195,33 +136,36 @@
     }
 
     @Override
-    public LocalResource getResourceById(long id) throws HyracksDataException {
-        return id2ResourceMap.get(id);
-    }
-
-    @Override
     public LocalResource getResourceByName(String name) throws HyracksDataException {
-        return name2ResourceMap.get(name);
+        LocalResource resource = resourcesCache.getIfPresent(name);
+        if (resource == null) {
+            File resourceFile = getLocalResourceFileByName(name);
+            if (resourceFile.exists()) {
+                resource = readLocalResource(resourceFile);
+                resourcesCache.put(name, resource);
+                return readLocalResource(resourceFile);
+            }
+        }
+        return resource;
     }
 
     @Override
     public synchronized void insert(LocalResource resource) throws HyracksDataException {
-        long id = resource.getResourceId();
+        File resourceFile = new File(getFileName(resource.getResourceName(), resource.getResourceId()));
 
-        if (id2ResourceMap.containsKey(id)) {
+        if (resourceFile.exists()) {
             throw new HyracksDataException("Duplicate resource");
         }
 
         if (resource.getResourceId() != ROOT_LOCAL_RESOURCE_ID) {
-            id2ResourceMap.put(id, resource);
-            name2ResourceMap.put(resource.getResourceName(), resource);
+            resourcesCache.put(resource.getResourceName(), resource);
         }
 
         FileOutputStream fos = null;
         ObjectOutputStream oosToFos = null;
 
         try {
-            fos = new FileOutputStream(getFileName(resource.getResourceName(), resource.getResourceId()));
+            fos = new FileOutputStream(resourceFile);
             oosToFos = new ObjectOutputStream(fos);
             oosToFos.writeObject(resource);
             oosToFos.flush();
@@ -246,40 +190,89 @@
     }
 
     @Override
-    public synchronized void deleteResourceById(long id) throws HyracksDataException {
-        LocalResource resource = id2ResourceMap.get(id);
-        if (resource == null) {
+    public synchronized void deleteResourceByName(String name) throws HyracksDataException {
+        File resourceFile = getLocalResourceFileByName(name);
+        if (resourceFile.exists()) {
+            resourceFile.delete();
+            resourcesCache.invalidate(name);
+        } else {
             throw new HyracksDataException("Resource doesn't exist");
         }
-        id2ResourceMap.remove(id);
-        name2ResourceMap.remove(resource.getResourceName());
-        File file = new File(getFileName(resource.getResourceName(), resource.getResourceId()));
-        file.delete();
     }
 
-    @Override
-    public synchronized void deleteResourceByName(String name) throws HyracksDataException {
-        LocalResource resource = name2ResourceMap.get(name);
-        if (resource == null) {
-            throw new HyracksDataException("Resource doesn't exist");
+    private static File getLocalResourceFileByName(String resourceName) {
+        return new File(resourceName + File.separator + METADATA_FILE_NAME);
+    }
+
+    public HashMap<Long, LocalResource> loadAndGetAllResources() throws HyracksDataException {
+        HashMap<Long, LocalResource> resourcesMap = new HashMap<Long, LocalResource>();
+
+        for (int i = 0; i < mountPoints.length; i++) {
+            String rootMetadataFileName = prepareRootMetaDataFileName(mountPoints[i], nodeId, i) + File.separator
+                    + ROOT_METADATA_FILE_NAME_PREFIX;
+            File rootMetadataFile = new File(rootMetadataFileName);
+            if (!rootMetadataFile.exists()) {
+                continue;
+            }
+            //#. if the rootMetadataFile exists, read it and set it as mounting point root
+            LocalResource rootLocalResource = readLocalResource(rootMetadataFile);
+            String mountedRootDir = (String) rootLocalResource.getResourceObject();
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("The root directory of the local resource repository is " + mountedRootDir);
+            }
+
+            File rootDirFile = new File(mountedRootDir);
+            if (!rootDirFile.exists()) {
+                //rootDir may not exist if this node is not the metadata node and doesn't have any user data.
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("The root directory of the local resource repository doesn't exist: there is no local resource.");
+                    LOGGER.info("Completed the initialization of the local resource repository");
+                }
+                continue;
+            }
+
+            //#. load all local resources. 
+            File[] dataverseFileList = rootDirFile.listFiles();
+            if (dataverseFileList == null) {
+                throw new HyracksDataException("Metadata dataverse doesn't exist.");
+            }
+            for (File dataverseFile : dataverseFileList) {
+                if (dataverseFile.isDirectory()) {
+                    File[] indexFileList = dataverseFile.listFiles();
+                    if (indexFileList != null) {
+                        for (File indexFile : indexFileList) {
+                            if (indexFile.isDirectory()) {
+                                File[] ioDevicesList = indexFile.listFiles();
+                                if (ioDevicesList != null) {
+                                    for (File ioDeviceFile : ioDevicesList) {
+                                        if (ioDeviceFile.isDirectory()) {
+                                            File[] metadataFiles = ioDeviceFile.listFiles(METADATA_FILES_FILTER);
+                                            if (metadataFiles != null) {
+                                                for (File metadataFile : metadataFiles) {
+                                                    LocalResource localResource = readLocalResource(metadataFile);
+                                                    resourcesMap.put(localResource.getResourceId(), localResource);
+                                                }
+                                            }
+                                        }
+                                    }
+                                }
+                            }
+                        }
+                    }
+                }
+            }
         }
-        id2ResourceMap.remove(resource.getResourceId());
-        name2ResourceMap.remove(name);
-        File file = new File(getFileName(resource.getResourceName(), resource.getResourceId()));
-        file.delete();
+
+        return resourcesMap;
     }
 
     @Override
     public List<LocalResource> getAllResources() throws HyracksDataException {
-        List<LocalResource> resources = new ArrayList<LocalResource>();
-        for (LocalResource resource : id2ResourceMap.values()) {
-            resources.add(resource);
-        }
-        return resources;
+        HashMap<Long, LocalResource> resourcesMap = loadAndGetAllResources();
+        return new ArrayList<LocalResource>(resourcesMap.values());
     }
 
     private String getFileName(String baseDir, long resourceId) {
-
         if (resourceId == ROOT_LOCAL_RESOURCE_ID) {
             return baseDir;
         } else {
@@ -319,4 +312,24 @@
             }
         }
     }
+
+    @Override
+    public synchronized void deleteResourceById(long id) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public LocalResource getResourceById(long id) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    private static final FilenameFilter METADATA_FILES_FILTER = new FilenameFilter() {
+        public boolean accept(File dir, String name) {
+            if (name.equalsIgnoreCase(METADATA_FILE_NAME)) {
+                return true;
+            } else {
+                return false;
+            }
+        }
+    };
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
index 1e6767b..74b92e0 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
@@ -20,14 +20,16 @@
 import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepositoryFactory;
 
 public class PersistentLocalResourceRepositoryFactory implements ILocalResourceRepositoryFactory {
-    private IIOManager ioManager;
+    private final IIOManager ioManager;
+    private final String nodeId;
 
-    public PersistentLocalResourceRepositoryFactory(IIOManager ioManager) {
+    public PersistentLocalResourceRepositoryFactory(IIOManager ioManager, String nodeId) {
         this.ioManager = ioManager;
+        this.nodeId = nodeId;
     }
 
     @Override
     public ILocalResourceRepository createRepository() throws HyracksDataException {
-        return new PersistentLocalResourceRepository(ioManager.getIODevices());
+        return new PersistentLocalResourceRepository(ioManager.getIODevices(), nodeId);
     }
 }
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/TestRuntimeContextProvider.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/TestRuntimeContextProvider.java
index 8b5e026..7cfc50a 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/TestRuntimeContextProvider.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/TestRuntimeContextProvider.java
@@ -35,9 +35,9 @@
 
 class TestRuntimeContextProvider implements IAsterixAppRuntimeContextProvider {
 
-    AsterixThreadExecutor ate = new AsterixThreadExecutor(Executors.defaultThreadFactory());    
+    AsterixThreadExecutor ate = new AsterixThreadExecutor(Executors.defaultThreadFactory());
     IIndexLifecycleManager ilm = new IndexLifecycleManager();
-    
+
     @Override
     public AsterixThreadExecutor getThreadExecutor() {
         return ate;
@@ -134,6 +134,40 @@
         public List<IIndex> getOpenIndexes() {
             throw new UnsupportedOperationException();
         }
-        
+
+        @Override
+        public void register(String resourceName, IIndex index) throws HyracksDataException {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public void open(String resourceName) throws HyracksDataException {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public boolean supportsPersistentLocalResources() {
+            return false;
+        }
+
+        @Override
+        public void close(String resourceName) throws HyracksDataException {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public IIndex getIndex(String resourceName) throws HyracksDataException {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public void unregister(String resourceName) throws HyracksDataException {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public IIndex getIndex(int datasetID, long resourceID) throws HyracksDataException {
+            throw new UnsupportedOperationException();
+        }
     }
 }
\ No newline at end of file
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 d415935..8996566 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
@@ -47,6 +47,7 @@
 import edu.uci.ics.asterix.common.transactions.IRecoveryManager;
 import edu.uci.ics.asterix.common.transactions.ITransactionContext;
 import edu.uci.ics.asterix.common.transactions.LogType;
+import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceRepository;
 import edu.uci.ics.asterix.transaction.management.service.logging.LogManager;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManager;
@@ -78,7 +79,7 @@
     private final LogManager logMgr;
     private final int checkpointHistory;
     private final long SHARP_CHECKPOINT_LSN = -1;
-
+    private Map<Long, LocalResource> resourcesMap;
     /**
      * A file at a known location that contains the LSN of the last log record
      * traversed doing a successful checkpoint.
@@ -245,6 +246,7 @@
         IIndexLifecycleManager indexLifecycleManager = appRuntimeContext.getIndexLifecycleManager();
         ILocalResourceRepository localResourceRepository = appRuntimeContext.getLocalResourceRepository();
 
+        resourcesMap = ((PersistentLocalResourceRepository) localResourceRepository).loadAndGetAllResources();
         //#. set log reader to the lowWaterMarkLsn again.
         logReader.initializeScan(lowWaterMarkLSN);
         logRecord = logReader.next();
@@ -269,12 +271,12 @@
                     }
                     if (foundWinner) {
                         resourceId = logRecord.getResourceId();
-                        localResource = localResourceRepository.getResourceById(resourceId);
+                        localResource = resourcesMap.get(resourceId);
 
                         //get index instance from IndexLifeCycleManager
                         //if index is not registered into IndexLifeCycleManager,
                         //create the index using LocalMetadata stored in LocalResourceRepository
-                        index = (ILSMIndex) indexLifecycleManager.getIndex(resourceId);
+                        index = (ILSMIndex) indexLifecycleManager.getIndex(localResource.getResourceName());
                         if (index == null) {
 
                             /*******************************************************************
@@ -300,8 +302,8 @@
                             localResourceMetadata = (ILocalResourceMetadata) localResource.getResourceObject();
                             index = localResourceMetadata.createIndexInstance(appRuntimeContext,
                                     localResource.getResourceName(), localResource.getPartition());
-                            indexLifecycleManager.register(resourceId, index);
-                            indexLifecycleManager.open(resourceId);
+                            indexLifecycleManager.register(localResource.getResourceName(), index);
+                            indexLifecycleManager.open(localResource.getResourceName());
 
                             //#. get maxDiskLastLSN
                             ILSMIndex lsmIndex = (ILSMIndex) index;
@@ -338,7 +340,7 @@
         //close all indexes
         Set<Long> resourceIdList = resourceId2MaxLSNMap.keySet();
         for (long r : resourceIdList) {
-            indexLifecycleManager.close(r);
+            indexLifecycleManager.close(resourcesMap.get(r).getResourceName());
         }
 
         logReader.close();
@@ -352,11 +354,12 @@
     }
 
     @Override
-    public synchronized long checkpoint(boolean isSharpCheckpoint, long nonSharpCheckpointTargetLSN) throws ACIDException, HyracksDataException {
+    public synchronized long checkpoint(boolean isSharpCheckpoint, long nonSharpCheckpointTargetLSN)
+            throws ACIDException, HyracksDataException {
 
         long minMCTFirstLSN;
         boolean nonSharpCheckpointSucceeded = false;
-        
+
         if (isSharpCheckpoint && LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Starting sharp checkpoint ... ");
         }
@@ -368,7 +371,8 @@
         //   right after the new checkpoint file is written.
         File[] prevCheckpointFiles = getPreviousCheckpointFiles();
 
-        DatasetLifecycleManager datasetLifecycleManager = (DatasetLifecycleManager)txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager();
+        DatasetLifecycleManager datasetLifecycleManager = (DatasetLifecycleManager) txnSubsystem
+                .getAsterixAppRuntimeContextProvider().getIndexLifecycleManager();
         //#. flush all in-memory components if it is the sharp checkpoint
         if (isSharpCheckpoint) {
 
@@ -378,11 +382,10 @@
         } else {
 
             minMCTFirstLSN = getMinFirstLSN();
-            
-            if(minMCTFirstLSN >= nonSharpCheckpointTargetLSN){
+
+            if (minMCTFirstLSN >= nonSharpCheckpointTargetLSN) {
                 nonSharpCheckpointSucceeded = true;
-            }
-            else{
+            } else {
                 //flush datasets with indexes behind target checkpoint LSN
                 datasetLifecycleManager.scheduleAsyncFlushForLaggingDatasets(nonSharpCheckpointTargetLSN);
             }
@@ -435,10 +438,10 @@
             }
         }
 
-        if(nonSharpCheckpointSucceeded){
+        if (nonSharpCheckpointSucceeded) {
             logMgr.deleteOldLogFiles(minMCTFirstLSN);
         }
-        
+
         if (isSharpCheckpoint && LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Completed sharp checkpoint.");
         }
@@ -447,9 +450,9 @@
         return minMCTFirstLSN;
     }
 
-    public long getMinFirstLSN() throws HyracksDataException
-    {
-        IIndexLifecycleManager indexLifecycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager();
+    public long getMinFirstLSN() throws HyracksDataException {
+        IIndexLifecycleManager indexLifecycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
+                .getIndexLifecycleManager();
         List<IIndex> openIndexList = indexLifecycleManager.getOpenIndexes();
         long firstLSN;
         //the min first lsn can only be the current append or smaller
@@ -459,8 +462,9 @@
 
             for (IIndex index : openIndexList) {
 
-                AbstractLSMIOOperationCallback ioCallback =  (AbstractLSMIOOperationCallback)((ILSMIndex) index).getIOOperationCallback();
-                if(!((AbstractLSMIndex)index).isCurrentMutableComponentEmpty() || ioCallback.hasPendingFlush()){
+                AbstractLSMIOOperationCallback ioCallback = (AbstractLSMIOOperationCallback) ((ILSMIndex) index)
+                        .getIOOperationCallback();
+                if (!((AbstractLSMIndex) index).isCurrentMutableComponentEmpty() || ioCallback.hasPendingFlush()) {
                     firstLSN = ioCallback.getFirstLSN();
                     minFirstLSN = Math.min(minFirstLSN, firstLSN);
                 }
@@ -590,6 +594,7 @@
         ILogReader logReader = logMgr.getLogReader(false);
         logReader.initializeScan(firstLSN);
         ILogRecord logRecord = null;
+
         while (currentLSN < lastLSN) {
             logRecord = logReader.next();
             if (logRecord == null) {
@@ -706,7 +711,7 @@
     private void undo(ILogRecord logRecord) {
         try {
             ILSMIndex index = (ILSMIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
-                    .getIndex(logRecord.getResourceId());
+                    .getIndex(logRecord.getDatasetId(), logRecord.getResourceId());
             ILSMIndexAccessor indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE,
                     NoOpOperationCallback.INSTANCE);
             if (logRecord.getNewOp() == IndexOperation.INSERT.ordinal()) {
@@ -724,7 +729,7 @@
     private void redo(ILogRecord logRecord) {
         try {
             ILSMIndex index = (ILSMIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
-                    .getIndex(logRecord.getResourceId());
+                    .getIndex(logRecord.getDatasetId(), logRecord.getResourceId());
             ILSMIndexAccessor indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE,
                     NoOpOperationCallback.INSTANCE);
             if (logRecord.getNewOp() == IndexOperation.INSERT.ordinal()) {

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newchange
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>

Change in asterixdb[master]: ASTERIXDB-1053: Allow lazy loading for persistent local reso...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1053: Allow lazy loading for persistent local resources
......................................................................


Patch Set 10: Verified-1

Build Failed 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/269/ : ABORTED

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 10
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Jenkins (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Jenkins has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/204/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Yingyi Bu (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Yingyi Bu has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 5:

(2 comments)

https://asterix-gerrit.ics.uci.edu/#/c/344/5/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
File asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java:

Line 48:     private final Cache<String, LocalResource> resourcesCache;
> I'm not sure if we have a naming standard. I think resourcesCache is better
OK, fine for me. (though in many other places in the codebase, we usually use singular for attributive nouns).


Line 205:     public HashMap<Long, LocalResource> loadAndGetAllResources() throws HyracksDataException {
> RecoveryManager needs to have the complete map to use it during recovery.
In my opinion, a disk-access-transparent ResourceCache would serve the same purpose.  If a lookup hit the cache, just return the resource; otherwise, read the resource from disk and invalidate another cache line.  

The purpose for this change is for the situation that there are ~1M datasets in AsterixDB.  But if we load everything here into the map, we probably will run into the same memory issue.


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Murtadha Hubail (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Murtadha Hubail has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 5:

(2 comments)

https://asterix-gerrit.ics.uci.edu/#/c/344/5/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
File asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java:

Line 205:     public HashMap<Long, LocalResource> loadAndGetAllResources() throws HyracksDataException {
> In my opinion, a disk-access-transparent ResourceCache would serve the same
During recovery, the only piece of information that you have from the log is the resource id. You wouldn't know where to look to begin with. Also, while the system is recovering, all the memory is dedicated for that. There are no concurrent operations. So the memory issue isn't likely to happen.


Line 205:     public HashMap<Long, LocalResource> loadAndGetAllResources() throws HyracksDataException {
> Is it possible to have yet-another resourcesCache for that maps an ID to th
Yes, we would have to persist a new file at the root of every IO Device and load these files gradually during recovery until the required resource is found. Another option is to store a file per resource in a common location and make each file name to be the resource id. Then from that common location, you would get where the actual resource is.

As I mentioned, I don't think this is going to be an issue during recovery, and when it does, we could make loading the resources to the cache gradual which will make the recovery slower, but limit the amount of memory used.


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Murtadha Hubail (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Hello abdullah alamoudi, Young-Seok Kim, Ian Maxon, Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/344

to look at the new patch set (#7).

Change subject: Allow lazy loading for persistent local resources
......................................................................

Allow lazy loading for persistent local resources

- Allow indexes to be accessed by name.
- Allow lazy loading for persistent local resources.
- Caching for local resources.

Change-Id: I48b9260a3280750145f6ddb3783673a299055910
---
M asterix-app/src/main/java/org/apache/asterix/api/common/AsterixAppRuntimeContext.java
M asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
M asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
M asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
M asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
M asterix-transactions/pom.xml
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/TestRuntimeContextProvider.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/RecoveryManager.java
16 files changed, 414 insertions(+), 328 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/44/344/7
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Murtadha Hubail (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Murtadha Hubail has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 5:

(7 comments)

https://asterix-gerrit.ics.uci.edu/#/c/344/5/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
File asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java:

Line 290:         ILSMIndex lsmIndex = (ILSMIndex) indexLifecycleManager.getIndex(metadataIndex.getFile().toString());
> Could we create a local variable for "metadataIndex.getFile().toString()"?
Done


Line 639:         ILSMIndex lsmIndex = (ILSMIndex) indexLifecycleManager.getIndex(metadataIndex.getFile().toString());
> Could we create a local variable for "metadataIndex.getFile().toString()"?
Done


Line 973:             IIndex indexInstance = indexLifecycleManager.getIndex(index.getFile().toString());
> Could we create a local variable for "index.getFile().toString()"?
Done


Line 1049:         IIndex indexInstance = indexLifecycleManager.getIndex(index.getFile().toString());
> Could we create a local variable for "index.getFile().toString()"?
Done


Line 1087:                     .toString());
> Another local variable here?
Done


https://asterix-gerrit.ics.uci.edu/#/c/344/5/asterix-transactions/pom.xml
File asterix-transactions/pom.xml:

Line 11: 	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
> This seems to be a modification in the wrong direction. We shouldn't introd
This is eclipse auto formatting for XML files. I will revert the formatting changes to this file and just add the new dependency until there is an agreement on whether to auto format the POM files or not.


https://asterix-gerrit.ics.uci.edu/#/c/344/5/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
File asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java:

Line 48:     private final Cache<String, LocalResource> resourcesCache;
> I'd prefer "resourceCache" as well.
Alright, I will give you this one :)


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Jenkins (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Jenkins has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 5: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/205/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Murtadha Hubail (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/344

to look at the new patch set (#2).

Change subject: Allow lazy loading for persistent local resources
......................................................................

Allow lazy loading for persistent local resources

- Allow indexes to be accessed by name.
- Allow lazy loading for persistent local resources.
- Caching for local resources.
- Fix server start timing issue. 

Change-Id: I48b9260a3280750145f6ddb3783673a299055910
---
M asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
M asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
M asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java
M asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
M asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
M asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
M asterix-transactions/pom.xml
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/TestRuntimeContextProvider.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
15 files changed, 435 insertions(+), 301 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/44/344/2
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: ASTERIXDB-1053: Allow lazy loading for persistent local reso...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1053: Allow lazy loading for persistent local resources
......................................................................


Patch Set 10:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/268/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 10
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Jenkins (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Jenkins has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 1: Verified-1

Build Failed 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/186/ : FAILURE

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1053: Allow lazy loading for persistent local reso...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1053: Allow lazy loading for persistent local resources
......................................................................


Patch Set 11: -Verified

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/279/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 11
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Young-Seok Kim (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Young-Seok Kim has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 3:

(4 comments)

I added comments.
Please address them.

https://asterix-gerrit.ics.uci.edu/#/c/344/3/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
File asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java:

Line 1085:         long resourceID = MetadataPrimaryIndexes.DATASET_DATASET.getResourceID();
This is not used anymore. So let's remove this.


https://asterix-gerrit.ics.uci.edu/#/c/344/3/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
File asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java:

Line 51:     private static final int MAX_CACHED_RESOURCES = 100;
Why 100? 100 seems small number considering the number of resources = (#dataset + #secondary index) * #partition


Line 146:                 return readLocalResource(resourceFile);
why should this be read again? Shouldn't be?


https://asterix-gerrit.ics.uci.edu/#/c/344/3/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
File asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java:

Line 249:         resourcesMap = ((PersistentLocalResourceRepository) localResourceRepository).loadAndGetAllResources();
When is resourceMap cleaned up after this recovery? Seems like all resource in the map are never cleaned up in this code, right? This map is supposed be cleaned up at the end of recovery, right?


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: ASTERIXDB-1053: Allow lazy loading for persistent local reso...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1053: Allow lazy loading for persistent local resources
......................................................................


Patch Set 11: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/279/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 11
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Murtadha Hubail (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Murtadha Hubail has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 7:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/344/7/asterix-transactions/pom.xml
File asterix-transactions/pom.xml:

Line 120: 	</dependencies>
> Why do we get a new guava dependency here?
For using their Cache class in PersistentLocalResourceRepository.


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Till Westmann (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Till Westmann has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 7:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/344/7/asterix-transactions/pom.xml
File asterix-transactions/pom.xml:

Line 120: 	</dependencies>
Why do we get a new guava dependency here?


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Murtadha Hubail (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Murtadha Hubail has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 3:

(4 comments)

https://asterix-gerrit.ics.uci.edu/#/c/344/3/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
File asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java:

Line 1085:         long resourceID = MetadataPrimaryIndexes.DATASET_DATASET.getResourceID();
> This is not used anymore. So let's remove this.
Done


https://asterix-gerrit.ics.uci.edu/#/c/344/3/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
File asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java:

Line 51:     private static final int MAX_CACHED_RESOURCES = 100;
> Why 100? 100 seems small number considering the number of resources = (#dat
Changed to 1000.


Line 146:                 return readLocalResource(resourceFile);
> why should this be read again? Shouldn't be?
It shouldn't be. Just an old code before introducing the cache. Done.


https://asterix-gerrit.ics.uci.edu/#/c/344/3/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
File asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java:

Line 249:         resourcesMap = ((PersistentLocalResourceRepository) localResourceRepository).loadAndGetAllResources();
> When is resourceMap cleaned up after this recovery? Seems like all resource
That's right. I made the map initially local to startRecovery method then changed it to global because I thought I would need to use it on redo. I changed it back to be local to startRecovery.


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: ASTERIXDB-1053: Allow lazy loading for persistent local reso...

Posted by "Murtadha Hubail (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello abdullah alamoudi, Young-Seok Kim, Yingyi Bu, Ian Maxon, Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/344

to look at the new patch set (#10).

Change subject: ASTERIXDB-1053: Allow lazy loading for persistent local resources
......................................................................

ASTERIXDB-1053: Allow lazy loading for persistent local resources

- Allow indexes to be accessed by name.
- Allow lazy loading for persistent local resources.
- Caching for local resources.

Change-Id: I48b9260a3280750145f6ddb3783673a299055910
---
M asterix-app/src/main/java/org/apache/asterix/api/common/AsterixAppRuntimeContext.java
M asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
M asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
M asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
M asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
M asterix-transactions/pom.xml
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/TestRuntimeContextProvider.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/RecoveryManager.java
M pom.xml
17 files changed, 385 insertions(+), 329 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/44/344/10
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 10
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Till Westmann (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Till Westmann has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 7:

(2 comments)

https://asterix-gerrit.ics.uci.edu/#/c/344/7/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
File asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java:

Line 77:         DatasetInfo dsInfo = datasetInfos.get(getDIDfromResourceName(resourceName));
Would it make sense to just determine the datasetID and the resourceID here and to then call the other getIndexMethod?


https://asterix-gerrit.ics.uci.edu/#/c/344/7/asterix-transactions/pom.xml
File asterix-transactions/pom.xml:

Line 120: 	</dependencies>
> For using their Cache class in PersistentLocalResourceRepository.
Ah, yes, missed that. Thanks!


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Jenkins (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Jenkins has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/205/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1053: Allow lazy loading for persistent local reso...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1053: Allow lazy loading for persistent local resources
......................................................................


Patch Set 11:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/274/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 11
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "abdullah alamoudi (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
abdullah alamoudi has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 5: Code-Review+2

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1053: Allow lazy loading for persistent local reso...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1053: Allow lazy loading for persistent local resources
......................................................................


Patch Set 10: -Verified

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/269/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 10
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Murtadha Hubail (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Hello Ian Maxon, Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/344

to look at the new patch set (#5).

Change subject: Allow lazy loading for persistent local resources
......................................................................

Allow lazy loading for persistent local resources

- Allow indexes to be accessed by name.
- Allow lazy loading for persistent local resources.
- Caching for local resources.
- Fix server start timing issue. 

Change-Id: I48b9260a3280750145f6ddb3783673a299055910
---
M asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
M asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
M asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java
M asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
M asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
M asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
M asterix-transactions/pom.xml
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/TestRuntimeContextProvider.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
15 files changed, 452 insertions(+), 322 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/44/344/5
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: ASTERIXDB-1053: Allow lazy loading for persistent local reso...

Posted by "Murtadha Hubail (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello abdullah alamoudi, Young-Seok Kim, Yingyi Bu, Ian Maxon, Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/344

to look at the new patch set (#8).

Change subject: ASTERIXDB-1053: Allow lazy loading for persistent local resources
......................................................................

ASTERIXDB-1053: Allow lazy loading for persistent local resources

- Allow indexes to be accessed by name.
- Allow lazy loading for persistent local resources.
- Caching for local resources.
- Fix server start timing issue.

Change-Id: I48b9260a3280750145f6ddb3783673a299055910
---
M asterix-app/src/main/java/org/apache/asterix/api/common/AsterixAppRuntimeContext.java
M asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
M asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
M asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
M asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
M asterix-transactions/pom.xml
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/TestRuntimeContextProvider.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/RecoveryManager.java
16 files changed, 383 insertions(+), 327 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/44/344/8
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Yingyi Bu (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Yingyi Bu has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 5:

(5 comments)

It looks all the resources are still loaded at the very beginning into a temporary map in getAndLoadAllResources(...).  
I looked at the callers --- it seems the callers are only interested in the max resource id instead of the whole map.
Am I right?

Detailed comments are in the code.

https://asterix-gerrit.ics.uci.edu/#/c/344/5/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
File asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java:

Line 68:     
trim space.


https://asterix-gerrit.ics.uci.edu/#/c/344/5/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
File asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java:

Line 48:     private final Cache<String, LocalResource> resourcesCache;
s/resourcesCache/resourceCache?


Line 205:     public HashMap<Long, LocalResource> loadAndGetAllResources() throws HyracksDataException {
s/public/private?
By looking at the callers of this method, I guess you don't need to load everything into the temporary map resourcesMap, but just need to return the max Id which is a long here.  Am I right?


Line 232:             //#. load all local resources. 
trim space.


Line 270:         return new ArrayList<LocalResource>(resourcesMap.values());
Change the method signature here?
Based on my understanding, it is only called in the ResourceIdFactoryProvider which is just to get the largest resource Id.


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: ASTERIXDB-1053: Allow lazy loading for persistent local reso...

Posted by "Ian Maxon (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello abdullah alamoudi, Young-Seok Kim, Yingyi Bu, Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/344

to look at the new patch set (#11).

Change subject: ASTERIXDB-1053: Allow lazy loading for persistent local resources
......................................................................

ASTERIXDB-1053: Allow lazy loading for persistent local resources

- Allow indexes to be accessed by name.
- Allow lazy loading for persistent local resources.
- Caching for local resources.

Change-Id: I48b9260a3280750145f6ddb3783673a299055910
---
M asterix-app/src/main/java/org/apache/asterix/api/common/AsterixAppRuntimeContext.java
M asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
M asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
M asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
M asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
M asterix-transactions/pom.xml
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/TestRuntimeContextProvider.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/RecoveryManager.java
16 files changed, 383 insertions(+), 327 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/44/344/11
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 11
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Jenkins (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Jenkins has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 6: Verified-1

Build Failed 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/224/ : ABORTED

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Murtadha Hubail (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Hello abdullah alamoudi, Young-Seok Kim, Ian Maxon, Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/344

to look at the new patch set (#6).

Change subject: Allow lazy loading for persistent local resources
......................................................................

Allow lazy loading for persistent local resources

- Allow indexes to be accessed by name.
- Allow lazy loading for persistent local resources.
- Caching for local resources.
- Fix server start timing issue.

Change-Id: I48b9260a3280750145f6ddb3783673a299055910
---
M asterix-app/src/main/java/org/apache/asterix/api/common/AsterixAppRuntimeContext.java
M asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
M asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
M asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
M asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
M asterix-transactions/pom.xml
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/TestRuntimeContextProvider.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/RecoveryManager.java
16 files changed, 414 insertions(+), 328 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/44/344/6
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: ASTERIXDB-1053: Allow lazy loading for persistent local reso...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1053: Allow lazy loading for persistent local resources
......................................................................


Patch Set 10: Verified-1

Build Failed 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/268/ : ABORTED

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 10
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1053: Allow lazy loading for persistent local reso...

Posted by "Murtadha Hubail (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello abdullah alamoudi, Young-Seok Kim, Yingyi Bu, Ian Maxon, Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/344

to look at the new patch set (#9).

Change subject: ASTERIXDB-1053: Allow lazy loading for persistent local resources
......................................................................

ASTERIXDB-1053: Allow lazy loading for persistent local resources

- Allow indexes to be accessed by name.
- Allow lazy loading for persistent local resources.
- Caching for local resources.

Change-Id: I48b9260a3280750145f6ddb3783673a299055910
---
M asterix-app/src/main/java/org/apache/asterix/api/common/AsterixAppRuntimeContext.java
M asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
M asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
M asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
M asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
M asterix-transactions/pom.xml
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/TestRuntimeContextProvider.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/RecoveryManager.java
16 files changed, 383 insertions(+), 327 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/44/344/9
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 9
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Jenkins (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Jenkins has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 2: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/196/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1053: Allow lazy loading for persistent local reso...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1053: Allow lazy loading for persistent local resources
......................................................................


Patch Set 8: Verified-1

Build Failed 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/266/ : FAILURE

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Jenkins (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Jenkins has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 4: Verified-1

Build Failed 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/204/ : FAILURE

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Jenkins (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Jenkins has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 3: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/197/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Murtadha Hubail (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Murtadha Hubail has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 5:

(5 comments)

https://asterix-gerrit.ics.uci.edu/#/c/344/5/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
File asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java:

Line 68:     
> trim space.
Done


https://asterix-gerrit.ics.uci.edu/#/c/344/5/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
File asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java:

Line 48:     private final Cache<String, LocalResource> resourcesCache;
> s/resourcesCache/resourceCache?
I'm not sure if we have a naming standard. I think resourcesCache is better for readability :)


Line 205:     public HashMap<Long, LocalResource> loadAndGetAllResources() throws HyracksDataException {
> s/public/private?
RecoveryManager needs to have the complete map to use it during recovery.


Line 232:             //#. load all local resources. 
> trim space.
Done


Line 270:         return new ArrayList<LocalResource>(resourcesMap.values());
> Change the method signature here?
This method is part of ILocalResourceRepository. Both PersistentLocalResourceRepository and TransientLocalResourceRepository (in Hyracks) are implementing it. I didn't want to change the signature and break the TransientLocalResourceRepository. We could however add a new method called GetMaxResourceID in both of them, which will basically do the same but will be reading the resources one by one and updating a max. After that, to factor out the code for reading the resources, we will end up with the current implementation. Since all resources are loaded only during startup, there shouldn't be any memory problems.


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Till Westmann (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Till Westmann has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 5:

(7 comments)

Just a few small comments.

https://asterix-gerrit.ics.uci.edu/#/c/344/5/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
File asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java:

Line 290:         ILSMIndex lsmIndex = (ILSMIndex) indexLifecycleManager.getIndex(metadataIndex.getFile().toString());
Could we create a local variable for "metadataIndex.getFile().toString()"?


Line 639:         ILSMIndex lsmIndex = (ILSMIndex) indexLifecycleManager.getIndex(metadataIndex.getFile().toString());
Could we create a local variable for "metadataIndex.getFile().toString()"?


Line 973:             IIndex indexInstance = indexLifecycleManager.getIndex(index.getFile().toString());
Could we create a local variable for "index.getFile().toString()"?


Line 1049:         IIndex indexInstance = indexLifecycleManager.getIndex(index.getFile().toString());
Could we create a local variable for "index.getFile().toString()"?


Line 1087:                     .toString());
Another local variable here?


https://asterix-gerrit.ics.uci.edu/#/c/344/5/asterix-transactions/pom.xml
File asterix-transactions/pom.xml:

Line 11: 	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
This seems to be a modification in the wrong direction. We shouldn't introduce tabs.


https://asterix-gerrit.ics.uci.edu/#/c/344/5/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
File asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java:

Line 48:     private final Cache<String, LocalResource> resourcesCache;
> OK, fine for me. (though in many other places in the codebase, we usually u
I'd prefer "resourceCache" as well.


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Murtadha Hubail (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Murtadha Hubail has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 2:

(2 comments)

Thanks Ian for the review. I addressed your comments.

https://asterix-gerrit.ics.uci.edu/#/c/344/2/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
File asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java:

Line 98: 
> Why the change of order here?
To make sure the servers have started before setting up the external libraries. I was getting many Connection Refused exceptions in test cases before the reordering. I didn't get any after afterwards. It might be unrelated and just the timing difference made them disappear.


https://asterix-gerrit.ics.uci.edu/#/c/344/2/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
File asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java:

Line 279:                         index = (ILSMIndex) indexLifecycleManager.getIndex(localResource.getResourceName());
> Won't this act differently now, if the local resource doesn't exist? If the
Great catch. Done.


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: ASTERIXDB-1053: Allow lazy loading for persistent local reso...

Posted by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org>.
Yingyi Bu has posted comments on this change.

Change subject: ASTERIXDB-1053: Allow lazy loading for persistent local resources
......................................................................


Patch Set 11: Code-Review+2

LGTM.

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 11
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Murtadha Hubail (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Murtadha Hubail has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 7:

(2 comments)

https://asterix-gerrit.ics.uci.edu/#/c/344/7/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
File asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java:

Line 77:         DatasetInfo dsInfo = datasetInfos.get(getDIDfromResourceName(resourceName));
> Would it make sense to just determine the datasetID and the resourceID here
Yes, it would. It will just cause an extra call to getResourceIDfromResourceName() if the dataset is not found.
We could also avoid calling getResourceByName() twice in all the methods by doing this at the beginning of each method:
LocalResource lr = resourceRepository.getResourceByName(resourceName);
if (lr == null) {
    return null;
}
int did = ((ILocalResourceMetadata)lr.getResourceObject()).getDatasetID();
long resourceID = lr.getResourceId();

These 6 lines will be repeated at the beginning of each method. 
What do you think?


https://asterix-gerrit.ics.uci.edu/#/c/344/7/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
File asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java:

Line 75:         throw new UnsupportedOperationException();
> Do we still need this interface method because it seems no one really imple
It is used in Hyracks IndexInsertUpdateDeleteOperatorNodePushable class. I modified it to use the new one and removed this method from the interface.


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Jenkins (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Jenkins has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 7:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/225/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Jenkins (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Jenkins has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/197/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Yingyi Bu (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Yingyi Bu has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 5:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/344/5/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
File asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java:

Line 205:     public HashMap<Long, LocalResource> loadAndGetAllResources() throws HyracksDataException {
> In my opinion, a disk-access-transparent ResourceCache would serve the same
Is it possible to have yet-another resourcesCache for that maps an ID to the Resource (or name string), so that we always use bounded amount of memory?

It looks not possible with the current physical metadata storage because the resources are physically identified by the metadata file name, unless we add other two files that one contains a <id, name_offset> map and the other contains all the names.


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Murtadha Hubail (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Murtadha Hubail has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 5:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/344/5/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
File asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java:

Line 205:     public HashMap<Long, LocalResource> loadAndGetAllResources() throws HyracksDataException {
> IMO, "slower" is better than "crashing when too many resources"...
1. I added the comment in the coming patch. When I'm done with fault tolerance implementation, we will have the partition number in the log. That could narrow down the search for specific resource on disk.

2. The call in initialize was a redundant call. I removed it completely. I implemented getMaxResourceId and used it in ResourceIdFactoryProvider.


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Ian Maxon (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Ian Maxon has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 3: Code-Review+1

Cool. Looks great to me, then.

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Jenkins (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Jenkins has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/196/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Ian Maxon (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Ian Maxon has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 2: Code-Review+1

(2 comments)

Just a few thoughts...

https://asterix-gerrit.ics.uci.edu/#/c/344/2/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
File asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java:

Line 98: 
Why the change of order here?


https://asterix-gerrit.ics.uci.edu/#/c/344/2/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
File asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java:

Line 279:                         index = (ILSMIndex) indexLifecycleManager.getIndex(localResource.getResourceName());
Won't this act differently now, if the local resource doesn't exist? If the resource doesn't exist, the map will return null, and you'll get a NPE when calling localResource.getName(), instead of having index get set to null.


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Jenkins (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Jenkins has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/186/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Young-Seok Kim (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Young-Seok Kim has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 5: Code-Review+1

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Jenkins (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Jenkins has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/224/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1053: Allow lazy loading for persistent local reso...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1053: Allow lazy loading for persistent local resources
......................................................................


Patch Set 11: Verified-1

Build Failed 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/274/ : ABORTED

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 11
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Till Westmann (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Till Westmann has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 7:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/344/7/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
File asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java:

Line 77:         DatasetInfo dsInfo = datasetInfos.get(getDIDfromResourceName(resourceName));
> Yes, it would. It will just cause an extra call to getResourceIDfromResourc
Depends on how often we call this. But as this is in the DatasetLifecycleManager I think that we don't call it too often, and so reducing the redundancy in the code would be worth it.


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Murtadha Hubail (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Hello Ian Maxon, Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/344

to look at the new patch set (#3).

Change subject: Allow lazy loading for persistent local resources
......................................................................

Allow lazy loading for persistent local resources

- Allow indexes to be accessed by name.
- Allow lazy loading for persistent local resources.
- Caching for local resources.
- Fix server start timing issue.

Change-Id: I48b9260a3280750145f6ddb3783673a299055910
---
M asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
M asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
M asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java
M asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
M asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
M asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
M asterix-transactions/pom.xml
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/TestRuntimeContextProvider.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
15 files changed, 454 insertions(+), 321 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/44/344/3
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: ASTERIXDB-1053: Allow lazy loading for persistent local reso...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1053: Allow lazy loading for persistent local resources
......................................................................


Patch Set 9: Verified-1

Build Failed 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/267/ : FAILURE

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 9
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1053: Allow lazy loading for persistent local reso...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1053: Allow lazy loading for persistent local resources
......................................................................


Patch Set 9:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/267/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 9
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1053: Allow lazy loading for persistent local reso...

Posted by "Murtadha Hubail (Code Review)" <do...@asterixdb.incubator.apache.org>.
Murtadha Hubail has submitted this change and it was merged.

Change subject: ASTERIXDB-1053: Allow lazy loading for persistent local resources
......................................................................


ASTERIXDB-1053: Allow lazy loading for persistent local resources

- Allow indexes to be accessed by name.
- Allow lazy loading for persistent local resources.
- Caching for local resources.

Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Reviewed-on: https://asterix-gerrit.ics.uci.edu/344
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Yingyi Bu <bu...@gmail.com>
---
M asterix-app/src/main/java/org/apache/asterix/api/common/AsterixAppRuntimeContext.java
M asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
M asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
M asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
M asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
M asterix-transactions/pom.xml
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/TestRuntimeContextProvider.java
M asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/RecoveryManager.java
16 files changed, 383 insertions(+), 327 deletions(-)

Approvals:
  Yingyi Bu: Looks good to me, approved
  Jenkins: Verified



diff --git a/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixAppRuntimeContext.java b/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixAppRuntimeContext.java
index 13d7e21..35afd8b 100644
--- a/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixAppRuntimeContext.java
+++ b/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixAppRuntimeContext.java
@@ -96,7 +96,6 @@
     private AsterixTransactionProperties txnProperties;
     private AsterixFeedProperties feedProperties;
 
-
     private AsterixThreadExecutor threadExecutor;
     private DatasetLifecycleManager indexLifecycleManager;
     private IFileMapManager fileMapManager;
@@ -140,7 +139,7 @@
         metadataMergePolicyFactory = new PrefixMergePolicyFactory();
 
         ILocalResourceRepositoryFactory persistentLocalResourceRepositoryFactory = new PersistentLocalResourceRepositoryFactory(
-                ioManager);
+                ioManager, ncApplicationContext.getNodeId());
         localResourceRepository = (PersistentLocalResourceRepository) persistentLocalResourceRepositoryFactory
                 .createRepository();
         resourceIdFactory = (new ResourceIdFactoryProvider(localResourceRepository)).createResourceIdFactory();
@@ -149,10 +148,10 @@
                 this);
         txnSubsystem = new TransactionSubsystem(ncApplicationContext.getNodeId(), asterixAppRuntimeContextProvider,
                 txnProperties);
-        
+
         indexLifecycleManager = new DatasetLifecycleManager(storageProperties, localResourceRepository,
-                MetadataPrimaryIndexes.FIRST_AVAILABLE_USER_DATASET_ID,(LogManager)txnSubsystem.getLogManager());
-        
+                MetadataPrimaryIndexes.FIRST_AVAILABLE_USER_DATASET_ID, (LogManager) txnSubsystem.getLogManager());
+
         isShuttingdown = false;
 
         feedManager = new FeedManager(ncApplicationContext.getNodeId(), feedProperties,
@@ -243,7 +242,7 @@
     public AsterixExternalProperties getExternalProperties() {
         return externalProperties;
     }
-    
+
     @Override
     public AsterixFeedProperties getFeedProperties() {
         return feedProperties;
diff --git a/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java b/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
index 1a3953d..1ad34da 100644
--- a/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
+++ b/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
@@ -84,7 +84,7 @@
         MetadataManager.INSTANCE = new MetadataManager(proxy, metadataProperties);
 
         AsterixAppContextInfo.getInstance().getCCApplicationContext()
-        .addJobLifecycleListener(FeedLifecycleListener.INSTANCE);
+                .addJobLifecycleListener(FeedLifecycleListener.INSTANCE);
 
         AsterixExternalProperties externalProperties = AsterixAppContextInfo.getInstance().getExternalProperties();
         setupWebServer(externalProperties);
@@ -92,16 +92,17 @@
 
         setupJSONAPIServer(externalProperties);
         jsonAPIServer.start();
-        ExternalLibraryBootstrap.setUpExternaLibraries(false);
 
         setupFeedServer(externalProperties);
         feedServer.start();
-        centralFeedManager = CentralFeedManager.getInstance(); 
-        centralFeedManager.start();
 
         waitUntilServerStart(webServer);
         waitUntilServerStart(jsonAPIServer);
         waitUntilServerStart(feedServer);
+
+        ExternalLibraryBootstrap.setUpExternaLibraries(false);
+        centralFeedManager = CentralFeedManager.getInstance();
+        centralFeedManager.start();
 
         AsterixGlobalRecoveryManager.INSTANCE = new AsterixGlobalRecoveryManager(
                 (HyracksConnection) getNewHyracksClientConnection());
@@ -182,7 +183,7 @@
 
         feedServer.setHandler(context);
         context.addServlet(new ServletHolder(new FeedServlet()), "/");
-   
+
         // add paths here
     }
 }
\ No newline at end of file
diff --git a/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java b/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
index bb21465..63f862c 100644
--- a/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
+++ b/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
@@ -112,12 +112,6 @@
                 LOGGER.info("System is in a state: " + systemState);
             }
 
-            if (systemState != SystemState.NEW_UNIVERSE) {
-                PersistentLocalResourceRepository localResourceRepository = (PersistentLocalResourceRepository) runtimeContext
-                        .getLocalResourceRepository();
-                localResourceRepository.initialize(nodeId, null, false, runtimeContext.getResourceIdFactory());
-            }
-            
             if (systemState == SystemState.CORRUPTED) {
                 recoveryMgr.startRecovery(true);
             }
@@ -165,7 +159,7 @@
 
             PersistentLocalResourceRepository localResourceRepository = (PersistentLocalResourceRepository) runtimeContext
                     .getLocalResourceRepository();
-            localResourceRepository.initialize(nodeId, metadataProperties.getStores().get(nodeId)[0], true, null);
+            localResourceRepository.initialize(nodeId, metadataProperties.getStores().get(nodeId)[0]);
         }
 
         IAsterixStateProxy proxy = null;
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java b/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
index 9df4970..741e106 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
@@ -77,8 +77,15 @@
     }
 
     @Override
-    public synchronized IIndex getIndex(long resourceID) throws HyracksDataException {
-        DatasetInfo dsInfo = datasetInfos.get(getDIDfromRID(resourceID));
+    public synchronized IIndex getIndex(String resourceName) throws HyracksDataException {
+        int datasetID = getDIDfromResourceName(resourceName);
+        long resourceID = getResourceIDfromResourceName(resourceName);
+        return getIndex(datasetID, resourceID);
+    }
+
+    @Override
+    public synchronized IIndex getIndex(int datasetID, long resourceID) throws HyracksDataException {
+        DatasetInfo dsInfo = datasetInfos.get(datasetID);
         if (dsInfo == null) {
             return null;
         }
@@ -90,8 +97,9 @@
     }
 
     @Override
-    public synchronized void register(long resourceID, IIndex index) throws HyracksDataException {
-        int did = getDIDfromRID(resourceID);
+    public synchronized void register(String resourceName, IIndex index) throws HyracksDataException {
+        int did = getDIDfromResourceName(resourceName);
+        long resourceID = getResourceIDfromResourceName(resourceName);
         DatasetInfo dsInfo = datasetInfos.get(did);
         if (dsInfo == null) {
             dsInfo = getDatasetInfo(did);
@@ -106,17 +114,27 @@
         dsInfo.indexes.put(resourceID, new IndexInfo((ILSMIndex) index));
     }
 
-    private int getDIDfromRID(long resourceID) throws HyracksDataException {
-        LocalResource lr = resourceRepository.getResourceById(resourceID);
+    private int getDIDfromResourceName(String resourceName) throws HyracksDataException {
+        LocalResource lr = resourceRepository.getResourceByName(resourceName);
         if (lr == null) {
             return -1;
         }
         return ((ILocalResourceMetadata) lr.getResourceObject()).getDatasetID();
     }
 
+    private long getResourceIDfromResourceName(String resourceName) throws HyracksDataException {
+        LocalResource lr = resourceRepository.getResourceByName(resourceName);
+        if (lr == null) {
+            return -1;
+        }
+        return lr.getResourceId();
+    }
+
     @Override
-    public synchronized void unregister(long resourceID) throws HyracksDataException {
-        int did = getDIDfromRID(resourceID);
+    public synchronized void unregister(String resourceName) throws HyracksDataException {
+        int did = getDIDfromResourceName(resourceName);
+        long resourceID = getResourceIDfromResourceName(resourceName);
+
         DatasetInfo dsInfo = datasetInfos.get(did);
         IndexInfo iInfo = dsInfo.indexes.get(resourceID);
 
@@ -155,20 +173,15 @@
 
         dsInfo.indexes.remove(resourceID);
         if (dsInfo.referenceCount == 0 && dsInfo.isOpen && dsInfo.indexes.isEmpty() && !dsInfo.isExternal) {
-            List<IVirtualBufferCache> vbcs = getVirtualBufferCaches(did);
-            assert vbcs != null;
-            for (IVirtualBufferCache vbc : vbcs) {
-                used -= (vbc.getNumPages() * vbc.getPageSize());
-            }
-            datasetInfos.remove(did);
-            datasetVirtualBufferCaches.remove(did);
-            datasetOpTrackers.remove(did);
+            removeDatasetFromCache(dsInfo.datasetID);
         }
     }
 
     @Override
-    public synchronized void open(long resourceID) throws HyracksDataException {
-        int did = getDIDfromRID(resourceID);
+    public synchronized void open(String resourceName) throws HyracksDataException {
+        int did = getDIDfromResourceName(resourceName);
+        long resourceID = getResourceIDfromResourceName(resourceName);
+
         DatasetInfo dsInfo = datasetInfos.get(did);
         if (dsInfo == null || !dsInfo.isRegistered) {
             throw new HyracksDataException("Failed to open index with resource ID " + resourceID
@@ -260,8 +273,10 @@
     }
 
     @Override
-    public synchronized void close(long resourceID) throws HyracksDataException {
-        int did = getDIDfromRID(resourceID);
+    public synchronized void close(String resourceName) throws HyracksDataException {
+        int did = getDIDfromResourceName(resourceName);
+        long resourceID = getResourceIDfromResourceName(resourceName);
+
         DatasetInfo dsInfo = datasetInfos.get(did);
         if (dsInfo == null) {
             throw new HyracksDataException("No index found with resourceID " + resourceID);
@@ -304,6 +319,17 @@
             }
             return vbcs;
         }
+    }
+
+    private void removeDatasetFromCache(int datasetID) {
+        List<IVirtualBufferCache> vbcs = getVirtualBufferCaches(datasetID);
+        assert vbcs != null;
+        for (IVirtualBufferCache vbc : vbcs) {
+            used -= (vbc.getNumPages() * vbc.getPageSize());
+        }
+        datasetInfos.remove(datasetID);
+        datasetVirtualBufferCaches.remove(datasetID);
+        datasetOpTrackers.remove(datasetID);
     }
 
     public ILSMOperationTracker getOperationTracker(int datasetID) {
@@ -505,7 +531,6 @@
         }
 
         if (asyncFlush) {
-
             for (IndexInfo iInfo : dsInfo.indexes.values()) {
                 ILSMIndexAccessor accessor = iInfo.index.createAccessor(NoOpOperationCallback.INSTANCE,
                         NoOpOperationCallback.INSTANCE);
@@ -552,10 +577,7 @@
         }
         dsInfo.isOpen = false;
 
-        List<IVirtualBufferCache> vbcs = getVirtualBufferCaches(dsInfo.datasetID);
-        for (IVirtualBufferCache vbc : vbcs) {
-            used -= vbc.getNumPages() * vbc.getPageSize();
-        }
+        removeDatasetFromCache(dsInfo.datasetID);
     }
 
     @Override
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java b/asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
index 41f86b4..a5dcf18 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
@@ -58,7 +58,7 @@
         AbstractLSMIndex lsmIndex = (AbstractLSMIndex) indexHelper.getIndexInstance();
         try {
             modCallback = opDesc.getModificationOpCallbackFactory().createModificationOperationCallback(
-                    indexHelper.getResourceID(), lsmIndex, ctx);
+                    indexHelper.getResourceName(), indexHelper.getResourceID(), lsmIndex, ctx);
             indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
             ITupleFilterFactory tupleFilterFactory = opDesc.getTupleFilterFactory();
             if (tupleFilterFactory != null) {
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
index 2b1d678..d600f57 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
@@ -284,9 +284,10 @@
     private void insertTupleIntoIndex(JobId jobId, IMetadataIndex metadataIndex, ITupleReference tuple)
             throws Exception {
         long resourceID = metadataIndex.getResourceID();
-        ILSMIndex lsmIndex = (ILSMIndex) indexLifecycleManager.getIndex(resourceID);
+        String resourceName = metadataIndex.getFile().toString();
+        ILSMIndex lsmIndex = (ILSMIndex) indexLifecycleManager.getIndex(resourceName);
         try {
-            indexLifecycleManager.open(resourceID);
+            indexLifecycleManager.open(resourceName);
 
             // prepare a Callback for logging
             IModificationOperationCallback modCallback = createIndexModificationCallback(jobId, resourceID,
@@ -307,7 +308,7 @@
         } catch (Exception e) {
             throw e;
         } finally {
-            indexLifecycleManager.close(resourceID);
+            indexLifecycleManager.close(resourceName);
         }
     }
 
@@ -633,9 +634,10 @@
     private void deleteTupleFromIndex(JobId jobId, IMetadataIndex metadataIndex, ITupleReference tuple)
             throws Exception {
         long resourceID = metadataIndex.getResourceID();
-        ILSMIndex lsmIndex = (ILSMIndex) indexLifecycleManager.getIndex(resourceID);
+        String resourceName = metadataIndex.getFile().toString();
+        ILSMIndex lsmIndex = (ILSMIndex) indexLifecycleManager.getIndex(resourceName);
         try {
-            indexLifecycleManager.open(resourceID);
+            indexLifecycleManager.open(resourceName);
             // prepare a Callback for logging
             IModificationOperationCallback modCallback = createIndexModificationCallback(jobId, resourceID,
                     metadataIndex, lsmIndex, IndexOperation.DELETE);
@@ -653,7 +655,7 @@
         } catch (Exception e) {
             throw e;
         } finally {
-            indexLifecycleManager.close(resourceID);
+            indexLifecycleManager.close(resourceName);
         }
     }
 
@@ -966,9 +968,9 @@
         StringBuilder sb = new StringBuilder();
         try {
             IMetadataIndex index = MetadataPrimaryIndexes.DATAVERSE_DATASET;
-            long resourceID = index.getResourceID();
-            IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
-            indexLifecycleManager.open(resourceID);
+            String resourceName = index.getFile().toString();
+            IIndex indexInstance = indexLifecycleManager.getIndex(resourceName);
+            indexLifecycleManager.open(resourceName);
             IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
                     NoOpOperationCallback.INSTANCE);
             ITreeIndexCursor rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor(false);
@@ -986,12 +988,11 @@
             } finally {
                 rangeCursor.close();
             }
-            indexLifecycleManager.close(resourceID);
+            indexLifecycleManager.close(resourceName);
 
             index = MetadataPrimaryIndexes.DATASET_DATASET;
-            resourceID = index.getResourceID();
-            indexInstance = indexLifecycleManager.getIndex(resourceID);
-            indexLifecycleManager.open(resourceID);
+            indexInstance = indexLifecycleManager.getIndex(resourceName);
+            indexLifecycleManager.open(resourceName);
             indexAccessor = indexInstance
                     .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
             rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor(false);
@@ -1009,12 +1010,11 @@
             } finally {
                 rangeCursor.close();
             }
-            indexLifecycleManager.close(resourceID);
+            indexLifecycleManager.close(resourceName);
 
             index = MetadataPrimaryIndexes.INDEX_DATASET;
-            resourceID = index.getResourceID();
-            indexInstance = indexLifecycleManager.getIndex(resourceID);
-            indexLifecycleManager.open(resourceID);
+            indexInstance = indexLifecycleManager.getIndex(resourceName);
+            indexLifecycleManager.open(resourceName);
             indexAccessor = indexInstance
                     .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
             rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor(false);
@@ -1033,7 +1033,7 @@
             } finally {
                 rangeCursor.close();
             }
-            indexLifecycleManager.close(resourceID);
+            indexLifecycleManager.close(resourceName);
         } catch (Exception e) {
             e.printStackTrace();
         }
@@ -1043,9 +1043,9 @@
     private <ResultType> void searchIndex(JobId jobId, IMetadataIndex index, ITupleReference searchKey,
             IValueExtractor<ResultType> valueExtractor, List<ResultType> results) throws Exception {
         IBinaryComparatorFactory[] comparatorFactories = index.getKeyBinaryComparatorFactory();
-        long resourceID = index.getResourceID();
-        IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
-        indexLifecycleManager.open(resourceID);
+        String resourceName = index.getFile().toString();
+        IIndex indexInstance = indexLifecycleManager.getIndex(resourceName);
+        indexLifecycleManager.open(resourceName);
         IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
                 NoOpOperationCallback.INSTANCE);
         ITreeIndexCursor rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor(false);
@@ -1074,16 +1074,16 @@
         } finally {
             rangeCursor.close();
         }
-        indexLifecycleManager.close(resourceID);
+        indexLifecycleManager.close(resourceName);
     }
 
     @Override
     public void initializeDatasetIdFactory(JobId jobId) throws MetadataException, RemoteException {
         int mostRecentDatasetId = MetadataPrimaryIndexes.FIRST_AVAILABLE_USER_DATASET_ID;
-        long resourceID = MetadataPrimaryIndexes.DATASET_DATASET.getResourceID();
         try {
-            IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
-            indexLifecycleManager.open(resourceID);
+            String resourceName = MetadataPrimaryIndexes.DATASET_DATASET.getFile().toString();
+            IIndex indexInstance = indexLifecycleManager.getIndex(resourceName);
+            indexLifecycleManager.open(resourceName);
             try {
                 IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
                         NoOpOperationCallback.INSTANCE);
@@ -1110,7 +1110,7 @@
                     rangeCursor.close();
                 }
             } finally {
-                indexLifecycleManager.close(resourceID);
+                indexLifecycleManager.close(resourceName);
             }
 
         } catch (Exception e) {
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
index 83d6c8f..c94133a 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -420,11 +420,11 @@
                     localResourceMetadata, LocalResource.LSMBTreeResource);
             ILocalResourceFactory localResourceFactory = localResourceFactoryProvider.getLocalResourceFactory();
             localResourceRepository.insert(localResourceFactory.createLocalResource(resourceID, path, 0));
-            indexLifecycleManager.register(resourceID, lsmBtree);
+            indexLifecycleManager.register(path, lsmBtree);
         } else {
             final LocalResource resource = localResourceRepository.getResourceByName(path);
             resourceID = resource.getResourceId();
-            lsmBtree = (LSMBTree) indexLifecycleManager.getIndex(resourceID);
+            lsmBtree = (LSMBTree) indexLifecycleManager.getIndex(resource.getResourceName());
             if (lsmBtree == null) {
                 lsmBtree = LSMBTreeUtils.createLSMTree(
                         virtualBufferCaches,
@@ -439,7 +439,7 @@
                                 GlobalConfig.DEFAULT_COMPACTION_POLICY_PROPERTIES, indexLifecycleManager), opTracker,
                         runtimeContext.getLSMIOScheduler(), LSMBTreeIOOperationCallbackFactory.INSTANCE
                                 .createIOOperationCallback(), index.isPrimaryIndex(), null, null, null, null, true);
-                indexLifecycleManager.register(resourceID, lsmBtree);
+                indexLifecycleManager.register(path, lsmBtree);
             }
         }
 
diff --git a/asterix-transactions/pom.xml b/asterix-transactions/pom.xml
index 447c762..b9a26a0 100644
--- a/asterix-transactions/pom.xml
+++ b/asterix-transactions/pom.xml
@@ -109,13 +109,18 @@
 			<groupId>org.apache.hyracks</groupId>
 			<artifactId>hyracks-storage-am-lsm-invertedindex</artifactId>
 		</dependency>
-                <dependency>
-                        <groupId>org.apache.asterix</groupId>
-                        <artifactId>asterix-common</artifactId>
-                        <version>0.8.7-SNAPSHOT</version>
-                        <type>jar</type>
-                        <scope>compile</scope>
-                </dependency>
+        <dependency>
+            <groupId>org.apache.asterix</groupId>
+            <artifactId>asterix-common</artifactId>
+            <version>0.8.7-SNAPSHOT</version>
+            <type>jar</type>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+			<groupId>com.google.guava</groupId>
+			<artifactId>guava</artifactId>
+			<version>18.0</version>
+		</dependency>
 	</dependencies>
 
 </project>
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
index 21b90ed..4c2f25d 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
@@ -50,13 +50,13 @@
     }
 
     @Override
-    public IModificationOperationCallback createModificationOperationCallback(long resourceId, Object resource,
-            IHyracksTaskContext ctx) throws HyracksDataException {
+    public IModificationOperationCallback createModificationOperationCallback(String resourceName, long resourceId,
+            Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
 
         ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
                 .getIndexLifecycleManager();
-        ILSMIndex index = (ILSMIndex) indexLifeCycleManager.getIndex(resourceId);
+        ILSMIndex index = (ILSMIndex) indexLifeCycleManager.getIndex(resourceName);
         if (index == null) {
             throw new HyracksDataException("Index(id:" + resourceId + ") is not registered.");
         }
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
index df29491..9c897f2 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
@@ -47,13 +47,12 @@
     }
 
     @Override
-    public IModificationOperationCallback createModificationOperationCallback(long resourceId, Object resource,
-            IHyracksTaskContext ctx) throws HyracksDataException {
-
+    public IModificationOperationCallback createModificationOperationCallback(String resourceName, long resourceId,
+            Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
         ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
                 .getIndexLifecycleManager();
-        ILSMIndex index = (ILSMIndex) indexLifeCycleManager.getIndex(resourceId);
+        ILSMIndex index = (ILSMIndex) indexLifeCycleManager.getIndex(resourceName);
         if (index == null) {
             throw new HyracksDataException("Index(id:" + resourceId + ") is not registered.");
         }
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
index 5f9fd0e..4de0749 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
@@ -34,25 +34,26 @@
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 
-public class TempDatasetPrimaryIndexModificationOperationCallbackFactory extends AbstractOperationCallbackFactory implements
-        IModificationOperationCallbackFactory {
+public class TempDatasetPrimaryIndexModificationOperationCallbackFactory extends AbstractOperationCallbackFactory
+        implements IModificationOperationCallbackFactory {
 
     private static final long serialVersionUID = 1L;
     private final IndexOperation indexOp;
 
-    public TempDatasetPrimaryIndexModificationOperationCallbackFactory(JobId jobId, int datasetId, int[] primaryKeyFields,
-            ITransactionSubsystemProvider txnSubsystemProvider, IndexOperation indexOp, byte resourceType) {
+    public TempDatasetPrimaryIndexModificationOperationCallbackFactory(JobId jobId, int datasetId,
+            int[] primaryKeyFields, ITransactionSubsystemProvider txnSubsystemProvider, IndexOperation indexOp,
+            byte resourceType) {
         super(jobId, datasetId, primaryKeyFields, txnSubsystemProvider, resourceType);
         this.indexOp = indexOp;
     }
 
     @Override
-    public IModificationOperationCallback createModificationOperationCallback(long resourceId, Object resource,
-            IHyracksTaskContext ctx) throws HyracksDataException {
+    public IModificationOperationCallback createModificationOperationCallback(String resourceName, long resourceId,
+            Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
         ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
                 .getIndexLifecycleManager();
-        ILSMIndex index = (ILSMIndex) indexLifeCycleManager.getIndex(resourceId);
+        ILSMIndex index = (ILSMIndex) indexLifeCycleManager.getIndex(resourceName);
         if (index == null) {
             throw new HyracksDataException("Index(id:" + resourceId + ") is not registered.");
         }
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
index 8a51c20..c2f56a0 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
@@ -34,25 +34,26 @@
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 
-public class TempDatasetSecondaryIndexModificationOperationCallbackFactory extends AbstractOperationCallbackFactory implements
-        IModificationOperationCallbackFactory {
+public class TempDatasetSecondaryIndexModificationOperationCallbackFactory extends AbstractOperationCallbackFactory
+        implements IModificationOperationCallbackFactory {
 
     private static final long serialVersionUID = 1L;
     private final IndexOperation indexOp;
 
-    public TempDatasetSecondaryIndexModificationOperationCallbackFactory(JobId jobId, int datasetId, int[] primaryKeyFields,
-            ITransactionSubsystemProvider txnSubsystemProvider, IndexOperation indexOp, byte resourceType) {
+    public TempDatasetSecondaryIndexModificationOperationCallbackFactory(JobId jobId, int datasetId,
+            int[] primaryKeyFields, ITransactionSubsystemProvider txnSubsystemProvider, IndexOperation indexOp,
+            byte resourceType) {
         super(jobId, datasetId, primaryKeyFields, txnSubsystemProvider, resourceType);
         this.indexOp = indexOp;
     }
 
     @Override
-    public IModificationOperationCallback createModificationOperationCallback(long resourceId, Object resource,
-            IHyracksTaskContext ctx) throws HyracksDataException {
+    public IModificationOperationCallback createModificationOperationCallback(String resourceName, long resourceId,
+            Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
         ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
                 .getIndexLifecycleManager();
-        ILSMIndex index = (ILSMIndex) indexLifeCycleManager.getIndex(resourceId);
+        ILSMIndex index = (ILSMIndex) indexLifeCycleManager.getIndex(resourceName);
         if (index == null) {
             throw new HyracksDataException("Index(id:" + resourceId + ") is not registered.");
         }
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
index abc98c7..1eda9cc 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
@@ -25,10 +25,8 @@
 import java.io.IOException;
 import java.io.ObjectInputStream;
 import java.io.ObjectOutputStream;
-import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
@@ -36,7 +34,9 @@
 import org.apache.hyracks.api.io.IODeviceHandle;
 import org.apache.hyracks.storage.common.file.ILocalResourceRepository;
 import org.apache.hyracks.storage.common.file.LocalResource;
-import org.apache.hyracks.storage.common.file.ResourceIdFactory;
+
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
 
 public class PersistentLocalResourceRepository implements ILocalResourceRepository {
 
@@ -46,14 +46,14 @@
     private static final String ROOT_METADATA_FILE_NAME_PREFIX = ".asterix_root_metadata";
     private static final long ROOT_LOCAL_RESOURCE_ID = -4321;
     private static final String METADATA_FILE_NAME = ".metadata";
-    private Map<String, LocalResource> name2ResourceMap = new HashMap<String, LocalResource>();
-    private Map<Long, LocalResource> id2ResourceMap = new HashMap<Long, LocalResource>();
-    private final int numIODevices;
+    private final Cache<String, LocalResource> resourceCache;
+    private final String nodeId;
+    private static final int MAX_CACHED_RESOURCES = 1000;
 
-    public PersistentLocalResourceRepository(List<IODeviceHandle> devices) throws HyracksDataException {
-        numIODevices = devices.size();
-        this.mountPoints = new String[numIODevices];
-        for (int i = 0; i < numIODevices; i++) {
+    public PersistentLocalResourceRepository(List<IODeviceHandle> devices, String nodeId) throws HyracksDataException {
+        mountPoints = new String[devices.size()];
+        this.nodeId = nodeId;
+        for (int i = 0; i < mountPoints.length; i++) {
             String mountPoint = devices.get(i).getPath().getPath();
             File mountPointDir = new File(mountPoint);
             if (!mountPointDir.exists()) {
@@ -65,167 +65,89 @@
                 mountPoints[i] = new String(mountPoint);
             }
         }
+
+        resourceCache = CacheBuilder.newBuilder().maximumSize(MAX_CACHED_RESOURCES).build();
     }
 
     private String prepareRootMetaDataFileName(String mountPoint, String nodeId, int ioDeviceId) {
         return mountPoint + ROOT_METADATA_DIRECTORY + File.separator + nodeId + "_" + "iodevice" + ioDeviceId;
     }
 
-    public void initialize(String nodeId, String rootDir, boolean isNewUniverse, ResourceIdFactory resourceIdFactory)
-            throws HyracksDataException {
+    public void initialize(String nodeId, String rootDir) throws HyracksDataException {
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Initializing local resource repository ... ");
         }
 
-        if (isNewUniverse) {
-            //#. if the rootMetadataFile doesn't exist, create it and return.
-            for (int i = 0; i < numIODevices; i++) {
-                String rootMetadataFileName = prepareRootMetaDataFileName(mountPoints[i], nodeId, i) + File.separator
-                        + ROOT_METADATA_FILE_NAME_PREFIX;
-                File rootMetadataFile = new File(rootMetadataFileName);
-
-                File rootMetadataDir = new File(prepareRootMetaDataFileName(mountPoints[i], nodeId, i));
-                if (!rootMetadataDir.exists()) {
-                    boolean success = rootMetadataDir.mkdirs();
-                    if (!success) {
-                        throw new IllegalStateException(
-                                "Unable to create root metadata directory of PersistentLocalResourceRepository in "
-                                        + rootMetadataDir.getAbsolutePath());
-                    }
-                    if (LOGGER.isLoggable(Level.INFO)) {
-                        LOGGER.info("created the root-metadata-file's directory: " + rootMetadataDir.getAbsolutePath());
-                    }
-                }
-
-                rootMetadataFile.delete();
-                String mountedRootDir;
-                if (rootDir.startsWith(System.getProperty("file.separator"))) {
-                    mountedRootDir = new String(mountPoints[i]
-                            + rootDir.substring(System.getProperty("file.separator").length()));
-                } else {
-                    mountedRootDir = new String(mountPoints[i] + rootDir);
-                }
-                LocalResource rootLocalResource = new LocalResource(ROOT_LOCAL_RESOURCE_ID, rootMetadataFileName, 0, 0,
-                        mountedRootDir);
-                insert(rootLocalResource);
-                if (LOGGER.isLoggable(Level.INFO)) {
-                    LOGGER.info("created the root-metadata-file: " + rootMetadataFileName);
-                }
-
-                if (LOGGER.isLoggable(Level.INFO)) {
-                    LOGGER.info("Completed the initialization of the local resource repository");
-                }
-            }
-            return;
-        }
-
-        FilenameFilter filter = new FilenameFilter() {
-            public boolean accept(File dir, String name) {
-                if (name.equalsIgnoreCase(METADATA_FILE_NAME)) {
-                    return true;
-                } else {
-                    return false;
-                }
-            }
-        };
-
-        long maxResourceId = 0;
-        for (int i = 0; i < numIODevices; i++) {
+        //if the rootMetadataFile doesn't exist, create it.
+        for (int i = 0; i < mountPoints.length; i++) {
             String rootMetadataFileName = prepareRootMetaDataFileName(mountPoints[i], nodeId, i) + File.separator
                     + ROOT_METADATA_FILE_NAME_PREFIX;
             File rootMetadataFile = new File(rootMetadataFileName);
-            //#. if the rootMetadataFile exists, read it and set this.rootDir.
-            LocalResource rootLocalResource = readLocalResource(rootMetadataFile);
-            String mountedRootDir = (String) rootLocalResource.getResourceObject();
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("The root directory of the local resource repository is " + mountedRootDir);
-            }
 
-            //#. load all local resources. 
-            File rootDirFile = new File(mountedRootDir);
-            if (!rootDirFile.exists()) {
-                //rootDir may not exist if this node is not the metadata node and doesn't have any user data.
+            File rootMetadataDir = new File(prepareRootMetaDataFileName(mountPoints[i], nodeId, i));
+            if (!rootMetadataDir.exists()) {
+                boolean success = rootMetadataDir.mkdirs();
+                if (!success) {
+                    throw new IllegalStateException(
+                            "Unable to create root metadata directory of PersistentLocalResourceRepository in "
+                                    + rootMetadataDir.getAbsolutePath());
+                }
                 if (LOGGER.isLoggable(Level.INFO)) {
-                    LOGGER.info("The root directory of the local resource repository doesn't exist: there is no local resource.");
-                    LOGGER.info("Completed the initialization of the local resource repository");
+                    LOGGER.info("created the root-metadata-file's directory: " + rootMetadataDir.getAbsolutePath());
                 }
-                continue;
             }
 
-            File[] dataverseFileList = rootDirFile.listFiles();
-            if (dataverseFileList == null) {
-                throw new HyracksDataException("Metadata dataverse doesn't exist.");
+            rootMetadataFile.delete();
+            String mountedRootDir;
+            if (rootDir.startsWith(System.getProperty("file.separator"))) {
+                mountedRootDir = new String(mountPoints[i]
+                        + rootDir.substring(System.getProperty("file.separator").length()));
+            } else {
+                mountedRootDir = new String(mountPoints[i] + rootDir);
             }
-            for (File dataverseFile : dataverseFileList) {
-                if (dataverseFile.isDirectory()) {
-                    File[] indexFileList = dataverseFile.listFiles();
-                    if (indexFileList != null) {
-                        for (File indexFile : indexFileList) {
-                            if (indexFile.isDirectory()) {
-                                File[] ioDevicesList = indexFile.listFiles();
-                                if (ioDevicesList != null) {
-                                    for (File ioDeviceFile : ioDevicesList) {
-                                        if (ioDeviceFile.isDirectory()) {
-                                            File[] metadataFiles = ioDeviceFile.listFiles(filter);
-                                            if (metadataFiles != null) {
-                                                for (File metadataFile : metadataFiles) {
-                                                    LocalResource localResource = readLocalResource(metadataFile);
-                                                    id2ResourceMap.put(localResource.getResourceId(), localResource);
-                                                    name2ResourceMap
-                                                            .put(localResource.getResourceName(), localResource);
-                                                    maxResourceId = Math.max(localResource.getResourceId(),
-                                                            maxResourceId);
-                                                    if (LOGGER.isLoggable(Level.INFO)) {
-                                                        LOGGER.info("loaded local resource - [id: "
-                                                                + localResource.getResourceId() + ", name: "
-                                                                + localResource.getResourceName() + "]");
-                                                    }
-                                                }
-                                            }
-                                        }
-                                    }
-                                }
-                            }
-                        }
-                    }
-                }
+            LocalResource rootLocalResource = new LocalResource(ROOT_LOCAL_RESOURCE_ID, rootMetadataFileName, 0, 0,
+                    mountedRootDir);
+            insert(rootLocalResource);
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("created the root-metadata-file: " + rootMetadataFileName);
             }
         }
-        resourceIdFactory.initId(maxResourceId + 1);
+
         if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("The resource id factory is intialized with the value: " + (maxResourceId + 1));
             LOGGER.info("Completed the initialization of the local resource repository");
         }
     }
 
     @Override
-    public LocalResource getResourceById(long id) throws HyracksDataException {
-        return id2ResourceMap.get(id);
-    }
-
-    @Override
     public LocalResource getResourceByName(String name) throws HyracksDataException {
-        return name2ResourceMap.get(name);
+        LocalResource resource = resourceCache.getIfPresent(name);
+        if (resource == null) {
+            File resourceFile = getLocalResourceFileByName(name);
+            if (resourceFile.exists()) {
+                resource = readLocalResource(resourceFile);
+                resourceCache.put(name, resource);
+            }
+        }
+        return resource;
     }
 
     @Override
     public synchronized void insert(LocalResource resource) throws HyracksDataException {
-        long id = resource.getResourceId();
+        File resourceFile = new File(getFileName(resource.getResourceName(), resource.getResourceId()));
 
-        if (id2ResourceMap.containsKey(id)) {
+        if (resourceFile.exists()) {
             throw new HyracksDataException("Duplicate resource");
         }
 
         if (resource.getResourceId() != ROOT_LOCAL_RESOURCE_ID) {
-            id2ResourceMap.put(id, resource);
-            name2ResourceMap.put(resource.getResourceName(), resource);
+            resourceCache.put(resource.getResourceName(), resource);
         }
 
         FileOutputStream fos = null;
         ObjectOutputStream oosToFos = null;
 
         try {
-            fos = new FileOutputStream(getFileName(resource.getResourceName(), resource.getResourceId()));
+            fos = new FileOutputStream(resourceFile);
             oosToFos = new ObjectOutputStream(fos);
             oosToFos.writeObject(resource);
             oosToFos.flush();
@@ -250,40 +172,133 @@
     }
 
     @Override
-    public synchronized void deleteResourceById(long id) throws HyracksDataException {
-        LocalResource resource = id2ResourceMap.get(id);
-        if (resource == null) {
-            throw new HyracksDataException("Resource doesn't exist");
-        }
-        id2ResourceMap.remove(id);
-        name2ResourceMap.remove(resource.getResourceName());
-        File file = new File(getFileName(resource.getResourceName(), resource.getResourceId()));
-        file.delete();
-    }
-
-    @Override
     public synchronized void deleteResourceByName(String name) throws HyracksDataException {
-        LocalResource resource = name2ResourceMap.get(name);
-        if (resource == null) {
+        File resourceFile = getLocalResourceFileByName(name);
+        if (resourceFile.exists()) {
+            resourceFile.delete();
+            resourceCache.invalidate(name);
+        } else {
             throw new HyracksDataException("Resource doesn't exist");
         }
-        id2ResourceMap.remove(resource.getResourceId());
-        name2ResourceMap.remove(name);
-        File file = new File(getFileName(resource.getResourceName(), resource.getResourceId()));
-        file.delete();
+    }
+
+    private static File getLocalResourceFileByName(String resourceName) {
+        return new File(resourceName + File.separator + METADATA_FILE_NAME);
+    }
+
+    public HashMap<Long, LocalResource> loadAndGetAllResources() throws HyracksDataException {
+        //TODO During recovery, the memory usage currently is proportional to the number of resources available.
+        //This could be fixed by traversing all resources on disk until the required resource is found.
+        HashMap<Long, LocalResource> resourcesMap = new HashMap<Long, LocalResource>();
+
+        for (int i = 0; i < mountPoints.length; i++) {
+            String rootMetadataFileName = prepareRootMetaDataFileName(mountPoints[i], nodeId, i) + File.separator
+                    + ROOT_METADATA_FILE_NAME_PREFIX;
+            File rootMetadataFile = new File(rootMetadataFileName);
+            if (!rootMetadataFile.exists()) {
+                continue;
+            }
+            //if the rootMetadataFile exists, read it and set it as mounting point root
+            LocalResource rootLocalResource = readLocalResource(rootMetadataFile);
+            String mountedRootDir = (String) rootLocalResource.getResourceObject();
+
+            File rootDirFile = new File(mountedRootDir);
+            if (!rootDirFile.exists()) {
+                //rootDir may not exist if this node is not the metadata node and doesn't have any user data.
+                continue;
+            }
+
+            //load all local resources.
+            File[] dataverseFileList = rootDirFile.listFiles();
+            if (dataverseFileList != null) {
+                for (File dataverseFile : dataverseFileList) {
+                    if (dataverseFile.isDirectory()) {
+                        File[] indexFileList = dataverseFile.listFiles();
+                        if (indexFileList != null) {
+                            for (File indexFile : indexFileList) {
+                                if (indexFile.isDirectory()) {
+                                    File[] ioDevicesList = indexFile.listFiles();
+                                    if (ioDevicesList != null) {
+                                        for (File ioDeviceFile : ioDevicesList) {
+                                            if (ioDeviceFile.isDirectory()) {
+                                                File[] metadataFiles = ioDeviceFile.listFiles(METADATA_FILES_FILTER);
+                                                if (metadataFiles != null) {
+                                                    for (File metadataFile : metadataFiles) {
+                                                        LocalResource localResource = readLocalResource(metadataFile);
+                                                        resourcesMap.put(localResource.getResourceId(), localResource);
+                                                    }
+                                                }
+                                            }
+                                        }
+                                    }
+                                }
+                            }
+                        }
+                    }
+                }
+            }
+        }
+
+        return resourcesMap;
     }
 
     @Override
-    public List<LocalResource> getAllResources() throws HyracksDataException {
-        List<LocalResource> resources = new ArrayList<LocalResource>();
-        for (LocalResource resource : id2ResourceMap.values()) {
-            resources.add(resource);
+    public long getMaxResourceID() throws HyracksDataException {
+        long maxResourceId = 0;
+
+        for (int i = 0; i < mountPoints.length; i++) {
+            String rootMetadataFileName = prepareRootMetaDataFileName(mountPoints[i], nodeId, i) + File.separator
+                    + ROOT_METADATA_FILE_NAME_PREFIX;
+            File rootMetadataFile = new File(rootMetadataFileName);
+            if (!rootMetadataFile.exists()) {
+                continue;
+            }
+
+            //if the rootMetadataFile exists, read it and set it as mounting point root
+            LocalResource rootLocalResource = readLocalResource(rootMetadataFile);
+            String mountedRootDir = (String) rootLocalResource.getResourceObject();
+
+            File rootDirFile = new File(mountedRootDir);
+            if (!rootDirFile.exists()) {
+                continue;
+            }
+
+            //traverse all local resources.
+            File[] dataverseFileList = rootDirFile.listFiles();
+            if (dataverseFileList != null) {
+                for (File dataverseFile : dataverseFileList) {
+                    if (dataverseFile.isDirectory()) {
+                        File[] indexFileList = dataverseFile.listFiles();
+                        if (indexFileList != null) {
+                            for (File indexFile : indexFileList) {
+                                if (indexFile.isDirectory()) {
+                                    File[] ioDevicesList = indexFile.listFiles();
+                                    if (ioDevicesList != null) {
+                                        for (File ioDeviceFile : ioDevicesList) {
+                                            if (ioDeviceFile.isDirectory()) {
+                                                File[] metadataFiles = ioDeviceFile.listFiles(METADATA_FILES_FILTER);
+                                                if (metadataFiles != null) {
+                                                    for (File metadataFile : metadataFiles) {
+                                                        LocalResource localResource = readLocalResource(metadataFile);
+                                                        maxResourceId = Math.max(maxResourceId,
+                                                                localResource.getResourceId());
+                                                    }
+                                                }
+                                            }
+                                        }
+                                    }
+                                }
+                            }
+                        }
+                    }
+                }
+            }
         }
-        return resources;
+
+        return maxResourceId;
     }
 
     private String getFileName(String baseDir, long resourceId) {
-
         if (resourceId == ROOT_LOCAL_RESOURCE_ID) {
             return baseDir;
         } else {
@@ -323,4 +338,14 @@
             }
         }
     }
+
+    private static final FilenameFilter METADATA_FILES_FILTER = new FilenameFilter() {
+        public boolean accept(File dir, String name) {
+            if (name.equalsIgnoreCase(METADATA_FILE_NAME)) {
+                return true;
+            } else {
+                return false;
+            }
+        }
+    };
 }
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
index 0638cae..b6bb7dc 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
@@ -24,14 +24,16 @@
 import org.apache.hyracks.storage.common.file.ILocalResourceRepositoryFactory;
 
 public class PersistentLocalResourceRepositoryFactory implements ILocalResourceRepositoryFactory {
-    private IIOManager ioManager;
+    private final IIOManager ioManager;
+    private final String nodeId;
 
-    public PersistentLocalResourceRepositoryFactory(IIOManager ioManager) {
+    public PersistentLocalResourceRepositoryFactory(IIOManager ioManager, String nodeId) {
         this.ioManager = ioManager;
+        this.nodeId = nodeId;
     }
 
     @Override
     public ILocalResourceRepository createRepository() throws HyracksDataException {
-        return new PersistentLocalResourceRepository(ioManager.getIODevices());
+        return new PersistentLocalResourceRepository(ioManager.getIODevices(), nodeId);
     }
 }
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/TestRuntimeContextProvider.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/TestRuntimeContextProvider.java
index 730116a..f6880db 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/TestRuntimeContextProvider.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/TestRuntimeContextProvider.java
@@ -39,9 +39,9 @@
 
 class TestRuntimeContextProvider implements IAsterixAppRuntimeContextProvider {
 
-    AsterixThreadExecutor ate = new AsterixThreadExecutor(Executors.defaultThreadFactory());    
+    AsterixThreadExecutor ate = new AsterixThreadExecutor(Executors.defaultThreadFactory());
     IIndexLifecycleManager ilm = new IndexLifecycleManager();
-    
+
     @Override
     public AsterixThreadExecutor getThreadExecutor() {
         return ate;
@@ -108,36 +108,39 @@
     }
 
     static class IndexLifecycleManager implements IIndexLifecycleManager {
-
-        @Override
-        public IIndex getIndex(long resourceID) throws HyracksDataException {
-            throw new UnsupportedOperationException();
-        }
-
-        @Override
-        public void register(long resourceID, IIndex index) throws HyracksDataException {
-            throw new UnsupportedOperationException();
-        }
-
-        @Override
-        public void unregister(long resourceID) throws HyracksDataException {
-            throw new UnsupportedOperationException();
-        }
-
-        @Override
-        public void open(long resourceID) throws HyracksDataException {
-            throw new UnsupportedOperationException();
-        }
-
-        @Override
-        public void close(long resourceID) throws HyracksDataException {
-            throw new UnsupportedOperationException();
-        }
-
         @Override
         public List<IIndex> getOpenIndexes() {
             throw new UnsupportedOperationException();
         }
-        
+
+        @Override
+        public void register(String resourceName, IIndex index) throws HyracksDataException {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public void open(String resourceName) throws HyracksDataException {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public void close(String resourceName) throws HyracksDataException {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public IIndex getIndex(String resourceName) throws HyracksDataException {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public void unregister(String resourceName) throws HyracksDataException {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public IIndex getIndex(int datasetID, long resourceID) throws HyracksDataException {
+            throw new UnsupportedOperationException();
+        }
     }
 }
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/RecoveryManager.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/RecoveryManager.java
index f59bc84..82ad32d 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/RecoveryManager.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/RecoveryManager.java
@@ -51,6 +51,7 @@
 import org.apache.asterix.common.transactions.IRecoveryManager;
 import org.apache.asterix.common.transactions.ITransactionContext;
 import org.apache.asterix.common.transactions.LogType;
+import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
 import org.apache.asterix.transaction.management.service.logging.LogManager;
 import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants;
 import org.apache.asterix.transaction.management.service.transaction.TransactionManager;
@@ -82,7 +83,6 @@
     private final LogManager logMgr;
     private final int checkpointHistory;
     private final long SHARP_CHECKPOINT_LSN = -1;
-
     /**
      * A file at a known location that contains the LSN of the last log record
      * traversed doing a successful checkpoint.
@@ -249,6 +249,8 @@
         IIndexLifecycleManager indexLifecycleManager = appRuntimeContext.getIndexLifecycleManager();
         ILocalResourceRepository localResourceRepository = appRuntimeContext.getLocalResourceRepository();
 
+        Map<Long, LocalResource> resourcesMap = ((PersistentLocalResourceRepository) localResourceRepository)
+                .loadAndGetAllResources();
         //#. set log reader to the lowWaterMarkLsn again.
         logReader.initializeScan(lowWaterMarkLSN);
         logRecord = logReader.next();
@@ -273,39 +275,38 @@
                     }
                     if (foundWinner) {
                         resourceId = logRecord.getResourceId();
-                        localResource = localResourceRepository.getResourceById(resourceId);
+                        localResource = resourcesMap.get(resourceId);
+
+                        /*******************************************************************
+                         * [Notice]
+                         * -> Issue
+                         * Delete index may cause a problem during redo.
+                         * The index operation to be redone couldn't be redone because the corresponding index
+                         * may not exist in NC due to the possible index drop DDL operation.
+                         * -> Approach
+                         * Avoid the problem during redo.
+                         * More specifically, the problem will be detected when the localResource of
+                         * the corresponding index is retrieved, which will end up with 'null'.
+                         * If null is returned, then just go and process the next
+                         * log record.
+                         *******************************************************************/
+                        if (localResource == null) {
+                            logRecord = logReader.next();
+                            continue;
+                        }
+                        /*******************************************************************/
 
                         //get index instance from IndexLifeCycleManager
                         //if index is not registered into IndexLifeCycleManager,
                         //create the index using LocalMetadata stored in LocalResourceRepository
-                        index = (ILSMIndex) indexLifecycleManager.getIndex(resourceId);
+                        index = (ILSMIndex) indexLifecycleManager.getIndex(localResource.getResourceName());
                         if (index == null) {
-
-                            /*******************************************************************
-                             * [Notice]
-                             * -> Issue
-                             * Delete index may cause a problem during redo.
-                             * The index operation to be redone couldn't be redone because the corresponding index
-                             * may not exist in NC due to the possible index drop DDL operation.
-                             * -> Approach
-                             * Avoid the problem during redo.
-                             * More specifically, the problem will be detected when the localResource of
-                             * the corresponding index is retrieved, which will end up with 'null' return from
-                             * localResourceRepository. If null is returned, then just go and process the next
-                             * log record.
-                             *******************************************************************/
-                            if (localResource == null) {
-                                logRecord = logReader.next();
-                                continue;
-                            }
-                            /*******************************************************************/
-
                             //#. create index instance and register to indexLifeCycleManager
                             localResourceMetadata = (ILocalResourceMetadata) localResource.getResourceObject();
                             index = localResourceMetadata.createIndexInstance(appRuntimeContext,
                                     localResource.getResourceName(), localResource.getPartition());
-                            indexLifecycleManager.register(resourceId, index);
-                            indexLifecycleManager.open(resourceId);
+                            indexLifecycleManager.register(localResource.getResourceName(), index);
+                            indexLifecycleManager.open(localResource.getResourceName());
 
                             //#. get maxDiskLastLSN
                             ILSMIndex lsmIndex = (ILSMIndex) index;
@@ -342,7 +343,7 @@
         //close all indexes
         Set<Long> resourceIdList = resourceId2MaxLSNMap.keySet();
         for (long r : resourceIdList) {
-            indexLifecycleManager.close(r);
+            indexLifecycleManager.close(resourcesMap.get(r).getResourceName());
         }
 
         logReader.close();
@@ -356,11 +357,12 @@
     }
 
     @Override
-    public synchronized long checkpoint(boolean isSharpCheckpoint, long nonSharpCheckpointTargetLSN) throws ACIDException, HyracksDataException {
+    public synchronized long checkpoint(boolean isSharpCheckpoint, long nonSharpCheckpointTargetLSN)
+            throws ACIDException, HyracksDataException {
 
         long minMCTFirstLSN;
         boolean nonSharpCheckpointSucceeded = false;
-        
+
         if (isSharpCheckpoint && LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Starting sharp checkpoint ... ");
         }
@@ -372,7 +374,8 @@
         //   right after the new checkpoint file is written.
         File[] prevCheckpointFiles = getPreviousCheckpointFiles();
 
-        DatasetLifecycleManager datasetLifecycleManager = (DatasetLifecycleManager)txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager();
+        DatasetLifecycleManager datasetLifecycleManager = (DatasetLifecycleManager) txnSubsystem
+                .getAsterixAppRuntimeContextProvider().getIndexLifecycleManager();
         //#. flush all in-memory components if it is the sharp checkpoint
         if (isSharpCheckpoint) {
 
@@ -382,11 +385,10 @@
         } else {
 
             minMCTFirstLSN = getMinFirstLSN();
-            
-            if(minMCTFirstLSN >= nonSharpCheckpointTargetLSN){
+
+            if (minMCTFirstLSN >= nonSharpCheckpointTargetLSN) {
                 nonSharpCheckpointSucceeded = true;
-            }
-            else{
+            } else {
                 //flush datasets with indexes behind target checkpoint LSN
                 datasetLifecycleManager.scheduleAsyncFlushForLaggingDatasets(nonSharpCheckpointTargetLSN);
             }
@@ -439,10 +441,10 @@
             }
         }
 
-        if(nonSharpCheckpointSucceeded){
+        if (nonSharpCheckpointSucceeded) {
             logMgr.deleteOldLogFiles(minMCTFirstLSN);
         }
-        
+
         if (isSharpCheckpoint && LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Completed sharp checkpoint.");
         }
@@ -451,9 +453,9 @@
         return minMCTFirstLSN;
     }
 
-    public long getMinFirstLSN() throws HyracksDataException
-    {
-        IIndexLifecycleManager indexLifecycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager();
+    public long getMinFirstLSN() throws HyracksDataException {
+        IIndexLifecycleManager indexLifecycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
+                .getIndexLifecycleManager();
         List<IIndex> openIndexList = indexLifecycleManager.getOpenIndexes();
         long firstLSN;
         //the min first lsn can only be the current append or smaller
@@ -463,8 +465,9 @@
 
             for (IIndex index : openIndexList) {
 
-                AbstractLSMIOOperationCallback ioCallback =  (AbstractLSMIOOperationCallback)((ILSMIndex) index).getIOOperationCallback();
-                if(!((AbstractLSMIndex)index).isCurrentMutableComponentEmpty() || ioCallback.hasPendingFlush()){
+                AbstractLSMIOOperationCallback ioCallback = (AbstractLSMIOOperationCallback) ((ILSMIndex) index)
+                        .getIOOperationCallback();
+                if (!((AbstractLSMIndex) index).isCurrentMutableComponentEmpty() || ioCallback.hasPendingFlush()) {
                     firstLSN = ioCallback.getFirstLSN();
                     minFirstLSN = Math.min(minFirstLSN, firstLSN);
                 }
@@ -594,6 +597,7 @@
         ILogReader logReader = logMgr.getLogReader(false);
         logReader.initializeScan(firstLSN);
         ILogRecord logRecord = null;
+
         while (currentLSN < lastLSN) {
             logRecord = logReader.next();
             if (logRecord == null) {
@@ -710,7 +714,7 @@
     private void undo(ILogRecord logRecord) {
         try {
             ILSMIndex index = (ILSMIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
-                    .getIndex(logRecord.getResourceId());
+                    .getIndex(logRecord.getDatasetId(), logRecord.getResourceId());
             ILSMIndexAccessor indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE,
                     NoOpOperationCallback.INSTANCE);
             if (logRecord.getNewOp() == IndexOperation.INSERT.ordinal()) {
@@ -728,7 +732,7 @@
     private void redo(ILogRecord logRecord) {
         try {
             ILSMIndex index = (ILSMIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
-                    .getIndex(logRecord.getResourceId());
+                    .getIndex(logRecord.getDatasetId(), logRecord.getResourceId());
             ILSMIndexAccessor indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE,
                     NoOpOperationCallback.INSTANCE);
             if (logRecord.getNewOp() == IndexOperation.INSERT.ordinal()) {

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: merged
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 12
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Jenkins (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Jenkins has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 7: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/225/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Murtadha Hubail (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Hello Ian Maxon, Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/344

to look at the new patch set (#4).

Change subject: Allow lazy loading for persistent local resources
......................................................................

Allow lazy loading for persistent local resources

- Allow indexes to be accessed by name.
- Allow lazy loading for persistent local resources.
- Caching for local resources.
- Fix server start timing issue.

Change-Id: I48b9260a3280750145f6ddb3783673a299055910
---
M asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
M asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
M asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java
M asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
M asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
M asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
M asterix-transactions/pom.xml
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/TestRuntimeContextProvider.java
M asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
15 files changed, 452 insertions(+), 322 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/44/344/4
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Yingyi Bu (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Yingyi Bu has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 5:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/344/5/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
File asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java:

Line 205:     public HashMap<Long, LocalResource> loadAndGetAllResources() throws HyracksDataException {
> Yes, we would have to persist a new file at the root of every IO Device and
IMO, "slower" is better than "crashing when too many resources"...
During recovery,  the buffer cache and in-memory components of indexes also consume memory.

Since this CL is monotonically better than the current master, I'm OK for merging it back. I propose two minor changes:

1.  Add a TODO comment to say that during recovery, the memory usage currently is proportional to the number of created indexes (this includes deleted indexes, I guess, because their directories are not deleted and resource ids are not released.) and should be fixed.  (Or create an issue for that.) 

2.  In ResourceIdFactoryProvider,  call getMaxResourceId() instead of getAllResources().   In initialize(...),  call getMaxResourceId() instead of loadAndGetAllResources().   In getMaxResourceId(), it just iterates over resources without loading them into a map.


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Allow lazy loading for persistent local resources

Posted by "Yingyi Bu (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Yingyi Bu has posted comments on this change.

Change subject: Allow lazy loading for persistent local resources
......................................................................


Patch Set 7: Code-Review+1

(1 comment)

LGTM.  Just a minor comment.

https://asterix-gerrit.ics.uci.edu/#/c/344/7/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
File asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java:

Line 75:         throw new UnsupportedOperationException();
Do we still need this interface method because it seems no one really implements that?


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/344
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I48b9260a3280750145f6ddb3783673a299055910
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes