This is an automated email from the ASF dual-hosted git repository.

mblow pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/asterixdb.git

commit 2b16b5585f006309c715a737fd3b4b92da4cd56e
Author: Ritik Raj <[email protected]>
AuthorDate: Thu Oct 23 19:08:56 2025 +0530

    [NO ISSUE][CLOUD] Use Azure async parallel downloader APIs
    
    - user model changes: no
    - storage format changes: no
    - interface changes: no
    
    Details:
    Introduced async apis for parallel downloading
    the files
    
    Ext-ref:MB-69079
    
    Change-Id: Iaab692c8d70019175738fdb39699796cf9ef72b8
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/20513
    Integration-Tests: Jenkins <[email protected]>
    Tested-by: Jenkins <[email protected]>
    Reviewed-by: Ali Alsuliman <[email protected]>
---
 .../blobstorage/AzBlobStorageClientConfig.java     |  38 ++++-
 .../blobstorage/AzBlobStorageCloudClient.java      |   7 +-
 .../azure/blobstorage/AzureParallelDownloader.java | 154 +++++++++++++--------
 .../asterix/cloud/azure/LSMAzBlobStorageTest.java  |   4 +-
 .../asterix/common/config/CloudProperties.java     |  10 +-
 .../apache/hyracks/api/io/IFileDeviceResolver.java |   4 +-
 .../java/org/apache/hyracks/api/io/IIOManager.java |   2 +-
 .../control/nc/io/DefaultDeviceResolver.java       |   3 +-
 .../apache/hyracks/control/nc/io/IOManager.java    |   2 +-
 9 files changed, 144 insertions(+), 80 deletions(-)

diff --git 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzBlobStorageClientConfig.java
 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzBlobStorageClientConfig.java
index 9abfbcc50c..548ad04633 100644
--- 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzBlobStorageClientConfig.java
+++ 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzBlobStorageClientConfig.java
@@ -46,18 +46,23 @@ public class AzBlobStorageClientConfig {
     private final int writeMaxRequestsPerSeconds;
     private final int readMaxRequestsPerSeconds;
     private final boolean storageDisableSSLVerify;
+    private final int requestsMaxHttpConnections;
+    private final int requestsMaxPendingHttpConnections;
+    private final int requestsHttpConnectionAcquireTimeout;
     private final AccessTier accessTier;
 
     public AzBlobStorageClientConfig(String region, String endpoint, String 
prefix, boolean anonymousAuth,
             long profilerLogInterval, String bucket, int writeBufferSize) {
         this(region, endpoint, prefix, anonymousAuth, profilerLogInterval, 
bucket, 1, 0, 0, writeBufferSize, false,
-                null);
+                null, CloudProperties.MAX_HTTP_CONNECTIONS, 
CloudProperties.MAX_PENDING_HTTP_CONNECTIONS,
+                CloudProperties.HTTP_CONNECTION_ACQUIRE_TIMEOUT);
     }
 
     public AzBlobStorageClientConfig(String region, String endpoint, String 
prefix, boolean anonymousAuth,
             long profilerLogInterval, String bucket, long tokenAcquireTimeout, 
int writeMaxRequestsPerSeconds,
-            int readMaxRequestsPerSeconds, int writeBufferSize, boolean 
storageDisableSSLVerify,
-            AccessTier accessTier) {
+            int readMaxRequestsPerSeconds, int writeBufferSize, boolean 
storageDisableSSLVerify, AccessTier accessTier,
+            int requestsMaxHttpConnections, int 
requestsMaxPendingHttpConnections,
+            int requestsHttpConnectionAcquireTimeout) {
         this.region = Objects.requireNonNull(region, "region");
         this.endpoint = endpoint;
         this.prefix = Objects.requireNonNull(prefix, "prefix");
@@ -69,6 +74,10 @@ public class AzBlobStorageClientConfig {
         this.readMaxRequestsPerSeconds = readMaxRequestsPerSeconds;
         this.writeBufferSize = writeBufferSize;
         this.storageDisableSSLVerify = storageDisableSSLVerify;
+        this.requestsMaxHttpConnections = requestsMaxHttpConnections;
+        this.requestsMaxPendingHttpConnections =
+                
getRequestsMaxPendingHttpConnections(requestsMaxPendingHttpConnections);
+        this.requestsHttpConnectionAcquireTimeout = 
requestsHttpConnectionAcquireTimeout;
         this.accessTier = accessTier;
     }
 
@@ -78,7 +87,9 @@ public class AzBlobStorageClientConfig {
                 cloudProperties.getProfilerLogInterval(), 
cloudProperties.getStorageBucket(),
                 cloudProperties.getTokenAcquireTimeout(), 
cloudProperties.getWriteMaxRequestsPerSecond(),
                 cloudProperties.getReadMaxRequestsPerSecond(), 
cloudProperties.getWriteBufferSize(),
-                cloudProperties.isStorageDisableSSLVerify(), 
INTERNAL_STORAGE_ACCESS_TIER);
+                cloudProperties.isStorageDisableSSLVerify(), 
INTERNAL_STORAGE_ACCESS_TIER,
+                cloudProperties.getRequestsMaxHttpConnections(), 
cloudProperties.getRequestsMaxPendingHttpConnections(),
+                cloudProperties.getRequestsHttpConnectionAcquireTimeout());
     }
 
     public static AzBlobStorageClientConfig of(Map<String, String> 
configuration, int writeBufferSize) {
@@ -148,4 +159,23 @@ public class AzBlobStorageClientConfig {
     public AccessTier getAccessTier() {
         return accessTier;
     }
+
+    public int getRequestsMaxHttpConnections() {
+        return requestsMaxHttpConnections;
+    }
+
+    public int getRequestsMaxPendingHttpConnections() {
+        return requestsMaxPendingHttpConnections;
+    }
+
+    public int getRequestsHttpConnectionAcquireTimeout() {
+        return requestsHttpConnectionAcquireTimeout;
+    }
+
+    private static int getRequestsMaxPendingHttpConnections(int 
requestsMaxPendingHttpConnections) {
+        if (requestsMaxPendingHttpConnections <= 0) {
+            throw new 
IllegalArgumentException("requestsMaxPendingHttpConnections must be greater 
than 0");
+        }
+        return requestsMaxPendingHttpConnections;
+    }
 }
diff --git 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzBlobStorageCloudClient.java
 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzBlobStorageCloudClient.java
index 43e5fd231e..293907f21d 100644
--- 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzBlobStorageCloudClient.java
+++ 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzBlobStorageCloudClient.java
@@ -19,8 +19,6 @@
 
 package org.apache.asterix.cloud.clients.azure.blobstorage;
 
-import static 
org.apache.asterix.cloud.clients.azure.blobstorage.AzBlobStorageClientConfig.MAX_CONCURRENT_REQUESTS;
-
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.FilenameFilter;
@@ -299,7 +297,8 @@ public class AzBlobStorageCloudClient implements 
ICloudClient {
         }
 
         try {
-            Flux.fromIterable(deleteMonos).flatMap(mono -> mono, 
MAX_CONCURRENT_REQUESTS).then().block();
+            Flux.fromIterable(deleteMonos).flatMap(mono -> mono, 
config.getRequestsMaxPendingHttpConnections()).then()
+                    .block();
         } catch (Exception ex) {
             throw new RuntimeDataException(ErrorCode.CLOUD_IO_FAILURE, 
"DELETE", ex, paths.toString());
         }
@@ -354,7 +353,7 @@ public class AzBlobStorageCloudClient implements 
ICloudClient {
 
     @Override
     public IParallelDownloader createParallelDownloader(String bucket, 
IOManager ioManager) {
-        return new AzureParallelDownloader(ioManager, blobContainerClient, 
profiler, config);
+        return new AzureParallelDownloader(ioManager, 
blobContainerAsyncClient, profiler, config);
     }
 
     @Override
diff --git 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzureParallelDownloader.java
 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzureParallelDownloader.java
index 498058703f..25bc217afd 100644
--- 
a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzureParallelDownloader.java
+++ 
b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzureParallelDownloader.java
@@ -19,129 +19,150 @@
 
 package org.apache.asterix.cloud.clients.azure.blobstorage;
 
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
 import java.io.IOException;
-import java.io.OutputStream;
 import java.nio.file.Files;
 import java.nio.file.InvalidPathException;
 import java.nio.file.Path;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Set;
 
 import org.apache.asterix.cloud.clients.IParallelDownloader;
 import org.apache.asterix.cloud.clients.profiler.IRequestProfilerLimiter;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.util.ExceptionUtils;
 import org.apache.hyracks.control.nc.io.IOManager;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
 
-import com.azure.core.http.rest.PagedIterable;
-import com.azure.storage.blob.BlobClient;
-import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.BlobAsyncClient;
+import com.azure.storage.blob.BlobContainerAsyncClient;
 import com.azure.storage.blob.models.BlobItem;
 import com.azure.storage.blob.models.ListBlobsOptions;
 
+import reactor.core.publisher.Flux;
+import reactor.core.publisher.Mono;
+
 public class AzureParallelDownloader implements IParallelDownloader {
     public static final String STORAGE_SUB_DIR = "storage";
     private final IOManager ioManager;
-    private final BlobContainerClient blobContainerClient;
+    private final BlobContainerAsyncClient blobContainerAsyncClient;
     private final IRequestProfilerLimiter profiler;
     private final AzBlobStorageClientConfig config;
-    private static final Logger LOGGER = LogManager.getLogger();
 
-    public AzureParallelDownloader(IOManager ioManager, BlobContainerClient 
blobContainerClient,
+    public AzureParallelDownloader(IOManager ioManager, 
BlobContainerAsyncClient blobContainerAsyncClient,
             IRequestProfilerLimiter profiler, AzBlobStorageClientConfig 
config) {
         this.ioManager = ioManager;
-        this.blobContainerClient = blobContainerClient;
+        this.blobContainerAsyncClient = blobContainerAsyncClient;
         this.profiler = profiler;
         this.config = config;
     }
 
     @Override
     public void downloadFiles(Collection<FileReference> toDownload) throws 
HyracksDataException {
+        try {
+            downloadFilesAndWait(toDownload);
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    private void downloadFilesAndWait(Collection<FileReference> toDownload) 
throws IOException {
+        List<Mono<Void>> downloads = new ArrayList<>();
+        int maxConcurrent = config.getRequestsMaxPendingHttpConnections();
+
         for (FileReference fileReference : toDownload) {
-            BlobClient blobClient =
-                    blobContainerClient.getBlobClient(config.getPrefix() + 
fileReference.getRelativePath());
+            profiler.objectGet();
+
             Path absPath = Path.of(fileReference.getAbsolutePath());
             Path parentPath = absPath.getParent();
-            OutputStream fileOutputStream = null;
-            try {
-                createDirectories(parentPath);
-                fileOutputStream = Files.newOutputStream(absPath);
-                blobClient.downloadStream(fileOutputStream);
-                fileOutputStream.close();
-            } catch (IOException e) {
-                throw HyracksDataException.create(e);
-            } finally {
-                closeOutputStream(fileOutputStream);
+            createDirectories(parentPath);
+
+            BlobAsyncClient blobAsyncClient =
+                    
blobContainerAsyncClient.getBlobAsyncClient(config.getPrefix() + 
fileReference.getRelativePath());
+
+            Mono<Void> downloadTask = 
blobAsyncClient.downloadToFile(absPath.toString()).then();
+            downloads.add(downloadTask);
+
+            if (maxConcurrent > 0 && downloads.size() >= maxConcurrent) {
+                waitForFileDownloads(downloads);
+                downloads.clear();
             }
         }
-    }
 
-    private static void closeOutputStream(OutputStream fileOutputStream) 
throws HyracksDataException {
-        if (fileOutputStream != null) {
-            try {
-                fileOutputStream.close();
-            } catch (IOException e) {
-                throw HyracksDataException.create(e);
-            }
+        if (!downloads.isEmpty()) {
+            waitForFileDownloads(downloads);
         }
     }
 
+    private void waitForFileDownloads(List<Mono<Void>> downloads) throws 
HyracksDataException {
+        runBlockingWithExceptionHandling(
+                () -> Flux.fromIterable(downloads).flatMap(mono -> mono, 
downloads.size()).then().block());
+    }
+
     @Override
     public Collection<FileReference> 
downloadDirectories(Collection<FileReference> directories)
             throws HyracksDataException {
+
         Set<FileReference> failedFiles = new HashSet<>();
+        List<Mono<Void>> directoryDownloads = new ArrayList<>();
+
         for (FileReference directory : directories) {
-            PagedIterable<BlobItem> blobsInDir = getBlobItems(directory);
-            for (BlobItem blobItem : blobsInDir) {
-                profiler.objectGet();
-                download(blobItem, failedFiles);
-            }
+            Mono<Void> directoryTask = downloadDirectoryAsync(directory, 
failedFiles).onErrorResume(e -> Mono.empty()); // Continue even if a directory 
fails
+            directoryDownloads.add(directoryTask);
         }
+
+        runBlockingWithExceptionHandling(() -> 
Flux.fromIterable(directoryDownloads)
+                .flatMap(mono -> mono, 
config.getRequestsMaxPendingHttpConnections()).then().block());
+
         return failedFiles;
     }
 
-    private void download(BlobItem blobItem, Set<FileReference> failedFiles) 
throws HyracksDataException {
-        BlobClient blobClient = 
blobContainerClient.getBlobClient(blobItem.getName());
-        FileReference diskDestFile = 
ioManager.resolve(createDiskSubPath(blobItem.getName()));
-        Path absDiskBlobPath = getDiskDestPath(diskDestFile);
-        Path parentDiskPath = absDiskBlobPath.getParent();
-        createDirectories(parentDiskPath);
-        FileOutputStream outputStreamToDest = 
getOutputStreamToDest(diskDestFile);
+    private Mono<Void> downloadDirectoryAsync(FileReference directory, 
Set<FileReference> failedFiles) {
+        return getBlobItems(directory).flatMap(blobItem -> {
+            profiler.objectGet();
+            return downloadBlobAsync(blobItem, failedFiles);
+        }, 
config.getRequestsMaxPendingHttpConnections()).then().doOnError(error -> 
failedFiles.add(directory)); // Record directory failure
+    }
+
+    private Mono<Void> downloadBlobAsync(BlobItem blobItem, Set<FileReference> 
failedFiles) {
         try {
-            blobClient.downloadStream(outputStreamToDest);
+            // Resolve destination path
+            FileReference diskDestFile = 
ioManager.resolve(createDiskSubPath(blobItem.getName()));
+            Path absDiskBlobPath = getDiskDestPath(diskDestFile);
+            Path parentDiskPath = absDiskBlobPath.getParent();
+
+            createDirectories(parentDiskPath);
+
+            BlobAsyncClient blobAsyncClient = 
blobContainerAsyncClient.getBlobAsyncClient(blobItem.getName());
+
+            return 
blobAsyncClient.downloadToFile(absDiskBlobPath.toString()).doOnError(error -> {
+                FileReference failedFile = 
ioManager.resolve(blobItem.getName());
+                failedFiles.add(failedFile);
+            }).then();
         } catch (Exception e) {
-            FileReference failedFile = ioManager.resolve(blobItem.getName());
-            failedFiles.add(failedFile);
+            failedFiles.add(ioManager.resolve(blobItem.getName()));
+            return Mono.error(HyracksDataException.create(e));
         }
     }
 
     private String createDiskSubPath(String blobName) {
-        if (!blobName.startsWith(STORAGE_SUB_DIR)) {
-            blobName = blobName.substring(blobName.indexOf(STORAGE_SUB_DIR));
+        int idx = blobName.indexOf(STORAGE_SUB_DIR);
+        if (idx >= 0) {
+            return blobName.substring(idx);
         }
         return blobName;
     }
 
-    private FileOutputStream getOutputStreamToDest(FileReference destFile) 
throws HyracksDataException {
-        try {
-            return new FileOutputStream(destFile.getAbsolutePath());
-        } catch (FileNotFoundException ex) {
-            throw HyracksDataException.create(ex);
-        }
-    }
-
     private void createDirectories(Path parentPath) throws 
HyracksDataException {
-        if (Files.notExists(parentPath))
+        if (Files.notExists(parentPath)) {
             try {
                 Files.createDirectories(parentPath);
             } catch (IOException ex) {
                 throw HyracksDataException.create(ex);
             }
+        }
     }
 
     private Path getDiskDestPath(FileReference destFile) throws 
HyracksDataException {
@@ -152,10 +173,10 @@ public class AzureParallelDownloader implements 
IParallelDownloader {
         }
     }
 
-    private PagedIterable<BlobItem> getBlobItems(FileReference 
directoryToDownload) {
+    private Flux<BlobItem> getBlobItems(FileReference directoryToDownload) {
         ListBlobsOptions listBlobsOptions =
                 new ListBlobsOptions().setPrefix(config.getPrefix() + 
directoryToDownload.getRelativePath());
-        return blobContainerClient.listBlobs(listBlobsOptions, null);
+        return blobContainerAsyncClient.listBlobs(listBlobsOptions);
     }
 
     @Override
@@ -165,4 +186,15 @@ public class AzureParallelDownloader implements 
IParallelDownloader {
         // Ref: https://github.com/Azure/azure-sdk-for-java/issues/17903
         // Hence this implementation is a no op.
     }
-}
+
+    private static void runBlockingWithExceptionHandling(Runnable runnable) 
throws HyracksDataException {
+        try {
+            runnable.run();
+        } catch (Exception e) {
+            if (ExceptionUtils.causedByInterrupt(e)) {
+                Thread.currentThread().interrupt();
+            }
+            throw HyracksDataException.create(e);
+        }
+    }
+}
\ No newline at end of file
diff --git 
a/asterixdb/asterix-cloud/src/test/java/org/apache/asterix/cloud/azure/LSMAzBlobStorageTest.java
 
b/asterixdb/asterix-cloud/src/test/java/org/apache/asterix/cloud/azure/LSMAzBlobStorageTest.java
index d9424a88be..fcd6e17b91 100644
--- 
a/asterixdb/asterix-cloud/src/test/java/org/apache/asterix/cloud/azure/LSMAzBlobStorageTest.java
+++ 
b/asterixdb/asterix-cloud/src/test/java/org/apache/asterix/cloud/azure/LSMAzBlobStorageTest.java
@@ -35,6 +35,7 @@ import org.apache.asterix.cloud.AbstractLSMTest;
 import org.apache.asterix.cloud.clients.ICloudGuardian;
 import 
org.apache.asterix.cloud.clients.azure.blobstorage.AzBlobStorageClientConfig;
 import 
org.apache.asterix.cloud.clients.azure.blobstorage.AzBlobStorageCloudClient;
+import org.apache.asterix.common.config.CloudProperties;
 import org.apache.hyracks.util.StorageUtil;
 import org.bouncycastle.asn1.x500.X500Name;
 import org.bouncycastle.asn1.x509.SubjectPublicKeyInfo;
@@ -117,7 +118,8 @@ public class LSMAzBlobStorageTest extends AbstractLSMTest {
         URI blobStore = URI.create(blobServiceClient.getAccountUrl());
         String endpoint = blobStore.getScheme() + "://" + 
blobStore.getAuthority() + "/devstoreaccount1";
         AzBlobStorageClientConfig config = new 
AzBlobStorageClientConfig(MOCK_SERVER_REGION, endpoint, "", false, 0,
-                PLAYGROUND_CONTAINER, 1, 0, 0, writeBufferSize, true, null);
+                PLAYGROUND_CONTAINER, 1, 0, 0, writeBufferSize, true, null, 
CloudProperties.MAX_HTTP_CONNECTIONS,
+                CloudProperties.MAX_PENDING_HTTP_CONNECTIONS, 
CloudProperties.HTTP_CONNECTION_ACQUIRE_TIMEOUT);
         CLOUD_CLIENT = new AzBlobStorageCloudClient(config, 
ICloudGuardian.NoOpCloudGuardian.INSTANCE);
     }
 
diff --git 
a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CloudProperties.java
 
b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CloudProperties.java
index 392282f6ec..4b6c928951 100644
--- 
a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CloudProperties.java
+++ 
b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CloudProperties.java
@@ -39,6 +39,10 @@ import org.apache.hyracks.util.StorageUtil;
 
 public class CloudProperties extends AbstractProperties {
 
+    public static final int MAX_HTTP_CONNECTIONS = 1000;
+    public static final int MAX_PENDING_HTTP_CONNECTIONS = 10000;
+    public static final int HTTP_CONNECTION_ACQUIRE_TIMEOUT = 120;
+
     public CloudProperties(PropertiesAccessor accessor) {
         super(accessor);
     }
@@ -67,9 +71,9 @@ public class CloudProperties extends AbstractProperties {
                 getRangedIntegerType(5, Integer.MAX_VALUE),
                 StorageUtil.getIntSizeInBytes(8, 
StorageUtil.StorageUnit.MEGABYTE)),
         CLOUD_EVICTION_PLAN_REEVALUATE_THRESHOLD(POSITIVE_INTEGER, 50),
-        CLOUD_REQUESTS_MAX_HTTP_CONNECTIONS(POSITIVE_INTEGER, 1000),
-        CLOUD_REQUESTS_MAX_PENDING_HTTP_CONNECTIONS(POSITIVE_INTEGER, 10000),
-        CLOUD_REQUESTS_HTTP_CONNECTION_ACQUIRE_TIMEOUT(POSITIVE_INTEGER, 120),
+        CLOUD_REQUESTS_MAX_HTTP_CONNECTIONS(POSITIVE_INTEGER, 
MAX_HTTP_CONNECTIONS),
+        CLOUD_REQUESTS_MAX_PENDING_HTTP_CONNECTIONS(POSITIVE_INTEGER, 
MAX_PENDING_HTTP_CONNECTIONS),
+        CLOUD_REQUESTS_HTTP_CONNECTION_ACQUIRE_TIMEOUT(POSITIVE_INTEGER, 
HTTP_CONNECTION_ACQUIRE_TIMEOUT),
         CLOUD_STORAGE_FORCE_PATH_STYLE(BOOLEAN, false),
         CLOUD_STORAGE_DISABLE_SSL_VERIFY(BOOLEAN, false),
         CLOUD_STORAGE_LIST_EVENTUALLY_CONSISTENT(BOOLEAN, false),
diff --git 
a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IFileDeviceResolver.java
 
b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IFileDeviceResolver.java
index 2598caf285..4db5400964 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IFileDeviceResolver.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IFileDeviceResolver.java
@@ -20,8 +20,6 @@ package org.apache.hyracks.api.io;
 
 import java.util.List;
 
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
 /**
  * Resolves the device destination for a file from its relative path
  */
@@ -35,6 +33,6 @@ public interface IFileDeviceResolver {
      *            a relative file path.
      * @return the resident IO device of the file.
      */
-    IODeviceHandle resolve(String relativePath, List<IODeviceHandle> devices) 
throws HyracksDataException;
+    IODeviceHandle resolve(String relativePath, List<IODeviceHandle> devices);
 
 }
diff --git 
a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IIOManager.java
 
b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IIOManager.java
index 75cd23d6a3..93232d27ab 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IIOManager.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IIOManager.java
@@ -181,7 +181,7 @@ public interface IIOManager extends Closeable {
      * @param path relative path
      * @return A file reference based on the mounting point of {@code 
ioDeviceId} and the passed {@code Path}
      */
-    FileReference resolve(String path) throws HyracksDataException;
+    FileReference resolve(String path);
 
     /**
      * Gets a file reference from an absolute path
diff --git 
a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/DefaultDeviceResolver.java
 
b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/DefaultDeviceResolver.java
index 61a1cd8a1b..420ff2d8ac 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/DefaultDeviceResolver.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/DefaultDeviceResolver.java
@@ -22,7 +22,6 @@ import java.io.File;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.IFileDeviceResolver;
 import org.apache.hyracks.api.io.IODeviceHandle;
 
@@ -30,7 +29,7 @@ public class DefaultDeviceResolver implements 
IFileDeviceResolver {
     private AtomicInteger next = new AtomicInteger(0);
 
     @Override
-    public IODeviceHandle resolve(String relPath, List<IODeviceHandle> 
devices) throws HyracksDataException {
+    public IODeviceHandle resolve(String relPath, List<IODeviceHandle> 
devices) {
         int numDevices = devices.size();
         String path = relPath;
         // if number of devices is 1, we return the device
diff --git 
a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java
 
b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java
index 37e14779df..17a5f03844 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java
@@ -433,7 +433,7 @@ public class IOManager implements IIOManager {
     }
 
     @Override
-    public FileReference resolve(String path) throws HyracksDataException {
+    public FileReference resolve(String path) {
         return new FileReference(deviceComputer.resolve(path, getIODevices()), 
path);
     }
 

Reply via email to