hadoop-common-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Chris Nauroth (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HADOOP-13130) s3a failures can surface as RTEs, not IOEs
Date Wed, 18 May 2016 00:18:13 GMT

    [ https://issues.apache.org/jira/browse/HADOOP-13130?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15287946#comment-15287946
] 

Chris Nauroth commented on HADOOP-13130:
----------------------------------------

[~stevel@apache.org], thank you for the patch.  Here are a few comments.

{code}
        if (LOG.isDebugEnabled()) {
          LOG.debug("Completing multi-part upload for key '{}', id '{}'", key,
              uploadId);
        }
{code}

The log level guard is unnecessary.

{code}
   public FSDataOutputStream append(Path f, int bufferSize,
       Progressable progress) throws IOException {
-    throw new IOException("Not supported");
+    throw new UnsupportedOperationException("Not supported");
   }
{code}

Possibly backwards-incompatible?  Maybe someone coded error handling that catches {{IOException}}
and falls back to a non-append strategy for non-HDFS?

{code}
    String header = operation
        + (path != null ? ("on " + path) : "")
        + ": ";
    String message = header + exception;
{code}

The message will have no space between {{operation}} and {{"on "}}.

{code}
      // this exception is non-null if the service exception is an s3 on
{code}

Typo at end of comment?

{code}
  public static void eventually(int timeout, Callable<Void> callback)
      throws Exception {
    Exception lastException;
    long endtime = System.currentTimeMillis() + timeout;
    do {
      try {
        callback.call();
        return;
      } catch (FailFastException e) {
        throw e;
      } catch (Exception e) {
        lastException = e;
      }
      Thread.sleep(500);
    } while (endtime > System.currentTimeMillis());
    throw lastException;
  }
{code}

{{eventually}} doesn't appear to be interested in the results returned from the callback,
so would {{Runnable}} be a better fit than {{Callable<Void>}}?

{code}
      assertEquals("Expected EOF got char " + (char) c, -1, c);

      byte[] buf = new byte[256];

      assertEquals(-1, instream.read(buf));
      assertEquals(-1, instream.read(instream.getPos(), buf, 0, buf.length));

      // now do a block read fully, again, backwards from the current pos
      try {
        instream.readFully(shortLen + 512, buf);
        fail("Expected readFully to fail");
      } catch (EOFException expected) {
        LOG.debug("Expected: ", expected);
      }

      assertEquals(-1, instream.read(shortLen + 510, buf, 0, buf.length));
{code}

Do you want to use the descriptive "Expected EOF" message on all of these EOF assertions?

See below for several test failures I got after applying the patch to branch-2.  I see these
failures consistently, running both with and without the parallel-tests profile.  If these
failures don't repro for you, let me know, and I'll dig deeper on my side.

{code}
testProxyConnection(org.apache.hadoop.fs.s3a.TestS3AConfiguration)  Time elapsed: 1.635 sec
 <<< ERROR!
java.io.IOException: doesBucketExiston cnauroth-test-aws-s3a: com.amazonaws.AmazonClientException:
Unable to execute HTTP request: Connection to http://127.0.0.1:1 refused
	at java.net.PlainSocketImpl.socketConnect(Native Method)
	at java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:350)
	at java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:206)
	at java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:188)
	at java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392)
	at java.net.Socket.connect(Socket.java:589)
	at org.apache.http.conn.scheme.PlainSocketFactory.connectSocket(PlainSocketFactory.java:127)
	at org.apache.http.impl.conn.DefaultClientConnectionOperator.openConnection(DefaultClientConnectionOperator.java:180)
	at org.apache.http.impl.conn.ManagedClientConnectionImpl.open(ManagedClientConnectionImpl.java:294)
	at org.apache.http.impl.client.DefaultRequestDirector.tryConnect(DefaultRequestDirector.java:643)
	at org.apache.http.impl.client.DefaultRequestDirector.execute(DefaultRequestDirector.java:479)
	at org.apache.http.impl.client.AbstractHttpClient.execute(AbstractHttpClient.java:906)
	at org.apache.http.impl.client.AbstractHttpClient.execute(AbstractHttpClient.java:805)
	at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:728)
	at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
	at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
	at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
	at com.amazonaws.services.s3.AmazonS3Client.headBucket(AmazonS3Client.java:1107)
	at com.amazonaws.services.s3.AmazonS3Client.doesBucketExist(AmazonS3Client.java:1070)
	at org.apache.hadoop.fs.s3a.S3AFileSystem.verifyBucketExists(S3AFileSystem.java:289)
	at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:272)
	at org.apache.hadoop.fs.s3a.S3ATestUtils.createTestFileSystem(S3ATestUtils.java:53)
	at org.apache.hadoop.fs.s3a.TestS3AConfiguration.testProxyConnection(TestS3AConfiguration.java:123)

testAutomaticProxyPortSelection(org.apache.hadoop.fs.s3a.TestS3AConfiguration)  Time elapsed:
0.928 sec  <<< ERROR!
java.io.IOException: doesBucketExiston cnauroth-test-aws-s3a: com.amazonaws.AmazonClientException:
Unable to execute HTTP request: Connection to http://127.0.0.1:443 refused
	at java.net.PlainSocketImpl.socketConnect(Native Method)
	at java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:350)
	at java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:206)
	at java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:188)
	at java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392)
	at java.net.Socket.connect(Socket.java:589)
	at org.apache.http.conn.scheme.PlainSocketFactory.connectSocket(PlainSocketFactory.java:127)
	at org.apache.http.impl.conn.DefaultClientConnectionOperator.openConnection(DefaultClientConnectionOperator.java:180)
	at org.apache.http.impl.conn.ManagedClientConnectionImpl.open(ManagedClientConnectionImpl.java:294)
	at org.apache.http.impl.client.DefaultRequestDirector.tryConnect(DefaultRequestDirector.java:643)
	at org.apache.http.impl.client.DefaultRequestDirector.execute(DefaultRequestDirector.java:479)
	at org.apache.http.impl.client.AbstractHttpClient.execute(AbstractHttpClient.java:906)
	at org.apache.http.impl.client.AbstractHttpClient.execute(AbstractHttpClient.java:805)
	at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:728)
	at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
	at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
	at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
	at com.amazonaws.services.s3.AmazonS3Client.headBucket(AmazonS3Client.java:1107)
	at com.amazonaws.services.s3.AmazonS3Client.doesBucketExist(AmazonS3Client.java:1070)
	at org.apache.hadoop.fs.s3a.S3AFileSystem.verifyBucketExists(S3AFileSystem.java:289)
	at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:272)
	at org.apache.hadoop.fs.s3a.S3ATestUtils.createTestFileSystem(S3ATestUtils.java:53)
	at org.apache.hadoop.fs.s3a.TestS3AConfiguration.testAutomaticProxyPortSelection(TestS3AConfiguration.java:158)

shouldBeAbleToSwitchOnS3PathStyleAccessViaConfigProperty(org.apache.hadoop.fs.s3a.TestS3AConfiguration)
 Time elapsed: 0.646 sec  <<< ERROR!
org.apache.hadoop.fs.s3a.AwsS3IOException: purging multipart uploadson cnauroth-test-aws-s3a:
com.amazonaws.services.s3.model.AmazonS3Exception: The bucket you are attempting to access
must be addressed using the specified endpoint. Please send all future requests to this endpoint.
(Service: Amazon S3; Status Code: 301; Error Code: PermanentRedirect; Request ID: 8FE330E9D3BFA908),
S3 Extended Request ID: i38YD4/pNstx3Wjddju8/+fTKwFuHSBDIh5fHxn9HKtye2Lr1USYeHALVbvJoEa1EtMP4xz3wHA=:
The bucket you are attempting to access must be addressed using the specified endpoint. Please
send all future requests to this endpoint. (Service: Amazon S3; Status Code: 301; Error Code:
PermanentRedirect; Request ID: 8FE330E9D3BFA908)
	at com.amazonaws.http.AmazonHttpClient.handleErrorResponse(AmazonHttpClient.java:1182)
	at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:770)
	at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
	at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
	at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
	at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3738)
	at com.amazonaws.services.s3.AmazonS3Client.listMultipartUploads(AmazonS3Client.java:2796)
	at com.amazonaws.services.s3.transfer.TransferManager.abortMultipartUploads(TransferManager.java:1217)
	at org.apache.hadoop.fs.s3a.S3AFileSystem.initMultipartUploads(S3AFileSystem.java:417)
	at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:274)
	at org.apache.hadoop.fs.s3a.S3ATestUtils.createTestFileSystem(S3ATestUtils.java:53)
	at org.apache.hadoop.fs.s3a.TestS3AConfiguration.shouldBeAbleToSwitchOnS3PathStyleAccessViaConfigProperty(TestS3AConfiguration.java:375)

test401isNotPermittedFound(org.apache.hadoop.fs.s3a.TestS3AFailureHandling)  Time elapsed:
0.922 sec  <<< ERROR!
java.nio.file.AccessDeniedException: /: teston /: com.amazonaws.services.s3.model.AmazonS3Exception:
 (Service: null; Status Code: 401; Error Code: null; Request ID: null), S3 Extended Request
ID: null
	at org.apache.hadoop.fs.s3a.TestS3AFailureHandling.createS3Exception(TestS3AFailureHandling.java:151)
	at org.apache.hadoop.fs.s3a.TestS3AFailureHandling.test401isNotPermittedFound(TestS3AFailureHandling.java:147)
{code}


> s3a failures can surface as RTEs, not IOEs
> ------------------------------------------
>
>                 Key: HADOOP-13130
>                 URL: https://issues.apache.org/jira/browse/HADOOP-13130
>             Project: Hadoop Common
>          Issue Type: Sub-task
>          Components: fs/s3
>    Affects Versions: 2.7.2
>            Reporter: Steve Loughran
>            Assignee: Steve Loughran
>         Attachments: HADOOP-13130-001.patch, HADOOP-13130-002.patch, HADOOP-13130-002.patch,
HADOOP-13130-003.patch, HADOOP-13130-004.patch, HADOOP-13130-005.patch, HADOOP-13130-branch-2-006.patch
>
>
> S3A failures happening in the AWS library surface as {{AmazonClientException}} derivatives,
rather than IOEs. As the amazon exceptions are runtime exceptions, any code which catches
IOEs for error handling breaks.
> The fix will be to catch and wrap. The hard thing will be to wrap it with meaningful
exceptions rather than a generic IOE. Furthermore, if anyone has been catching AWS exceptions,
they are going to be disappointed. That means that fixing this situation could be considered
"incompatible" —but only for code which contains assumptions about the underlying FS and
the exceptions they raise.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


Mime
View raw message