hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Hadoop QA (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HBASE-8755) A new write thread model for HLog to improve the overall HBase write throughput
Date Fri, 13 Dec 2013 00:44:17 GMT

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

Hadoop QA commented on HBASE-8755:
----------------------------------

{color:red}-1 overall{color}.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12618499/8755v8.txt
  against trunk revision .

    {color:green}+1 @author{color}.  The patch does not contain any @author tags.

    {color:green}+1 tests included{color}.  The patch appears to include 6 new or modified
tests.

    {color:red}-1 hadoop1.0{color}.  The patch failed to compile against the hadoop 1.0 profile.
    Here is snippet of errors:
    {code}[ERROR] Failed to execute goal org.apache.maven.plugins:maven-compiler-plugin:2.5.1:compile
(default-compile) on project hbase-server: Compilation failure: Compilation failure:
[ERROR] /home/jenkins/jenkins-slave/workspace/PreCommit-HBASE-Build/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java:[436,42]
unclosed string literal
[ERROR] /home/jenkins/jenkins-slave/workspace/PreCommit-HBASE-Build/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java:[436,63]
';' expected
[ERROR] /home/jenkins/jenkins-slave/workspace/PreCommit-HBASE-Build/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java:[437,17]
illegal start of expression
[ERROR] /home/jenkins/jenkins-slave/workspace/PreCommit-HBASE-Build/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java:[437,23]
';' expected
[ERROR] -> [Help 1]
org.apache.maven.lifecycle.LifecycleExecutionException: Failed to execute goal org.apache.maven.plugins:maven-compiler-plugin:2.5.1:compile
(default-compile) on project hbase-server: Compilation failure
	at org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:213)
	at org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:153)
	at org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:145)
	at org.apache.maven.lifecycle.internal.LifecycleModuleBuilder.buildProject(LifecycleModuleBuilder.java:84)
	at org.apache.maven.lifecycle.internal.LifecycleModuleBuilder.buildProject(LifecycleModuleBuilder.java:59)
--
Caused by: org.apache.maven.plugin.CompilationFailureException: Compilation failure
	at org.apache.maven.plugin.AbstractCompilerMojo.execute(AbstractCompilerMojo.java:729)
	at org.apache.maven.plugin.CompilerMojo.execute(CompilerMojo.java:128)
	at org.apache.maven.plugin.DefaultBuildPluginManager.executeMojo(DefaultBuildPluginManager.java:101)
	at org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:209)
	... 19 more{code}

Console output: https://builds.apache.org/job/PreCommit-HBASE-Build/8148//console

This message is automatically generated.

> A new write thread model for HLog to improve the overall HBase write throughput
> -------------------------------------------------------------------------------
>
>                 Key: HBASE-8755
>                 URL: https://issues.apache.org/jira/browse/HBASE-8755
>             Project: HBase
>          Issue Type: Improvement
>          Components: Performance, wal
>            Reporter: Feng Honghua
>            Assignee: stack
>            Priority: Critical
>         Attachments: 8755-syncer.patch, 8755trunkV2.txt, 8755v8.txt, HBASE-8755-0.94-V0.patch,
HBASE-8755-0.94-V1.patch, HBASE-8755-0.96-v0.patch, HBASE-8755-trunk-V0.patch, HBASE-8755-trunk-V1.patch,
HBASE-8755-trunk-v4.patch, HBASE-8755-trunk-v6.patch, HBASE-8755-trunk-v7.patch, HBASE-8755-v5.patch,
thread.out
>
>
> In current write model, each write handler thread (executing put()) will individually
go through a full 'append (hlog local buffer) => HLog writer append (write to hdfs) =>
HLog writer sync (sync hdfs)' cycle for each write, which incurs heavy race condition on updateLock
and flushLock.
> The only optimization where checking if current syncTillHere > txid in expectation
for other thread help write/sync its own txid to hdfs and omitting the write/sync actually
help much less than expectation.
> Three of my colleagues(Ye Hangjun / Wu Zesheng / Zhang Peng) at Xiaomi proposed a new
write thread model for writing hdfs sequence file and the prototype implementation shows a
4X improvement for throughput (from 17000 to 70000+). 
> I apply this new write thread model in HLog and the performance test in our test cluster
shows about 3X throughput improvement (from 12150 to 31520 for 1 RS, from 22000 to 70000 for
5 RS), the 1 RS write throughput (1K row-size) even beats the one of BigTable (Precolator
published in 2011 says Bigtable's write throughput then is 31002). I can provide the detailed
performance test results if anyone is interested.
> The change for new write thread model is as below:
>  1> All put handler threads append the edits to HLog's local pending buffer; (it notifies
AsyncWriter thread that there is new edits in local buffer)
>  2> All put handler threads wait in HLog.syncer() function for underlying threads
to finish the sync that contains its txid;
>  3> An single AsyncWriter thread is responsible for retrieve all the buffered edits
in HLog's local pending buffer and write to the hdfs (hlog.writer.append); (it notifies AsyncFlusher
thread that there is new writes to hdfs that needs a sync)
>  4> An single AsyncFlusher thread is responsible for issuing a sync to hdfs to persist
the writes by AsyncWriter; (it notifies the AsyncNotifier thread that sync watermark increases)
>  5> An single AsyncNotifier thread is responsible for notifying all pending put handler
threads which are waiting in the HLog.syncer() function
>  6> No LogSyncer thread any more (since there is always AsyncWriter/AsyncFlusher threads
do the same job it does)



--
This message was sent by Atlassian JIRA
(v6.1.4#6159)

Mime
View raw message