[ 
https://issues.apache.org/jira/browse/SPARK-17564?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Adam Roberts updated SPARK-17564:
---------------------------------
    Description: 
Could be related to [SPARK-10680]

This is the test and one fix would be to increase the timeouts from 1.2 seconds 
to 5 seconds

// The timeout is relative to the LAST request sent, which is kinda weird, but 
still.
  // This test also makes sure the timeout works for Fetch requests as well as 
RPCs.
  @Test
  public void furtherRequestsDelay() throws Exception {
    final byte[] response = new byte[16];
    final StreamManager manager = new StreamManager() {
      @Override
      public ManagedBuffer getChunk(long streamId, int chunkIndex) {
        Uninterruptibles.sleepUninterruptibly(FOREVER, TimeUnit.MILLISECONDS);
        return new NioManagedBuffer(ByteBuffer.wrap(response));
      }
    };
    RpcHandler handler = new RpcHandler() {
      @Override
      public void receive(
          TransportClient client,
          ByteBuffer message,
          RpcResponseCallback callback) {
        throw new UnsupportedOperationException();
      }

      @Override
      public StreamManager getStreamManager() {
        return manager;
      }
    };

    TransportContext context = new TransportContext(conf, handler);
    server = context.createServer();
    clientFactory = context.createClientFactory();
    TransportClient client = 
clientFactory.createClient(TestUtils.getLocalHost(), server.getPort());

    // Send one request, which will eventually fail.
    TestCallback callback0 = new TestCallback();
    client.fetchChunk(0, 0, callback0);
    Uninterruptibles.sleepUninterruptibly(1200, TimeUnit.MILLISECONDS);
    // This would be one timeout to increase

    // Send a second request before the first has failed.
    TestCallback callback1 = new TestCallback();
    client.fetchChunk(0, 1, callback1);
    Uninterruptibles.sleepUninterruptibly(1200, TimeUnit.MILLISECONDS);
    // This would be another timeout to increase

    synchronized (callback0) {
      // not complete yet, but should complete soon
      assertEquals(-1, callback0.successLength);
      assertNull(callback0.failure);
      callback0.wait(2 * 1000);
      assertTrue(callback0.failure instanceof IOException);
    }

    synchronized (callback1) {
      // failed at same time as previous
      assert (callback0.failure instanceof IOException);
    }
  }

The suite fails with this 1/3 of the time

Tests run: 3, Failures: 1, Errors: 0, Skipped: 0, Time elapsed: 31.487 sec <<< 
FAILURE! - in org.apache.spark.network.RequestTimeoutIntegrationSuite
furtherRequestsDelay(org.apache.spark.network.RequestTimeoutIntegrationSuite)  
Time elapsed: 11.297 sec  <<< FAILURE!
java.lang.AssertionError
        at 
org.apache.spark.network.RequestTimeoutIntegrationSuite.furtherRequestsDelay(RequestTimeoutIntegrationSuite.java:230)

If there are better suggestions for improving this test let's take them 
onboard, I think using 5 sec timeout periods would be a place to start so folks 
don't need to needlessly triage this failure. Will add a few prints and report 
back

  was:
Could be related to [SPARK-10680]

This is the test and one fix would be to increase the timeouts from 1.2 seconds 
to 5 seconds

// The timeout is relative to the LAST request sent, which is kinda weird, but 
still.
  // This test also makes sure the timeout works for Fetch requests as well as 
RPCs.
  @Test
  public void furtherRequestsDelay() throws Exception {
    final byte[] response = new byte[16];
    final StreamManager manager = new StreamManager() {
      @Override
      public ManagedBuffer getChunk(long streamId, int chunkIndex) {
        Uninterruptibles.sleepUninterruptibly(FOREVER, TimeUnit.MILLISECONDS);
        return new NioManagedBuffer(ByteBuffer.wrap(response));
      }
    };
    RpcHandler handler = new RpcHandler() {
      @Override
      public void receive(
          TransportClient client,
          ByteBuffer message,
          RpcResponseCallback callback) {
        throw new UnsupportedOperationException();
      }

      @Override
      public StreamManager getStreamManager() {
        return manager;
      }
    };

    TransportContext context = new TransportContext(conf, handler);
    server = context.createServer();
    clientFactory = context.createClientFactory();
    TransportClient client = 
clientFactory.createClient(TestUtils.getLocalHost(), server.getPort());

    // Send one request, which will eventually fail.
    TestCallback callback0 = new TestCallback();
    client.fetchChunk(0, 0, callback0);
    Uninterruptibles.sleepUninterruptibly(1200, TimeUnit.MILLISECONDS);
    // This would be one timeout to increase

    // Send a second request before the first has failed.
    TestCallback callback1 = new TestCallback();
    client.fetchChunk(0, 1, callback1);
    Uninterruptibles.sleepUninterruptibly(1200, TimeUnit.MILLISECONDS);
    // This would be another timeout to increase

    synchronized (callback0) {
      // not complete yet, but should complete soon
      assertEquals(-1, callback0.successLength);
      assertNull(callback0.failure);
      callback0.wait(2 * 1000);
      assertTrue(callback0.failure instanceof IOException);
    }

    synchronized (callback1) {
      // failed at same time as previous
      assert (callback0.failure instanceof IOException);
    }
  }

If there are better suggestions for improving this test let's take them 
onboard, I'll create a pull request using 5 sec timeout periods as a place to 
start so folks don't need to needlessly triage this failure


> Flaky RequestTimeoutIntegrationSuite, furtherRequestsDelay
> ----------------------------------------------------------
>
>                 Key: SPARK-17564
>                 URL: https://issues.apache.org/jira/browse/SPARK-17564
>             Project: Spark
>          Issue Type: Improvement
>          Components: Tests
>    Affects Versions: 2.0.1, 2.1.0
>            Reporter: Adam Roberts
>            Priority: Minor
>
> Could be related to [SPARK-10680]
> This is the test and one fix would be to increase the timeouts from 1.2 
> seconds to 5 seconds
> // The timeout is relative to the LAST request sent, which is kinda weird, 
> but still.
>   // This test also makes sure the timeout works for Fetch requests as well 
> as RPCs.
>   @Test
>   public void furtherRequestsDelay() throws Exception {
>     final byte[] response = new byte[16];
>     final StreamManager manager = new StreamManager() {
>       @Override
>       public ManagedBuffer getChunk(long streamId, int chunkIndex) {
>         Uninterruptibles.sleepUninterruptibly(FOREVER, TimeUnit.MILLISECONDS);
>         return new NioManagedBuffer(ByteBuffer.wrap(response));
>       }
>     };
>     RpcHandler handler = new RpcHandler() {
>       @Override
>       public void receive(
>           TransportClient client,
>           ByteBuffer message,
>           RpcResponseCallback callback) {
>         throw new UnsupportedOperationException();
>       }
>       @Override
>       public StreamManager getStreamManager() {
>         return manager;
>       }
>     };
>     TransportContext context = new TransportContext(conf, handler);
>     server = context.createServer();
>     clientFactory = context.createClientFactory();
>     TransportClient client = 
> clientFactory.createClient(TestUtils.getLocalHost(), server.getPort());
>     // Send one request, which will eventually fail.
>     TestCallback callback0 = new TestCallback();
>     client.fetchChunk(0, 0, callback0);
>     Uninterruptibles.sleepUninterruptibly(1200, TimeUnit.MILLISECONDS);
>     // This would be one timeout to increase
>     // Send a second request before the first has failed.
>     TestCallback callback1 = new TestCallback();
>     client.fetchChunk(0, 1, callback1);
>     Uninterruptibles.sleepUninterruptibly(1200, TimeUnit.MILLISECONDS);
>     // This would be another timeout to increase
>     synchronized (callback0) {
>       // not complete yet, but should complete soon
>       assertEquals(-1, callback0.successLength);
>       assertNull(callback0.failure);
>       callback0.wait(2 * 1000);
>       assertTrue(callback0.failure instanceof IOException);
>     }
>     synchronized (callback1) {
>       // failed at same time as previous
>       assert (callback0.failure instanceof IOException);
>     }
>   }
> The suite fails with this 1/3 of the time
> Tests run: 3, Failures: 1, Errors: 0, Skipped: 0, Time elapsed: 31.487 sec 
> <<< FAILURE! - in org.apache.spark.network.RequestTimeoutIntegrationSuite
> furtherRequestsDelay(org.apache.spark.network.RequestTimeoutIntegrationSuite) 
>  Time elapsed: 11.297 sec  <<< FAILURE!
> java.lang.AssertionError
>         at 
> org.apache.spark.network.RequestTimeoutIntegrationSuite.furtherRequestsDelay(RequestTimeoutIntegrationSuite.java:230)
> If there are better suggestions for improving this test let's take them 
> onboard, I think using 5 sec timeout periods would be a place to start so 
> folks don't need to needlessly triage this failure. Will add a few prints and 
> report back



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

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org

Reply via email to