You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@hbase.apache.org by lars hofhansl <la...@apache.org> on 2013/11/05 23:44:51 UTC

Re: [VOTE] The 1st hbase 0.94.13 release candidate is available for download

Please have a look.
I know testing a release is like pulling teeth (I think I said this before).

-- Lars



________________________________
 From: Ted Yu <yu...@gmail.com>
To: "dev@hbase.apache.org" <de...@hbase.apache.org>; lars hofhansl <la...@apache.org> 
Sent: Wednesday, October 30, 2013 11:54 AM
Subject: Re: [VOTE] The 1st hbase 0.94.13 release candidate is available for download
 

+1

- checked the documentation
- ran unit test suite with secure profile
- built Phoenix with 0.94.13



On Wed, Oct 30, 2013 at 10:05 AM, lars hofhansl <la...@apache.org> wrote:

> +1
>
> - checked the tarball
> - checked the documentation
> - created a table, inserted 100's on millions of KVs, scanned, flushed,
> compacted, scanned again
> - ran in both local and distributed mode
>
>
> ________________________________
> From: lars hofhansl <la...@apache.org>
> To: "dev@hbase.apache.org" <de...@hbase.apache.org>; lars hofhansl <
> larsh@apache.org>
> Sent: Tuesday, October 29, 2013 10:46 PM
> Subject: Re: [VOTE] The 1st hbase 0.94.13 release candidate is available
> for download
>
>
> In case it was not obvious... I mean the first 0.94.13 RC.
>
> -- Lars
>
>
>
> ________________________________
>
> From: lars hofhansl <la...@apache.org>
> To: hbase-dev <de...@hbase.apache.org>
> Sent: Tuesday, October 29, 2013 10:42 PM
> Subject: [VOTE] The 1st hbase 0.94.12 release candidate is available for
> download
>
>
> The 1st 0.94.12 RC is available for download at
> http://people.apache.org/~larsh/hbase-0.94.13-rc0/
> Signed with my code signing key: C7CFE328
>
> As before 0.94.13 is mostly a bug fix release along with a few performance
> improvements:
>     [HBASE-9711] - Improve HBASE-9428 - avoid copying bytes for
> RegexFilter unless necessary
>     [HBASE-7600] - TestAdmin.testCreateBadTables is failing occasionally
>     [HBASE-8521] - Cells cannot be overwritten with bulk loaded HFiles
>     [HBASE-9430] - Memstore heapSize calculation - DEEP_OVERHEAD is
> incorrect
>     [HBASE-9504] - Backport HBASE-1212 to 0.94
>     [HBASE-9548] - Cleanup SnapshotTestingUtils
>     [HBASE-9607] - Data loss after snapshot restore into cloned table
>     [HBASE-9649] - HFilePrettyPrinter should not throw a NPE if FirstKey
> or LastKey is null.
>     [HBASE-9651] - Backport HBASE-3890 'Scheduled tasks in distributed log
> splitting not in sync with ZK' to 0.94
>     [HBASE-9727] - HBase Rest Server - DELETE scanner operation is a no-op
>     [HBASE-9731] - updatesBlockedSeconds RegionServer metric should not be
> a histogram
>     [HBASE-9732] - Static AtomicLong updated in StoreFileScanner every
> (re)seek
>     [HBASE-9737] - Corrupt HFile cause resource leak leading to Region
> Server OOM
>     [HBASE-9745] - Append HBASE_CLASSPATH to end of Java classpath and use
> another env var for prefix
>     [HBASE-9747] - PrefixFilter with OR condition gives wrong results
>     [HBASE-9751] - Excessive readpoints checks in StoreFileScanner
>     [HBASE-9753] - Excessive readpoint checks in MemstoreScanner
>     [HBASE-9783] - o.a.h.h.r.HRegion.mutateRow() with non-existent CF
> cause NPE
>     [HBASE-9789] - Change logging for Coprocessor exec call to trace
>     [HBASE-9807] - block encoder unnecessarily copies the key for each
> reseek
>     [HBASE-9842] - Backport HBASE-9593 and HBASE-8667 to 0.94
>     [HBASE-9847] - HConnectionImplementation does not connect to new
> active master
>     [HBASE-9221] - Provide interface for getting a User in the client
>     [HBASE-9488] - Improve performance for small scan
>     [HBASE-9716] - LoadTestTool should provide default min and max
> settings to the data generator
>     [HBASE-9749] - Custom threadpool for Coprocessor obtained HTables
>     [HBASE-9819] - Backport HBASE-8372 'Provide mutability to
> CompoundConfiguration' to 0.94
>     [HBASE-8553] - improve unit-test coverage of package
> org.apache.hadoop.hbase.mapreduce.hadoopbackport
>     [HBASE-9851] - TestHBaseFsck.testQuarantineMissingHFile is flaky
>     [HBASE-9852] - TestRpcMetrics.testCustomMetrics is flaky
>
> The list of changes is also available here:
> https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12310753&version=12324885
>
> Here're the jenkins runs for this RC:
> https://builds.apache.org/job/HBase-0.94.13/11/ and
> https://builds.apache.org/job/HBase-0.94.13-security/11/
>
> Please try out the RC, check out the doc, take it for a spin, etc, and
> vote +1/-1 by November 5th on whether we should release this as 0.94.13.
>
> Thanks.
>
> -- Lars
>

Re: [VOTE] The 1st hbase 0.94.13 release candidate is available for download

Posted by Jean-Marc Spaggiari <je...@spaggiari.org>.
No. I just use mvn test -P runAllTests -Dsurefire.secondPartThreadCount=1
(I tried with 16 initially, then 8, 4 and move to 1 to see if it was more
stable).


2013/11/7 Ted Yu <yu...@gmail.com>

> TestRegionObserverScannerOpenHook has been fixed through HBASE-9836
>
> I can log a JIRA to backport the fix to 0.94
>
> Did you use secure profile for running the tests ?
>
> Cheers
>
>
> On Thu, Nov 7, 2013 at 5:33 AM, Jean-Marc Spaggiari <
> jean-marc@spaggiari.org
> > wrote:
>
> > I tried multiple times on different servers to run the test suite but it
> > always fail on those test cases:
> >
> > Tests in error:
> >
> >
> >
> testRegionObserverCompactionTimeStacking(org.apache.hadoop.hbase.coprocessor.TestRegionObserverScannerOpenHook):
> > Shutting down
> >   testCalls(org.apache.hadoop.hbase.ipc.TestProtocolExtension): 20000
> > millis timeout while waiting for channel to be ready for connect. ch :
> > java.nio.channels.SocketChannel[connection-pending
> > remote=/0:0:0:0:0:0:0:0:6016]
> >
> >
> > Does any one have faced the same? I will still deploy it on my test
> > cluster, but I m wondering the impact.
> >
> >
> > 2013/11/5 Jean-Marc Spaggiari <je...@spaggiari.org>
> >
> > > Sorry Lars. I will have time on Wednesday, i will run my tests against
> is
> > > that day...
> > >
> > > JM
> > > Le 2013-11-05 17:45, "lars hofhansl" <la...@apache.org> a écrit :
> > >
> > > Please have a look.
> > >> I know testing a release is like pulling teeth (I think I said this
> > >> before).
> > >>
> > >> -- Lars
> > >>
> > >>
> > >>
> > >> ________________________________
> > >>  From: Ted Yu <yu...@gmail.com>
> > >> To: "dev@hbase.apache.org" <de...@hbase.apache.org>; lars hofhansl <
> > >> larsh@apache.org>
> > >> Sent: Wednesday, October 30, 2013 11:54 AM
> > >> Subject: Re: [VOTE] The 1st hbase 0.94.13 release candidate is
> available
> > >> for download
> > >>
> > >>
> > >> +1
> > >>
> > >> - checked the documentation
> > >> - ran unit test suite with secure profile
> > >> - built Phoenix with 0.94.13
> > >>
> > >>
> > >>
> > >> On Wed, Oct 30, 2013 at 10:05 AM, lars hofhansl <la...@apache.org>
> > wrote:
> > >>
> > >> > +1
> > >> >
> > >> > - checked the tarball
> > >> > - checked the documentation
> > >> > - created a table, inserted 100's on millions of KVs, scanned,
> > flushed,
> > >> > compacted, scanned again
> > >> > - ran in both local and distributed mode
> > >> >
> > >> >
> > >> > ________________________________
> > >> > From: lars hofhansl <la...@apache.org>
> > >> > To: "dev@hbase.apache.org" <de...@hbase.apache.org>; lars hofhansl <
> > >> > larsh@apache.org>
> > >> > Sent: Tuesday, October 29, 2013 10:46 PM
> > >> > Subject: Re: [VOTE] The 1st hbase 0.94.13 release candidate is
> > available
> > >> > for download
> > >> >
> > >> >
> > >> > In case it was not obvious... I mean the first 0.94.13 RC.
> > >> >
> > >> > -- Lars
> > >> >
> > >> >
> > >> >
> > >> > ________________________________
> > >> >
> > >> > From: lars hofhansl <la...@apache.org>
> > >> > To: hbase-dev <de...@hbase.apache.org>
> > >> > Sent: Tuesday, October 29, 2013 10:42 PM
> > >> > Subject: [VOTE] The 1st hbase 0.94.12 release candidate is available
> > for
> > >> > download
> > >> >
> > >> >
> > >> > The 1st 0.94.12 RC is available for download at
> > >> > http://people.apache.org/~larsh/hbase-0.94.13-rc0/
> > >> > Signed with my code signing key: C7CFE328
> > >> >
> > >> > As before 0.94.13 is mostly a bug fix release along with a few
> > >> performance
> > >> > improvements:
> > >> >     [HBASE-9711] - Improve HBASE-9428 - avoid copying bytes for
> > >> > RegexFilter unless necessary
> > >> >     [HBASE-7600] - TestAdmin.testCreateBadTables is failing
> > occasionally
> > >> >     [HBASE-8521] - Cells cannot be overwritten with bulk loaded
> HFiles
> > >> >     [HBASE-9430] - Memstore heapSize calculation - DEEP_OVERHEAD is
> > >> > incorrect
> > >> >     [HBASE-9504] - Backport HBASE-1212 to 0.94
> > >> >     [HBASE-9548] - Cleanup SnapshotTestingUtils
> > >> >     [HBASE-9607] - Data loss after snapshot restore into cloned
> table
> > >> >     [HBASE-9649] - HFilePrettyPrinter should not throw a NPE if
> > FirstKey
> > >> > or LastKey is null.
> > >> >     [HBASE-9651] - Backport HBASE-3890 'Scheduled tasks in
> distributed
> > >> log
> > >> > splitting not in sync with ZK' to 0.94
> > >> >     [HBASE-9727] - HBase Rest Server - DELETE scanner operation is a
> > >> no-op
> > >> >     [HBASE-9731] - updatesBlockedSeconds RegionServer metric should
> > not
> > >> be
> > >> > a histogram
> > >> >     [HBASE-9732] - Static AtomicLong updated in StoreFileScanner
> every
> > >> > (re)seek
> > >> >     [HBASE-9737] - Corrupt HFile cause resource leak leading to
> Region
> > >> > Server OOM
> > >> >     [HBASE-9745] - Append HBASE_CLASSPATH to end of Java classpath
> and
> > >> use
> > >> > another env var for prefix
> > >> >     [HBASE-9747] - PrefixFilter with OR condition gives wrong
> results
> > >> >     [HBASE-9751] - Excessive readpoints checks in StoreFileScanner
> > >> >     [HBASE-9753] - Excessive readpoint checks in MemstoreScanner
> > >> >     [HBASE-9783] - o.a.h.h.r.HRegion.mutateRow() with non-existent
> CF
> > >> > cause NPE
> > >> >     [HBASE-9789] - Change logging for Coprocessor exec call to trace
> > >> >     [HBASE-9807] - block encoder unnecessarily copies the key for
> each
> > >> > reseek
> > >> >     [HBASE-9842] - Backport HBASE-9593 and HBASE-8667 to 0.94
> > >> >     [HBASE-9847] - HConnectionImplementation does not connect to new
> > >> > active master
> > >> >     [HBASE-9221] - Provide interface for getting a User in the
> client
> > >> >     [HBASE-9488] - Improve performance for small scan
> > >> >     [HBASE-9716] - LoadTestTool should provide default min and max
> > >> > settings to the data generator
> > >> >     [HBASE-9749] - Custom threadpool for Coprocessor obtained
> HTables
> > >> >     [HBASE-9819] - Backport HBASE-8372 'Provide mutability to
> > >> > CompoundConfiguration' to 0.94
> > >> >     [HBASE-8553] - improve unit-test coverage of package
> > >> > org.apache.hadoop.hbase.mapreduce.hadoopbackport
> > >> >     [HBASE-9851] - TestHBaseFsck.testQuarantineMissingHFile is flaky
> > >> >     [HBASE-9852] - TestRpcMetrics.testCustomMetrics is flaky
> > >> >
> > >> > The list of changes is also available here:
> > >> >
> > >>
> >
> https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12310753&version=12324885
> > >> >
> > >> > Here're the jenkins runs for this RC:
> > >> > https://builds.apache.org/job/HBase-0.94.13/11/ and
> > >> > https://builds.apache.org/job/HBase-0.94.13-security/11/
> > >> >
> > >> > Please try out the RC, check out the doc, take it for a spin, etc,
> and
> > >> > vote +1/-1 by November 5th on whether we should release this as
> > 0.94.13.
> > >> >
> > >> > Thanks.
> > >> >
> > >> > -- Lars
> > >> >
> > >
> > >
> >
>

Re: [VOTE] The 1st hbase 0.94.13 release candidate is available for download

Posted by Ted Yu <yu...@gmail.com>.
TestRegionObserverScannerOpenHook has been fixed through HBASE-9836

I can log a JIRA to backport the fix to 0.94

Did you use secure profile for running the tests ?

Cheers


On Thu, Nov 7, 2013 at 5:33 AM, Jean-Marc Spaggiari <jean-marc@spaggiari.org
> wrote:

> I tried multiple times on different servers to run the test suite but it
> always fail on those test cases:
>
> Tests in error:
>
>
> testRegionObserverCompactionTimeStacking(org.apache.hadoop.hbase.coprocessor.TestRegionObserverScannerOpenHook):
> Shutting down
>   testCalls(org.apache.hadoop.hbase.ipc.TestProtocolExtension): 20000
> millis timeout while waiting for channel to be ready for connect. ch :
> java.nio.channels.SocketChannel[connection-pending
> remote=/0:0:0:0:0:0:0:0:6016]
>
>
> Does any one have faced the same? I will still deploy it on my test
> cluster, but I m wondering the impact.
>
>
> 2013/11/5 Jean-Marc Spaggiari <je...@spaggiari.org>
>
> > Sorry Lars. I will have time on Wednesday, i will run my tests against is
> > that day...
> >
> > JM
> > Le 2013-11-05 17:45, "lars hofhansl" <la...@apache.org> a écrit :
> >
> > Please have a look.
> >> I know testing a release is like pulling teeth (I think I said this
> >> before).
> >>
> >> -- Lars
> >>
> >>
> >>
> >> ________________________________
> >>  From: Ted Yu <yu...@gmail.com>
> >> To: "dev@hbase.apache.org" <de...@hbase.apache.org>; lars hofhansl <
> >> larsh@apache.org>
> >> Sent: Wednesday, October 30, 2013 11:54 AM
> >> Subject: Re: [VOTE] The 1st hbase 0.94.13 release candidate is available
> >> for download
> >>
> >>
> >> +1
> >>
> >> - checked the documentation
> >> - ran unit test suite with secure profile
> >> - built Phoenix with 0.94.13
> >>
> >>
> >>
> >> On Wed, Oct 30, 2013 at 10:05 AM, lars hofhansl <la...@apache.org>
> wrote:
> >>
> >> > +1
> >> >
> >> > - checked the tarball
> >> > - checked the documentation
> >> > - created a table, inserted 100's on millions of KVs, scanned,
> flushed,
> >> > compacted, scanned again
> >> > - ran in both local and distributed mode
> >> >
> >> >
> >> > ________________________________
> >> > From: lars hofhansl <la...@apache.org>
> >> > To: "dev@hbase.apache.org" <de...@hbase.apache.org>; lars hofhansl <
> >> > larsh@apache.org>
> >> > Sent: Tuesday, October 29, 2013 10:46 PM
> >> > Subject: Re: [VOTE] The 1st hbase 0.94.13 release candidate is
> available
> >> > for download
> >> >
> >> >
> >> > In case it was not obvious... I mean the first 0.94.13 RC.
> >> >
> >> > -- Lars
> >> >
> >> >
> >> >
> >> > ________________________________
> >> >
> >> > From: lars hofhansl <la...@apache.org>
> >> > To: hbase-dev <de...@hbase.apache.org>
> >> > Sent: Tuesday, October 29, 2013 10:42 PM
> >> > Subject: [VOTE] The 1st hbase 0.94.12 release candidate is available
> for
> >> > download
> >> >
> >> >
> >> > The 1st 0.94.12 RC is available for download at
> >> > http://people.apache.org/~larsh/hbase-0.94.13-rc0/
> >> > Signed with my code signing key: C7CFE328
> >> >
> >> > As before 0.94.13 is mostly a bug fix release along with a few
> >> performance
> >> > improvements:
> >> >     [HBASE-9711] - Improve HBASE-9428 - avoid copying bytes for
> >> > RegexFilter unless necessary
> >> >     [HBASE-7600] - TestAdmin.testCreateBadTables is failing
> occasionally
> >> >     [HBASE-8521] - Cells cannot be overwritten with bulk loaded HFiles
> >> >     [HBASE-9430] - Memstore heapSize calculation - DEEP_OVERHEAD is
> >> > incorrect
> >> >     [HBASE-9504] - Backport HBASE-1212 to 0.94
> >> >     [HBASE-9548] - Cleanup SnapshotTestingUtils
> >> >     [HBASE-9607] - Data loss after snapshot restore into cloned table
> >> >     [HBASE-9649] - HFilePrettyPrinter should not throw a NPE if
> FirstKey
> >> > or LastKey is null.
> >> >     [HBASE-9651] - Backport HBASE-3890 'Scheduled tasks in distributed
> >> log
> >> > splitting not in sync with ZK' to 0.94
> >> >     [HBASE-9727] - HBase Rest Server - DELETE scanner operation is a
> >> no-op
> >> >     [HBASE-9731] - updatesBlockedSeconds RegionServer metric should
> not
> >> be
> >> > a histogram
> >> >     [HBASE-9732] - Static AtomicLong updated in StoreFileScanner every
> >> > (re)seek
> >> >     [HBASE-9737] - Corrupt HFile cause resource leak leading to Region
> >> > Server OOM
> >> >     [HBASE-9745] - Append HBASE_CLASSPATH to end of Java classpath and
> >> use
> >> > another env var for prefix
> >> >     [HBASE-9747] - PrefixFilter with OR condition gives wrong results
> >> >     [HBASE-9751] - Excessive readpoints checks in StoreFileScanner
> >> >     [HBASE-9753] - Excessive readpoint checks in MemstoreScanner
> >> >     [HBASE-9783] - o.a.h.h.r.HRegion.mutateRow() with non-existent CF
> >> > cause NPE
> >> >     [HBASE-9789] - Change logging for Coprocessor exec call to trace
> >> >     [HBASE-9807] - block encoder unnecessarily copies the key for each
> >> > reseek
> >> >     [HBASE-9842] - Backport HBASE-9593 and HBASE-8667 to 0.94
> >> >     [HBASE-9847] - HConnectionImplementation does not connect to new
> >> > active master
> >> >     [HBASE-9221] - Provide interface for getting a User in the client
> >> >     [HBASE-9488] - Improve performance for small scan
> >> >     [HBASE-9716] - LoadTestTool should provide default min and max
> >> > settings to the data generator
> >> >     [HBASE-9749] - Custom threadpool for Coprocessor obtained HTables
> >> >     [HBASE-9819] - Backport HBASE-8372 'Provide mutability to
> >> > CompoundConfiguration' to 0.94
> >> >     [HBASE-8553] - improve unit-test coverage of package
> >> > org.apache.hadoop.hbase.mapreduce.hadoopbackport
> >> >     [HBASE-9851] - TestHBaseFsck.testQuarantineMissingHFile is flaky
> >> >     [HBASE-9852] - TestRpcMetrics.testCustomMetrics is flaky
> >> >
> >> > The list of changes is also available here:
> >> >
> >>
> https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12310753&version=12324885
> >> >
> >> > Here're the jenkins runs for this RC:
> >> > https://builds.apache.org/job/HBase-0.94.13/11/ and
> >> > https://builds.apache.org/job/HBase-0.94.13-security/11/
> >> >
> >> > Please try out the RC, check out the doc, take it for a spin, etc, and
> >> > vote +1/-1 by November 5th on whether we should release this as
> 0.94.13.
> >> >
> >> > Thanks.
> >> >
> >> > -- Lars
> >> >
> >
> >
>

Re: [VOTE] The 1st hbase 0.94.13 release candidate is available for download

Posted by Jean-Marc Spaggiari <je...@spaggiari.org>.
I tried multiple times on different servers to run the test suite but it
always fail on those test cases:

Tests in error:

testRegionObserverCompactionTimeStacking(org.apache.hadoop.hbase.coprocessor.TestRegionObserverScannerOpenHook):
Shutting down
  testCalls(org.apache.hadoop.hbase.ipc.TestProtocolExtension): 20000
millis timeout while waiting for channel to be ready for connect. ch :
java.nio.channels.SocketChannel[connection-pending
remote=/0:0:0:0:0:0:0:0:6016]


Does any one have faced the same? I will still deploy it on my test
cluster, but I m wondering the impact.


2013/11/5 Jean-Marc Spaggiari <je...@spaggiari.org>

> Sorry Lars. I will have time on Wednesday, i will run my tests against is
> that day...
>
> JM
> Le 2013-11-05 17:45, "lars hofhansl" <la...@apache.org> a écrit :
>
> Please have a look.
>> I know testing a release is like pulling teeth (I think I said this
>> before).
>>
>> -- Lars
>>
>>
>>
>> ________________________________
>>  From: Ted Yu <yu...@gmail.com>
>> To: "dev@hbase.apache.org" <de...@hbase.apache.org>; lars hofhansl <
>> larsh@apache.org>
>> Sent: Wednesday, October 30, 2013 11:54 AM
>> Subject: Re: [VOTE] The 1st hbase 0.94.13 release candidate is available
>> for download
>>
>>
>> +1
>>
>> - checked the documentation
>> - ran unit test suite with secure profile
>> - built Phoenix with 0.94.13
>>
>>
>>
>> On Wed, Oct 30, 2013 at 10:05 AM, lars hofhansl <la...@apache.org> wrote:
>>
>> > +1
>> >
>> > - checked the tarball
>> > - checked the documentation
>> > - created a table, inserted 100's on millions of KVs, scanned, flushed,
>> > compacted, scanned again
>> > - ran in both local and distributed mode
>> >
>> >
>> > ________________________________
>> > From: lars hofhansl <la...@apache.org>
>> > To: "dev@hbase.apache.org" <de...@hbase.apache.org>; lars hofhansl <
>> > larsh@apache.org>
>> > Sent: Tuesday, October 29, 2013 10:46 PM
>> > Subject: Re: [VOTE] The 1st hbase 0.94.13 release candidate is available
>> > for download
>> >
>> >
>> > In case it was not obvious... I mean the first 0.94.13 RC.
>> >
>> > -- Lars
>> >
>> >
>> >
>> > ________________________________
>> >
>> > From: lars hofhansl <la...@apache.org>
>> > To: hbase-dev <de...@hbase.apache.org>
>> > Sent: Tuesday, October 29, 2013 10:42 PM
>> > Subject: [VOTE] The 1st hbase 0.94.12 release candidate is available for
>> > download
>> >
>> >
>> > The 1st 0.94.12 RC is available for download at
>> > http://people.apache.org/~larsh/hbase-0.94.13-rc0/
>> > Signed with my code signing key: C7CFE328
>> >
>> > As before 0.94.13 is mostly a bug fix release along with a few
>> performance
>> > improvements:
>> >     [HBASE-9711] - Improve HBASE-9428 - avoid copying bytes for
>> > RegexFilter unless necessary
>> >     [HBASE-7600] - TestAdmin.testCreateBadTables is failing occasionally
>> >     [HBASE-8521] - Cells cannot be overwritten with bulk loaded HFiles
>> >     [HBASE-9430] - Memstore heapSize calculation - DEEP_OVERHEAD is
>> > incorrect
>> >     [HBASE-9504] - Backport HBASE-1212 to 0.94
>> >     [HBASE-9548] - Cleanup SnapshotTestingUtils
>> >     [HBASE-9607] - Data loss after snapshot restore into cloned table
>> >     [HBASE-9649] - HFilePrettyPrinter should not throw a NPE if FirstKey
>> > or LastKey is null.
>> >     [HBASE-9651] - Backport HBASE-3890 'Scheduled tasks in distributed
>> log
>> > splitting not in sync with ZK' to 0.94
>> >     [HBASE-9727] - HBase Rest Server - DELETE scanner operation is a
>> no-op
>> >     [HBASE-9731] - updatesBlockedSeconds RegionServer metric should not
>> be
>> > a histogram
>> >     [HBASE-9732] - Static AtomicLong updated in StoreFileScanner every
>> > (re)seek
>> >     [HBASE-9737] - Corrupt HFile cause resource leak leading to Region
>> > Server OOM
>> >     [HBASE-9745] - Append HBASE_CLASSPATH to end of Java classpath and
>> use
>> > another env var for prefix
>> >     [HBASE-9747] - PrefixFilter with OR condition gives wrong results
>> >     [HBASE-9751] - Excessive readpoints checks in StoreFileScanner
>> >     [HBASE-9753] - Excessive readpoint checks in MemstoreScanner
>> >     [HBASE-9783] - o.a.h.h.r.HRegion.mutateRow() with non-existent CF
>> > cause NPE
>> >     [HBASE-9789] - Change logging for Coprocessor exec call to trace
>> >     [HBASE-9807] - block encoder unnecessarily copies the key for each
>> > reseek
>> >     [HBASE-9842] - Backport HBASE-9593 and HBASE-8667 to 0.94
>> >     [HBASE-9847] - HConnectionImplementation does not connect to new
>> > active master
>> >     [HBASE-9221] - Provide interface for getting a User in the client
>> >     [HBASE-9488] - Improve performance for small scan
>> >     [HBASE-9716] - LoadTestTool should provide default min and max
>> > settings to the data generator
>> >     [HBASE-9749] - Custom threadpool for Coprocessor obtained HTables
>> >     [HBASE-9819] - Backport HBASE-8372 'Provide mutability to
>> > CompoundConfiguration' to 0.94
>> >     [HBASE-8553] - improve unit-test coverage of package
>> > org.apache.hadoop.hbase.mapreduce.hadoopbackport
>> >     [HBASE-9851] - TestHBaseFsck.testQuarantineMissingHFile is flaky
>> >     [HBASE-9852] - TestRpcMetrics.testCustomMetrics is flaky
>> >
>> > The list of changes is also available here:
>> >
>> https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12310753&version=12324885
>> >
>> > Here're the jenkins runs for this RC:
>> > https://builds.apache.org/job/HBase-0.94.13/11/ and
>> > https://builds.apache.org/job/HBase-0.94.13-security/11/
>> >
>> > Please try out the RC, check out the doc, take it for a spin, etc, and
>> > vote +1/-1 by November 5th on whether we should release this as 0.94.13.
>> >
>> > Thanks.
>> >
>> > -- Lars
>> >
>
>

Re: [VOTE] The 1st hbase 0.94.13 release candidate is available for download

Posted by Jean-Marc Spaggiari <je...@spaggiari.org>.
Sorry Lars. I will have time on Wednesday, i will run my tests against is
that day...

JM
Le 2013-11-05 17:45, "lars hofhansl" <la...@apache.org> a écrit :

> Please have a look.
> I know testing a release is like pulling teeth (I think I said this
> before).
>
> -- Lars
>
>
>
> ________________________________
>  From: Ted Yu <yu...@gmail.com>
> To: "dev@hbase.apache.org" <de...@hbase.apache.org>; lars hofhansl <
> larsh@apache.org>
> Sent: Wednesday, October 30, 2013 11:54 AM
> Subject: Re: [VOTE] The 1st hbase 0.94.13 release candidate is available
> for download
>
>
> +1
>
> - checked the documentation
> - ran unit test suite with secure profile
> - built Phoenix with 0.94.13
>
>
>
> On Wed, Oct 30, 2013 at 10:05 AM, lars hofhansl <la...@apache.org> wrote:
>
> > +1
> >
> > - checked the tarball
> > - checked the documentation
> > - created a table, inserted 100's on millions of KVs, scanned, flushed,
> > compacted, scanned again
> > - ran in both local and distributed mode
> >
> >
> > ________________________________
> > From: lars hofhansl <la...@apache.org>
> > To: "dev@hbase.apache.org" <de...@hbase.apache.org>; lars hofhansl <
> > larsh@apache.org>
> > Sent: Tuesday, October 29, 2013 10:46 PM
> > Subject: Re: [VOTE] The 1st hbase 0.94.13 release candidate is available
> > for download
> >
> >
> > In case it was not obvious... I mean the first 0.94.13 RC.
> >
> > -- Lars
> >
> >
> >
> > ________________________________
> >
> > From: lars hofhansl <la...@apache.org>
> > To: hbase-dev <de...@hbase.apache.org>
> > Sent: Tuesday, October 29, 2013 10:42 PM
> > Subject: [VOTE] The 1st hbase 0.94.12 release candidate is available for
> > download
> >
> >
> > The 1st 0.94.12 RC is available for download at
> > http://people.apache.org/~larsh/hbase-0.94.13-rc0/
> > Signed with my code signing key: C7CFE328
> >
> > As before 0.94.13 is mostly a bug fix release along with a few
> performance
> > improvements:
> >     [HBASE-9711] - Improve HBASE-9428 - avoid copying bytes for
> > RegexFilter unless necessary
> >     [HBASE-7600] - TestAdmin.testCreateBadTables is failing occasionally
> >     [HBASE-8521] - Cells cannot be overwritten with bulk loaded HFiles
> >     [HBASE-9430] - Memstore heapSize calculation - DEEP_OVERHEAD is
> > incorrect
> >     [HBASE-9504] - Backport HBASE-1212 to 0.94
> >     [HBASE-9548] - Cleanup SnapshotTestingUtils
> >     [HBASE-9607] - Data loss after snapshot restore into cloned table
> >     [HBASE-9649] - HFilePrettyPrinter should not throw a NPE if FirstKey
> > or LastKey is null.
> >     [HBASE-9651] - Backport HBASE-3890 'Scheduled tasks in distributed
> log
> > splitting not in sync with ZK' to 0.94
> >     [HBASE-9727] - HBase Rest Server - DELETE scanner operation is a
> no-op
> >     [HBASE-9731] - updatesBlockedSeconds RegionServer metric should not
> be
> > a histogram
> >     [HBASE-9732] - Static AtomicLong updated in StoreFileScanner every
> > (re)seek
> >     [HBASE-9737] - Corrupt HFile cause resource leak leading to Region
> > Server OOM
> >     [HBASE-9745] - Append HBASE_CLASSPATH to end of Java classpath and
> use
> > another env var for prefix
> >     [HBASE-9747] - PrefixFilter with OR condition gives wrong results
> >     [HBASE-9751] - Excessive readpoints checks in StoreFileScanner
> >     [HBASE-9753] - Excessive readpoint checks in MemstoreScanner
> >     [HBASE-9783] - o.a.h.h.r.HRegion.mutateRow() with non-existent CF
> > cause NPE
> >     [HBASE-9789] - Change logging for Coprocessor exec call to trace
> >     [HBASE-9807] - block encoder unnecessarily copies the key for each
> > reseek
> >     [HBASE-9842] - Backport HBASE-9593 and HBASE-8667 to 0.94
> >     [HBASE-9847] - HConnectionImplementation does not connect to new
> > active master
> >     [HBASE-9221] - Provide interface for getting a User in the client
> >     [HBASE-9488] - Improve performance for small scan
> >     [HBASE-9716] - LoadTestTool should provide default min and max
> > settings to the data generator
> >     [HBASE-9749] - Custom threadpool for Coprocessor obtained HTables
> >     [HBASE-9819] - Backport HBASE-8372 'Provide mutability to
> > CompoundConfiguration' to 0.94
> >     [HBASE-8553] - improve unit-test coverage of package
> > org.apache.hadoop.hbase.mapreduce.hadoopbackport
> >     [HBASE-9851] - TestHBaseFsck.testQuarantineMissingHFile is flaky
> >     [HBASE-9852] - TestRpcMetrics.testCustomMetrics is flaky
> >
> > The list of changes is also available here:
> >
> https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12310753&version=12324885
> >
> > Here're the jenkins runs for this RC:
> > https://builds.apache.org/job/HBase-0.94.13/11/ and
> > https://builds.apache.org/job/HBase-0.94.13-security/11/
> >
> > Please try out the RC, check out the doc, take it for a spin, etc, and
> > vote +1/-1 by November 5th on whether we should release this as 0.94.13.
> >
> > Thanks.
> >
> > -- Lars
> >