You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by GitBox <gi...@apache.org> on 2021/07/27 13:14:17 UTC

[GitHub] [hive] ayushtkn commented on a change in pull request #2516: HIVE-25330: Make FS calls in CopyUtils retryable

ayushtkn commented on a change in pull request #2516:
URL: https://github.com/apache/hive/pull/2516#discussion_r677418736



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/parse/repl/CopyUtils.java
##########
@@ -297,21 +343,21 @@ public void renameFileCopiedFromCmPath(Path toPath, FileSystem dstFs, List<ReplC
       String destFileName = srcFile.getCmPath().getName();
       Path destRoot = CopyUtils.getCopyDestination(srcFile, toPath);
       Path destFile = new Path(destRoot, destFileName);
-      if (dstFs.exists(destFile)) {
+      if (exists(dstFs, destFile)) {
         String destFileWithSourceName = srcFile.getSourcePath().getName();
         Path newDestFile = new Path(destRoot, destFileWithSourceName);
 
         // if the new file exist then delete it before renaming, to avoid rename failure. If the copy is done
         // directly to table path (bypassing staging directory) then there might be some stale files from previous
         // incomplete/failed load. No need of recycle as this is a case of stale file.
         try {
-          dstFs.delete(newDestFile, true);
+          delete(dstFs, newDestFile, true);
           LOG.debug(" file " + newDestFile + " is deleted before renaming");
         } catch (FileNotFoundException e) {
           // no problem
         }

Review comment:
       Post this change, Will we get `FileNotFoundException`? `FNF` is part of `IOException` it will be retried, hence in the end, it will throw `IOException` with `REPL_FILE_SYSTEM_OPERATION_RETRY ` which won't get caught.

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/parse/repl/CopyUtils.java
##########
@@ -76,6 +76,57 @@ public CopyUtils(String distCpDoAsUser, HiveConf hiveConf, FileSystem destinatio
     this.destinationFs = destinationFs;
   }
 
+  private <T> T retryableFxn(Callable<T> callable) throws IOException {
+    Retryable retryable = Retryable.builder()
+            .withHiveConf(hiveConf)
+            .withRetryOnException(IOException.class).build();

Review comment:
       Retrying all `IOE` might not be apt, We might land up retrying doing nothing, as in case of `FileNotFoundException`, it can not get solved even in 10 iteration, So is an `IOE` in case of FileSystem Closed, even if we retry. the result will be same.
   Some example Exceptions which we can include-
   ``ConnectException`` ``EOFException`` ``ConnectTimeoutException`` ``StandbyException`` ``SafemodeException`` ``NoRouteToHostException`` ``SocketException`` and ``RetriableException``

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/parse/repl/CopyUtils.java
##########
@@ -333,7 +379,7 @@ private boolean isSourceFileMismatch(FileSystem sourceFs, ReplChangeManager.File
         } catch (IOException e) {

Review comment:
       ``` 
   ReplChangeManager.checksumFor(srcFile.getSourcePath(), sourceFs);
   ```
   This internally calls `` FileChecksum checksum = fs.getFileChecksum(path);``, can make ``ReplChangeManager.checksumFor(srcFile.getSourcePath(), sourceFs);`` also retryable 

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/parse/repl/CopyUtils.java
##########
@@ -190,11 +240,12 @@ private void doCopyRetry(FileSystem sourceFs, List<ReplChangeManager.FileInfo> s
         // If copy fails, fall through the retry logic
         LOG.info("file operation failed", e);
 
-        if (repeat >= (MAX_IO_RETRY - 1)) {
-          //no need to wait in the last iteration
+        if (repeat >= (MAX_IO_RETRY - 1)
+                || e.getMessage().equals(ErrorMsg.REPL_FILE_SYSTEM_OPERATION_RETRY.format(e.getCause().getMessage()))) {
+          //Don't retry if this is last iteration or retry is already exhausted by FS operations.
           break;
         }
-
+        closeAllForUGI((proxyUser == null) ? Utils.getUGI() : proxyUser);

Review comment:
       Now we closed the filesystem here. Below we recreate the filesystem only if exception is not `FNF` What will happen in case of a ``FileNotFoundException`` the filesystem will get closed but won't get recreated? 

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/parse/repl/CopyUtils.java
##########
@@ -145,14 +196,13 @@ ExecutorService getExecutorService() {
   @VisibleForTesting
   void doCopy(Map.Entry<Path, List<ReplChangeManager.FileInfo>> destMapEntry, UserGroupInformation proxyUser,
                       boolean useRegularCopy, boolean overwrite) throws IOException, LoginException,
-    HiveFatalException {
+          HiveFatalException {

Review comment:
       nit:
   Avoid indentation change




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org