Return-Path: Delivered-To: apmail-hadoop-hbase-issues-archive@minotaur.apache.org Received: (qmail 1535 invoked from network); 4 May 2010 12:43:21 -0000 Received: from unknown (HELO mail.apache.org) (140.211.11.3) by 140.211.11.9 with SMTP; 4 May 2010 12:43:21 -0000 Received: (qmail 83452 invoked by uid 500); 4 May 2010 12:43:21 -0000 Delivered-To: apmail-hadoop-hbase-issues-archive@hadoop.apache.org Received: (qmail 83429 invoked by uid 500); 4 May 2010 12:43:20 -0000 Mailing-List: contact hbase-issues-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list hbase-issues@hadoop.apache.org Received: (qmail 83421 invoked by uid 99); 4 May 2010 12:43:20 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 04 May 2010 12:43:20 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=10.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.22] (HELO thor.apache.org) (140.211.11.22) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 04 May 2010 12:43:17 +0000 Received: from thor (localhost [127.0.0.1]) by thor.apache.org (8.13.8+Sun/8.13.8) with ESMTP id o44CguEU009979 for ; Tue, 4 May 2010 12:42:56 GMT Message-ID: <31900309.35571272976976105.JavaMail.jira@thor> Date: Tue, 4 May 2010 08:42:56 -0400 (EDT) From: "Miklos Kurucz (JIRA)" To: hbase-issues@hadoop.apache.org Subject: [jira] Commented: (HBASE-2445) Clean up client retry policies MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 X-Virus-Checked: Checked by ClamAV on apache.org [ https://issues.apache.org/jira/browse/HBASE-2445?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12863789#action_12863789 ] Miklos Kurucz commented on HBASE-2445: -------------------------------------- I am having problems with setting hbase.client.retries.number = 1 In that case cache will never be updated. Jean-Daniel Cryans asked me: "Yeah I understand that retries are unusable at that level, but you still want retries in order to be able to recalibrate the .META. cache right?" My answer is that I want HCM to update cache when it is necessary, but I don't think that should only happen in retries. For me it seems that the two things can be separated: Whenever a NotServingRegionException is caught the cache entry should be cleared. When an exception is caugh that is not related to bad cache entries the cache should not be cleared. The current exception handling is done this way if I'm correct: Server sends NSRE client/ScannerCallable.java:84 84 if (ioe instanceof NotServingRegionException) { 85 // Throw a DNRE so that we break out of cycle of calling NSRE 86 // when what we need is to open scanner against new location. 87 // Attach NSRE to signal client that it needs to resetup scanner. 88 throw new DoNotRetryIOException("Reset scanner", ioe); client/HConnectionManager.java:1063 1063 } catch (Throwable t) { 1064 t = translateException(t); client/HConnectionManager.java:1431 1431 if (t instanceof DoNotRetryIOException) { 1432 throw (DoNotRetryIOException)t; client/HTable.java:824 824 } catch (DoNotRetryIOException e) { ... 835 Throwable cause = e.getCause(); 836 if (cause == null || !(cause instanceof NotServingRegionException)) { 837 throw e; 838 } 839 // Else, its signal from depths of ScannerCallable that we got an 840 // NSRE on a next and that we need to reset the scanner. And after resetting the scanner we get to: client/HTable.java:776 776 callable = getScannerCallable(localStartKey, nbRows); 777 // Open a scanner on the region server starting at the 778 // beginning of the region 779 getConnection().getRegionServerWithRetries(callable); Which will still use the bad cache entry first? And runs on the same problem? Perhaps I am mistaken and somewhere we do delete bad cache entry, but in case Server sends java.net.ConnectException client/HConnectionManager.java:1063 1063 } catch (Throwable t) { 1064 t = translateException(t); 1065 exceptions.add(t); 1066 if (tries == numRetries - 1) { 1067 throw new RetriesExhaustedException(callable.getServerName(), 1068 callable.getRegionName(), callable.getRow(), tries, exceptions); We will not delete bad entry. I understand that ConnectException can be thrown for various reasons, still deleting the cache entry might be a good idea even when hbase.client.retries.number = 1 Also retrying a "connection refused" does not make sense to me either. > Clean up client retry policies > ------------------------------ > > Key: HBASE-2445 > URL: https://issues.apache.org/jira/browse/HBASE-2445 > Project: Hadoop HBase > Issue Type: Improvement > Components: client > Reporter: Todd Lipcon > Priority: Critical > Fix For: 0.20.5, 0.21.0 > > > Right now almost all retry behavior is governed by a single parameter that determines the number of retries. In a few places, there are also conf for the number of millis to sleep between retries. This isn't quite flexible enough. If we can refactor some of the retry logic into a RetryPolicy class, we could introduce exponential backoff where appropriate, clean up some of the config, etc -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.