You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by mh...@apache.org on 2016/01/01 02:16:17 UTC

incubator-asterixdb-hyracks git commit: Make Index Path Relative to NC IO Devices

Repository: incubator-asterixdb-hyracks
Updated Branches:
  refs/heads/master 7dd479923 -> 84790762f


Make Index Path Relative to NC IO Devices

This change includes the following:
- Store relative index path in persisted local resource as resource name.
- Fetch index absolute path based on FileSplit IO device and index relative path.
This change will enable a node to perform operations on an index that was created
on another node by finding the absolve path based on the node IO deviec and
the index relative path. This is needed in case a node leaves the cluster and
a replica node is assigned to take over and recover the index partitions
that belonged to the failed node.

Change-Id: I09fabd097b8a995ca959fe60286acdb43edfcfa1
Reviewed-on: https://asterix-gerrit.ics.uci.edu/563
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Yingyi Bu <bu...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/commit/84790762
Tree: http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/tree/84790762
Diff: http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/diff/84790762

Branch: refs/heads/master
Commit: 84790762fb7c06ed5cdbb83a51c872ccb7ac6b77
Parents: 7dd4799
Author: Murtadha Hubail <mh...@uci.edu>
Authored: Thu Dec 31 08:30:25 2015 -0800
Committer: Murtadha Hubail <hu...@gmail.com>
Committed: Thu Dec 31 17:11:32 2015 -0800

----------------------------------------------------------------------
 .../org/apache/hyracks/api/io/IIOManager.java   |  7 +++
 .../apache/hyracks/control/nc/io/IOManager.java |  6 +++
 .../hyracks/dataflow/std/file/FileSplit.java    | 11 ++++-
 .../am/common/api/IIndexDataflowHelper.java     |  2 +-
 .../am/common/api/IIndexLifecycleManager.java   | 10 ++--
 .../IModificationOperationCallbackFactory.java  |  2 +-
 .../am/common/dataflow/IndexDataflowHelper.java | 46 +++++++++----------
 ...xInsertUpdateDeleteOperatorNodePushable.java |  2 +-
 .../common/dataflow/IndexLifecycleManager.java  | 48 ++++++++++----------
 .../impls/NoOpOperationCallbackFactory.java     |  2 +-
 .../am/common/util/IndexFileNameUtil.java       | 13 +++++-
 .../dataflow/ExternalBTreeDataflowHelper.java   |  2 +-
 .../ExternalBTreeWithBuddyDataflowHelper.java   |  2 +-
 .../dataflow/ExternalRTreeDataflowHelper.java   |  2 +-
 .../storage/common/IResourceMemoryManager.java  |  2 +-
 .../common/file/ILocalResourceFactory.java      |  2 +-
 .../common/file/ILocalResourceRepository.java   |  4 +-
 .../storage/common/file/LocalResource.java      | 13 +++++-
 .../file/TransientLocalResourceFactory.java     |  4 +-
 .../file/TransientLocalResourceRepository.java  | 14 +++---
 20 files changed, 118 insertions(+), 76 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/84790762/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IIOManager.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IIOManager.java b/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IIOManager.java
index b5c7aa0..47eac7e 100644
--- a/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IIOManager.java
+++ b/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IIOManager.java
@@ -58,4 +58,11 @@ public interface IIOManager {
     public long getSize(IFileHandle fileHandle);
 
     public void deleteWorkspaceFiles();
+
+    /**
+     * @param ioDeviceId
+     * @param relativePath
+     * @return A file reference based on the mounting point of {@code ioDeviceId} and the passed {@code relativePath}
+     */
+    public FileReference getAbsoluteFileRef(int ioDeviceId, String relativePath);
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/84790762/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java
index 6881f73..519a9d7 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java
@@ -282,4 +282,10 @@ public class IOManager implements IIOManager {
             return name.endsWith(WORKSPACE_FILE_SUFFIX);
         }
     };
+
+    @Override
+    public FileReference getAbsoluteFileRef(int ioDeviceId, String relativePath) {
+        IODeviceHandle devHandle = ioDevices.get(ioDeviceId);
+        return new FileReference(devHandle, relativePath);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/84790762/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/file/FileSplit.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/file/FileSplit.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/file/FileSplit.java
index b66122e..384b180 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/file/FileSplit.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/file/FileSplit.java
@@ -29,29 +29,34 @@ public class FileSplit implements Serializable {
     private final String nodeName;
     private final FileReference file;
     private final int ioDeviceId;
+    private final int partition;
 
     public FileSplit(String nodeName, FileReference file) {
         this.nodeName = nodeName;
         this.file = file;
         this.ioDeviceId = 0;
+        this.partition = -1;
     }
 
-    public FileSplit(String nodeName, FileReference file, int ioDeviceId) {
+    public FileSplit(String nodeName, FileReference file, int ioDeviceId, int partition) {
         this.nodeName = nodeName;
         this.file = file;
         this.ioDeviceId = ioDeviceId;
+        this.partition = partition;
     }
 
     public FileSplit(String nodeName, String path, int ioDeviceId) {
         this.nodeName = nodeName;
         this.file = new FileReference(new File(path));
         this.ioDeviceId = ioDeviceId;
+        this.partition = -1;
     }
 
     public FileSplit(String nodeName, String path) {
         this.nodeName = nodeName;
         this.file = new FileReference(new File(path));
         this.ioDeviceId = 0;
+        this.partition = -1;
     }
 
     public String getNodeName() {
@@ -66,6 +71,10 @@ public class FileSplit implements Serializable {
         return ioDeviceId;
     }
 
+    public int getPartition() {
+        return partition;
+    }
+
     @Override
     public String toString() {
         return file.toString();

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/84790762/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexDataflowHelper.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexDataflowHelper.java b/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexDataflowHelper.java
index 9ee7969..4357770 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexDataflowHelper.java
@@ -45,5 +45,5 @@ public interface IIndexDataflowHelper {
 
     public IHyracksTaskContext getTaskContext();
 
-    public String getResourceName();
+    public String getResourcePath();
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/84790762/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexLifecycleManager.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexLifecycleManager.java b/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexLifecycleManager.java
index 6d16921..55ec2f8 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexLifecycleManager.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexLifecycleManager.java
@@ -26,13 +26,13 @@ import org.apache.hyracks.storage.common.IResourceMemoryManager;
 public interface IIndexLifecycleManager extends IResourceMemoryManager {
     public List<IIndex> getOpenIndexes();
 
-    public void register(String resourceName, IIndex index) throws HyracksDataException;
+    public void register(String resourcePath, IIndex index) throws HyracksDataException;
 
-    public void open(String resourceName) throws HyracksDataException;
+    public void open(String resourcePath) throws HyracksDataException;
 
-    public void close(String resourceName) throws HyracksDataException;
+    public void close(String resourcePath) throws HyracksDataException;
 
-    public IIndex getIndex(String resourceName) throws HyracksDataException;
+    public IIndex getIndex(String resourcePath) throws HyracksDataException;
 
-    public void unregister(String resourceName) throws HyracksDataException;
+    public void unregister(String resourcePath) throws HyracksDataException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/84790762/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IModificationOperationCallbackFactory.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IModificationOperationCallbackFactory.java b/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IModificationOperationCallbackFactory.java
index 212190c..120d880 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IModificationOperationCallbackFactory.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IModificationOperationCallbackFactory.java
@@ -24,6 +24,6 @@ import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public interface IModificationOperationCallbackFactory extends Serializable {
-    public IModificationOperationCallback createModificationOperationCallback(String resourceName, long resourceId,
+    public IModificationOperationCallback createModificationOperationCallback(String resourcePath, long resourceId,
             Object resource, IHyracksTaskContext ctx) throws HyracksDataException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/84790762/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexDataflowHelper.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexDataflowHelper.java b/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexDataflowHelper.java
index c6680bc..e465aa0 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexDataflowHelper.java
@@ -19,7 +19,6 @@
 
 package org.apache.hyracks.storage.am.common.dataflow;
 
-import java.io.File;
 import java.io.IOException;
 
 import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -43,10 +42,9 @@ public abstract class IndexDataflowHelper implements IIndexDataflowHelper {
     protected final IResourceIdFactory resourceIdFactory;
     protected final FileReference file;
     protected final int partition;
-    protected final int ioDeviceId;
     protected final boolean durable;
-    protected final String resourceName;
     protected IIndex index;
+    protected final String resourcePath;
 
     public IndexDataflowHelper(IIndexOperatorDescriptor opDesc, final IHyracksTaskContext ctx, int partition,
             boolean durable) {
@@ -56,12 +54,9 @@ public abstract class IndexDataflowHelper implements IIndexDataflowHelper {
         this.localResourceRepository = opDesc.getStorageManager().getLocalResourceRepository(ctx);
         this.resourceIdFactory = opDesc.getStorageManager().getResourceIdFactory(ctx);
         this.partition = partition;
-        this.ioDeviceId = opDesc.getFileSplitProvider().getFileSplits()[partition].getIODeviceId();
-        this.file = new FileReference(new File(IndexFileNameUtil.prepareFileName(
-                opDesc.getFileSplitProvider().getFileSplits()[partition].getLocalFile().getFile().getPath(),
-                ioDeviceId)));
+        this.file = IndexFileNameUtil.getIndexAbsoluteFileRef(opDesc, partition, ctx.getIOManager());
+        this.resourcePath = file.getFile().getPath();
         this.durable = durable;
-        this.resourceName = file.getFile().getPath();
     }
 
     protected abstract IIndex createIndexInstance() throws HyracksDataException;
@@ -74,9 +69,9 @@ public abstract class IndexDataflowHelper implements IIndexDataflowHelper {
     @Override
     public void create() throws HyracksDataException {
         synchronized (lcManager) {
-            index = lcManager.getIndex(resourceName);
+            index = lcManager.getIndex(resourcePath);
             if (index != null) {
-                lcManager.unregister(resourceName);
+                lcManager.unregister(resourcePath);
             } else {
                 index = createIndexInstance();
             }
@@ -86,19 +81,22 @@ public abstract class IndexDataflowHelper implements IIndexDataflowHelper {
             // Once the index has been created, a new resource ID can be generated.
             long resourceID = getResourceID();
             if (resourceID != -1) {
-                localResourceRepository.deleteResourceByName(resourceName);
+                localResourceRepository.deleteResourceByPath(resourcePath);
             }
             index.create();
             try {
                 resourceID = resourceIdFactory.createId();
                 ILocalResourceFactory localResourceFactory = opDesc.getLocalResourceFactoryProvider()
                         .getLocalResourceFactory();
-                localResourceRepository
-                        .insert(localResourceFactory.createLocalResource(resourceID, resourceName, partition));
+                int resourcePartition = opDesc.getFileSplitProvider().getFileSplits()[partition].getPartition();
+                String resourceName = opDesc.getFileSplitProvider().getFileSplits()[partition].getLocalFile().getFile()
+                        .getPath();
+                localResourceRepository.insert(localResourceFactory.createLocalResource(resourceID, resourceName,
+                        resourcePartition, resourcePath));
             } catch (IOException e) {
                 throw new HyracksDataException(e);
             }
-            lcManager.register(resourceName, index);
+            lcManager.register(resourcePath, index);
         }
     }
 
@@ -109,34 +107,34 @@ public abstract class IndexDataflowHelper implements IIndexDataflowHelper {
                 throw new HyracksDataException("Index does not have a valid resource ID. Has it been created yet?");
             }
 
-            index = lcManager.getIndex(resourceName);
+            index = lcManager.getIndex(resourcePath);
             if (index == null) {
                 index = createIndexInstance();
-                lcManager.register(resourceName, index);
+                lcManager.register(resourcePath, index);
             }
-            lcManager.open(resourceName);
+            lcManager.open(resourcePath);
         }
     }
 
     @Override
     public void close() throws HyracksDataException {
         synchronized (lcManager) {
-            lcManager.close(resourceName);
+            lcManager.close(resourcePath);
         }
     }
 
     @Override
     public void destroy() throws HyracksDataException {
         synchronized (lcManager) {
-            index = lcManager.getIndex(resourceName);
+            index = lcManager.getIndex(resourcePath);
             if (index != null) {
-                lcManager.unregister(resourceName);
+                lcManager.unregister(resourcePath);
             } else {
                 index = createIndexInstance();
             }
 
             if (getResourceID() != -1) {
-                localResourceRepository.deleteResourceByName(resourceName);
+                localResourceRepository.deleteResourceByPath(resourcePath);
             }
             index.destroy();
         }
@@ -149,7 +147,7 @@ public abstract class IndexDataflowHelper implements IIndexDataflowHelper {
 
     @Override
     public long getResourceID() throws HyracksDataException {
-        LocalResource lr = localResourceRepository.getResourceByName(resourceName);
+        LocalResource lr = localResourceRepository.getResourceByPath(resourcePath);
         return lr == null ? -1 : lr.getResourceId();
     }
 
@@ -159,7 +157,7 @@ public abstract class IndexDataflowHelper implements IIndexDataflowHelper {
     }
 
     @Override
-    public String getResourceName() {
-        return resourceName;
+    public String getResourcePath() {
+        return resourcePath;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/84790762/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexInsertUpdateDeleteOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexInsertUpdateDeleteOperatorNodePushable.java b/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexInsertUpdateDeleteOperatorNodePushable.java
index a2913f4..6c7e8d0 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexInsertUpdateDeleteOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexInsertUpdateDeleteOperatorNodePushable.java
@@ -77,7 +77,7 @@ public class IndexInsertUpdateDeleteOperatorNodePushable extends AbstractUnaryIn
         try {
             writer.open();
             modCallback = opDesc.getModificationOpCallbackFactory().createModificationOperationCallback(
-                    indexHelper.getResourceName(), indexHelper.getResourceID(), index, ctx);
+                    indexHelper.getResourcePath(), indexHelper.getResourceID(), index, ctx);
             indexAccessor = index.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
             ITupleFilterFactory tupleFilterFactory = opDesc.getTupleFilterFactory();
             if (tupleFilterFactory != null) {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/84790762/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexLifecycleManager.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexLifecycleManager.java b/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexLifecycleManager.java
index 4ba2b80..adf4723 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexLifecycleManager.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexLifecycleManager.java
@@ -183,23 +183,23 @@ public class IndexLifecycleManager implements IIndexLifecycleManager, ILifeCycle
     }
 
     @Override
-    public void register(String resourceName, IIndex index) throws HyracksDataException {
-        if (indexInfos.containsKey(resourceName)) {
-            throw new HyracksDataException("Index with resource name " + resourceName + " already exists.");
+    public void register(String resourcePath, IIndex index) throws HyracksDataException {
+        if (indexInfos.containsKey(resourcePath)) {
+            throw new HyracksDataException("Index with resource name " + resourcePath + " already exists.");
         }
-        indexInfos.put(resourceName, new IndexInfo(index));
+        indexInfos.put(resourcePath, new IndexInfo(index));
     }
 
     @Override
-    public void open(String resourceName) throws HyracksDataException {
-        IndexInfo info = indexInfos.get(resourceName);
+    public void open(String resourcePath) throws HyracksDataException {
+        IndexInfo info = indexInfos.get(resourcePath);
         if (info == null) {
             throw new HyracksDataException(
-                    "Failed to open index with resource name " + resourceName + " since it does not exist.");
+                    "Failed to open index with resource name " + resourcePath + " since it does not exist.");
         }
 
         if (!info.isOpen) {
-            allocateMemory(resourceName);
+            allocateMemory(resourcePath);
             info.index.activate();
             info.isOpen = true;
         }
@@ -207,40 +207,40 @@ public class IndexLifecycleManager implements IIndexLifecycleManager, ILifeCycle
     }
 
     @Override
-    public void close(String resourceName) throws HyracksDataException {
-        indexInfos.get(resourceName).untouch();
+    public void close(String resourcePath) throws HyracksDataException {
+        indexInfos.get(resourcePath).untouch();
     }
 
     @Override
-    public IIndex getIndex(String resourceName) throws HyracksDataException {
-        IndexInfo info = indexInfos.get(resourceName);
+    public IIndex getIndex(String resourcePath) throws HyracksDataException {
+        IndexInfo info = indexInfos.get(resourcePath);
         return info == null ? null : info.index;
     }
 
     @Override
-    public void unregister(String resourceName) throws HyracksDataException {
-        IndexInfo info = indexInfos.get(resourceName);
+    public void unregister(String resourcePath) throws HyracksDataException {
+        IndexInfo info = indexInfos.get(resourcePath);
         if (info == null) {
-            throw new HyracksDataException("Index with resource name " + resourceName + " does not exist.");
+            throw new HyracksDataException("Index with resource name " + resourcePath + " does not exist.");
         }
 
         if (info.referenceCount != 0) {
-            indexInfos.put(resourceName, info);
+            indexInfos.put(resourcePath, info);
             throw new HyracksDataException("Cannot remove index while it is open.");
         }
 
         if (info.isOpen) {
             info.index.deactivate();
-            deallocateMemory(resourceName);
+            deallocateMemory(resourcePath);
         }
-        indexInfos.remove(resourceName);
+        indexInfos.remove(resourcePath);
     }
 
     @Override
-    public void allocateMemory(String resourceName) throws HyracksDataException {
-        IndexInfo info = indexInfos.get(resourceName);
+    public void allocateMemory(String resourcePath) throws HyracksDataException {
+        IndexInfo info = indexInfos.get(resourcePath);
         if (info == null) {
-            throw new HyracksDataException("Failed to allocate memory for index with resource ID " + resourceName
+            throw new HyracksDataException("Failed to allocate memory for index with resource ID " + resourcePath
                     + " since it does not exist.");
         }
         if (!info.memoryAllocated) {
@@ -256,10 +256,10 @@ public class IndexLifecycleManager implements IIndexLifecycleManager, ILifeCycle
         }
     }
 
-    private void deallocateMemory(String resourceName) throws HyracksDataException {
-        IndexInfo info = indexInfos.get(resourceName);
+    private void deallocateMemory(String resourcePath) throws HyracksDataException {
+        IndexInfo info = indexInfos.get(resourcePath);
         if (info == null) {
-            throw new HyracksDataException("Failed to deallocate memory for index with resource name " + resourceName
+            throw new HyracksDataException("Failed to deallocate memory for index with resource name " + resourcePath
                     + " since it does not exist.");
         }
         if (info.isOpen && info.memoryAllocated) {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/84790762/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/NoOpOperationCallbackFactory.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/NoOpOperationCallbackFactory.java b/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/NoOpOperationCallbackFactory.java
index 076f7f8..17f9265 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/NoOpOperationCallbackFactory.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/NoOpOperationCallbackFactory.java
@@ -39,7 +39,7 @@ public enum NoOpOperationCallbackFactory implements ISearchOperationCallbackFact
     }
 
     @Override
-    public IModificationOperationCallback createModificationOperationCallback(String resourceName, long resourceId,
+    public IModificationOperationCallback createModificationOperationCallback(String resourcePath, long resourceId,
             Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
         return NoOpOperationCallback.INSTANCE;
     }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/84790762/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/IndexFileNameUtil.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/IndexFileNameUtil.java b/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/IndexFileNameUtil.java
index d1d59ed..5d08e23 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/IndexFileNameUtil.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/IndexFileNameUtil.java
@@ -21,11 +21,22 @@ package org.apache.hyracks.storage.am.common.util;
 
 import java.io.File;
 
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
+
 public class IndexFileNameUtil {
 
     public static final String IO_DEVICE_NAME_PREFIX = "device_id_";
-    
+
+    @Deprecated
     public static String prepareFileName(String path, int ioDeviceId) {
         return path + File.separator + IO_DEVICE_NAME_PREFIX + ioDeviceId;
     }
+
+    public static FileReference getIndexAbsoluteFileRef(IIndexOperatorDescriptor opDesc, int partition, IIOManager ioManager){
+        String indexName = opDesc.getFileSplitProvider().getFileSplits()[partition].getLocalFile().getFile().getPath();
+        int ioDeviceId = opDesc.getFileSplitProvider().getFileSplits()[partition].getIODeviceId();
+        return ioManager.getAbsoluteFileRef(ioDeviceId, indexName);
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/84790762/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeDataflowHelper.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeDataflowHelper.java
index 0e23fa4..44d81ab 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeDataflowHelper.java
@@ -60,7 +60,7 @@ public class ExternalBTreeDataflowHelper extends LSMBTreeDataflowHelper {
             return index;
         synchronized (lcManager) {
             try {
-                index = lcManager.getIndex(resourceName);
+                index = lcManager.getIndex(resourcePath);
             } catch (HyracksDataException e) {
                 return null;
             }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/84790762/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyDataflowHelper.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyDataflowHelper.java
index 325ff91..81beadd 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyDataflowHelper.java
@@ -61,7 +61,7 @@ public class ExternalBTreeWithBuddyDataflowHelper extends AbstractLSMIndexDatafl
             return index;
         synchronized (lcManager) {
             try {
-                index = lcManager.getIndex(resourceName);
+                index = lcManager.getIndex(resourcePath);
             } catch (HyracksDataException e) {
                 return null;
             }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/84790762/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeDataflowHelper.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeDataflowHelper.java
index b6f5941..6f1c34a 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeDataflowHelper.java
@@ -76,7 +76,7 @@ public class ExternalRTreeDataflowHelper extends LSMRTreeDataflowHelper {
             return index;
         synchronized (lcManager) {
             try {
-                index = lcManager.getIndex(resourceName);
+                index = lcManager.getIndex(resourcePath);
             } catch (HyracksDataException e) {
                 return null;
             }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/84790762/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IResourceMemoryManager.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IResourceMemoryManager.java b/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IResourceMemoryManager.java
index c8b530f..036520a 100644
--- a/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IResourceMemoryManager.java
+++ b/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IResourceMemoryManager.java
@@ -21,5 +21,5 @@ package org.apache.hyracks.storage.common;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public interface IResourceMemoryManager {
-    void allocateMemory(String resourceName) throws HyracksDataException;
+    void allocateMemory(String resourcePath) throws HyracksDataException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/84790762/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/ILocalResourceFactory.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/ILocalResourceFactory.java b/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/ILocalResourceFactory.java
index 9133a7e..f5e7010 100644
--- a/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/ILocalResourceFactory.java
+++ b/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/ILocalResourceFactory.java
@@ -19,5 +19,5 @@
 package org.apache.hyracks.storage.common.file;
 
 public interface ILocalResourceFactory {
-    public LocalResource createLocalResource(long resourceId, String resourceName, int partition);
+    public LocalResource createLocalResource(long resourceId, String resourceName, int partition, String resourcePath);
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/84790762/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/ILocalResourceRepository.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/ILocalResourceRepository.java b/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/ILocalResourceRepository.java
index 6f2e285..eef76a8 100644
--- a/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/ILocalResourceRepository.java
+++ b/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/ILocalResourceRepository.java
@@ -22,11 +22,11 @@ import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public interface ILocalResourceRepository {
 
-    public LocalResource getResourceByName(String name) throws HyracksDataException;
+    public LocalResource getResourceByPath(String name) throws HyracksDataException;
 
     public void insert(LocalResource resource) throws HyracksDataException;
 
-    public void deleteResourceByName(String name) throws HyracksDataException;
+    public void deleteResourceByPath(String name) throws HyracksDataException;
 
     public long getMaxResourceID() throws HyracksDataException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/84790762/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/LocalResource.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/LocalResource.java b/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/LocalResource.java
index 59190b1..eba0119 100644
--- a/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/LocalResource.java
+++ b/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/LocalResource.java
@@ -27,6 +27,7 @@ public class LocalResource implements Serializable {
     private final int partition;
     private final int resourceType;
     private final Object object;
+    private String resourcePath;
 
     public static final int TransientResource = 0;
     public static final int LSMBTreeResource = 1;
@@ -36,10 +37,12 @@ public class LocalResource implements Serializable {
     public static final int ExternalRTreeResource = 5;
     public static final int ExternalBTreeWithBuddyResource = 6;
 
-    public LocalResource(long resourceId, String resourceName, int partition, int resourceType, Object object) {
+    public LocalResource(long resourceId, String resourceName, int partition, String resourcePath, int resourceType,
+            Object object) {
         this.resourceId = resourceId;
         this.resourceName = resourceName;
         this.partition = partition;
+        this.resourcePath = resourcePath;
         this.resourceType = resourceType;
         this.object = object;
     }
@@ -63,4 +66,12 @@ public class LocalResource implements Serializable {
     public Object getResourceObject() {
         return object;
     }
+
+    public String getResourcePath() {
+        return resourcePath;
+    }
+
+    public void setResourcePath(String resourcePath) {
+        this.resourcePath = resourcePath;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/84790762/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/TransientLocalResourceFactory.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/TransientLocalResourceFactory.java b/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/TransientLocalResourceFactory.java
index 49c6842..527f55e 100644
--- a/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/TransientLocalResourceFactory.java
+++ b/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/TransientLocalResourceFactory.java
@@ -21,7 +21,7 @@ package org.apache.hyracks.storage.common.file;
 public class TransientLocalResourceFactory implements ILocalResourceFactory {
 
     @Override
-    public LocalResource createLocalResource(long resourceId, String resourceName, int partition) {
-        return new LocalResource(resourceId, resourceName, partition, LocalResource.TransientResource, null);
+    public LocalResource createLocalResource(long resourceId, String resourceName, int partition, String resourcePath) {
+        return new LocalResource(resourceId, resourceName, partition, resourcePath, LocalResource.TransientResource, null);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/84790762/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/TransientLocalResourceRepository.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/TransientLocalResourceRepository.java b/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/TransientLocalResourceRepository.java
index 3bc763d..35add93 100644
--- a/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/TransientLocalResourceRepository.java
+++ b/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/TransientLocalResourceRepository.java
@@ -25,12 +25,12 @@ import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class TransientLocalResourceRepository implements ILocalResourceRepository {
 
-    private Map<String, LocalResource> name2ResourceMap = new HashMap<String, LocalResource>();
+    private Map<String, LocalResource> path2ResourceMap = new HashMap<String, LocalResource>();
     private Map<Long, LocalResource> id2ResourceMap = new HashMap<Long, LocalResource>();
 
     @Override
-    public LocalResource getResourceByName(String name) throws HyracksDataException {
-        return name2ResourceMap.get(name);
+    public LocalResource getResourceByPath(String path) throws HyracksDataException {
+        return path2ResourceMap.get(path);
     }
 
     @Override
@@ -41,17 +41,17 @@ public class TransientLocalResourceRepository implements ILocalResourceRepositor
             throw new HyracksDataException("Duplicate resource");
         }
         id2ResourceMap.put(id, resource);
-        name2ResourceMap.put(resource.getResourceName(), resource);
+        path2ResourceMap.put(resource.getResourcePath(), resource);
     }
 
     @Override
-    public synchronized void deleteResourceByName(String name) throws HyracksDataException {
-        LocalResource resource = name2ResourceMap.get(name);
+    public synchronized void deleteResourceByPath(String path) throws HyracksDataException {
+        LocalResource resource = path2ResourceMap.get(path);
         if (resource == null) {
             throw new HyracksDataException("Resource doesn't exist");
         }
         id2ResourceMap.remove(resource.getResourceId());
-        name2ResourceMap.remove(name);
+        path2ResourceMap.remove(path);
     }
 
     @Override