Return-Path: Delivered-To: apmail-hadoop-hdfs-issues-archive@minotaur.apache.org Received: (qmail 46745 invoked from network); 13 Jan 2010 16:51:16 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.3) by minotaur.apache.org with SMTP; 13 Jan 2010 16:51:16 -0000 Received: (qmail 64627 invoked by uid 500); 13 Jan 2010 16:51:15 -0000 Delivered-To: apmail-hadoop-hdfs-issues-archive@hadoop.apache.org Received: (qmail 64523 invoked by uid 500); 13 Jan 2010 16:51:15 -0000 Mailing-List: contact hdfs-issues-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hdfs-issues@hadoop.apache.org Delivered-To: mailing list hdfs-issues@hadoop.apache.org Received: (qmail 64444 invoked by uid 99); 13 Jan 2010 16:51:15 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 13 Jan 2010 16:51:15 +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.140] (HELO brutus.apache.org) (140.211.11.140) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 13 Jan 2010 16:51:14 +0000 Received: from brutus.apache.org (localhost [127.0.0.1]) by brutus.apache.org (Postfix) with ESMTP id 7BBC1234C48C for ; Wed, 13 Jan 2010 08:50:54 -0800 (PST) Message-ID: <1434004888.217191263401454505.JavaMail.jira@brutus.apache.org> Date: Wed, 13 Jan 2010 16:50:54 +0000 (UTC) From: "dhruba borthakur (JIRA)" To: hdfs-issues@hadoop.apache.org Subject: [jira] Created: (HDFS-895) Allow hflush/sync to occur in parallel with new writes to the file MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 Allow hflush/sync to occur in parallel with new writes to the file ------------------------------------------------------------------ Key: HDFS-895 URL: https://issues.apache.org/jira/browse/HDFS-895 Project: Hadoop HDFS Issue Type: Improvement Components: hdfs client Reporter: dhruba borthakur In the current trunk, the HDFS client methods writeChunk() and hflush./sync are syncronized. This means that if a hflush/sync is in progress, an applicationn cannot write data to the HDFS client buffer. This reduces the write throughput of the transaction log in HBase. The hflush/sync should allow new writes to happen to the HDFS client even when a hflush/sync is in progress. It can record the seqno of the message for which it should receice the ack, indicate to the DataStream thread to star flushing those messages, exit the synchronized section and just wai for that ack to arrive. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.