[ 
https://issues.apache.org/jira/browse/HIVE-25330?focusedWorklogId=642682&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-642682
 ]

ASF GitHub Bot logged work on HIVE-25330:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 27/Aug/21 03:38
            Start Date: 27/Aug/21 03:38
    Worklog Time Spent: 10m 
      Work Description: ayushtkn commented on a change in pull request #2516:
URL: https://github.com/apache/hive/pull/2516#discussion_r697128887



##########
File path: ql/src/test/org/apache/hadoop/hive/ql/parse/repl/TestCopyUtils.java
##########
@@ -110,6 +112,40 @@ public void shouldThrowExceptionOnDistcpFailure() throws 
Exception {
     copyUtils.doCopy(destination, srcPaths);
   }
 
+  @Test
+  public void testRetryableFSCalls() throws Exception {
+    mockStatic(UserGroupInformation.class);
+    mockStatic(ReplChangeManager.class);
+    
when(UserGroupInformation.getCurrentUser()).thenReturn(mock(UserGroupInformation.class));
+    HiveConf conf = mock(HiveConf.class);
+    conf.set(HiveConf.ConfVars.REPL_RETRY_INTIAL_DELAY.varname, "1s");
+    FileSystem fs = mock(FileSystem.class);
+    Path source = mock(Path.class);
+    Path destination = mock(Path.class);
+    ContentSummary cs = mock(ContentSummary.class);
+
+    when(ReplChangeManager.checksumFor(source, fs)).thenThrow(new 
IOException("Failed")).thenReturn("dummy");
+    when(fs.exists(same(source))).thenThrow(new 
IOException("Failed")).thenReturn(true);
+    when(fs.delete(same(source), anyBoolean())).thenThrow(new 
IOException("Failed")).thenReturn(true);
+    when(fs.mkdirs(same(source))).thenThrow(new 
IOException("Failed")).thenReturn(true);
+    when(fs.rename(same(source), same(destination))).thenThrow(new 
IOException("Failed")).thenReturn(true);
+    when(fs.getContentSummary(same(source))).thenThrow(new 
IOException("Failed")).thenReturn(cs);
+
+    CopyUtils copyUtils = new 
CopyUtils(UserGroupInformation.getCurrentUser().getUserName(), conf, fs);
+    CopyUtils copyUtilsSpy = Mockito.spy(copyUtils);
+    assertEquals (copyUtilsSpy.exists(fs, source), true);

Review comment:
       change to `assertTrue` similarly for the others as well

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/parse/repl/CopyUtils.java
##########
@@ -66,6 +67,16 @@
   private FileSystem destinationFs;
   private final int maxParallelCopyTask;
 
+  private List<Class<? extends Exception>> failOnExceptions = 
Arrays.asList(org.apache.hadoop.fs.PathIOException.class,
+          org.apache.hadoop.fs.UnsupportedFileSystemException.class,
+          org.apache.hadoop.fs.InvalidPathException.class,
+          org.apache.hadoop.fs.InvalidRequestException.class,
+          org.apache.hadoop.fs.FileAlreadyExistsException.class,
+          org.apache.hadoop.fs.ChecksumException.class,
+          org.apache.hadoop.fs.ParentNotDirectoryException.class,
+          org.apache.hadoop.hdfs.protocol.NSQuotaExceededException.class,

Review comment:
       We can include other quota exceptions also, say the children and 
grandchildren of `ClusterStorageCapacityExceededException` or directly 
`ClusterStorageCapacityExceededException` if retryable function can take the 
parent class and block its children as well.

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/parse/repl/CopyUtils.java
##########
@@ -190,11 +247,14 @@ 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) || 
failOnExceptions.stream().anyMatch(k -> e.getClass().equals(k))
+                || 
ErrorMsg.REPL_FILE_SYSTEM_OPERATION_RETRY.getMsg().equals(e.getMessage())) {
+          //Don't retry in the following cases:

Review comment:
       pull the comment above the if statement

##########
File path: ql/src/test/org/apache/hadoop/hive/ql/parse/repl/TestCopyUtils.java
##########
@@ -110,6 +112,40 @@ public void shouldThrowExceptionOnDistcpFailure() throws 
Exception {
     copyUtils.doCopy(destination, srcPaths);
   }
 
+  @Test
+  public void testRetryableFSCalls() throws Exception {
+    mockStatic(UserGroupInformation.class);
+    mockStatic(ReplChangeManager.class);
+    
when(UserGroupInformation.getCurrentUser()).thenReturn(mock(UserGroupInformation.class));
+    HiveConf conf = mock(HiveConf.class);
+    conf.set(HiveConf.ConfVars.REPL_RETRY_INTIAL_DELAY.varname, "1s");
+    FileSystem fs = mock(FileSystem.class);
+    Path source = mock(Path.class);
+    Path destination = mock(Path.class);
+    ContentSummary cs = mock(ContentSummary.class);
+
+    when(ReplChangeManager.checksumFor(source, fs)).thenThrow(new 
IOException("Failed")).thenReturn("dummy");
+    when(fs.exists(same(source))).thenThrow(new 
IOException("Failed")).thenReturn(true);
+    when(fs.delete(same(source), anyBoolean())).thenThrow(new 
IOException("Failed")).thenReturn(true);
+    when(fs.mkdirs(same(source))).thenThrow(new 
IOException("Failed")).thenReturn(true);
+    when(fs.rename(same(source), same(destination))).thenThrow(new 
IOException("Failed")).thenReturn(true);
+    when(fs.getContentSummary(same(source))).thenThrow(new 
IOException("Failed")).thenReturn(cs);
+
+    CopyUtils copyUtils = new 
CopyUtils(UserGroupInformation.getCurrentUser().getUserName(), conf, fs);
+    CopyUtils copyUtilsSpy = Mockito.spy(copyUtils);
+    assertEquals (copyUtilsSpy.exists(fs, source), true);
+    Mockito.verify(fs, Mockito.times(2)).exists(source);
+    assertEquals (copyUtils.delete(fs, source, true), true);
+    Mockito.verify(fs, Mockito.times(2)).delete(source, true);
+    assertEquals (copyUtils.mkdirs(fs, source), true);
+    Mockito.verify(fs, Mockito.times(2)).mkdirs(source);
+    assertEquals (copyUtils.rename(fs, source, destination), true);
+    Mockito.verify(fs, Mockito.times(2)).rename(source, destination);
+    assertEquals (copyUtilsSpy.getContentSummary(fs, source), cs);
+    Mockito.verify(fs, Mockito.times(2)).getContentSummary(source);
+    assertEquals (copyUtilsSpy.checkSumFor(source, fs), "dummy");

Review comment:
       flip the entries, say ``assertEquals ("dummy", 
copyUtilsSpy.checkSumFor(source, fs));``
   in case of ``assertEquals`` the expected goes first and the second arg is 
the actual value. :-)




-- 
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-unsubscr...@hive.apache.org

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


Issue Time Tracking
-------------------

    Worklog Id:     (was: 642682)
    Time Spent: 0.5h  (was: 20m)

> Make FS calls in CopyUtils retryable
> ------------------------------------
>
>                 Key: HIVE-25330
>                 URL: https://issues.apache.org/jira/browse/HIVE-25330
>             Project: Hive
>          Issue Type: Improvement
>            Reporter: Pravin Sinha
>            Assignee: Haymant Mangla
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 0.5h
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to