On Tue, Oct 4, 2011 at 3:41 PM, Matt Foley <mfo...@hortonworks.com> wrote: > I am going to spin an RC2 early tomorrow. Does anyone have other issues > they consider critical for 205.0?
I've been playing with it. Recovering the lease on an open file (An HBase WAL) the length is always zero and I don't seem to be able to recover any edits from the file we writing at time of the crash: 2011-10-04 21:17:04,486 DEBUG org.apache.hadoop.hbase.regionserver.wal.HLogSplitter: Splitting hlog 34 of 34: hdfs://sv4r11s38:7000/hbase/.logs/sv4r8s38,7003,1317760866490/sv4r8s38%3A7003.1317762914728, length=0 2011-10-04 21:17:04,486 INFO org.apache.hadoop.hbase.util.FSUtils: Recovering file hdfs://sv4r11s38:7000/hbase/.logs/sv4r8s38,7003,1317760866490/sv4r8s38%3A7003.1317762914728 2011-10-04 21:17:05,487 INFO org.apache.hadoop.hbase.util.FSUtils: Finished lease recover attempt for hdfs://sv4r11s38:7000/hbase/.logs/sv4r8s38,7003,1317760866490/sv4r8s38%3A7003.1317762914728 2011-10-04 21:17:05,488 WARN org.apache.hadoop.hbase.regionserver.wal.HLogSplitter: File hdfs://sv4r11s38:7000/hbase/.logs/sv4r8s38,7003,1317760866490/sv4r8s38%3A7003.1317762914728 might be still open, length is 0 Its probably me misconfiguring 205 compared to 0.20-append. I got some of these tooo though I'd just opened the file a few seconds earlier: 2011-10-04 21:16:28,439 DEBUG org.apache.hadoop.hbase.regionserver.wal.HLogSplitter: Creating writer path=hdfs://sv4r11s38:7000/hbase/TestTable/62ff2cb514838519e5fa4282a8af4c35/recovered.edits/0000000000000008111 region=62ff2cb514838519e5fa4282a8af4c35 .... 2011-10-04 21:17:06,883 ERROR org.apache.hadoop.hbase.regionserver.wal.HLogSplitter: Couldn't close log at hdfs://sv4r11s38:7000/hbase/TestTable/62ff2cb514838519e5fa4282a8af4c35/recovered.edits/0000000000000008111 org.apache.hadoop.ipc.RemoteException: org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException: No lease on /hbase/TestTable/62ff2cb514838519e5fa4282a8af4c35/recovered.edits/0000000000000008111 File does not exist. [Lease. Holder: DFSClient_hb_m_sv4r11s38:7001_1317760883384, pendingcreates: 3] at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease(FSNamesystem.java:1604) at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease(FSNamesystem.java:1595) at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.completeFileInternal(FSNamesystem.java:1650) at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.completeFile(FSNamesystem.java:1638) at org.apache.hadoop.hdfs.server.namenode.NameNode.complete(NameNode.java:682) at sun.reflect.GeneratedMethodAccessor15.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25) at java.lang.reflect.Method.invoke(Method.java:597) at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:563) at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1388) at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1384) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:396) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1059) at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1382) at org.apache.hadoop.ipc.Client.call(Client.java:1066) at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:225) at $Proxy6.complete(Unknown Source) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25) at java.lang.reflect.Method.invoke(Method.java:597) at org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:82) at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:59) at $Proxy6.complete(Unknown Source) at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.closeInternal(DFSClient.java:3711) at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.close(DFSClient.java:3626) at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:61) at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:86) at org.apache.hadoop.io.SequenceFile$Writer.close(SequenceFile.java:966) at org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogWriter.close(SequenceFileLogWriter.java:138) at org.apache.hadoop.hbase.regionserver.wal.HLogSplitter$OutputSink.closeStreams(HLogSplitter.java:768) at org.apache.hadoop.hbase.regionserver.wal.HLogSplitter$OutputSink.finishWritingAndClose(HLogSplitter.java:753) at org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.splitLog(HLogSplitter.java:300) at org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.splitLog(HLogSplitter.java:188) at org.apache.hadoop.hbase.master.MasterFileSystem.splitLog(MasterFileSystem.java:201) at org.apache.hadoop.hbase.master.handler.ServerShutdownHandler.process(ServerShutdownHandler.java:153) at org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:156) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) I'll keep banging at it. St.Ack