[jira] [Commented] (HBASE-19371) Running WALPerformanceEvaluation against asyncfswal throws exceptions
[ https://issues.apache.org/jira/browse/HBASE-19371?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16286609#comment-16286609 ] stack commented on HBASE-19371: --- I tried the patch and it works for me on 2.8.2; no more exceptions. Verification runs too. I also tried to run against hadoop 3.0.0 RC1 -- which took a while to setup -- and it seems to work fine too. Let me push this patch. > Running WALPerformanceEvaluation against asyncfswal throws exceptions > - > > Key: HBASE-19371 > URL: https://issues.apache.org/jira/browse/HBASE-19371 > Project: HBase > Issue Type: Bug >Reporter: stack > Fix For: 2.0.0-beta-1 > > Attachments: HBASE-19371.patch > > > Was trying to do a perf eval on asyncfswal. I ran w/ these args: > Performance counter stats for '/home/stack/hbase/bin/hbase --config > /home/stack/conf_hbase org.apache.hadoop.hbase.wal.WALPerformanceEvaluation > -path /user/stack/logs/ -verify -threads 25 -iterations 100 -keySize 50 > -valueSize 100 -syncInterval 10': > The verify fails on all runs: > Exception in thread "main" java.lang.IllegalStateException: Counted=12390228, > expected=2500 > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.run(WALPerformanceEvaluation.java:368) > > at > org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.innerMain(WALPerformanceEvaluation.java:597) > > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.main(WALPerformanceEvaluation.java:601) > I need to fix test or figure what is wrong in asyncfswal. > Also seeing these when I run w/ one thread only: > 2017-11-28 21:25:49,952 WARN [AsyncFSWAL-1-1] wal.AsyncFSWAL: > RingBufferTruck with unexpected type: EMPTY > Log has a spew of them. > Has stuff like this too: > 2017-11-28 21:25:40,065 WARN [Close-WAL-Writer-3] > wal.AsyncProtobufLogWriter: normal close failed, try recover > > java.io.IOException: stream already broken > > at > org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.endBlock(FanOutOneBlockAsyncDFSOutput.java:510) > > at > org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.lambda$close$12(FanOutOneBlockAsyncDFSOutput.java:550) > >at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:163) > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:403) > at > org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:462) > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858) > > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138) > at java.lang.Thread.run(Thread.java:748) > Starts out spewing EMPTY here: > 2017-11-28 21:16:52,051 INFO [main] regionserver.HRegion: Setting > FlushNonSloppyStoresFirstPolicy for the > region=WALPerformanceEvaluation:0,,1511932610787.deca03e0ca447fa25d02fe9cd6e31aa4. > 2017-11-28 21:16:52,058 DEBUG [main] regionserver.HRegion: Found 0 recovered > edits file(s) under > hdfs://ve0524.halxg.cloudera.com:8020/user/stack/logs/data/WALPerformanceEvaluation/0/deca03e0ca447fa25d02fe9cd6e31aa4 > 2017-11-28 21:16:52,068 DEBUG [main] regionserver.FlushLargeStoresPolicy: No > hbase.hregion.percolumnfamilyflush.size.lower.bound set in description of > table WALPerformanceEvaluation:0, use config (134217728) instead > 2017-11-28 21:16:52,084 DEBUG [main] wal.WALSplitter: Wrote > file=hdfs://ve0524.halxg.cloudera.com:8020/user/stack/logs/data/WALPerformanceEvaluation/0/deca03e0ca447fa25d02fe9cd6e31aa4/recovered.edits/2.seqid, > newSeqId=2, maxSeqId=0 > 2017-11-28 21:16:52,084 INFO [main] regionserver.HRegion: Onlined > deca03e0ca447fa25d02fe9cd6e31aa4; next sequenceid=2 > 2017-11-28 21:16:52,185 WARN [AsyncFSWAL-1-1]
[jira] [Commented] (HBASE-19371) Running WALPerformanceEvaluation against asyncfswal throws exceptions
[ https://issues.apache.org/jira/browse/HBASE-19371?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16284767#comment-16284767 ] stack commented on HBASE-19371: --- Thanks [~Apache9] Will try it after finish numbers on HBASE-16890 > Running WALPerformanceEvaluation against asyncfswal throws exceptions > - > > Key: HBASE-19371 > URL: https://issues.apache.org/jira/browse/HBASE-19371 > Project: HBase > Issue Type: Bug >Reporter: stack > Fix For: 2.0.0-beta-1 > > Attachments: HBASE-19371.patch > > > Was trying to do a perf eval on asyncfswal. I ran w/ these args: > Performance counter stats for '/home/stack/hbase/bin/hbase --config > /home/stack/conf_hbase org.apache.hadoop.hbase.wal.WALPerformanceEvaluation > -path /user/stack/logs/ -verify -threads 25 -iterations 100 -keySize 50 > -valueSize 100 -syncInterval 10': > The verify fails on all runs: > Exception in thread "main" java.lang.IllegalStateException: Counted=12390228, > expected=2500 > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.run(WALPerformanceEvaluation.java:368) > > at > org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.innerMain(WALPerformanceEvaluation.java:597) > > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.main(WALPerformanceEvaluation.java:601) > I need to fix test or figure what is wrong in asyncfswal. > Also seeing these when I run w/ one thread only: > 2017-11-28 21:25:49,952 WARN [AsyncFSWAL-1-1] wal.AsyncFSWAL: > RingBufferTruck with unexpected type: EMPTY > Log has a spew of them. > Has stuff like this too: > 2017-11-28 21:25:40,065 WARN [Close-WAL-Writer-3] > wal.AsyncProtobufLogWriter: normal close failed, try recover > > java.io.IOException: stream already broken > > at > org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.endBlock(FanOutOneBlockAsyncDFSOutput.java:510) > > at > org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.lambda$close$12(FanOutOneBlockAsyncDFSOutput.java:550) > >at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:163) > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:403) > at > org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:462) > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858) > > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138) > at java.lang.Thread.run(Thread.java:748) > Starts out spewing EMPTY here: > 2017-11-28 21:16:52,051 INFO [main] regionserver.HRegion: Setting > FlushNonSloppyStoresFirstPolicy for the > region=WALPerformanceEvaluation:0,,1511932610787.deca03e0ca447fa25d02fe9cd6e31aa4. > 2017-11-28 21:16:52,058 DEBUG [main] regionserver.HRegion: Found 0 recovered > edits file(s) under > hdfs://ve0524.halxg.cloudera.com:8020/user/stack/logs/data/WALPerformanceEvaluation/0/deca03e0ca447fa25d02fe9cd6e31aa4 > 2017-11-28 21:16:52,068 DEBUG [main] regionserver.FlushLargeStoresPolicy: No > hbase.hregion.percolumnfamilyflush.size.lower.bound set in description of > table WALPerformanceEvaluation:0, use config (134217728) instead > 2017-11-28 21:16:52,084 DEBUG [main] wal.WALSplitter: Wrote > file=hdfs://ve0524.halxg.cloudera.com:8020/user/stack/logs/data/WALPerformanceEvaluation/0/deca03e0ca447fa25d02fe9cd6e31aa4/recovered.edits/2.seqid, > newSeqId=2, maxSeqId=0 > 2017-11-28 21:16:52,084 INFO [main] regionserver.HRegion: Onlined > deca03e0ca447fa25d02fe9cd6e31aa4; next sequenceid=2 > 2017-11-28 21:16:52,185 WARN [AsyncFSWAL-1-1] wal.AsyncFSWAL: > RingBufferTruck with unexpected type: EMPTY > 2017-11-28 21:16:52,185 WARN [AsyncFSWAL-1-1] wal.AsyncFSWAL: > RingBufferTruck with unexpected type:
[jira] [Commented] (HBASE-19371) Running WALPerformanceEvaluation against asyncfswal throws exceptions
[ https://issues.apache.org/jira/browse/HBASE-19371?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16284683#comment-16284683 ] Hadoop QA commented on HBASE-19371: --- | (x) *{color:red}-1 overall{color}* | \\ \\ || Vote || Subsystem || Runtime || Comment || | {color:blue}0{color} | {color:blue} reexec {color} | {color:blue} 0m 9s{color} | {color:blue} Docker mode activated. {color} | || || || || {color:brown} Prechecks {color} || | {color:blue}0{color} | {color:blue} findbugs {color} | {color:blue} 0m 0s{color} | {color:blue} Findbugs executables are not available. {color} | | {color:green}+1{color} | {color:green} hbaseanti {color} | {color:green} 0m 0s{color} | {color:green} Patch does not have any anti-patterns. {color} | | {color:green}+1{color} | {color:green} @author {color} | {color:green} 0m 0s{color} | {color:green} The patch does not contain any @author tags. {color} | | {color:red}-1{color} | {color:red} test4tests {color} | {color:red} 0m 0s{color} | {color:red} The patch doesn't appear to include any new or modified tests. Please justify why no new tests are needed for this patch. Also please list what manual steps were performed to verify this patch. {color} | || || || || {color:brown} master Compile Tests {color} || | {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 4m 37s{color} | {color:green} master passed {color} | | {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 40s{color} | {color:green} master passed {color} | | {color:green}+1{color} | {color:green} checkstyle {color} | {color:green} 1m 3s{color} | {color:green} master passed {color} | | {color:green}+1{color} | {color:green} shadedjars {color} | {color:green} 5m 28s{color} | {color:green} branch has no errors when building our shaded downstream artifacts. {color} | | {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 27s{color} | {color:green} master passed {color} | || || || || {color:brown} Patch Compile Tests {color} || | {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 4m 33s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 40s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} javac {color} | {color:green} 0m 40s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} checkstyle {color} | {color:green} 1m 3s{color} | {color:green} hbase-server: The patch generated 0 new + 1 unchanged - 2 fixed = 1 total (was 3) {color} | | {color:green}+1{color} | {color:green} whitespace {color} | {color:green} 0m 0s{color} | {color:green} The patch has no whitespace issues. {color} | | {color:green}+1{color} | {color:green} shadedjars {color} | {color:green} 4m 24s{color} | {color:green} patch has no errors when building our shaded downstream artifacts. {color} | | {color:green}+1{color} | {color:green} hadoopcheck {color} | {color:green} 53m 16s{color} | {color:green} Patch does not cause any errors with Hadoop 2.6.1 2.6.2 2.6.3 2.6.4 2.6.5 2.7.1 2.7.2 2.7.3 2.7.4 or 3.0.0-alpha4. {color} | | {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 28s{color} | {color:green} the patch passed {color} | || || || || {color:brown} Other Tests {color} || | {color:green}+1{color} | {color:green} unit {color} | {color:green}105m 39s{color} | {color:green} hbase-server in the patch passed. {color} | | {color:green}+1{color} | {color:green} asflicense {color} | {color:green} 0m 24s{color} | {color:green} The patch does not generate ASF License warnings. {color} | | {color:black}{color} | {color:black} {color} | {color:black}177m 37s{color} | {color:black} {color} | \\ \\ || Subsystem || Report/Notes || | Docker | Client=17.05.0-ce Server=17.05.0-ce Image:yetus/hbase:eee3b01 | | JIRA Issue | HBASE-19371 | | JIRA Patch URL | https://issues.apache.org/jira/secure/attachment/12901349/HBASE-19371.patch | | Optional Tests | asflicense javac javadoc unit findbugs shadedjars hadoopcheck hbaseanti checkstyle compile | | uname | Linux 0c4d9f677a87 3.13.0-133-generic #182-Ubuntu SMP Tue Sep 19 15:49:21 UTC 2017 x86_64 GNU/Linux | | Build tool | maven | | Personality | /home/jenkins/jenkins-slave/workspace/PreCommit-HBASE-Build/component/dev-support/hbase-personality.sh | | git revision | master / 7092b814bd | | maven | version: Apache Maven 3.5.2 (138edd61fd100ec658bfa2d307c43b76940a5d7d; 2017-10-18T07:58:13Z) | | Default Java | 1.8.0_151 | | Test Results | https://builds.apache.org/job/PreCommit-HBASE-Build/10312/testReport/ | | modules | C: hbase-server U: hbase-server | | Console output | https://builds.apache.org/job/PreCommit-HBASE-Build/10312/console | | Powered by | Apache Yetus 0.6.0 http://yetus.apache.org | This message was automatically
[jira] [Commented] (HBASE-19371) Running WALPerformanceEvaluation against asyncfswal throws exceptions
[ https://issues.apache.org/jira/browse/HBASE-19371?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16284610#comment-16284610 ] Duo Zhang commented on HBASE-19371: --- The problem is that, after hadoop 2.8, DFSClien$Conf is changed to DFSClientConf. And JVM will include the return type to generate the method signature so... Let me think how to deal with it, just some configurations so maybe just copy the construction code from HDFS... Thanks. > Running WALPerformanceEvaluation against asyncfswal throws exceptions > - > > Key: HBASE-19371 > URL: https://issues.apache.org/jira/browse/HBASE-19371 > Project: HBase > Issue Type: Bug >Reporter: stack > Fix For: 2.0.0-beta-1 > > > Was trying to do a perf eval on asyncfswal. I ran w/ these args: > Performance counter stats for '/home/stack/hbase/bin/hbase --config > /home/stack/conf_hbase org.apache.hadoop.hbase.wal.WALPerformanceEvaluation > -path /user/stack/logs/ -verify -threads 25 -iterations 100 -keySize 50 > -valueSize 100 -syncInterval 10': > The verify fails on all runs: > Exception in thread "main" java.lang.IllegalStateException: Counted=12390228, > expected=2500 > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.run(WALPerformanceEvaluation.java:368) > > at > org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.innerMain(WALPerformanceEvaluation.java:597) > > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.main(WALPerformanceEvaluation.java:601) > I need to fix test or figure what is wrong in asyncfswal. > Also seeing these when I run w/ one thread only: > 2017-11-28 21:25:49,952 WARN [AsyncFSWAL-1-1] wal.AsyncFSWAL: > RingBufferTruck with unexpected type: EMPTY > Log has a spew of them. > Has stuff like this too: > 2017-11-28 21:25:40,065 WARN [Close-WAL-Writer-3] > wal.AsyncProtobufLogWriter: normal close failed, try recover > > java.io.IOException: stream already broken > > at > org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.endBlock(FanOutOneBlockAsyncDFSOutput.java:510) > > at > org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.lambda$close$12(FanOutOneBlockAsyncDFSOutput.java:550) > >at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:163) > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:403) > at > org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:462) > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858) > > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138) > at java.lang.Thread.run(Thread.java:748) > Starts out spewing EMPTY here: > 2017-11-28 21:16:52,051 INFO [main] regionserver.HRegion: Setting > FlushNonSloppyStoresFirstPolicy for the > region=WALPerformanceEvaluation:0,,1511932610787.deca03e0ca447fa25d02fe9cd6e31aa4. > 2017-11-28 21:16:52,058 DEBUG [main] regionserver.HRegion: Found 0 recovered > edits file(s) under > hdfs://ve0524.halxg.cloudera.com:8020/user/stack/logs/data/WALPerformanceEvaluation/0/deca03e0ca447fa25d02fe9cd6e31aa4 > 2017-11-28 21:16:52,068 DEBUG [main] regionserver.FlushLargeStoresPolicy: No > hbase.hregion.percolumnfamilyflush.size.lower.bound set in description of > table WALPerformanceEvaluation:0, use config (134217728) instead > 2017-11-28 21:16:52,084 DEBUG [main] wal.WALSplitter: Wrote > file=hdfs://ve0524.halxg.cloudera.com:8020/user/stack/logs/data/WALPerformanceEvaluation/0/deca03e0ca447fa25d02fe9cd6e31aa4/recovered.edits/2.seqid, > newSeqId=2, maxSeqId=0 > 2017-11-28 21:16:52,084 INFO [main] regionserver.HRegion: Onlined > deca03e0ca447fa25d02fe9cd6e31aa4; next sequenceid=2 > 2017-11-28 21:16:52,185 WARN
[jira] [Commented] (HBASE-19371) Running WALPerformanceEvaluation against asyncfswal throws exceptions
[ https://issues.apache.org/jira/browse/HBASE-19371?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16284411#comment-16284411 ] stack commented on HBASE-19371: --- Switch up on to hadoop 2.8.2 again. This time I get this: {code} 2017-12-08 15:18:16,720 DEBUG [main] asyncfs.FanOutOneBlockAsyncDFSOutputHelper: ClientProtocol::create wrong number of arguments, should be hadoop 2.x 2017-12-08 15:18:16,747 INFO [main] util.FSHDFSUtils: Recover lease on dfs file /user/stack/logs/WALs/wals/wals.1512775096639 2017-12-08 15:18:16,759 INFO [main] util.FSHDFSUtils: Recovered lease, attempt=0 on file=/user/stack/logs/WALs/wals/wals.1512775096639 after 10ms 2017-12-08 15:18:16,759 INFO [main] hbase.MockRegionServerServices: Shutting down due to request 'test clean up.' 2017-12-08 15:18:16,759 INFO [main] wal.WALPerformanceEvaluation: shutting down log roller. 2017-12-08 15:18:16,759 INFO [WALPerfEval.logRoller] regionserver.LogRoller: LogRoller exiting. Exception in thread "main" java.lang.NoSuchMethodError: org.apache.hadoop.hdfs.DFSClient.getConf()Lorg/apache/hadoop/hdfs/DFSClient$Conf; at org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.createChecksum(FanOutOneBlockAsyncDFSOutputHelper.java:569) at org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.createOutput(FanOutOneBlockAsyncDFSOutputHelper.java:762) at org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.access$400(FanOutOneBlockAsyncDFSOutputHelper.java:118) at org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper$16.doCall(FanOutOneBlockAsyncDFSOutputHelper.java:812) at org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper$16.doCall(FanOutOneBlockAsyncDFSOutputHelper.java:807) at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81) at org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.createOutput(FanOutOneBlockAsyncDFSOutputHelper.java:820) at org.apache.hadoop.hbase.io.asyncfs.AsyncFSOutputHelper.createOutput(AsyncFSOutputHelper.java:62) at org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.initOutput(AsyncProtobufLogWriter.java:158) at org.apache.hadoop.hbase.regionserver.wal.AbstractProtobufLogWriter.init(AbstractProtobufLogWriter.java:167) at org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createAsyncWriter(AsyncFSWALProvider.java:100) at org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(AsyncFSWAL.java:621) at org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(AsyncFSWAL.java:131) at org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.rollWriter(AbstractFSWAL.java:751) at org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.rollWriter(AbstractFSWAL.java:489) at org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.(AsyncFSWAL.java:257) at org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createWAL(AsyncFSWALProvider.java:70) at org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createWAL(AsyncFSWALProvider.java:45) at org.apache.hadoop.hbase.wal.AbstractFSWALProvider.getWAL(AbstractFSWALProvider.java:139) at org.apache.hadoop.hbase.wal.AbstractFSWALProvider.getWAL(AbstractFSWALProvider.java:55) at org.apache.hadoop.hbase.wal.WALFactory.getWAL(WALFactory.java:244) at org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.openRegion(WALPerformanceEvaluation.java:502) at org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.run(WALPerformanceEvaluation.java:336) at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) at org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.innerMain(WALPerformanceEvaluation.java:597) at org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.main(WALPerformanceEvaluation.java:601) {code} We are trying hadoop3 and then hadoop2 create Client method. Looks like we need a 2.8 version? I can work on it if get confirmation. Thanks [~Apache9] > Running WALPerformanceEvaluation against asyncfswal throws exceptions > - > > Key: HBASE-19371 > URL: https://issues.apache.org/jira/browse/HBASE-19371 > Project: HBase > Issue Type: Bug >Reporter: stack > Fix For: 2.0.0-beta-1 > > > Was trying to do a perf eval on asyncfswal. I ran w/ these args: > Performance counter stats for '/home/stack/hbase/bin/hbase --config > /home/stack/conf_hbase org.apache.hadoop.hbase.wal.WALPerformanceEvaluation > -path /user/stack/logs/ -verify -threads 25 -iterations 100 -keySize 50 > -valueSize 100 -syncInterval 10': > The verify fails on all runs: > Exception in thread "main" java.lang.IllegalStateException: Counted=12390228, > expected=2500 > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.run(WALPerformanceEvaluation.java:368) >
[jira] [Commented] (HBASE-19371) Running WALPerformanceEvaluation against asyncfswal throws exceptions
[ https://issues.apache.org/jira/browse/HBASE-19371?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16280981#comment-16280981 ] stack commented on HBASE-19371: --- HBASE-15628 Implement an AsyncOutputStream which can work with any FileSystem implementation; ADDENDUM -- Change Log output ... took care of the complaint about "java.lang.ClassNotFoundException: org.apache.hadoop.hdfs.protocolPB.PBHelperClient". I don't have the EMPTY issues now I've updated. I switched back to 2.7 hdfs and i don't have the validation complaints anymore. I'll be back to try 2.8 hfs again. > Running WALPerformanceEvaluation against asyncfswal throws exceptions > - > > Key: HBASE-19371 > URL: https://issues.apache.org/jira/browse/HBASE-19371 > Project: HBase > Issue Type: Bug >Reporter: stack > Fix For: 2.0.0-beta-1 > > > Was trying to do a perf eval on asyncfswal. I ran w/ these args: > Performance counter stats for '/home/stack/hbase/bin/hbase --config > /home/stack/conf_hbase org.apache.hadoop.hbase.wal.WALPerformanceEvaluation > -path /user/stack/logs/ -verify -threads 25 -iterations 100 -keySize 50 > -valueSize 100 -syncInterval 10': > The verify fails on all runs: > Exception in thread "main" java.lang.IllegalStateException: Counted=12390228, > expected=2500 > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.run(WALPerformanceEvaluation.java:368) > > at > org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.innerMain(WALPerformanceEvaluation.java:597) > > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.main(WALPerformanceEvaluation.java:601) > I need to fix test or figure what is wrong in asyncfswal. > Also seeing these when I run w/ one thread only: > 2017-11-28 21:25:49,952 WARN [AsyncFSWAL-1-1] wal.AsyncFSWAL: > RingBufferTruck with unexpected type: EMPTY > Log has a spew of them. > Has stuff like this too: > 2017-11-28 21:25:40,065 WARN [Close-WAL-Writer-3] > wal.AsyncProtobufLogWriter: normal close failed, try recover > > java.io.IOException: stream already broken > > at > org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.endBlock(FanOutOneBlockAsyncDFSOutput.java:510) > > at > org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.lambda$close$12(FanOutOneBlockAsyncDFSOutput.java:550) > >at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:163) > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:403) > at > org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:462) > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858) > > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138) > at java.lang.Thread.run(Thread.java:748) > Starts out spewing EMPTY here: > 2017-11-28 21:16:52,051 INFO [main] regionserver.HRegion: Setting > FlushNonSloppyStoresFirstPolicy for the > region=WALPerformanceEvaluation:0,,1511932610787.deca03e0ca447fa25d02fe9cd6e31aa4. > 2017-11-28 21:16:52,058 DEBUG [main] regionserver.HRegion: Found 0 recovered > edits file(s) under > hdfs://ve0524.halxg.cloudera.com:8020/user/stack/logs/data/WALPerformanceEvaluation/0/deca03e0ca447fa25d02fe9cd6e31aa4 > 2017-11-28 21:16:52,068 DEBUG [main] regionserver.FlushLargeStoresPolicy: No > hbase.hregion.percolumnfamilyflush.size.lower.bound set in description of > table WALPerformanceEvaluation:0, use config (134217728) instead > 2017-11-28 21:16:52,084 DEBUG [main] wal.WALSplitter: Wrote > file=hdfs://ve0524.halxg.cloudera.com:8020/user/stack/logs/data/WALPerformanceEvaluation/0/deca03e0ca447fa25d02fe9cd6e31aa4/recovered.edits/2.seqid, > newSeqId=2, maxSeqId=0 > 2017-11-28 21:16:52,084 INFO
[jira] [Commented] (HBASE-19371) Running WALPerformanceEvaluation against asyncfswal throws exceptions
[ https://issues.apache.org/jira/browse/HBASE-19371?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16271697#comment-16271697 ] Duo Zhang commented on HBASE-19371: --- {quote} And it seems like asyncfswal requires 2.8.0? {quote} No, 2.7.x is fine. Maybe the error message when setting up the bridge reflection interface is a bit confusing... {quote} Any comment on the validation not working? {quote} I haven't tried this yet since [~ram_krish] and [~zghaobac] are both worked on PE right now. Do you guys observer the same problem? Thanks. > Running WALPerformanceEvaluation against asyncfswal throws exceptions > - > > Key: HBASE-19371 > URL: https://issues.apache.org/jira/browse/HBASE-19371 > Project: HBase > Issue Type: Bug >Reporter: stack > Fix For: 2.0.0-beta-1 > > > Was trying to do a perf eval on asyncfswal. I ran w/ these args: > Performance counter stats for '/home/stack/hbase/bin/hbase --config > /home/stack/conf_hbase org.apache.hadoop.hbase.wal.WALPerformanceEvaluation > -path /user/stack/logs/ -verify -threads 25 -iterations 100 -keySize 50 > -valueSize 100 -syncInterval 10': > The verify fails on all runs: > Exception in thread "main" java.lang.IllegalStateException: Counted=12390228, > expected=2500 > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.run(WALPerformanceEvaluation.java:368) > > at > org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.innerMain(WALPerformanceEvaluation.java:597) > > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.main(WALPerformanceEvaluation.java:601) > I need to fix test or figure what is wrong in asyncfswal. > Also seeing these when I run w/ one thread only: > 2017-11-28 21:25:49,952 WARN [AsyncFSWAL-1-1] wal.AsyncFSWAL: > RingBufferTruck with unexpected type: EMPTY > Log has a spew of them. > Has stuff like this too: > 2017-11-28 21:25:40,065 WARN [Close-WAL-Writer-3] > wal.AsyncProtobufLogWriter: normal close failed, try recover > > java.io.IOException: stream already broken > > at > org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.endBlock(FanOutOneBlockAsyncDFSOutput.java:510) > > at > org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.lambda$close$12(FanOutOneBlockAsyncDFSOutput.java:550) > >at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:163) > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:403) > at > org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:462) > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858) > > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138) > at java.lang.Thread.run(Thread.java:748) > Starts out spewing EMPTY here: > 2017-11-28 21:16:52,051 INFO [main] regionserver.HRegion: Setting > FlushNonSloppyStoresFirstPolicy for the > region=WALPerformanceEvaluation:0,,1511932610787.deca03e0ca447fa25d02fe9cd6e31aa4. > 2017-11-28 21:16:52,058 DEBUG [main] regionserver.HRegion: Found 0 recovered > edits file(s) under > hdfs://ve0524.halxg.cloudera.com:8020/user/stack/logs/data/WALPerformanceEvaluation/0/deca03e0ca447fa25d02fe9cd6e31aa4 > 2017-11-28 21:16:52,068 DEBUG [main] regionserver.FlushLargeStoresPolicy: No > hbase.hregion.percolumnfamilyflush.size.lower.bound set in description of > table WALPerformanceEvaluation:0, use config (134217728) instead > 2017-11-28 21:16:52,084 DEBUG [main] wal.WALSplitter: Wrote > file=hdfs://ve0524.halxg.cloudera.com:8020/user/stack/logs/data/WALPerformanceEvaluation/0/deca03e0ca447fa25d02fe9cd6e31aa4/recovered.edits/2.seqid, > newSeqId=2, maxSeqId=0 > 2017-11-28 21:16:52,084 INFO [main] regionserver.HRegion: Onlined
[jira] [Commented] (HBASE-19371) Running WALPerformanceEvaluation against asyncfswal throws exceptions
[ https://issues.apache.org/jira/browse/HBASE-19371?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16271022#comment-16271022 ] stack commented on HBASE-19371: --- And it seems like asyncfswal requires 2.8.0? S > Running WALPerformanceEvaluation against asyncfswal throws exceptions > - > > Key: HBASE-19371 > URL: https://issues.apache.org/jira/browse/HBASE-19371 > Project: HBase > Issue Type: Bug >Reporter: stack > Fix For: 2.0.0-beta-1 > > > Was trying to do a perf eval on asyncfswal. I ran w/ these args: > Performance counter stats for '/home/stack/hbase/bin/hbase --config > /home/stack/conf_hbase org.apache.hadoop.hbase.wal.WALPerformanceEvaluation > -path /user/stack/logs/ -verify -threads 25 -iterations 100 -keySize 50 > -valueSize 100 -syncInterval 10': > The verify fails on all runs: > Exception in thread "main" java.lang.IllegalStateException: Counted=12390228, > expected=2500 > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.run(WALPerformanceEvaluation.java:368) > > at > org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.innerMain(WALPerformanceEvaluation.java:597) > > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.main(WALPerformanceEvaluation.java:601) > I need to fix test or figure what is wrong in asyncfswal. > Also seeing these when I run w/ one thread only: > 2017-11-28 21:25:49,952 WARN [AsyncFSWAL-1-1] wal.AsyncFSWAL: > RingBufferTruck with unexpected type: EMPTY > Log has a spew of them. > Has stuff like this too: > 2017-11-28 21:25:40,065 WARN [Close-WAL-Writer-3] > wal.AsyncProtobufLogWriter: normal close failed, try recover > > java.io.IOException: stream already broken > > at > org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.endBlock(FanOutOneBlockAsyncDFSOutput.java:510) > > at > org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.lambda$close$12(FanOutOneBlockAsyncDFSOutput.java:550) > >at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:163) > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:403) > at > org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:462) > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858) > > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138) > at java.lang.Thread.run(Thread.java:748) > Starts out spewing EMPTY here: > 2017-11-28 21:16:52,051 INFO [main] regionserver.HRegion: Setting > FlushNonSloppyStoresFirstPolicy for the > region=WALPerformanceEvaluation:0,,1511932610787.deca03e0ca447fa25d02fe9cd6e31aa4. > 2017-11-28 21:16:52,058 DEBUG [main] regionserver.HRegion: Found 0 recovered > edits file(s) under > hdfs://ve0524.halxg.cloudera.com:8020/user/stack/logs/data/WALPerformanceEvaluation/0/deca03e0ca447fa25d02fe9cd6e31aa4 > 2017-11-28 21:16:52,068 DEBUG [main] regionserver.FlushLargeStoresPolicy: No > hbase.hregion.percolumnfamilyflush.size.lower.bound set in description of > table WALPerformanceEvaluation:0, use config (134217728) instead > 2017-11-28 21:16:52,084 DEBUG [main] wal.WALSplitter: Wrote > file=hdfs://ve0524.halxg.cloudera.com:8020/user/stack/logs/data/WALPerformanceEvaluation/0/deca03e0ca447fa25d02fe9cd6e31aa4/recovered.edits/2.seqid, > newSeqId=2, maxSeqId=0 > 2017-11-28 21:16:52,084 INFO [main] regionserver.HRegion: Onlined > deca03e0ca447fa25d02fe9cd6e31aa4; next sequenceid=2 > 2017-11-28 21:16:52,185 WARN [AsyncFSWAL-1-1] wal.AsyncFSWAL: > RingBufferTruck with unexpected type: EMPTY > 2017-11-28 21:16:52,185 WARN [AsyncFSWAL-1-1] wal.AsyncFSWAL: > RingBufferTruck with unexpected type: EMPTY > 2017-11-28 21:16:52,185 WARN [AsyncFSWAL-1-1]
[jira] [Commented] (HBASE-19371) Running WALPerformanceEvaluation against asyncfswal throws exceptions
[ https://issues.apache.org/jira/browse/HBASE-19371?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16271019#comment-16271019 ] stack commented on HBASE-19371: --- Thanks [~Apache9] Will try. Any comment on the validation not working? I could dig in on this > Running WALPerformanceEvaluation against asyncfswal throws exceptions > - > > Key: HBASE-19371 > URL: https://issues.apache.org/jira/browse/HBASE-19371 > Project: HBase > Issue Type: Bug >Reporter: stack > Fix For: 2.0.0-beta-1 > > > Was trying to do a perf eval on asyncfswal. I ran w/ these args: > Performance counter stats for '/home/stack/hbase/bin/hbase --config > /home/stack/conf_hbase org.apache.hadoop.hbase.wal.WALPerformanceEvaluation > -path /user/stack/logs/ -verify -threads 25 -iterations 100 -keySize 50 > -valueSize 100 -syncInterval 10': > The verify fails on all runs: > Exception in thread "main" java.lang.IllegalStateException: Counted=12390228, > expected=2500 > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.run(WALPerformanceEvaluation.java:368) > > at > org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.innerMain(WALPerformanceEvaluation.java:597) > > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.main(WALPerformanceEvaluation.java:601) > I need to fix test or figure what is wrong in asyncfswal. > Also seeing these when I run w/ one thread only: > 2017-11-28 21:25:49,952 WARN [AsyncFSWAL-1-1] wal.AsyncFSWAL: > RingBufferTruck with unexpected type: EMPTY > Log has a spew of them. > Has stuff like this too: > 2017-11-28 21:25:40,065 WARN [Close-WAL-Writer-3] > wal.AsyncProtobufLogWriter: normal close failed, try recover > > java.io.IOException: stream already broken > > at > org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.endBlock(FanOutOneBlockAsyncDFSOutput.java:510) > > at > org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.lambda$close$12(FanOutOneBlockAsyncDFSOutput.java:550) > >at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:163) > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:403) > at > org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:462) > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858) > > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138) > at java.lang.Thread.run(Thread.java:748) > Starts out spewing EMPTY here: > 2017-11-28 21:16:52,051 INFO [main] regionserver.HRegion: Setting > FlushNonSloppyStoresFirstPolicy for the > region=WALPerformanceEvaluation:0,,1511932610787.deca03e0ca447fa25d02fe9cd6e31aa4. > 2017-11-28 21:16:52,058 DEBUG [main] regionserver.HRegion: Found 0 recovered > edits file(s) under > hdfs://ve0524.halxg.cloudera.com:8020/user/stack/logs/data/WALPerformanceEvaluation/0/deca03e0ca447fa25d02fe9cd6e31aa4 > 2017-11-28 21:16:52,068 DEBUG [main] regionserver.FlushLargeStoresPolicy: No > hbase.hregion.percolumnfamilyflush.size.lower.bound set in description of > table WALPerformanceEvaluation:0, use config (134217728) instead > 2017-11-28 21:16:52,084 DEBUG [main] wal.WALSplitter: Wrote > file=hdfs://ve0524.halxg.cloudera.com:8020/user/stack/logs/data/WALPerformanceEvaluation/0/deca03e0ca447fa25d02fe9cd6e31aa4/recovered.edits/2.seqid, > newSeqId=2, maxSeqId=0 > 2017-11-28 21:16:52,084 INFO [main] regionserver.HRegion: Onlined > deca03e0ca447fa25d02fe9cd6e31aa4; next sequenceid=2 > 2017-11-28 21:16:52,185 WARN [AsyncFSWAL-1-1] wal.AsyncFSWAL: > RingBufferTruck with unexpected type: EMPTY > 2017-11-28 21:16:52,185 WARN [AsyncFSWAL-1-1] wal.AsyncFSWAL: > RingBufferTruck with unexpected type: EMPTY >
[jira] [Commented] (HBASE-19371) Running WALPerformanceEvaluation against asyncfswal throws exceptions
[ https://issues.apache.org/jira/browse/HBASE-19371?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16270428#comment-16270428 ] Duo Zhang commented on HBASE-19371: --- OK, it is exactly what I concerned in HBASE-18601... We haven't call TraceUtil.initTracer in WALPE so the sync method in AsyncFSWAL just bypass the truck loading and cause an empty truck being published. You can add this line in setConf to temporary fix the problem sir [~stack]. {code} @Override public void setConf(Configuration conf) { super.setConf(conf); TEST_UTIL = new HBaseTestingUtility(conf); TraceUtil.initTracer(conf); // <-- add this } {code} I will file a new issue to address the problem. Seems the span in SyncFuture is not used any more. Thanks. > Running WALPerformanceEvaluation against asyncfswal throws exceptions > - > > Key: HBASE-19371 > URL: https://issues.apache.org/jira/browse/HBASE-19371 > Project: HBase > Issue Type: Bug >Reporter: stack > Fix For: 2.0.0-beta-1 > > > Was trying to do a perf eval on asyncfswal. I ran w/ these args: > Performance counter stats for '/home/stack/hbase/bin/hbase --config > /home/stack/conf_hbase org.apache.hadoop.hbase.wal.WALPerformanceEvaluation > -path /user/stack/logs/ -verify -threads 25 -iterations 100 -keySize 50 > -valueSize 100 -syncInterval 10': > The verify fails on all runs: > Exception in thread "main" java.lang.IllegalStateException: Counted=12390228, > expected=2500 > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.run(WALPerformanceEvaluation.java:368) > > at > org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.innerMain(WALPerformanceEvaluation.java:597) > > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.main(WALPerformanceEvaluation.java:601) > I need to fix test or figure what is wrong in asyncfswal. > Also seeing these when I run w/ one thread only: > 2017-11-28 21:25:49,952 WARN [AsyncFSWAL-1-1] wal.AsyncFSWAL: > RingBufferTruck with unexpected type: EMPTY > Log has a spew of them. > Has stuff like this too: > 2017-11-28 21:25:40,065 WARN [Close-WAL-Writer-3] > wal.AsyncProtobufLogWriter: normal close failed, try recover > > java.io.IOException: stream already broken > > at > org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.endBlock(FanOutOneBlockAsyncDFSOutput.java:510) > > at > org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.lambda$close$12(FanOutOneBlockAsyncDFSOutput.java:550) > >at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:163) > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:403) > at > org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:462) > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858) > > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138) > at java.lang.Thread.run(Thread.java:748) > Starts out spewing EMPTY here: > 2017-11-28 21:16:52,051 INFO [main] regionserver.HRegion: Setting > FlushNonSloppyStoresFirstPolicy for the > region=WALPerformanceEvaluation:0,,1511932610787.deca03e0ca447fa25d02fe9cd6e31aa4. > 2017-11-28 21:16:52,058 DEBUG [main] regionserver.HRegion: Found 0 recovered > edits file(s) under > hdfs://ve0524.halxg.cloudera.com:8020/user/stack/logs/data/WALPerformanceEvaluation/0/deca03e0ca447fa25d02fe9cd6e31aa4 > 2017-11-28 21:16:52,068 DEBUG [main] regionserver.FlushLargeStoresPolicy: No > hbase.hregion.percolumnfamilyflush.size.lower.bound set in description of > table WALPerformanceEvaluation:0, use config (134217728) instead > 2017-11-28 21:16:52,084 DEBUG [main] wal.WALSplitter: Wrote >
[jira] [Commented] (HBASE-19371) Running WALPerformanceEvaluation against asyncfswal throws exceptions
[ https://issues.apache.org/jira/browse/HBASE-19371?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16270328#comment-16270328 ] Duo Zhang commented on HBASE-19371: --- Tried reverting HBASE-17049 but did not work. Let me dig more. > Running WALPerformanceEvaluation against asyncfswal throws exceptions > - > > Key: HBASE-19371 > URL: https://issues.apache.org/jira/browse/HBASE-19371 > Project: HBase > Issue Type: Bug >Reporter: stack > Fix For: 2.0.0-beta-1 > > > Was trying to do a perf eval on asyncfswal. I ran w/ these args: > Performance counter stats for '/home/stack/hbase/bin/hbase --config > /home/stack/conf_hbase org.apache.hadoop.hbase.wal.WALPerformanceEvaluation > -path /user/stack/logs/ -verify -threads 25 -iterations 100 -keySize 50 > -valueSize 100 -syncInterval 10': > The verify fails on all runs: > Exception in thread "main" java.lang.IllegalStateException: Counted=12390228, > expected=2500 > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.run(WALPerformanceEvaluation.java:368) > > at > org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.innerMain(WALPerformanceEvaluation.java:597) > > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.main(WALPerformanceEvaluation.java:601) > I need to fix test or figure what is wrong in asyncfswal. > Also seeing these when I run w/ one thread only: > 2017-11-28 21:25:49,952 WARN [AsyncFSWAL-1-1] wal.AsyncFSWAL: > RingBufferTruck with unexpected type: EMPTY > Log has a spew of them. > Has stuff like this too: > 2017-11-28 21:25:40,065 WARN [Close-WAL-Writer-3] > wal.AsyncProtobufLogWriter: normal close failed, try recover > > java.io.IOException: stream already broken > > at > org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.endBlock(FanOutOneBlockAsyncDFSOutput.java:510) > > at > org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.lambda$close$12(FanOutOneBlockAsyncDFSOutput.java:550) > >at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:163) > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:403) > at > org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:462) > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858) > > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138) > at java.lang.Thread.run(Thread.java:748) > Starts out spewing EMPTY here: > 2017-11-28 21:16:52,051 INFO [main] regionserver.HRegion: Setting > FlushNonSloppyStoresFirstPolicy for the > region=WALPerformanceEvaluation:0,,1511932610787.deca03e0ca447fa25d02fe9cd6e31aa4. > 2017-11-28 21:16:52,058 DEBUG [main] regionserver.HRegion: Found 0 recovered > edits file(s) under > hdfs://ve0524.halxg.cloudera.com:8020/user/stack/logs/data/WALPerformanceEvaluation/0/deca03e0ca447fa25d02fe9cd6e31aa4 > 2017-11-28 21:16:52,068 DEBUG [main] regionserver.FlushLargeStoresPolicy: No > hbase.hregion.percolumnfamilyflush.size.lower.bound set in description of > table WALPerformanceEvaluation:0, use config (134217728) instead > 2017-11-28 21:16:52,084 DEBUG [main] wal.WALSplitter: Wrote > file=hdfs://ve0524.halxg.cloudera.com:8020/user/stack/logs/data/WALPerformanceEvaluation/0/deca03e0ca447fa25d02fe9cd6e31aa4/recovered.edits/2.seqid, > newSeqId=2, maxSeqId=0 > 2017-11-28 21:16:52,084 INFO [main] regionserver.HRegion: Onlined > deca03e0ca447fa25d02fe9cd6e31aa4; next sequenceid=2 > 2017-11-28 21:16:52,185 WARN [AsyncFSWAL-1-1] wal.AsyncFSWAL: > RingBufferTruck with unexpected type: EMPTY > 2017-11-28 21:16:52,185 WARN [AsyncFSWAL-1-1] wal.AsyncFSWAL: > RingBufferTruck with unexpected type: EMPTY > 2017-11-28 21:16:52,185 WARN
[jira] [Commented] (HBASE-19371) Running WALPerformanceEvaluation against asyncfswal throws exceptions
[ https://issues.apache.org/jira/browse/HBASE-19371?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16270321#comment-16270321 ] Duo Zhang commented on HBASE-19371: --- OK, I got the same problem {quote} 2017-11-29 15:29:40,910 WARN [AsyncFSWAL-1-1] wal.AsyncFSWAL: RingBufferTruck with unexpected type: EMPTY {quote} Let me check. > Running WALPerformanceEvaluation against asyncfswal throws exceptions > - > > Key: HBASE-19371 > URL: https://issues.apache.org/jira/browse/HBASE-19371 > Project: HBase > Issue Type: Bug >Reporter: stack > Fix For: 2.0.0-beta-1 > > > Was trying to do a perf eval on asyncfswal. I ran w/ these args: > Performance counter stats for '/home/stack/hbase/bin/hbase --config > /home/stack/conf_hbase org.apache.hadoop.hbase.wal.WALPerformanceEvaluation > -path /user/stack/logs/ -verify -threads 25 -iterations 100 -keySize 50 > -valueSize 100 -syncInterval 10': > The verify fails on all runs: > Exception in thread "main" java.lang.IllegalStateException: Counted=12390228, > expected=2500 > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.run(WALPerformanceEvaluation.java:368) > > at > org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.innerMain(WALPerformanceEvaluation.java:597) > > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.main(WALPerformanceEvaluation.java:601) > I need to fix test or figure what is wrong in asyncfswal. > Also seeing these when I run w/ one thread only: > 2017-11-28 21:25:49,952 WARN [AsyncFSWAL-1-1] wal.AsyncFSWAL: > RingBufferTruck with unexpected type: EMPTY > Log has a spew of them. > Has stuff like this too: > 2017-11-28 21:25:40,065 WARN [Close-WAL-Writer-3] > wal.AsyncProtobufLogWriter: normal close failed, try recover > > java.io.IOException: stream already broken > > at > org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.endBlock(FanOutOneBlockAsyncDFSOutput.java:510) > > at > org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.lambda$close$12(FanOutOneBlockAsyncDFSOutput.java:550) > >at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:163) > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:403) > at > org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:462) > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858) > > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138) > at java.lang.Thread.run(Thread.java:748) > Starts out spewing EMPTY here: > 2017-11-28 21:16:52,051 INFO [main] regionserver.HRegion: Setting > FlushNonSloppyStoresFirstPolicy for the > region=WALPerformanceEvaluation:0,,1511932610787.deca03e0ca447fa25d02fe9cd6e31aa4. > 2017-11-28 21:16:52,058 DEBUG [main] regionserver.HRegion: Found 0 recovered > edits file(s) under > hdfs://ve0524.halxg.cloudera.com:8020/user/stack/logs/data/WALPerformanceEvaluation/0/deca03e0ca447fa25d02fe9cd6e31aa4 > 2017-11-28 21:16:52,068 DEBUG [main] regionserver.FlushLargeStoresPolicy: No > hbase.hregion.percolumnfamilyflush.size.lower.bound set in description of > table WALPerformanceEvaluation:0, use config (134217728) instead > 2017-11-28 21:16:52,084 DEBUG [main] wal.WALSplitter: Wrote > file=hdfs://ve0524.halxg.cloudera.com:8020/user/stack/logs/data/WALPerformanceEvaluation/0/deca03e0ca447fa25d02fe9cd6e31aa4/recovered.edits/2.seqid, > newSeqId=2, maxSeqId=0 > 2017-11-28 21:16:52,084 INFO [main] regionserver.HRegion: Onlined > deca03e0ca447fa25d02fe9cd6e31aa4; next sequenceid=2 > 2017-11-28 21:16:52,185 WARN [AsyncFSWAL-1-1] wal.AsyncFSWAL: > RingBufferTruck with unexpected type: EMPTY > 2017-11-28 21:16:52,185 WARN
[jira] [Commented] (HBASE-19371) Running WALPerformanceEvaluation against asyncfswal throws exceptions
[ https://issues.apache.org/jira/browse/HBASE-19371?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16270319#comment-16270319 ] Duo Zhang commented on HBASE-19371: --- Oh shit I forget to change the config to AsyncFSWAL... Let me try again. > Running WALPerformanceEvaluation against asyncfswal throws exceptions > - > > Key: HBASE-19371 > URL: https://issues.apache.org/jira/browse/HBASE-19371 > Project: HBase > Issue Type: Bug >Reporter: stack > Fix For: 2.0.0-beta-1 > > > Was trying to do a perf eval on asyncfswal. I ran w/ these args: > Performance counter stats for '/home/stack/hbase/bin/hbase --config > /home/stack/conf_hbase org.apache.hadoop.hbase.wal.WALPerformanceEvaluation > -path /user/stack/logs/ -verify -threads 25 -iterations 100 -keySize 50 > -valueSize 100 -syncInterval 10': > The verify fails on all runs: > Exception in thread "main" java.lang.IllegalStateException: Counted=12390228, > expected=2500 > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.run(WALPerformanceEvaluation.java:368) > > at > org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.innerMain(WALPerformanceEvaluation.java:597) > > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.main(WALPerformanceEvaluation.java:601) > I need to fix test or figure what is wrong in asyncfswal. > Also seeing these when I run w/ one thread only: > 2017-11-28 21:25:49,952 WARN [AsyncFSWAL-1-1] wal.AsyncFSWAL: > RingBufferTruck with unexpected type: EMPTY > Log has a spew of them. > Has stuff like this too: > 2017-11-28 21:25:40,065 WARN [Close-WAL-Writer-3] > wal.AsyncProtobufLogWriter: normal close failed, try recover > > java.io.IOException: stream already broken > > at > org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.endBlock(FanOutOneBlockAsyncDFSOutput.java:510) > > at > org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.lambda$close$12(FanOutOneBlockAsyncDFSOutput.java:550) > >at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:163) > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:403) > at > org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:462) > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858) > > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138) > at java.lang.Thread.run(Thread.java:748) > Starts out spewing EMPTY here: > 2017-11-28 21:16:52,051 INFO [main] regionserver.HRegion: Setting > FlushNonSloppyStoresFirstPolicy for the > region=WALPerformanceEvaluation:0,,1511932610787.deca03e0ca447fa25d02fe9cd6e31aa4. > 2017-11-28 21:16:52,058 DEBUG [main] regionserver.HRegion: Found 0 recovered > edits file(s) under > hdfs://ve0524.halxg.cloudera.com:8020/user/stack/logs/data/WALPerformanceEvaluation/0/deca03e0ca447fa25d02fe9cd6e31aa4 > 2017-11-28 21:16:52,068 DEBUG [main] regionserver.FlushLargeStoresPolicy: No > hbase.hregion.percolumnfamilyflush.size.lower.bound set in description of > table WALPerformanceEvaluation:0, use config (134217728) instead > 2017-11-28 21:16:52,084 DEBUG [main] wal.WALSplitter: Wrote > file=hdfs://ve0524.halxg.cloudera.com:8020/user/stack/logs/data/WALPerformanceEvaluation/0/deca03e0ca447fa25d02fe9cd6e31aa4/recovered.edits/2.seqid, > newSeqId=2, maxSeqId=0 > 2017-11-28 21:16:52,084 INFO [main] regionserver.HRegion: Onlined > deca03e0ca447fa25d02fe9cd6e31aa4; next sequenceid=2 > 2017-11-28 21:16:52,185 WARN [AsyncFSWAL-1-1] wal.AsyncFSWAL: > RingBufferTruck with unexpected type: EMPTY > 2017-11-28 21:16:52,185 WARN [AsyncFSWAL-1-1] wal.AsyncFSWAL: > RingBufferTruck with unexpected type: EMPTY > 2017-11-28 21:16:52,185
[jira] [Commented] (HBASE-19371) Running WALPerformanceEvaluation against asyncfswal throws exceptions
[ https://issues.apache.org/jira/browse/HBASE-19371?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16270315#comment-16270315 ] Duo Zhang commented on HBASE-19371: --- [~stack] I tried the same command with you, it worked. Which HDFS version do you use? Seems you use 2.8.0? That maybe a problem as we only run UTs with 2.7.x, so maybe AsyncFSWAL is not compatible with 2.8.x... Thanks. > Running WALPerformanceEvaluation against asyncfswal throws exceptions > - > > Key: HBASE-19371 > URL: https://issues.apache.org/jira/browse/HBASE-19371 > Project: HBase > Issue Type: Bug >Reporter: stack > Fix For: 2.0.0-beta-1 > > > Was trying to do a perf eval on asyncfswal. I ran w/ these args: > Performance counter stats for '/home/stack/hbase/bin/hbase --config > /home/stack/conf_hbase org.apache.hadoop.hbase.wal.WALPerformanceEvaluation > -path /user/stack/logs/ -verify -threads 25 -iterations 100 -keySize 50 > -valueSize 100 -syncInterval 10': > The verify fails on all runs: > Exception in thread "main" java.lang.IllegalStateException: Counted=12390228, > expected=2500 > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.run(WALPerformanceEvaluation.java:368) > > at > org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.innerMain(WALPerformanceEvaluation.java:597) > > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.main(WALPerformanceEvaluation.java:601) > I need to fix test or figure what is wrong in asyncfswal. > Also seeing these when I run w/ one thread only: > 2017-11-28 21:25:49,952 WARN [AsyncFSWAL-1-1] wal.AsyncFSWAL: > RingBufferTruck with unexpected type: EMPTY > Log has a spew of them. > Has stuff like this too: > 2017-11-28 21:25:40,065 WARN [Close-WAL-Writer-3] > wal.AsyncProtobufLogWriter: normal close failed, try recover > > java.io.IOException: stream already broken > > at > org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.endBlock(FanOutOneBlockAsyncDFSOutput.java:510) > > at > org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.lambda$close$12(FanOutOneBlockAsyncDFSOutput.java:550) > >at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:163) > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:403) > at > org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:462) > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858) > > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138) > at java.lang.Thread.run(Thread.java:748) > Starts out spewing EMPTY here: > 2017-11-28 21:16:52,051 INFO [main] regionserver.HRegion: Setting > FlushNonSloppyStoresFirstPolicy for the > region=WALPerformanceEvaluation:0,,1511932610787.deca03e0ca447fa25d02fe9cd6e31aa4. > 2017-11-28 21:16:52,058 DEBUG [main] regionserver.HRegion: Found 0 recovered > edits file(s) under > hdfs://ve0524.halxg.cloudera.com:8020/user/stack/logs/data/WALPerformanceEvaluation/0/deca03e0ca447fa25d02fe9cd6e31aa4 > 2017-11-28 21:16:52,068 DEBUG [main] regionserver.FlushLargeStoresPolicy: No > hbase.hregion.percolumnfamilyflush.size.lower.bound set in description of > table WALPerformanceEvaluation:0, use config (134217728) instead > 2017-11-28 21:16:52,084 DEBUG [main] wal.WALSplitter: Wrote > file=hdfs://ve0524.halxg.cloudera.com:8020/user/stack/logs/data/WALPerformanceEvaluation/0/deca03e0ca447fa25d02fe9cd6e31aa4/recovered.edits/2.seqid, > newSeqId=2, maxSeqId=0 > 2017-11-28 21:16:52,084 INFO [main] regionserver.HRegion: Onlined > deca03e0ca447fa25d02fe9cd6e31aa4; next sequenceid=2 > 2017-11-28 21:16:52,185 WARN [AsyncFSWAL-1-1] wal.AsyncFSWAL: > RingBufferTruck with unexpected
[jira] [Commented] (HBASE-19371) Running WALPerformanceEvaluation against asyncfswal throws exceptions
[ https://issues.apache.org/jira/browse/HBASE-19371?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16270207#comment-16270207 ] Duo Zhang commented on HBASE-19371: --- {quote} This is tip of branch-2 minus the commit of the change to the ringbuffer (I'd not pulled in that change). {quote} Which commit? > Running WALPerformanceEvaluation against asyncfswal throws exceptions > - > > Key: HBASE-19371 > URL: https://issues.apache.org/jira/browse/HBASE-19371 > Project: HBase > Issue Type: Bug >Reporter: stack > Fix For: 2.0.0-beta-1 > > > Was trying to do a perf eval on asyncfswal. I ran w/ these args: > Performance counter stats for '/home/stack/hbase/bin/hbase --config > /home/stack/conf_hbase org.apache.hadoop.hbase.wal.WALPerformanceEvaluation > -path /user/stack/logs/ -verify -threads 25 -iterations 100 -keySize 50 > -valueSize 100 -syncInterval 10': > The verify fails on all runs: > Exception in thread "main" java.lang.IllegalStateException: Counted=12390228, > expected=2500 > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.run(WALPerformanceEvaluation.java:368) > > at > org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.innerMain(WALPerformanceEvaluation.java:597) > > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.main(WALPerformanceEvaluation.java:601) > I need to fix test or figure what is wrong in asyncfswal. > Also seeing these when I run w/ one thread only: > 2017-11-28 21:25:49,952 WARN [AsyncFSWAL-1-1] wal.AsyncFSWAL: > RingBufferTruck with unexpected type: EMPTY > Log has a spew of them. > Has stuff like this too: > 2017-11-28 21:25:40,065 WARN [Close-WAL-Writer-3] > wal.AsyncProtobufLogWriter: normal close failed, try recover > > java.io.IOException: stream already broken > > at > org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.endBlock(FanOutOneBlockAsyncDFSOutput.java:510) > > at > org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.lambda$close$12(FanOutOneBlockAsyncDFSOutput.java:550) > >at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:163) > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:403) > at > org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:462) > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858) > > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138) > at java.lang.Thread.run(Thread.java:748) > Starts out spewing EMPTY here: > 2017-11-28 21:16:52,051 INFO [main] regionserver.HRegion: Setting > FlushNonSloppyStoresFirstPolicy for the > region=WALPerformanceEvaluation:0,,1511932610787.deca03e0ca447fa25d02fe9cd6e31aa4. > 2017-11-28 21:16:52,058 DEBUG [main] regionserver.HRegion: Found 0 recovered > edits file(s) under > hdfs://ve0524.halxg.cloudera.com:8020/user/stack/logs/data/WALPerformanceEvaluation/0/deca03e0ca447fa25d02fe9cd6e31aa4 > 2017-11-28 21:16:52,068 DEBUG [main] regionserver.FlushLargeStoresPolicy: No > hbase.hregion.percolumnfamilyflush.size.lower.bound set in description of > table WALPerformanceEvaluation:0, use config (134217728) instead > 2017-11-28 21:16:52,084 DEBUG [main] wal.WALSplitter: Wrote > file=hdfs://ve0524.halxg.cloudera.com:8020/user/stack/logs/data/WALPerformanceEvaluation/0/deca03e0ca447fa25d02fe9cd6e31aa4/recovered.edits/2.seqid, > newSeqId=2, maxSeqId=0 > 2017-11-28 21:16:52,084 INFO [main] regionserver.HRegion: Onlined > deca03e0ca447fa25d02fe9cd6e31aa4; next sequenceid=2 > 2017-11-28 21:16:52,185 WARN [AsyncFSWAL-1-1] wal.AsyncFSWAL: > RingBufferTruck with unexpected type: EMPTY > 2017-11-28 21:16:52,185 WARN [AsyncFSWAL-1-1] wal.AsyncFSWAL: >
[jira] [Commented] (HBASE-19371) Running WALPerformanceEvaluation against asyncfswal throws exceptions
[ https://issues.apache.org/jira/browse/HBASE-19371?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16270206#comment-16270206 ] Duo Zhang commented on HBASE-19371: --- I haven't seen this before. Let me try. > Running WALPerformanceEvaluation against asyncfswal throws exceptions > - > > Key: HBASE-19371 > URL: https://issues.apache.org/jira/browse/HBASE-19371 > Project: HBase > Issue Type: Bug >Reporter: stack > Fix For: 2.0.0-beta-1 > > > Was trying to do a perf eval on asyncfswal. I ran w/ these args: > Performance counter stats for '/home/stack/hbase/bin/hbase --config > /home/stack/conf_hbase org.apache.hadoop.hbase.wal.WALPerformanceEvaluation > -path /user/stack/logs/ -verify -threads 25 -iterations 100 -keySize 50 > -valueSize 100 -syncInterval 10': > The verify fails on all runs: > Exception in thread "main" java.lang.IllegalStateException: Counted=12390228, > expected=2500 > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.run(WALPerformanceEvaluation.java:368) > > at > org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.innerMain(WALPerformanceEvaluation.java:597) > > at > org.apache.hadoop.hbase.wal.WALPerformanceEvaluation.main(WALPerformanceEvaluation.java:601) > I need to fix test or figure what is wrong in asyncfswal. > Also seeing these when I run w/ one thread only: > 2017-11-28 21:25:49,952 WARN [AsyncFSWAL-1-1] wal.AsyncFSWAL: > RingBufferTruck with unexpected type: EMPTY > Log has a spew of them. > Has stuff like this too: > 2017-11-28 21:25:40,065 WARN [Close-WAL-Writer-3] > wal.AsyncProtobufLogWriter: normal close failed, try recover > > java.io.IOException: stream already broken > > at > org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.endBlock(FanOutOneBlockAsyncDFSOutput.java:510) > > at > org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.lambda$close$12(FanOutOneBlockAsyncDFSOutput.java:550) > >at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:163) > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:403) > at > org.apache.hadoop.hbase.shaded.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:462) > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858) > > at > org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138) > at java.lang.Thread.run(Thread.java:748) > Starts out spewing EMPTY here: > 2017-11-28 21:16:52,051 INFO [main] regionserver.HRegion: Setting > FlushNonSloppyStoresFirstPolicy for the > region=WALPerformanceEvaluation:0,,1511932610787.deca03e0ca447fa25d02fe9cd6e31aa4. > 2017-11-28 21:16:52,058 DEBUG [main] regionserver.HRegion: Found 0 recovered > edits file(s) under > hdfs://ve0524.halxg.cloudera.com:8020/user/stack/logs/data/WALPerformanceEvaluation/0/deca03e0ca447fa25d02fe9cd6e31aa4 > 2017-11-28 21:16:52,068 DEBUG [main] regionserver.FlushLargeStoresPolicy: No > hbase.hregion.percolumnfamilyflush.size.lower.bound set in description of > table WALPerformanceEvaluation:0, use config (134217728) instead > 2017-11-28 21:16:52,084 DEBUG [main] wal.WALSplitter: Wrote > file=hdfs://ve0524.halxg.cloudera.com:8020/user/stack/logs/data/WALPerformanceEvaluation/0/deca03e0ca447fa25d02fe9cd6e31aa4/recovered.edits/2.seqid, > newSeqId=2, maxSeqId=0 > 2017-11-28 21:16:52,084 INFO [main] regionserver.HRegion: Onlined > deca03e0ca447fa25d02fe9cd6e31aa4; next sequenceid=2 > 2017-11-28 21:16:52,185 WARN [AsyncFSWAL-1-1] wal.AsyncFSWAL: > RingBufferTruck with unexpected type: EMPTY > 2017-11-28 21:16:52,185 WARN [AsyncFSWAL-1-1] wal.AsyncFSWAL: > RingBufferTruck with unexpected type: EMPTY > 2017-11-28 21:16:52,185 WARN [AsyncFSWAL-1-1]