hadoop-yarn-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Zhijie Shen (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (YARN-2673) Add retry for timeline client put APIs
Date Fri, 17 Oct 2014 06:46:34 GMT

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

Zhijie Shen commented on YARN-2673:
-----------------------------------

[~gtCarrera], thanks for working on this issue.

bq. Then, on top of this, we can add retry mechanism to delegation token calls for secured
clusters.

GetDelegationToken is not via the jersey client. It needs to be fixed separately.

bq. We agreed that blocking the timeline client for a short while is better, since we may
not want to drop some critical timeline data.

Make sense to me.


The jersey filter approach looks fine to me. I've some comments w.r.t. the patch:

1. The variable is only used for testing? Shall we mark it \@VisibleForTesting, and let the
test case access it directly? Also maxRetries and retryInterval.

After retried is set to true first time. It is always true, which means it's not useful for
asserting the second request. Do we really need it? Or we can spy the filter, and check how
many times getNext() has been invoked?
{code}
    private boolean retried = false;
{code}

2. Use "public synchronized void changeRetrySettings(int maxRetries, long interval) \{"?
{code}
    // Customize retry settings synchronously
    public void changeRetrySettings(int maxRetries, long interval) {
      synchronized(this) {
        this.maxRetries = maxRetries;
        this.retryInterval = interval;
      }
    }
{code}

3. Why can it be -1 given the configuration is correct?
{code}
            } else {
              // note that maxRetries may be -1 at the very beginning
              LOG.info("Connection Timeout (" + cr.getURI()
                  + "), will keep retrying.");
            }
{code}

4. response is unused. And we need to add Assert.fail() after this statement to ensure the
test case will fail if the exception is not thrown.
{code}
      TimelinePutResponse response = client.putEntities(generateEntity());
{code}

5. Is this API really necessary? We don't have similar configurations for RPC based APIs,
for example YarnClient methods, right? According to the discussion on the umbrella YARN-1530,
it's still arguable about how to make timeline client reliable, which may affect this configuration.
If we add this API to timeline client, we have to support it forwardly.
{code}
  @Public
  public abstract void customizeRetrySettings(int maxRetries,
      long retryIntervalMs);
{code}

> Add retry for timeline client put APIs
> --------------------------------------
>
>                 Key: YARN-2673
>                 URL: https://issues.apache.org/jira/browse/YARN-2673
>             Project: Hadoop YARN
>          Issue Type: Sub-task
>            Reporter: Li Lu
>            Assignee: Li Lu
>         Attachments: YARN-2673-101414-1.patch, YARN-2673-101414-2.patch, YARN-2673-101414.patch
>
>
> Timeline client now does not handle the case gracefully when the server is down. Jobs
from distributed shell may fail due to ATS restart. We may need to add some retry mechanisms
to the client. 



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

Mime
View raw message