diff --git a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java
index f6a14d8ec9d63..aee0473be95eb 100644
--- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java
+++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java
@@ -88,8 +88,8 @@
 import org.opensearch.index.mapper.MapperService;
 import org.opensearch.index.mapper.MapperService.MergeReason;
 import org.opensearch.index.query.QueryShardContext;
-import org.opensearch.index.remote.RemoteStorePathResolver;
 import org.opensearch.index.remote.RemoteStorePathType;
+import org.opensearch.index.remote.RemoteStorePathTypeResolver;
 import org.opensearch.index.shard.IndexSettingProvider;
 import org.opensearch.index.translog.Translog;
 import org.opensearch.indices.IndexCreationException;
@@ -113,6 +113,7 @@
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
+import java.util.Objects;
 import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -170,7 +171,7 @@ public class MetadataCreateIndexService {
     private AwarenessReplicaBalance awarenessReplicaBalance;
 
     @Nullable
-    private final RemoteStorePathResolver remoteStorePathResolver;
+    private final RemoteStorePathTypeResolver remoteStorePathTypeResolver;
 
     public MetadataCreateIndexService(
         final Settings settings,
@@ -203,8 +204,8 @@ public MetadataCreateIndexService(
 
         // Task is onboarded for throttling, it will get retried from associated TransportClusterManagerNodeAction.
         createIndexTaskKey = clusterService.registerClusterManagerTask(ClusterManagerTaskKeys.CREATE_INDEX_KEY, true);
-        remoteStorePathResolver = isRemoteDataAttributePresent(settings)
-            ? new RemoteStorePathResolver(clusterService.getClusterSettings())
+        remoteStorePathTypeResolver = isRemoteDataAttributePresent(settings)
+            ? new RemoteStorePathTypeResolver(clusterService.getClusterSettings())
             : null;
     }
 
@@ -553,7 +554,7 @@ IndexMetadata buildAndValidateTemporaryIndexMetadata(
         tmpImdBuilder.setRoutingNumShards(routingNumShards);
         tmpImdBuilder.settings(indexSettings);
         tmpImdBuilder.system(isSystem);
-        addRemoteCustomData(tmpImdBuilder);
+        addRemoteStorePathTypeInCustomData(tmpImdBuilder, true);
 
         // Set up everything, now locally create the index to see that things are ok, and apply
         IndexMetadata tempMetadata = tmpImdBuilder.build();
@@ -562,8 +563,14 @@ IndexMetadata buildAndValidateTemporaryIndexMetadata(
         return tempMetadata;
     }
 
-    public void addRemoteCustomData(IndexMetadata.Builder tmpImdBuilder) {
-        if (remoteStorePathResolver != null) {
+    /**
+     * Adds the remote store path type information in custom data of index metadata.
+     *
+     * @param tmpImdBuilder     index metadata builder.
+     * @param assertNullOldType flag to verify that the old remote store path type is null
+     */
+    public void addRemoteStorePathTypeInCustomData(IndexMetadata.Builder tmpImdBuilder, boolean assertNullOldType) {
+        if (remoteStorePathTypeResolver != null) {
             // It is possible that remote custom data exists already. In such cases, we need to only update the path type
             // in the remote store custom data map.
             Map<String, String> existingRemoteCustomData = tmpImdBuilder.removeCustom(IndexMetadata.REMOTE_STORE_CUSTOM_KEY);
@@ -571,8 +578,9 @@ public void addRemoteCustomData(IndexMetadata.Builder tmpImdBuilder) {
                 ? new HashMap<>()
                 : new HashMap<>(existingRemoteCustomData);
             // Determine the path type for use using the remoteStorePathResolver.
-            String newPathType = remoteStorePathResolver.resolveType().toString();
+            String newPathType = remoteStorePathTypeResolver.getType().toString();
             String oldPathType = remoteCustomData.put(RemoteStorePathType.NAME, newPathType);
+            assert !assertNullOldType || Objects.isNull(oldPathType);
             logger.trace(() -> new ParameterizedMessage("Added new path type {}, replaced old path type {}", newPathType, oldPathType));
             tmpImdBuilder.putCustom(IndexMetadata.REMOTE_STORE_CUSTOM_KEY, remoteCustomData);
         }
diff --git a/server/src/main/java/org/opensearch/index/IndexService.java b/server/src/main/java/org/opensearch/index/IndexService.java
index 11dc4474cfa42..109bda65b1fd8 100644
--- a/server/src/main/java/org/opensearch/index/IndexService.java
+++ b/server/src/main/java/org/opensearch/index/IndexService.java
@@ -507,7 +507,8 @@ public synchronized IndexShard createShard(
                     remoteDirectory = ((RemoteSegmentStoreDirectoryFactory) remoteDirectoryFactory).newDirectory(
                         RemoteStoreNodeAttribute.getRemoteStoreSegmentRepo(this.indexSettings.getNodeSettings()),
                         this.indexSettings.getUUID(),
-                        shardId
+                        shardId,
+                        this.indexSettings.getRemoteStorePathType()
                     );
                 }
                 remoteStore = new Store(shardId, this.indexSettings, remoteDirectory, lock, Store.OnClose.EMPTY, path);
diff --git a/server/src/main/java/org/opensearch/index/IndexSettings.java b/server/src/main/java/org/opensearch/index/IndexSettings.java
index 7e49726c259cb..7e3d812974c79 100644
--- a/server/src/main/java/org/opensearch/index/IndexSettings.java
+++ b/server/src/main/java/org/opensearch/index/IndexSettings.java
@@ -48,6 +48,7 @@
 import org.opensearch.core.common.unit.ByteSizeUnit;
 import org.opensearch.core.common.unit.ByteSizeValue;
 import org.opensearch.core.index.Index;
+import org.opensearch.index.remote.RemoteStorePathType;
 import org.opensearch.index.translog.Translog;
 import org.opensearch.indices.replication.common.ReplicationType;
 import org.opensearch.ingest.IngestService;
@@ -59,6 +60,7 @@
 import java.util.Collections;
 import java.util.List;
 import java.util.Locale;
+import java.util.Map;
 import java.util.Optional;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Consumer;
@@ -1905,4 +1907,11 @@ public double getDocIdFuzzySetFalsePositiveProbability() {
     public void setDocIdFuzzySetFalsePositiveProbability(double docIdFuzzySetFalsePositiveProbability) {
         this.docIdFuzzySetFalsePositiveProbability = docIdFuzzySetFalsePositiveProbability;
     }
+
+    public RemoteStorePathType getRemoteStorePathType() {
+        Map<String, String> remoteCustomData = indexMetadata.getCustomData(IndexMetadata.REMOTE_STORE_CUSTOM_KEY);
+        return remoteCustomData != null && remoteCustomData.containsKey(RemoteStorePathType.NAME)
+            ? RemoteStorePathType.parseString(remoteCustomData.get(RemoteStorePathType.NAME))
+            : RemoteStorePathType.FIXED;
+    }
 }
diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStoreDataEnums.java b/server/src/main/java/org/opensearch/index/remote/RemoteStoreDataEnums.java
new file mode 100644
index 0000000000000..1afd73bf1f1b3
--- /dev/null
+++ b/server/src/main/java/org/opensearch/index/remote/RemoteStoreDataEnums.java
@@ -0,0 +1,65 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.index.remote;
+
+import java.util.Set;
+
+import static org.opensearch.index.remote.RemoteStoreDataEnums.DataType.DATA;
+import static org.opensearch.index.remote.RemoteStoreDataEnums.DataType.METADATA;
+
+/**
+ * This class contains the different enums related to remote store data categories and types.
+ *
+ * @opensearch.internal
+ */
+public class RemoteStoreDataEnums {
+
+    /**
+     * Categories of the data in Remote store.
+     */
+    public enum DataCategory {
+        SEGMENTS("segments", Set.of(DataType.values())),
+        TRANSLOG("translog", Set.of(DATA, METADATA));
+
+        private final String name;
+        private final Set<DataType> supportedDataTypes;
+
+        DataCategory(String name, Set<DataType> supportedDataTypes) {
+            this.name = name;
+            this.supportedDataTypes = supportedDataTypes;
+        }
+
+        public boolean isSupportedDataType(DataType dataType) {
+            return supportedDataTypes.contains(dataType);
+        }
+
+        public String getName() {
+            return name;
+        }
+    }
+
+    /**
+     * Types of data in remote store.
+     */
+    public enum DataType {
+        DATA("data"),
+        METADATA("metadata"),
+        LOCK_FILES("lock_files");
+
+        private final String name;
+
+        DataType(String name) {
+            this.name = name;
+        }
+
+        public String getName() {
+            return name;
+        }
+    }
+}
diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStorePathType.java b/server/src/main/java/org/opensearch/index/remote/RemoteStorePathType.java
index a64e07ab1f66f..d7d7a8cdfd644 100644
--- a/server/src/main/java/org/opensearch/index/remote/RemoteStorePathType.java
+++ b/server/src/main/java/org/opensearch/index/remote/RemoteStorePathType.java
@@ -8,6 +8,10 @@
 
 package org.opensearch.index.remote;
 
+import org.opensearch.common.blobstore.BlobPath;
+import org.opensearch.index.remote.RemoteStoreDataEnums.DataCategory;
+import org.opensearch.index.remote.RemoteStoreDataEnums.DataType;
+
 import java.util.Locale;
 
 /**
@@ -18,13 +22,46 @@
  */
 public enum RemoteStorePathType {
 
-    FIXED,
-    HASHED_PREFIX;
+    FIXED {
+        @Override
+        public BlobPath generatePath(BlobPath basePath, String indexUUID, String shardId, String dataCategory, String dataType) {
+            return basePath.add(indexUUID).add(shardId).add(dataCategory).add(dataType);
+        }
+    },
+    HASHED_PREFIX {
+        @Override
+        public BlobPath generatePath(BlobPath basePath, String indexUUID, String shardId, String dataCategory, String dataType) {
+            // TODO - We need to implement this, keeping the same path as Fixed for sake of multiple tests that can fail otherwise.
+            // throw new UnsupportedOperationException("Not implemented"); --> Not using this for unblocking couple of tests.
+            return basePath.add(indexUUID).add(shardId).add(dataCategory).add(dataType);
+        }
+    };
+
+    /**
+     * @param basePath     base path of the underlying blob store repository
+     * @param indexUUID    of the index
+     * @param shardId      shard id
+     * @param dataCategory is either translog or segment
+     * @param dataType     can be one of data, metadata or lock_files.
+     * @return the blob path for the underlying remote store path type.
+     */
+    public BlobPath path(BlobPath basePath, String indexUUID, String shardId, DataCategory dataCategory, DataType dataType) {
+        assert dataCategory.isSupportedDataType(dataType) : "category:"
+            + dataCategory
+            + " type:"
+            + dataType
+            + " are not supported together";
+        return generatePath(basePath, indexUUID, shardId, dataCategory.getName(), dataType.getName());
+    }
+
+    abstract BlobPath generatePath(BlobPath basePath, String indexUUID, String shardId, String dataCategory, String dataType);
 
     public static RemoteStorePathType parseString(String remoteStoreBlobPathType) {
         try {
             return RemoteStorePathType.valueOf(remoteStoreBlobPathType.toUpperCase(Locale.ROOT));
-        } catch (IllegalArgumentException e) {
+        } catch (IllegalArgumentException | NullPointerException e) {
+            // IllegalArgumentException is thrown when the input does not match any enum name
+            // NullPointerException is thrown when the input is null
             throw new IllegalArgumentException("Could not parse RemoteStorePathType for [" + remoteStoreBlobPathType + "]");
         }
     }
diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStorePathResolver.java b/server/src/main/java/org/opensearch/index/remote/RemoteStorePathTypeResolver.java
similarity index 50%
rename from server/src/main/java/org/opensearch/index/remote/RemoteStorePathResolver.java
rename to server/src/main/java/org/opensearch/index/remote/RemoteStorePathTypeResolver.java
index 6e8126fcce0ca..5d014c9862d45 100644
--- a/server/src/main/java/org/opensearch/index/remote/RemoteStorePathResolver.java
+++ b/server/src/main/java/org/opensearch/index/remote/RemoteStorePathTypeResolver.java
@@ -16,15 +16,20 @@
  *
  * @opensearch.internal
  */
-public class RemoteStorePathResolver {
+public class RemoteStorePathTypeResolver {
 
-    private final ClusterSettings clusterSettings;
+    private volatile RemoteStorePathType type;
 
-    public RemoteStorePathResolver(ClusterSettings clusterSettings) {
-        this.clusterSettings = clusterSettings;
+    public RemoteStorePathTypeResolver(ClusterSettings clusterSettings) {
+        type = clusterSettings.get(IndicesService.CLUSTER_REMOTE_STORE_PATH_PREFIX_TYPE_SETTING);
+        clusterSettings.addSettingsUpdateConsumer(IndicesService.CLUSTER_REMOTE_STORE_PATH_PREFIX_TYPE_SETTING, this::setType);
     }
 
-    public RemoteStorePathType resolveType() {
-        return clusterSettings.get(IndicesService.CLUSTER_REMOTE_STORE_PATH_PREFIX_TYPE_SETTING);
+    public RemoteStorePathType getType() {
+        return type;
+    }
+
+    public void setType(RemoteStorePathType type) {
+        this.type = type;
     }
 }
diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java
index 72ce858661031..1d7aa6ac4958b 100644
--- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java
+++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java
@@ -4932,7 +4932,7 @@ public void deleteTranslogFilesFromRemoteTranslog() throws IOException {
         TranslogFactory translogFactory = translogFactorySupplier.apply(indexSettings, shardRouting);
         assert translogFactory instanceof RemoteBlobStoreInternalTranslogFactory;
         Repository repository = ((RemoteBlobStoreInternalTranslogFactory) translogFactory).getRepository();
-        RemoteFsTranslog.cleanup(repository, shardId, getThreadPool());
+        RemoteFsTranslog.cleanup(repository, shardId, getThreadPool(), indexSettings.getRemoteStorePathType());
     }
 
     /*
@@ -4949,7 +4949,14 @@ public void syncTranslogFilesFromRemoteTranslog() throws IOException {
         TranslogFactory translogFactory = translogFactorySupplier.apply(indexSettings, shardRouting);
         assert translogFactory instanceof RemoteBlobStoreInternalTranslogFactory;
         Repository repository = ((RemoteBlobStoreInternalTranslogFactory) translogFactory).getRepository();
-        RemoteFsTranslog.download(repository, shardId, getThreadPool(), shardPath().resolveTranslog(), logger);
+        RemoteFsTranslog.download(
+            repository,
+            shardId,
+            getThreadPool(),
+            shardPath().resolveTranslog(),
+            indexSettings.getRemoteStorePathType(),
+            logger
+        );
     }
 
     /**
diff --git a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java
index 5f09b1a0802f3..9779a2320d79f 100644
--- a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java
+++ b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java
@@ -58,6 +58,7 @@
 import org.opensearch.index.engine.Engine;
 import org.opensearch.index.engine.EngineException;
 import org.opensearch.index.mapper.MapperService;
+import org.opensearch.index.remote.RemoteStorePathType;
 import org.opensearch.index.seqno.SequenceNumbers;
 import org.opensearch.index.snapshots.IndexShardRestoreFailedException;
 import org.opensearch.index.snapshots.blobstore.RemoteStoreShardShallowCopySnapshot;
@@ -409,7 +410,8 @@ void recoverFromSnapshotAndRemoteStore(
                 RemoteSegmentStoreDirectory sourceRemoteDirectory = (RemoteSegmentStoreDirectory) directoryFactory.newDirectory(
                     remoteStoreRepository,
                     indexUUID,
-                    shardId
+                    shardId,
+                    RemoteStorePathType.FIXED // TODO - The path type needs to be obtained from RemoteStoreShardShallowCopySnapshot
                 );
                 sourceRemoteDirectory.initializeToSpecificCommit(
                     primaryTerm,
diff --git a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java
index c9a238c6e3350..999625e0e579f 100644
--- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java
+++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java
@@ -31,6 +31,7 @@
 import org.opensearch.common.lucene.store.ByteArrayIndexInput;
 import org.opensearch.core.action.ActionListener;
 import org.opensearch.core.index.shard.ShardId;
+import org.opensearch.index.remote.RemoteStorePathType;
 import org.opensearch.index.remote.RemoteStoreUtils;
 import org.opensearch.index.store.lockmanager.FileLockInfo;
 import org.opensearch.index.store.lockmanager.RemoteStoreCommitLevelLockManager;
@@ -897,13 +898,15 @@ public static void remoteDirectoryCleanup(
         RemoteSegmentStoreDirectoryFactory remoteDirectoryFactory,
         String remoteStoreRepoForIndex,
         String indexUUID,
-        ShardId shardId
+        ShardId shardId,
+        RemoteStorePathType pathType
     ) {
         try {
             RemoteSegmentStoreDirectory remoteSegmentStoreDirectory = (RemoteSegmentStoreDirectory) remoteDirectoryFactory.newDirectory(
                 remoteStoreRepoForIndex,
                 indexUUID,
-                shardId
+                shardId,
+                pathType
             );
             remoteSegmentStoreDirectory.deleteStaleSegments(0);
             remoteSegmentStoreDirectory.deleteIfEmpty();
diff --git a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java
index d6d3f1fca833c..f0ecd96bcf1f7 100644
--- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java
+++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java
@@ -13,6 +13,7 @@
 import org.opensearch.common.blobstore.BlobPath;
 import org.opensearch.core.index.shard.ShardId;
 import org.opensearch.index.IndexSettings;
+import org.opensearch.index.remote.RemoteStorePathType;
 import org.opensearch.index.shard.ShardPath;
 import org.opensearch.index.store.lockmanager.RemoteStoreLockManager;
 import org.opensearch.index.store.lockmanager.RemoteStoreLockManagerFactory;
@@ -24,8 +25,13 @@
 import org.opensearch.threadpool.ThreadPool;
 
 import java.io.IOException;
+import java.util.Objects;
 import java.util.function.Supplier;
 
+import static org.opensearch.index.remote.RemoteStoreDataEnums.DataCategory.SEGMENTS;
+import static org.opensearch.index.remote.RemoteStoreDataEnums.DataType.DATA;
+import static org.opensearch.index.remote.RemoteStoreDataEnums.DataType.METADATA;
+
 /**
  * Factory for a remote store directory
  *
@@ -33,8 +39,6 @@
  */
 @PublicApi(since = "2.3.0")
 public class RemoteSegmentStoreDirectoryFactory implements IndexStorePlugin.DirectoryFactory {
-    private static final String SEGMENTS = "segments";
-
     private final Supplier<RepositoriesService> repositoriesService;
 
     private final ThreadPool threadPool;
@@ -48,29 +52,38 @@ public RemoteSegmentStoreDirectoryFactory(Supplier<RepositoriesService> reposito
     public Directory newDirectory(IndexSettings indexSettings, ShardPath path) throws IOException {
         String repositoryName = indexSettings.getRemoteStoreRepository();
         String indexUUID = indexSettings.getIndex().getUUID();
-        return newDirectory(repositoryName, indexUUID, path.getShardId());
+        return newDirectory(repositoryName, indexUUID, path.getShardId(), indexSettings.getRemoteStorePathType());
     }
 
-    public Directory newDirectory(String repositoryName, String indexUUID, ShardId shardId) throws IOException {
+    public Directory newDirectory(String repositoryName, String indexUUID, ShardId shardId, RemoteStorePathType pathType)
+        throws IOException {
+        assert Objects.nonNull(pathType);
         try (Repository repository = repositoriesService.get().repository(repositoryName)) {
+
             assert repository instanceof BlobStoreRepository : "repository should be instance of BlobStoreRepository";
             BlobStoreRepository blobStoreRepository = ((BlobStoreRepository) repository);
-            BlobPath commonBlobPath = blobStoreRepository.basePath();
-            commonBlobPath = commonBlobPath.add(indexUUID).add(String.valueOf(shardId.id())).add(SEGMENTS);
+            BlobPath repositoryBasePath = blobStoreRepository.basePath();
+            String shardIdStr = String.valueOf(shardId.id());
 
+            // Derive the path for data directory of SEGMENTS
+            BlobPath dataPath = pathType.path(repositoryBasePath, indexUUID, shardIdStr, SEGMENTS, DATA);
             RemoteDirectory dataDirectory = new RemoteDirectory(
-                blobStoreRepository.blobStore().blobContainer(commonBlobPath.add("data")),
+                blobStoreRepository.blobStore().blobContainer(dataPath),
                 blobStoreRepository::maybeRateLimitRemoteUploadTransfers,
                 blobStoreRepository::maybeRateLimitRemoteDownloadTransfers
             );
-            RemoteDirectory metadataDirectory = new RemoteDirectory(
-                blobStoreRepository.blobStore().blobContainer(commonBlobPath.add("metadata"))
-            );
+
+            // Derive the path for metadata directory of SEGMENTS
+            BlobPath mdPath = pathType.path(repositoryBasePath, indexUUID, shardIdStr, SEGMENTS, METADATA);
+            RemoteDirectory metadataDirectory = new RemoteDirectory(blobStoreRepository.blobStore().blobContainer(mdPath));
+
+            // The path for lock is derived within the RemoteStoreLockManagerFactory
             RemoteStoreLockManager mdLockManager = RemoteStoreLockManagerFactory.newLockManager(
                 repositoriesService.get(),
                 repositoryName,
                 indexUUID,
-                String.valueOf(shardId.id())
+                shardIdStr,
+                pathType
             );
 
             return new RemoteSegmentStoreDirectory(dataDirectory, metadataDirectory, mdLockManager, threadPool, shardId);
diff --git a/server/src/main/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManagerFactory.java b/server/src/main/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManagerFactory.java
index 00666ada11983..c033e4f7ad0aa 100644
--- a/server/src/main/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManagerFactory.java
+++ b/server/src/main/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManagerFactory.java
@@ -11,15 +11,18 @@
 import org.opensearch.common.annotation.PublicApi;
 import org.opensearch.common.blobstore.BlobContainer;
 import org.opensearch.common.blobstore.BlobPath;
+import org.opensearch.index.remote.RemoteStorePathType;
 import org.opensearch.index.store.RemoteBufferedOutputDirectory;
 import org.opensearch.repositories.RepositoriesService;
 import org.opensearch.repositories.Repository;
 import org.opensearch.repositories.RepositoryMissingException;
 import org.opensearch.repositories.blobstore.BlobStoreRepository;
 
-import java.io.IOException;
 import java.util.function.Supplier;
 
+import static org.opensearch.index.remote.RemoteStoreDataEnums.DataCategory.SEGMENTS;
+import static org.opensearch.index.remote.RemoteStoreDataEnums.DataType.LOCK_FILES;
+
 /**
  * Factory for remote store lock manager
  *
@@ -27,34 +30,29 @@
  */
 @PublicApi(since = "2.8.0")
 public class RemoteStoreLockManagerFactory {
-    private static final String SEGMENTS = "segments";
-    private static final String LOCK_FILES = "lock_files";
     private final Supplier<RepositoriesService> repositoriesService;
 
     public RemoteStoreLockManagerFactory(Supplier<RepositoriesService> repositoriesService) {
         this.repositoriesService = repositoriesService;
     }
 
-    public RemoteStoreLockManager newLockManager(String repositoryName, String indexUUID, String shardId) throws IOException {
-        return newLockManager(repositoriesService.get(), repositoryName, indexUUID, shardId);
+    public RemoteStoreLockManager newLockManager(String repositoryName, String indexUUID, String shardId, RemoteStorePathType pathType) {
+        return newLockManager(repositoriesService.get(), repositoryName, indexUUID, shardId, pathType);
     }
 
     public static RemoteStoreMetadataLockManager newLockManager(
         RepositoriesService repositoriesService,
         String repositoryName,
         String indexUUID,
-        String shardId
-    ) throws IOException {
+        String shardId,
+        RemoteStorePathType pathType
+    ) {
         try (Repository repository = repositoriesService.repository(repositoryName)) {
             assert repository instanceof BlobStoreRepository : "repository should be instance of BlobStoreRepository";
-            BlobPath shardLevelBlobPath = ((BlobStoreRepository) repository).basePath().add(indexUUID).add(shardId).add(SEGMENTS);
-            RemoteBufferedOutputDirectory shardMDLockDirectory = createRemoteBufferedOutputDirectory(
-                repository,
-                shardLevelBlobPath,
-                LOCK_FILES
-            );
-
-            return new RemoteStoreMetadataLockManager(shardMDLockDirectory);
+            BlobPath repositoryBasePath = ((BlobStoreRepository) repository).basePath();
+            BlobPath lockDirectoryPath = pathType.path(repositoryBasePath, indexUUID, shardId, SEGMENTS, LOCK_FILES);
+            BlobContainer lockDirectoryBlobContainer = ((BlobStoreRepository) repository).blobStore().blobContainer(lockDirectoryPath);
+            return new RemoteStoreMetadataLockManager(new RemoteBufferedOutputDirectory(lockDirectoryBlobContainer));
         } catch (RepositoryMissingException e) {
             throw new IllegalArgumentException("Repository should be present to acquire/release lock", e);
         }
@@ -65,14 +63,4 @@ public static RemoteStoreMetadataLockManager newLockManager(
     public Supplier<RepositoriesService> getRepositoriesService() {
         return repositoriesService;
     }
-
-    private static RemoteBufferedOutputDirectory createRemoteBufferedOutputDirectory(
-        Repository repository,
-        BlobPath commonBlobPath,
-        String extention
-    ) {
-        BlobPath extendedPath = commonBlobPath.add(extention);
-        BlobContainer dataBlobContainer = ((BlobStoreRepository) repository).blobStore().blobContainer(extendedPath);
-        return new RemoteBufferedOutputDirectory(dataBlobContainer);
-    }
 }
diff --git a/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java b/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java
index 43eec01b2d365..f0fb03cc905a4 100644
--- a/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java
+++ b/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java
@@ -11,6 +11,7 @@
 import org.apache.logging.log4j.Logger;
 import org.opensearch.cluster.service.ClusterService;
 import org.opensearch.common.SetOnce;
+import org.opensearch.common.blobstore.BlobPath;
 import org.opensearch.common.lease.Releasable;
 import org.opensearch.common.lease.Releasables;
 import org.opensearch.common.logging.Loggers;
@@ -18,6 +19,7 @@
 import org.opensearch.common.util.io.IOUtils;
 import org.opensearch.core.index.shard.ShardId;
 import org.opensearch.core.util.FileSystemUtils;
+import org.opensearch.index.remote.RemoteStorePathType;
 import org.opensearch.index.remote.RemoteTranslogTransferTracker;
 import org.opensearch.index.translog.transfer.BlobStoreTransferService;
 import org.opensearch.index.translog.transfer.FileTransferTracker;
@@ -38,6 +40,7 @@
 import java.util.HashSet;
 import java.util.Locale;
 import java.util.Map;
+import java.util.Objects;
 import java.util.Set;
 import java.util.concurrent.Semaphore;
 import java.util.concurrent.TimeUnit;
@@ -47,6 +50,10 @@
 import java.util.function.LongConsumer;
 import java.util.function.LongSupplier;
 
+import static org.opensearch.index.remote.RemoteStoreDataEnums.DataCategory.TRANSLOG;
+import static org.opensearch.index.remote.RemoteStoreDataEnums.DataType.DATA;
+import static org.opensearch.index.remote.RemoteStoreDataEnums.DataType.METADATA;
+
 /**
  * A Translog implementation which syncs local FS with a remote store
  * The current impl uploads translog , ckp and metadata to remote store
@@ -74,7 +81,6 @@ public class RemoteFsTranslog extends Translog {
 
     private static final int REMOTE_DELETION_PERMITS = 2;
     private static final int DOWNLOAD_RETRIES = 2;
-    public static final String TRANSLOG = "translog";
 
     // Semaphore used to allow only single remote generation to happen at a time
     private final Semaphore remoteGenerationDeletionPermits = new Semaphore(REMOTE_DELETION_PERMITS);
@@ -106,7 +112,8 @@ public RemoteFsTranslog(
             threadPool,
             shardId,
             fileTransferTracker,
-            remoteTranslogTransferTracker
+            remoteTranslogTransferTracker,
+            indexSettings().getRemoteStorePathType()
         );
         try {
             download(translogTransferManager, location, logger);
@@ -150,8 +157,14 @@ RemoteTranslogTransferTracker getRemoteTranslogTracker() {
         return remoteTranslogTransferTracker;
     }
 
-    public static void download(Repository repository, ShardId shardId, ThreadPool threadPool, Path location, Logger logger)
-        throws IOException {
+    public static void download(
+        Repository repository,
+        ShardId shardId,
+        ThreadPool threadPool,
+        Path location,
+        RemoteStorePathType pathType,
+        Logger logger
+    ) throws IOException {
         assert repository instanceof BlobStoreRepository : String.format(
             Locale.ROOT,
             "%s repository should be instance of BlobStoreRepository",
@@ -167,7 +180,8 @@ public static void download(Repository repository, ShardId shardId, ThreadPool t
             threadPool,
             shardId,
             fileTransferTracker,
-            remoteTranslogTransferTracker
+            remoteTranslogTransferTracker,
+            pathType
         );
         RemoteFsTranslog.download(translogTransferManager, location, logger);
         logger.trace(remoteTranslogTransferTracker.toString());
@@ -244,15 +258,16 @@ public static TranslogTransferManager buildTranslogTransferManager(
         ThreadPool threadPool,
         ShardId shardId,
         FileTransferTracker fileTransferTracker,
-        RemoteTranslogTransferTracker remoteTranslogTransferTracker
+        RemoteTranslogTransferTracker tracker,
+        RemoteStorePathType pathType
     ) {
-        return new TranslogTransferManager(
-            shardId,
-            new BlobStoreTransferService(blobStoreRepository.blobStore(), threadPool),
-            blobStoreRepository.basePath().add(shardId.getIndex().getUUID()).add(String.valueOf(shardId.id())).add(TRANSLOG),
-            fileTransferTracker,
-            remoteTranslogTransferTracker
-        );
+        assert Objects.nonNull(pathType);
+        String indexUUID = shardId.getIndex().getUUID();
+        String shardIdStr = String.valueOf(shardId.id());
+        BlobPath dataPath = pathType.path(blobStoreRepository.basePath(), indexUUID, shardIdStr, TRANSLOG, DATA);
+        BlobPath mdPath = pathType.path(blobStoreRepository.basePath(), indexUUID, shardIdStr, TRANSLOG, METADATA);
+        BlobStoreTransferService transferService = new BlobStoreTransferService(blobStoreRepository.blobStore(), threadPool);
+        return new TranslogTransferManager(shardId, transferService, dataPath, mdPath, fileTransferTracker, tracker);
     }
 
     @Override
@@ -524,7 +539,8 @@ private void deleteStaleRemotePrimaryTerms() {
         }
     }
 
-    public static void cleanup(Repository repository, ShardId shardId, ThreadPool threadPool) throws IOException {
+    public static void cleanup(Repository repository, ShardId shardId, ThreadPool threadPool, RemoteStorePathType pathType)
+        throws IOException {
         assert repository instanceof BlobStoreRepository : "repository should be instance of BlobStoreRepository";
         BlobStoreRepository blobStoreRepository = (BlobStoreRepository) repository;
         // We use a dummy stats tracker to ensure the flow doesn't break.
@@ -536,7 +552,8 @@ public static void cleanup(Repository repository, ShardId shardId, ThreadPool th
             threadPool,
             shardId,
             fileTransferTracker,
-            remoteTranslogTransferTracker
+            remoteTranslogTransferTracker,
+            pathType
         );
         // clean up all remote translog files
         translogTransferManager.deleteTranslogFiles();
diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java
index 2f6055df87804..c9e07ca3ef8c1 100644
--- a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java
+++ b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java
@@ -58,7 +58,6 @@ public class TranslogTransferManager {
     private final TransferService transferService;
     private final BlobPath remoteDataTransferPath;
     private final BlobPath remoteMetadataTransferPath;
-    private final BlobPath remoteBaseTransferPath;
     private final FileTransferTracker fileTransferTracker;
     private final RemoteTranslogTransferTracker remoteTranslogTransferTracker;
 
@@ -67,8 +66,6 @@ public class TranslogTransferManager {
     private static final int METADATA_FILES_TO_FETCH = 10;
 
     private final Logger logger;
-    private final static String METADATA_DIR = "metadata";
-    private final static String DATA_DIR = "data";
 
     private static final VersionedCodecStreamWrapper<TranslogTransferMetadata> metadataStreamWrapper = new VersionedCodecStreamWrapper<>(
         new TranslogTransferMetadataHandler(),
@@ -79,15 +76,15 @@ public class TranslogTransferManager {
     public TranslogTransferManager(
         ShardId shardId,
         TransferService transferService,
-        BlobPath remoteBaseTransferPath,
+        BlobPath remoteDataTransferPath,
+        BlobPath remoteMetadataTransferPath,
         FileTransferTracker fileTransferTracker,
         RemoteTranslogTransferTracker remoteTranslogTransferTracker
     ) {
         this.shardId = shardId;
         this.transferService = transferService;
-        this.remoteBaseTransferPath = remoteBaseTransferPath;
-        this.remoteDataTransferPath = remoteBaseTransferPath.add(DATA_DIR);
-        this.remoteMetadataTransferPath = remoteBaseTransferPath.add(METADATA_DIR);
+        this.remoteDataTransferPath = remoteDataTransferPath;
+        this.remoteMetadataTransferPath = remoteMetadataTransferPath;
         this.fileTransferTracker = fileTransferTracker;
         this.logger = Loggers.getLogger(getClass(), shardId);
         this.remoteTranslogTransferTracker = remoteTranslogTransferTracker;
@@ -456,17 +453,27 @@ public void onFailure(Exception e) {
         );
     }
 
+    /**
+     * Deletes all the translog content related to the underlying shard.
+     */
     public void delete() {
-        // cleans up all the translog contents in async fashion
-        transferService.deleteAsync(ThreadPool.Names.REMOTE_PURGE, remoteBaseTransferPath, new ActionListener<>() {
+        // Delete the translog data content from the remote store.
+        delete(remoteDataTransferPath);
+        // Delete the translog metadata content from the remote store.
+        delete(remoteMetadataTransferPath);
+    }
+
+    private void delete(BlobPath path) {
+        // cleans up all the translog contents in async fashion for the given path
+        transferService.deleteAsync(ThreadPool.Names.REMOTE_PURGE, path, new ActionListener<>() {
             @Override
             public void onResponse(Void unused) {
-                logger.info("Deleted all remote translog data");
+                logger.info("Deleted all remote translog data at path={}", path);
             }
 
             @Override
             public void onFailure(Exception e) {
-                logger.error("Exception occurred while cleaning translog", e);
+                logger.error(new ParameterizedMessage("Exception occurred while cleaning translog at path={}", path), e);
             }
         });
     }
diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java
index 076173177feee..a7c2fb03285b0 100644
--- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java
+++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java
@@ -108,6 +108,7 @@
 import org.opensearch.core.xcontent.NamedXContentRegistry;
 import org.opensearch.core.xcontent.XContentParser;
 import org.opensearch.index.mapper.MapperService;
+import org.opensearch.index.remote.RemoteStorePathType;
 import org.opensearch.index.snapshots.IndexShardRestoreFailedException;
 import org.opensearch.index.snapshots.IndexShardSnapshotStatus;
 import org.opensearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot;
@@ -669,7 +670,8 @@ public void cloneRemoteStoreIndexShardSnapshot(
             RemoteStoreLockManager remoteStoreMetadataLockManger = remoteStoreLockManagerFactory.newLockManager(
                 remoteStoreRepository,
                 indexUUID,
-                String.valueOf(shardId.shardId())
+                String.valueOf(shardId.shardId()),
+                RemoteStorePathType.FIXED // TODO - The path type needs to be obtained from RemoteStoreShardShallowCopySnapshot
             );
             remoteStoreMetadataLockManger.cloneLock(
                 FileLockInfo.getLockInfoBuilder().withAcquirerId(source.getUUID()).build(),
@@ -1107,7 +1109,8 @@ public static void remoteDirectoryCleanupAsync(
         String remoteStoreRepoForIndex,
         String indexUUID,
         ShardId shardId,
-        String threadPoolName
+        String threadPoolName,
+        RemoteStorePathType pathType
     ) {
         threadpool.executor(threadPoolName)
             .execute(
@@ -1116,7 +1119,8 @@ public static void remoteDirectoryCleanupAsync(
                         remoteDirectoryFactory,
                         remoteStoreRepoForIndex,
                         indexUUID,
-                        shardId
+                        shardId,
+                        pathType
                     ),
                     indexUUID,
                     shardId
@@ -1147,7 +1151,8 @@ protected void releaseRemoteStoreLockAndCleanup(
         RemoteStoreLockManager remoteStoreMetadataLockManager = remoteStoreLockManagerFactory.newLockManager(
             remoteStoreRepoForIndex,
             indexUUID,
-            shardId
+            shardId,
+            RemoteStorePathType.HASHED_PREFIX // TODO - The path type needs to be obtained from RemoteStoreShardShallowCopySnapshot
         );
         remoteStoreMetadataLockManager.release(FileLockInfo.getLockInfoBuilder().withAcquirerId(shallowSnapshotUUID).build());
         logger.debug("Successfully released lock for shard {} of index with uuid {}", shardId, indexUUID);
@@ -1169,7 +1174,8 @@ protected void releaseRemoteStoreLockAndCleanup(
                 remoteStoreRepoForIndex,
                 indexUUID,
                 new ShardId(Index.UNKNOWN_INDEX_NAME, indexUUID, Integer.parseInt(shardId)),
-                ThreadPool.Names.REMOTE_PURGE
+                ThreadPool.Names.REMOTE_PURGE,
+                RemoteStorePathType.FIXED // TODO - The path type needs to be obtained from RemoteStoreShardShallowCopySnapshot
             );
         }
     }
diff --git a/server/src/main/java/org/opensearch/snapshots/RestoreService.java b/server/src/main/java/org/opensearch/snapshots/RestoreService.java
index e5ac604e0a5e3..da2a36cbb0701 100644
--- a/server/src/main/java/org/opensearch/snapshots/RestoreService.java
+++ b/server/src/main/java/org/opensearch/snapshots/RestoreService.java
@@ -451,7 +451,7 @@ public ClusterState execute(ClusterState currentState) {
                                             .put(snapshotIndexMetadata.getSettings())
                                             .put(IndexMetadata.SETTING_INDEX_UUID, UUIDs.randomBase64UUID())
                                     );
-                                    createIndexService.addRemoteCustomData(indexMdBuilder);
+                                    createIndexService.addRemoteStorePathTypeInCustomData(indexMdBuilder, false);
                                     shardLimitValidator.validateShardLimit(
                                         renamedIndexName,
                                         snapshotIndexMetadata.getSettings(),
diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteStorePathTypeTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteStorePathTypeTests.java
new file mode 100644
index 0000000000000..0f108d1b45f5a
--- /dev/null
+++ b/server/src/test/java/org/opensearch/index/remote/RemoteStorePathTypeTests.java
@@ -0,0 +1,111 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.index.remote;
+
+import org.opensearch.common.blobstore.BlobPath;
+import org.opensearch.index.remote.RemoteStoreDataEnums.DataCategory;
+import org.opensearch.index.remote.RemoteStoreDataEnums.DataType;
+import org.opensearch.test.OpenSearchTestCase;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+
+import static org.opensearch.index.remote.RemoteStoreDataEnums.DataCategory.SEGMENTS;
+import static org.opensearch.index.remote.RemoteStoreDataEnums.DataCategory.TRANSLOG;
+import static org.opensearch.index.remote.RemoteStoreDataEnums.DataType.DATA;
+import static org.opensearch.index.remote.RemoteStoreDataEnums.DataType.LOCK_FILES;
+import static org.opensearch.index.remote.RemoteStoreDataEnums.DataType.METADATA;
+import static org.opensearch.index.remote.RemoteStorePathType.FIXED;
+import static org.opensearch.index.remote.RemoteStorePathType.parseString;
+
+public class RemoteStorePathTypeTests extends OpenSearchTestCase {
+
+    private static final String SEPARATOR = "/";
+
+    public void testParseString() {
+        // Case 1 - Pass values from the enum.
+        String typeString = FIXED.toString();
+        RemoteStorePathType type = parseString(randomFrom(typeString, typeString.toLowerCase(Locale.ROOT)));
+        assertEquals(FIXED, type);
+
+        typeString = RemoteStorePathType.HASHED_PREFIX.toString();
+        type = parseString(randomFrom(typeString, typeString.toLowerCase(Locale.ROOT)));
+        assertEquals(RemoteStorePathType.HASHED_PREFIX, type);
+
+        // Case 2 - Pass random string
+        String randomTypeString = randomAlphaOfLength(2);
+        IllegalArgumentException ex = assertThrows(IllegalArgumentException.class, () -> parseString(randomTypeString));
+        assertEquals("Could not parse RemoteStorePathType for [" + randomTypeString + "]", ex.getMessage());
+
+        // Case 3 - Null string
+        ex = assertThrows(IllegalArgumentException.class, () -> parseString(null));
+        assertEquals("Could not parse RemoteStorePathType for [null]", ex.getMessage());
+    }
+
+    public void testGeneratePathForFixedType() {
+        BlobPath blobPath = new BlobPath();
+        List<String> pathList = getPathList();
+        for (String path : pathList) {
+            blobPath = blobPath.add(path);
+        }
+
+        String indexUUID = randomAlphaOfLength(10);
+        String shardId = String.valueOf(randomInt(100));
+        DataCategory dataCategory = TRANSLOG;
+        DataType dataType = DATA;
+
+        String basePath = getPath(pathList) + indexUUID + SEPARATOR + shardId + SEPARATOR;
+        // Translog Data
+        BlobPath result = FIXED.path(blobPath, indexUUID, shardId, dataCategory, dataType);
+        assertEquals(basePath + dataCategory.getName() + SEPARATOR + dataType.getName() + SEPARATOR, result.buildAsString());
+
+        // Translog Metadata
+        dataType = METADATA;
+        result = FIXED.path(blobPath, indexUUID, shardId, dataCategory, dataType);
+        assertEquals(basePath + dataCategory.getName() + SEPARATOR + dataType.getName() + SEPARATOR, result.buildAsString());
+
+        // Translog Lock files - This is a negative case where the assertion will trip.
+        BlobPath finalBlobPath = blobPath;
+        assertThrows(AssertionError.class, () -> FIXED.path(finalBlobPath, indexUUID, shardId, TRANSLOG, LOCK_FILES));
+
+        // Segment Data
+        dataCategory = SEGMENTS;
+        dataType = DATA;
+        result = FIXED.path(blobPath, indexUUID, shardId, dataCategory, dataType);
+        assertEquals(basePath + dataCategory.getName() + SEPARATOR + dataType.getName() + SEPARATOR, result.buildAsString());
+
+        // Segment Metadata
+        dataType = METADATA;
+        result = FIXED.path(blobPath, indexUUID, shardId, dataCategory, dataType);
+        assertEquals(basePath + dataCategory.getName() + SEPARATOR + dataType.getName() + SEPARATOR, result.buildAsString());
+
+        // Segment Metadata
+        dataType = LOCK_FILES;
+        result = FIXED.path(blobPath, indexUUID, shardId, dataCategory, dataType);
+        assertEquals(basePath + dataCategory.getName() + SEPARATOR + dataType.getName() + SEPARATOR, result.buildAsString());
+    }
+
+    private List<String> getPathList() {
+        List<String> pathList = new ArrayList<>();
+        int length = randomIntBetween(0, 5);
+        for (int i = 0; i < length; i++) {
+            pathList.add(randomAlphaOfLength(randomIntBetween(2, 5)));
+        }
+        return pathList;
+    }
+
+    private String getPath(List<String> pathList) {
+        String p = String.join(SEPARATOR, pathList);
+        if (p.isEmpty() || p.endsWith(SEPARATOR)) {
+            return p;
+        }
+        return p + SEPARATOR;
+    }
+}
diff --git a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java
index 8b69c15dac9d3..59c8a9d92f07b 100644
--- a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java
+++ b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java
@@ -37,6 +37,7 @@
 import org.opensearch.core.index.Index;
 import org.opensearch.core.index.shard.ShardId;
 import org.opensearch.index.engine.NRTReplicationEngineFactory;
+import org.opensearch.index.remote.RemoteStorePathType;
 import org.opensearch.index.remote.RemoteStoreUtils;
 import org.opensearch.index.shard.IndexShard;
 import org.opensearch.index.shard.IndexShardTestCase;
@@ -697,13 +698,20 @@ public void testCleanupAsync() throws Exception {
             threadPool,
             indexShard.shardId()
         );
-        when(remoteSegmentStoreDirectoryFactory.newDirectory(any(), any(), any())).thenReturn(remoteSegmentDirectory);
+        when(remoteSegmentStoreDirectoryFactory.newDirectory(any(), any(), any(), any())).thenReturn(remoteSegmentDirectory);
         String repositoryName = "test-repository";
         String indexUUID = "test-idx-uuid";
         ShardId shardId = new ShardId(Index.UNKNOWN_INDEX_NAME, indexUUID, Integer.parseInt("0"));
-
-        RemoteSegmentStoreDirectory.remoteDirectoryCleanup(remoteSegmentStoreDirectoryFactory, repositoryName, indexUUID, shardId);
-        verify(remoteSegmentStoreDirectoryFactory).newDirectory(repositoryName, indexUUID, shardId);
+        RemoteStorePathType pathType = randomFrom(RemoteStorePathType.values());
+
+        RemoteSegmentStoreDirectory.remoteDirectoryCleanup(
+            remoteSegmentStoreDirectoryFactory,
+            repositoryName,
+            indexUUID,
+            shardId,
+            pathType
+        );
+        verify(remoteSegmentStoreDirectoryFactory).newDirectory(repositoryName, indexUUID, shardId, pathType);
         verify(threadPool, times(0)).executor(ThreadPool.Names.REMOTE_PURGE);
         verify(remoteMetadataDirectory).delete();
         verify(remoteDataDirectory).delete();
diff --git a/server/src/test/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManagerFactoryTests.java b/server/src/test/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManagerFactoryTests.java
index 897785849cf7b..0fe5557737447 100644
--- a/server/src/test/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManagerFactoryTests.java
+++ b/server/src/test/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManagerFactoryTests.java
@@ -11,6 +11,7 @@
 import org.opensearch.common.blobstore.BlobContainer;
 import org.opensearch.common.blobstore.BlobPath;
 import org.opensearch.common.blobstore.BlobStore;
+import org.opensearch.index.remote.RemoteStorePathType;
 import org.opensearch.repositories.RepositoriesService;
 import org.opensearch.repositories.blobstore.BlobStoreRepository;
 import org.opensearch.test.OpenSearchTestCase;
@@ -48,6 +49,7 @@ public void testNewLockManager() throws IOException {
         String testRepository = "testRepository";
         String testIndexUUID = "testIndexUUID";
         String testShardId = "testShardId";
+        RemoteStorePathType pathType = RemoteStorePathType.FIXED;
 
         BlobStoreRepository repository = mock(BlobStoreRepository.class);
         BlobStore blobStore = mock(BlobStore.class);
@@ -59,7 +61,12 @@ public void testNewLockManager() throws IOException {
 
         when(repositoriesService.repository(testRepository)).thenReturn(repository);
 
-        RemoteStoreLockManager lockManager = remoteStoreLockManagerFactory.newLockManager(testRepository, testIndexUUID, testShardId);
+        RemoteStoreLockManager lockManager = remoteStoreLockManagerFactory.newLockManager(
+            testRepository,
+            testIndexUUID,
+            testShardId,
+            pathType
+        );
 
         assertTrue(lockManager != null);
         ArgumentCaptor<BlobPath> blobPathCaptor = ArgumentCaptor.forClass(BlobPath.class);
diff --git a/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java b/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java
index 7ff4c3ecf5236..9f72d3c7bd825 100644
--- a/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java
+++ b/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java
@@ -99,7 +99,7 @@
 
 import static org.opensearch.common.util.BigArrays.NON_RECYCLING_INSTANCE;
 import static org.opensearch.index.IndexSettings.INDEX_REMOTE_TRANSLOG_KEEP_EXTRA_GEN_SETTING;
-import static org.opensearch.index.translog.RemoteFsTranslog.TRANSLOG;
+import static org.opensearch.index.remote.RemoteStoreDataEnums.DataCategory.TRANSLOG;
 import static org.opensearch.index.translog.SnapshotMatchers.containsOperationsInAnyOrder;
 import static org.opensearch.index.translog.TranslogDeletionPolicies.createTranslogDeletionPolicy;
 import static org.hamcrest.Matchers.contains;
@@ -907,7 +907,7 @@ public void testDrainSync() throws Exception {
     }
 
     private BlobPath getTranslogDirectory() {
-        return repository.basePath().add(shardId.getIndex().getUUID()).add(String.valueOf(shardId.id())).add(TRANSLOG);
+        return repository.basePath().add(shardId.getIndex().getUUID()).add(String.valueOf(shardId.id())).add(TRANSLOG.getName());
     }
 
     private Long populateTranslogOps(boolean withMissingOps) throws IOException {
diff --git a/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java b/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java
index e34bc078896f9..a9502dc051428 100644
--- a/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java
+++ b/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java
@@ -48,6 +48,8 @@
 
 import org.mockito.Mockito;
 
+import static org.opensearch.index.remote.RemoteStoreDataEnums.DataCategory.TRANSLOG;
+import static org.opensearch.index.remote.RemoteStoreDataEnums.DataType.METADATA;
 import static org.opensearch.index.translog.transfer.TranslogTransferMetadata.METADATA_SEPARATOR;
 import static org.mockito.ArgumentMatchers.anyInt;
 import static org.mockito.ArgumentMatchers.anyMap;
@@ -95,7 +97,8 @@ public void setUp() throws Exception {
         translogTransferManager = new TranslogTransferManager(
             shardId,
             transferService,
-            remoteBaseTransferPath,
+            remoteBaseTransferPath.add(TRANSLOG.getName()),
+            remoteBaseTransferPath.add(METADATA.getName()),
             tracker,
             remoteTranslogTransferTracker
         );
@@ -159,7 +162,8 @@ public void onFailure(TransferFileSnapshot fileSnapshot, Exception e) {
         TranslogTransferManager translogTransferManager = new TranslogTransferManager(
             shardId,
             transferService,
-            remoteBaseTransferPath,
+            remoteBaseTransferPath.add(TRANSLOG.getName()),
+            remoteBaseTransferPath.add(METADATA.getName()),
             fileTransferTracker,
             remoteTranslogTransferTracker
         );
@@ -194,7 +198,8 @@ public void testTransferSnapshotOnUploadTimeout() throws Exception {
         TranslogTransferManager translogTransferManager = new TranslogTransferManager(
             shardId,
             transferService,
-            remoteBaseTransferPath,
+            remoteBaseTransferPath.add(TRANSLOG.getName()),
+            remoteBaseTransferPath.add(METADATA.getName()),
             fileTransferTracker,
             remoteTranslogTransferTracker
         );
@@ -235,7 +240,8 @@ public void testTransferSnapshotOnThreadInterrupt() throws Exception {
         TranslogTransferManager translogTransferManager = new TranslogTransferManager(
             shardId,
             transferService,
-            remoteBaseTransferPath,
+            remoteBaseTransferPath.add(TRANSLOG.getName()),
+            remoteBaseTransferPath.add(METADATA.getName()),
             fileTransferTracker,
             remoteTranslogTransferTracker
         );
@@ -333,7 +339,8 @@ public void testReadMetadataNoFile() throws IOException {
         TranslogTransferManager translogTransferManager = new TranslogTransferManager(
             shardId,
             transferService,
-            remoteBaseTransferPath,
+            remoteBaseTransferPath.add(TRANSLOG.getName()),
+            remoteBaseTransferPath.add(METADATA.getName()),
             null,
             remoteTranslogTransferTracker
         );
@@ -354,7 +361,8 @@ public void testReadMetadataFile() throws IOException {
         TranslogTransferManager translogTransferManager = new TranslogTransferManager(
             shardId,
             transferService,
-            remoteBaseTransferPath,
+            remoteBaseTransferPath.add(TRANSLOG.getName()),
+            remoteBaseTransferPath.add(METADATA.getName()),
             null,
             remoteTranslogTransferTracker
         );
@@ -390,7 +398,8 @@ public void testReadMetadataReadException() throws IOException {
         TranslogTransferManager translogTransferManager = new TranslogTransferManager(
             shardId,
             transferService,
-            remoteBaseTransferPath,
+            remoteBaseTransferPath.add(TRANSLOG.getName()),
+            remoteBaseTransferPath.add(METADATA.getName()),
             null,
             remoteTranslogTransferTracker
         );
@@ -426,7 +435,8 @@ public void testReadMetadataListException() throws IOException {
         TranslogTransferManager translogTransferManager = new TranslogTransferManager(
             shardId,
             transferService,
-            remoteBaseTransferPath,
+            remoteBaseTransferPath.add(TRANSLOG.getName()),
+            remoteBaseTransferPath.add(METADATA.getName()),
             null,
             remoteTranslogTransferTracker
         );
@@ -499,7 +509,8 @@ public void testDeleteTranslogSuccess() throws Exception {
         TranslogTransferManager translogTransferManager = new TranslogTransferManager(
             shardId,
             blobStoreTransferService,
-            remoteBaseTransferPath,
+            remoteBaseTransferPath.add(TRANSLOG.getName()),
+            remoteBaseTransferPath.add(METADATA.getName()),
             tracker,
             remoteTranslogTransferTracker
         );
@@ -518,7 +529,8 @@ public void testDeleteStaleTranslogMetadata() {
         TranslogTransferManager translogTransferManager = new TranslogTransferManager(
             shardId,
             transferService,
-            remoteBaseTransferPath,
+            remoteBaseTransferPath.add(TRANSLOG.getName()),
+            remoteBaseTransferPath.add(METADATA.getName()),
             null,
             remoteTranslogTransferTracker
         );
@@ -569,7 +581,8 @@ public void testDeleteTranslogFailure() throws Exception {
         TranslogTransferManager translogTransferManager = new TranslogTransferManager(
             shardId,
             blobStoreTransferService,
-            remoteBaseTransferPath,
+            remoteBaseTransferPath.add(TRANSLOG.getName()),
+            remoteBaseTransferPath.add(METADATA.getName()),
             tracker,
             remoteTranslogTransferTracker
         );
@@ -612,7 +625,8 @@ public void testMetadataConflict() throws InterruptedException {
         TranslogTransferManager translogTransferManager = new TranslogTransferManager(
             shardId,
             transferService,
-            remoteBaseTransferPath,
+            remoteBaseTransferPath.add(TRANSLOG.getName()),
+            remoteBaseTransferPath.add(METADATA.getName()),
             null,
             remoteTranslogTransferTracker
         );