Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 3505D200C54 for ; Wed, 12 Apr 2017 22:56:46 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 3390B160BAC; Wed, 12 Apr 2017 20:56:46 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 7AFA4160B95 for ; Wed, 12 Apr 2017 22:56:45 +0200 (CEST) Received: (qmail 31108 invoked by uid 500); 12 Apr 2017 20:56:44 -0000 Mailing-List: contact common-issues-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list common-issues@hadoop.apache.org Received: (qmail 31073 invoked by uid 99); 12 Apr 2017 20:56:44 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd2-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 12 Apr 2017 20:56:44 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd2-us-west.apache.org (ASF Mail Server at spamd2-us-west.apache.org) with ESMTP id 180711A040B for ; Wed, 12 Apr 2017 20:56:44 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd2-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -99.202 X-Spam-Level: X-Spam-Status: No, score=-99.202 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, RP_MATCHES_RCVD=-0.001, SPF_PASS=-0.001, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd2-us-west.apache.org [10.40.0.9]) (amavisd-new, port 10024) with ESMTP id nCuog7p9HyTF for ; Wed, 12 Apr 2017 20:56:43 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with ESMTP id 06C8B5FC4A for ; Wed, 12 Apr 2017 20:56:43 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id 7B592E0D40 for ; Wed, 12 Apr 2017 20:56:42 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 017162406F for ; Wed, 12 Apr 2017 20:56:42 +0000 (UTC) Date: Wed, 12 Apr 2017 20:56:42 +0000 (UTC) From: "Steve Loughran (JIRA)" To: common-issues@hadoop.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (HADOOP-13726) Enforce that FileSystem initializes only a single instance of the requested FileSystem. MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Wed, 12 Apr 2017 20:56:46 -0000 [ https://issues.apache.org/jira/browse/HADOOP-13726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15966607#comment-15966607 ] Steve Loughran commented on HADOOP-13726: ----------------------------------------- I've realised there's another thing we can do here. We're assuming that every FS instance has to be initialized before its added to the cache. If we just make the cache the unique factory for an FS instance, but the init is checked for and done after, then the sync locks across the cache would be minimal, it'd only be threads awaiting access to the specific FS instance which would hold. What I don't see is how you'd implement this with the current FS code (no real state model, etc)., more specifically, no "isInitialized" property. Still, one could be added, for a sequence like {code} Filesystem.get(uri, conf) { fs = CACHE.get(...) if (!fs.isInitialized()} { sychronized(fs) { if(!fs.isInitialized()} { fs.initialize(...) fs.setInitialized(true) } } } return fs; } {code} > Enforce that FileSystem initializes only a single instance of the requested FileSystem. > --------------------------------------------------------------------------------------- > > Key: HADOOP-13726 > URL: https://issues.apache.org/jira/browse/HADOOP-13726 > Project: Hadoop Common > Issue Type: Improvement > Components: fs > Reporter: Chris Nauroth > Assignee: Manjunath Anand > > The {{FileSystem}} cache is intended to guarantee reuse of instances by multiple call sites or multiple threads. The current implementation does provide this guarantee, but there is a brief race condition window during which multiple threads could perform redundant initialization. If the file system implementation has expensive initialization logic, then this is wasteful. This issue proposes to eliminate that race condition and guarantee initialization of only a single instance. -- This message was sent by Atlassian JIRA (v6.3.15#6346) --------------------------------------------------------------------- To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org For additional commands, e-mail: common-issues-help@hadoop.apache.org