[jira] [Created] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Anoop Sam John (JIRA)
Anoop Sam John created HBASE-10412:
--

 Summary: Distributed log replay : Cell tags getting missed
 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker


This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10383) Secure Bulk Load for 'completebulkload' fails for version 0.94.15

2014-01-24 Thread Kashif J S (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10383?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13880811#comment-13880811
 ] 

Kashif J S commented on HBASE-10383:


Yes Lars Hofhansl.
I have tested the patch, by first only modifying only the secure test cases and 
having old SecureBulkLoadProtocol and SecureBulkLoadEndPoint. The secure test 
cases FAIL in that case. So the TCs LG as they detect the failure now unlike 
before.

After applying patch in SecureBulkLoadProtocol and SecureBulkLoadEndPoint, the 
test cases pass Successfully.

Also in my cluster setup with Kerberos, the patch works fine and secure bulk 
load is Fine.
Thanks.

 Secure Bulk Load for 'completebulkload' fails for version 0.94.15
 -

 Key: HBASE-10383
 URL: https://issues.apache.org/jira/browse/HBASE-10383
 Project: HBase
  Issue Type: Bug
  Components: Coprocessors
Affects Versions: 0.94.15
Reporter: Kashif J S
Assignee: Kashif J S
Priority: Critical
 Fix For: 0.94.17

 Attachments: 10383.txt, HBASE-10383-v2.patch, 
 hbase-10383-jyates-0.94-v0.patch


 Secure Bulk Load with kerberos enabled fails for Complete Bulk 
 LoadLoadIncrementalHfile with following exception ERROR 
 org.apache.hadoop.hbase.regionserver.HRegionServer: 
 org.apache.hadoop.hbase.ipc.HBaseRPC$UnknownProtocolException: No matching 
 handler for protocol 
 org.apache.hadoop.hbase.security.access.SecureBulkLoadProtocol in region 
 t1,,1389699438035.28bb0284d971d0676cf562efea80199b.
  at org.apache.hadoop.hbase.regionserver.HRegion.exec(HRegion.java)
  at 
 org.apache.hadoop.hbase.regionserver.HRegionServer.execCoprocessor(HRegionServer.java)
  at sun.reflect.GeneratedMethodAccessor18.invoke(Unknown Source)
  at 
 sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java)
  at java.lang.reflect.Method.invoke(Method.java)
  at 
 org.apache.hadoop.hbase.ipc.SecureRpcEngine$Server.call(SecureRpcEngine.java)
  at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java) 



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Anoop Sam John (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13880817#comment-13880817
 ] 

Anoop Sam John commented on HBASE-10412:


Again this can be solved by setting hbase.client.rpc.codec *at the HRS* to some 
codec which consider tags also.  (KeyValueCodecWithTags or CellCodecWithTags).  
With this way, the issue can be solved by a doc update.

For replication we have introduced a new config to specify a replication 
specific codec name. Might not be good to add a new config to specify a replay 
codec.



 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker

 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread ramkrishna.s.vasudevan (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13880820#comment-13880820
 ] 

ramkrishna.s.vasudevan commented on HBASE-10412:


Distributed log replay should be using WALCellCodec to read and write tags?  
Right?

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker

 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Anoop Sam John (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13880821#comment-13880821
 ] 

Anoop Sam John commented on HBASE-10412:


The HRS which reads WAL uses this codec. But that HRS will be calling 
HRS#replay() for the WALEdits and use codec to ship cells to destination RS. At 
this time the KVCodec is getting used and this strip tags. :(


 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker

 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread ramkrishna.s.vasudevan (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13880823#comment-13880823
 ] 

ramkrishna.s.vasudevan commented on HBASE-10412:


Ok, may be the new way of log replay. got it. So this will need codec change.  
May be can we make this by default WALCellCodec only?  anyway internally 
WALCellcodec and keyvaluecodec are the same execpt that after HBASe-10322 
walcellcodec writes tags too?

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker

 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread ramkrishna.s.vasudevan (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13880825#comment-13880825
 ] 

ramkrishna.s.vasudevan commented on HBASE-10412:


Comments clashed.

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker

 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Anoop Sam John (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13880843#comment-13880843
 ] 

Anoop Sam John commented on HBASE-10412:


bq.May be can we make this by default WALCellCodec only? 
Looks Fine for me.  [~jeffreyz], [~yuzhih...@gmail.com] , [~stack] what do you 
guys think?

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker

 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Updated] (HBASE-7386) Investigate providing some supervisor support for znode deletion

2014-01-24 Thread Samir Ahmic (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-7386?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Samir Ahmic updated HBASE-7386:
---

Attachment: HBASE-7386-conf-v3.patch
HBASE-7386-bin-v3.patch

Here is summary of v3 patches:

* PROCESS_STATE_UNKNOWN removed for zk_cleaner.py, now we clean znode in case 
that process is in state STOPING or EXITED
* mail notifications are moved to separate listener mail_notifications.py and 
disabled by default. I have updated README file with instructions how to enable 
mail notification
* some typos fixed 
* master failover time ~4s, dead rs detection time 3s  

 Investigate providing some supervisor support for znode deletion
 

 Key: HBASE-7386
 URL: https://issues.apache.org/jira/browse/HBASE-7386
 Project: HBase
  Issue Type: Task
  Components: master, regionserver, scripts
Reporter: Gregory Chanan
Assignee: stack
Priority: Blocker
 Attachments: HBASE-7386-bin-v2.patch, HBASE-7386-bin-v3.patch, 
 HBASE-7386-bin.patch, HBASE-7386-conf-v2.patch, HBASE-7386-conf-v3.patch, 
 HBASE-7386-conf.patch, HBASE-7386-src.patch, HBASE-7386-v0.patch, 
 supervisordconfigs-v0.patch


 There a couple of JIRAs for deleting the znode on a process failure:
 HBASE-5844 (RS)
 HBASE-5926 (Master)
 which are pretty neat; on process failure, they delete the znode of the 
 underlying process so HBase can recover faster.
 These JIRAs were implemented via the startup scripts; i.e. the script hangs 
 around and waits for the process to exit, then deletes the znode.
 There are a few problems associated with this approach, as listed in the 
 below JIRAs:
 1) Hides startup output in script
 https://issues.apache.org/jira/browse/HBASE-5844?focusedCommentId=13463401page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13463401
 2) two hbase processes listed per launched daemon
 https://issues.apache.org/jira/browse/HBASE-5844?focusedCommentId=13463409page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13463409
 3) Not run by a real supervisor
 https://issues.apache.org/jira/browse/HBASE-5844?focusedCommentId=13463409page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13463409
 4) Weird output after kill -9 actual process in standalone mode
 https://issues.apache.org/jira/browse/HBASE-5926?focusedCommentId=13506801page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13506801
 5) Can kill existing RS if called again
 https://issues.apache.org/jira/browse/HBASE-5844?focusedCommentId=13463401page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13463401
 6) Hides stdout/stderr[6]
 https://issues.apache.org/jira/browse/HBASE-5844?focusedCommentId=13506832page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13506832
 I suspect running in via something like supervisor.d can solve these issues 
 if we provide the right support.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10383) Secure Bulk Load for 'completebulkload' fails for version 0.94.15

2014-01-24 Thread rajeshbabu (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10383?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13880856#comment-13880856
 ] 

rajeshbabu commented on HBASE-10383:


Thanks for testing [~kashi...@huawei.com]. The patch working fine with 
kerberos. 
+1

 Secure Bulk Load for 'completebulkload' fails for version 0.94.15
 -

 Key: HBASE-10383
 URL: https://issues.apache.org/jira/browse/HBASE-10383
 Project: HBase
  Issue Type: Bug
  Components: Coprocessors
Affects Versions: 0.94.15
Reporter: Kashif J S
Assignee: Kashif J S
Priority: Critical
 Fix For: 0.94.17

 Attachments: 10383.txt, HBASE-10383-v2.patch, 
 hbase-10383-jyates-0.94-v0.patch


 Secure Bulk Load with kerberos enabled fails for Complete Bulk 
 LoadLoadIncrementalHfile with following exception ERROR 
 org.apache.hadoop.hbase.regionserver.HRegionServer: 
 org.apache.hadoop.hbase.ipc.HBaseRPC$UnknownProtocolException: No matching 
 handler for protocol 
 org.apache.hadoop.hbase.security.access.SecureBulkLoadProtocol in region 
 t1,,1389699438035.28bb0284d971d0676cf562efea80199b.
  at org.apache.hadoop.hbase.regionserver.HRegion.exec(HRegion.java)
  at 
 org.apache.hadoop.hbase.regionserver.HRegionServer.execCoprocessor(HRegionServer.java)
  at sun.reflect.GeneratedMethodAccessor18.invoke(Unknown Source)
  at 
 sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java)
  at java.lang.reflect.Method.invoke(Method.java)
  at 
 org.apache.hadoop.hbase.ipc.SecureRpcEngine$Server.call(SecureRpcEngine.java)
  at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java) 



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Anoop Sam John (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13880884#comment-13880884
 ] 

Anoop Sam John commented on HBASE-10412:


We can set the Codec, to be used by WAL, using hbase.regionserver.wal.codec
Phoenix is having custom WALCellCodec implementation for index. For the 
distributed log replay from one RS to another RS, the same custom WALCellCodec 
should get used?  
Ping [~jyates], [~lhofhansl]

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker

 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Comment Edited] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Anoop Sam John (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13880884#comment-13880884
 ] 

Anoop Sam John edited comment on HBASE-10412 at 1/24/14 10:47 AM:
--

We can set the Codec, to be used by WAL, using hbase.regionserver.wal.codec
Phoenix is having custom WALCellCodec implementation for index. For the 
distributed log replay from one RS to another RS, the same custom WALCellCodec 
should get used?  
Ping [~jesse_yates], [~lhofhansl]


was (Author: anoop.hbase):
We can set the Codec, to be used by WAL, using hbase.regionserver.wal.codec
Phoenix is having custom WALCellCodec implementation for index. For the 
distributed log replay from one RS to another RS, the same custom WALCellCodec 
should get used?  
Ping [~jyates], [~lhofhansl]

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker

 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Comment Edited] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Anoop Sam John (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13880884#comment-13880884
 ] 

Anoop Sam John edited comment on HBASE-10412 at 1/24/14 11:12 AM:
--

We can set the Codec, to be used by WAL, using hbase.regionserver.wal.codec
Phoenix is having custom WALCellCodec implementation for index.(WALEditCodec it 
is as we have that name in 0.94) For the distributed log replay from one RS to 
another RS, the same custom WALCellCodec should get used?  
Ping [~jesse_yates], [~lhofhansl]


was (Author: anoop.hbase):
We can set the Codec, to be used by WAL, using hbase.regionserver.wal.codec
Phoenix is having custom WALCellCodec implementation for index. For the 
distributed log replay from one RS to another RS, the same custom WALCellCodec 
should get used?  
Ping [~jesse_yates], [~lhofhansl]

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker

 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Created] (HBASE-10413) TableSplits are not sorted by size.

2014-01-24 Thread Lukas Nalezenec (JIRA)
Lukas Nalezenec created HBASE-10413:
---

 Summary: TableSplits are not sorted by size.
 Key: HBASE-10413
 URL: https://issues.apache.org/jira/browse/HBASE-10413
 Project: HBase
  Issue Type: Bug
  Components: Client
Affects Versions: 0.96.1.1
Reporter: Lukas Nalezenec


We had serious issue in our production today.

InputSplits should be sorted by length but TableSplit does not contain real 
getLength implementation:

  @Override
  public long getLength() {
// Not clear how to obtain this... seems to be used only for sorting splits
return 0;
  }

Can we implement this method ? 
What is the best way ?



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10081) Since HBASE-7091, HBASE_OPTS cannot be set on the command line

2014-01-24 Thread Nicolas Liochon (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10081?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881024#comment-13881024
 ] 

Nicolas Liochon commented on HBASE-10081:
-

Hum. Doesn't that mean that the first time it will erase the value set in the 
shell, but not the next time?
If yes, would it make sense to do something to have something like 
{code}
// pseudo code
HBASE_EXTRA_OPTS=-XX:+UseConcMarkSweepGC
if (!HBASE_OPTS.contains(HBASE_EXTRA_OPTS)) HBASE_OPTS  = HBASE_EXTRA_OPTS + 
HBASE_OPTS
{code}

 Since HBASE-7091, HBASE_OPTS cannot be set on the command line
 --

 Key: HBASE-10081
 URL: https://issues.apache.org/jira/browse/HBASE-10081
 Project: HBase
  Issue Type: Bug
  Components: scripts
Affects Versions: 0.98.0, 0.96.0
Reporter: Nicolas Liochon
Assignee: Liu Shaohui
Priority: Minor
 Attachments: HBASE-10081-v1.diff


 Discussed in HBASE-7091.
 It's not critical, but a little bit surprising, as the comments in bin/hbase 
 doesn't say anything about this. If you create your own hbase-env then it's 
 not an issue...



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10413) TableSplits are not sorted by size.

2014-01-24 Thread Jean-Marc Spaggiari (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881103#comment-13881103
 ] 

Jean-Marc Spaggiari commented on HBASE-10413:
-

Hum. Interesting. I will bring that to dev list...

 TableSplits are not sorted by size.
 ---

 Key: HBASE-10413
 URL: https://issues.apache.org/jira/browse/HBASE-10413
 Project: HBase
  Issue Type: Bug
  Components: Client
Affects Versions: 0.96.1.1
Reporter: Lukas Nalezenec

 We had serious issue in our production today.
 InputSplits should be sorted by length but TableSplit does not contain real 
 getLength implementation:
   @Override
   public long getLength() {
 // Not clear how to obtain this... seems to be used only for sorting 
 splits
 return 0;
   }
 Can we implement this method ? 
 What is the best way ?



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Andrew Purtell (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881129#comment-13881129
 ] 

Andrew Purtell commented on HBASE-10412:


Also, we should have a unit test to cover this. 

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker

 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Andrew Purtell (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881128#comment-13881128
 ] 

Andrew Purtell commented on HBASE-10412:


We should fix this so we are not using a RPC codec for handling a WAL. That is 
surprising and wrong. 

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker

 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Comment Edited] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Andrew Purtell (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881128#comment-13881128
 ] 

Andrew Purtell edited comment on HBASE-10412 at 1/24/14 4:59 PM:
-

We should fix this so we are not using a RPC codec for handling a WAL. That is 
surprising and wrong. 

bq. We can set the Codec, to be used by WAL, using hbase.regionserver.wal.codec
Phoenix is having custom WALCellCodec implementation for index.(WALEditCodec it 
is as we have that name in 0.94) For the distributed log replay from one RS to 
another RS, the same custom WALCellCodec should get used

+1, something like this


was (Author: apurtell):
We should fix this so we are not using a RPC codec for handling a WAL. That is 
surprising and wrong. 

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker

 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread ramkrishna.s.vasudevan (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881148#comment-13881148
 ] 

ramkrishna.s.vasudevan commented on HBASE-10412:


We planned to enable distributed log splitting for the existing 
TestvisibilityLabels itself.  That would ensure that it works at the same time 
we could solve the issue by using the wal.codec config.  Ideally for all WAL 
related codecs including writing to HLog we could use this config.  Using the 
same codec for all wal related operations ideally seems right to me.

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker

 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Andrew Purtell (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881151#comment-13881151
 ] 

Andrew Purtell commented on HBASE-10412:


bq. We planned to enable distributed log splitting for the existing 
TestvisibilityLabels itself. That would ensure that it works at the same time 
we could solve the issue by using the wal.codec config. Ideally for all WAL 
related codecs including writing to HLog we could use this config. Using the 
same codec for all wal related operations ideally seems right to me.

+1

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker

 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Updated] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Anoop Sam John (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Anoop Sam John updated HBASE-10412:
---

Fix Version/s: 0.99.0
   0.98.0
   Status: Patch Available  (was: Open)

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker
 Fix For: 0.98.0, 0.99.0

 Attachments: HBASE-10412.patch


 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Updated] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Anoop Sam John (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Anoop Sam John updated HBASE-10412:
---

Attachment: HBASE-10412.patch

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker
 Fix For: 0.98.0, 0.99.0

 Attachments: HBASE-10412.patch


 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Andrew Purtell (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881210#comment-13881210
 ] 

Andrew Purtell commented on HBASE-10412:


Patch looks good to me.

On this part:
{noformat}
@@ -169,22 +168,25 @@ public class HLogSplitter {
 
   HLogSplitter(Configuration conf, Path rootDir,
   FileSystem fs, LastSequenceId idChecker, ZooKeeperWatcher zkw) {
-this.conf = conf;
+this.conf = HBaseConfiguration.create(conf);
+String codecClassName = conf
+.get(WALCellCodec.WAL_CELL_CODEC_CLASS_KEY, 
WALCellCodec.class.getName());
+this.conf.set(HConstants.RPC_CODEC_CONF_KEY, codecClassName);
{noformat}

Here we are telling HLogSplitter to use a WAL codec for RPC. Makes sense but 
could be a future source of surprise. We should update the javadoc of 
WALCellCodec to let people know this is used not only server side but also for 
sending edits as part of the distributed splitting process.

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker
 Fix For: 0.98.0, 0.99.0

 Attachments: HBASE-10412.patch


 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-6048) Table Scan is failing if offheap cache enabled

2014-01-24 Thread Nick Dimiduk (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-6048?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881209#comment-13881209
 ] 

Nick Dimiduk commented on HBASE-6048:
-

FYI, I'm *not* seeing this issue with a build of 0.96.1.1/Hadoop2. Testing 
using PerfEval scan tests.

 Table Scan is failing if offheap cache enabled
 --

 Key: HBASE-6048
 URL: https://issues.apache.org/jira/browse/HBASE-6048
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.94.0
Reporter: Gopinathan A
Assignee: ramkrishna.s.vasudevan
 Attachments: hbase-6048-94.patch


 Table Scan is failing if offheap cache enabled.
 {noformat}
 2012-05-18 20:03:38,446 DEBUG org.apache.hadoop.hbase.io.hfile.HFileWriterV2: 
 Initialized with CacheConfig:enabled [cacheDataOnRead=true] 
 [cacheDataOnWrite=false] [cacheIndexesOnWrite=false] 
 [cacheBloomsOnWrite=false] [cacheEvictOnClose=false] [cacheCompressed=false]
 2012-05-18 20:03:38,446 INFO org.apache.hadoop.hbase.regionserver.StoreFile: 
 Delete Family Bloom filter type for 
 hdfs://10.18.40.217:9000/hbase/ufdr/1d4656fd417a07c9171a38b8f4d08510/.tmp/03742024b28f443bb63cfc338d4ca422:
  CompoundBloomFilterWriter
 2012-05-18 20:04:25,576 DEBUG org.apache.hadoop.hbase.io.hfile.LruBlockCache: 
 Block cache LRU eviction started; Attempting to free 120.57 MB of 
 total=1020.57 MB
 2012-05-18 20:04:25,655 DEBUG org.apache.hadoop.hbase.io.hfile.LruBlockCache: 
 Block cache LRU eviction completed; freed=120.82 MB, total=907.89 MB, 
 single=1012.11 MB, multi=6.12 MB, memory=0 KB
 2012-05-18 20:04:25,733 ERROR 
 org.apache.hadoop.hbase.regionserver.HRegionServer: Failed openScanner
 java.lang.IllegalStateException: Schema metrics requested before table/CF 
 name initialization: {tableName:null,cfName:null}
   at 
 org.apache.hadoop.hbase.regionserver.metrics.SchemaConfigured.getSchemaMetrics(SchemaConfigured.java:182)
   at 
 org.apache.hadoop.hbase.io.hfile.LruBlockCache.updateSizeMetrics(LruBlockCache.java:310)
   at 
 org.apache.hadoop.hbase.io.hfile.LruBlockCache.cacheBlock(LruBlockCache.java:274)
   at 
 org.apache.hadoop.hbase.io.hfile.LruBlockCache.cacheBlock(LruBlockCache.java:293)
   at 
 org.apache.hadoop.hbase.io.hfile.DoubleBlockCache.getBlock(DoubleBlockCache.java:102)
   at 
 org.apache.hadoop.hbase.io.hfile.HFileReaderV2.readBlock(HFileReaderV2.java:296)
   at 
 org.apache.hadoop.hbase.io.hfile.HFileBlockIndex$BlockIndexReader.seekToDataBlock(HFileBlockIndex.java:213)
   at 
 org.apache.hadoop.hbase.io.hfile.HFileReaderV2$AbstractScannerV2.seekTo(HFileReaderV2.java:455)
   at 
 org.apache.hadoop.hbase.io.hfile.HFileReaderV2$AbstractScannerV2.seekTo(HFileReaderV2.java:475)
   at 
 org.apache.hadoop.hbase.regionserver.StoreFileScanner.seekAtOrAfter(StoreFileScanner.java:226)
   at 
 org.apache.hadoop.hbase.regionserver.StoreFileScanner.seek(StoreFileScanner.java:145)
   at 
 org.apache.hadoop.hbase.regionserver.StoreScanner.init(StoreScanner.java:130)
   at 
 org.apache.hadoop.hbase.regionserver.Store.getScanner(Store.java:2001)
   at 
 org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.init(HRegion.java:3274)
   at 
 org.apache.hadoop.hbase.regionserver.HRegion.instantiateRegionScanner(HRegion.java:1604)
   at 
 org.apache.hadoop.hbase.regionserver.HRegion.getScanner(HRegion.java:1596)
   at 
 org.apache.hadoop.hbase.regionserver.HRegion.getScanner(HRegion.java:1572)
   at 
 org.apache.hadoop.hbase.regionserver.HRegionServer.openScanner(HRegionServer.java:2310)
   at sun.reflect.GeneratedMethodAccessor45.invoke(Unknown Source)
   at 
 sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
   at java.lang.reflect.Method.invoke(Method.java:597)
   at 
 org.apache.hadoop.hbase.ipc.WritableRpcEngine$Server.call(WritableRpcEngine.java:364)
   at 
 org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1376)
 2012-05-18 20:04:25,828 ERROR 
 org.apache.hadoop.hbase.regionserver.HRegionServer: Failed openScanner
 {noformat}



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Updated] (HBASE-10408) Intermittent TestDistributedLogSplitting#testLogReplayForDisablingTable failure

2014-01-24 Thread Ted Yu (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-10408?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ted Yu updated HBASE-10408:
---

Attachment: 10408-v2.txt

Thanks for the reminder, Jeff.

Here is patch v2 that filters out system table.

TestDistributedLogSplitting passed locally.

 Intermittent TestDistributedLogSplitting#testLogReplayForDisablingTable 
 failure
 ---

 Key: HBASE-10408
 URL: https://issues.apache.org/jira/browse/HBASE-10408
 Project: HBase
  Issue Type: Test
Reporter: Ted Yu
Assignee: Ted Yu
 Attachments: 10408-v1.txt, 10408-v2.txt, 
 TestDistributedLogSplitting.out, 
 testDistributedLogSplitting-disablingTable.html


 Here was the assertion failure:
 {code} java.lang.AssertionError: expected:lt;1000 but was:lt;0 at 
 org.junit.Assert.fail(Assert.java:88) at 
 org.junit.Assert.failNotEquals(Assert.java:743) at 
 org.junit.Assert.assertEquals(Assert.java:118) at 
 org.junit.Assert.assertEquals(Assert.java:555) at 
 org.junit.Assert.assertEquals(Assert.java:542) at 
 org.apache.hadoop.hbase.master.TestDistributedLogSplitting.testLogReplayForDisablingTable(TestDistributedLogSplitting.java:838)
  
 {code}
 This was due to the loop starting around line 823 not picking up any edits 



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10408) Intermittent TestDistributedLogSplitting#testLogReplayForDisablingTable failure

2014-01-24 Thread Andrew Purtell (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10408?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881222#comment-13881222
 ] 

Andrew Purtell commented on HBASE-10408:


Ted, can you explain how the test is now fixed?

AFAIK, this goto isn't necessary:
{code}
--- 
hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
  (revision 1561083)
+++ 
hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
  (working copy)
@@ -739,16 +739,15 @@
 HRegionServer hrs = null;
 boolean hasRegionsForBothTables = false;
 String tableName = null;
+OUTER:
 for (int i = 0; i  NUM_RS; i++) {
   tableName = null;
   hasRegionsForBothTables = false;
-  boolean isCarryingMeta = false;
   hrs = rsts.get(i).getRegionServer();
   regions = ProtobufUtil.getOnlineRegions(hrs);
   for (HRegionInfo region : regions) {
-if (region.isMetaRegion()) {
-  isCarryingMeta = true;
-  break;
+if (region.getTable().isSystemTable()) {
+  continue OUTER;
 }
 if (tableName != null 
 !tableName.equalsIgnoreCase(region.getTable().getNameAsString())) {
{code}

 Intermittent TestDistributedLogSplitting#testLogReplayForDisablingTable 
 failure
 ---

 Key: HBASE-10408
 URL: https://issues.apache.org/jira/browse/HBASE-10408
 Project: HBase
  Issue Type: Test
Reporter: Ted Yu
Assignee: Ted Yu
 Attachments: 10408-v1.txt, 10408-v2.txt, 
 TestDistributedLogSplitting.out, 
 testDistributedLogSplitting-disablingTable.html


 Here was the assertion failure:
 {code} java.lang.AssertionError: expected:lt;1000 but was:lt;0 at 
 org.junit.Assert.fail(Assert.java:88) at 
 org.junit.Assert.failNotEquals(Assert.java:743) at 
 org.junit.Assert.assertEquals(Assert.java:118) at 
 org.junit.Assert.assertEquals(Assert.java:555) at 
 org.junit.Assert.assertEquals(Assert.java:542) at 
 org.apache.hadoop.hbase.master.TestDistributedLogSplitting.testLogReplayForDisablingTable(TestDistributedLogSplitting.java:838)
  
 {code}
 This was due to the loop starting around line 823 not picking up any edits 



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Ted Yu (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881224#comment-13881224
 ] 

Ted Yu commented on HBASE-10412:


{code}
+conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
{code}
Should the test be parameterized so that both log splitting and log replay are 
covered ?

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker
 Fix For: 0.98.0, 0.99.0

 Attachments: HBASE-10412.patch


 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10408) Intermittent TestDistributedLogSplitting#testLogReplayForDisablingTable failure

2014-01-24 Thread stack (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10408?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881223#comment-13881223
 ] 

stack commented on HBASE-10408:
---

[~ted_yu] In future, when you put up a patch, can you explain how it fixes the 
issue?

 Intermittent TestDistributedLogSplitting#testLogReplayForDisablingTable 
 failure
 ---

 Key: HBASE-10408
 URL: https://issues.apache.org/jira/browse/HBASE-10408
 Project: HBase
  Issue Type: Test
Reporter: Ted Yu
Assignee: Ted Yu
 Attachments: 10408-v1.txt, 10408-v2.txt, 
 TestDistributedLogSplitting.out, 
 testDistributedLogSplitting-disablingTable.html


 Here was the assertion failure:
 {code} java.lang.AssertionError: expected:lt;1000 but was:lt;0 at 
 org.junit.Assert.fail(Assert.java:88) at 
 org.junit.Assert.failNotEquals(Assert.java:743) at 
 org.junit.Assert.assertEquals(Assert.java:118) at 
 org.junit.Assert.assertEquals(Assert.java:555) at 
 org.junit.Assert.assertEquals(Assert.java:542) at 
 org.apache.hadoop.hbase.master.TestDistributedLogSplitting.testLogReplayForDisablingTable(TestDistributedLogSplitting.java:838)
  
 {code}
 This was due to the loop starting around line 823 not picking up any edits 



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10408) Intermittent TestDistributedLogSplitting#testLogReplayForDisablingTable failure

2014-01-24 Thread Ted Yu (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10408?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881230#comment-13881230
 ] 

Ted Yu commented on HBASE-10408:


The failure shown in the attached test output was caused by 
hasRegionsForBothTables being set to true prematurely.
It happened when system table was hosted on the underlying region server but 
disableTable was not. This was not covered by region.isMetaRegion() check.
Patch v2 covers the above scenario.

 Intermittent TestDistributedLogSplitting#testLogReplayForDisablingTable 
 failure
 ---

 Key: HBASE-10408
 URL: https://issues.apache.org/jira/browse/HBASE-10408
 Project: HBase
  Issue Type: Test
Reporter: Ted Yu
Assignee: Ted Yu
 Attachments: 10408-v1.txt, 10408-v2.txt, 
 TestDistributedLogSplitting.out, 
 testDistributedLogSplitting-disablingTable.html


 Here was the assertion failure:
 {code} java.lang.AssertionError: expected:lt;1000 but was:lt;0 at 
 org.junit.Assert.fail(Assert.java:88) at 
 org.junit.Assert.failNotEquals(Assert.java:743) at 
 org.junit.Assert.assertEquals(Assert.java:118) at 
 org.junit.Assert.assertEquals(Assert.java:555) at 
 org.junit.Assert.assertEquals(Assert.java:542) at 
 org.apache.hadoop.hbase.master.TestDistributedLogSplitting.testLogReplayForDisablingTable(TestDistributedLogSplitting.java:838)
  
 {code}
 This was due to the loop starting around line 823 not picking up any edits 



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10408) Intermittent TestDistributedLogSplitting#testLogReplayForDisablingTable failure

2014-01-24 Thread Ted Yu (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10408?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881235#comment-13881235
 ] 

Ted Yu commented on HBASE-10408:


bq.  this goto isn't necessary

Pardon me. There is no goto with patch v2.

 Intermittent TestDistributedLogSplitting#testLogReplayForDisablingTable 
 failure
 ---

 Key: HBASE-10408
 URL: https://issues.apache.org/jira/browse/HBASE-10408
 Project: HBase
  Issue Type: Test
Reporter: Ted Yu
Assignee: Ted Yu
 Attachments: 10408-v1.txt, 10408-v2.txt, 
 TestDistributedLogSplitting.out, 
 testDistributedLogSplitting-disablingTable.html


 Here was the assertion failure:
 {code} java.lang.AssertionError: expected:lt;1000 but was:lt;0 at 
 org.junit.Assert.fail(Assert.java:88) at 
 org.junit.Assert.failNotEquals(Assert.java:743) at 
 org.junit.Assert.assertEquals(Assert.java:118) at 
 org.junit.Assert.assertEquals(Assert.java:555) at 
 org.junit.Assert.assertEquals(Assert.java:542) at 
 org.apache.hadoop.hbase.master.TestDistributedLogSplitting.testLogReplayForDisablingTable(TestDistributedLogSplitting.java:838)
  
 {code}
 This was due to the loop starting around line 823 not picking up any edits 



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Jesse Yates (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881245#comment-13881245
 ] 

Jesse Yates commented on HBASE-10412:
-

As Andy said:
bq. We should fix this so we are not using a RPC codec for handling a WAL. That 
is surprising and wrong.

Should be ok for now to just update javadoc, but I'm worried we'll start 
rolling in custom behavior for RPC that makes it very complicated.

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker
 Fix For: 0.98.0, 0.99.0

 Attachments: HBASE-10412.patch


 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Comment Edited] (HBASE-10081) Since HBASE-7091, HBASE_OPTS cannot be set on the command line

2014-01-24 Thread Jesse Yates (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10081?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13880499#comment-13880499
 ] 

Jesse Yates edited comment on HBASE-10081 at 1/24/14 6:22 PM:
--

Ok, guess others are using it :) Looking through hbase-config.sh there are 
these lines:
{code}
# Source the hbase-env.sh.  Will have JAVA_HOME defined.
# HBASE-7817 - Source the hbase-env.sh only if it has not already been done. 
HBASE_ENV_INIT keeps track of it.
if [ -z $HBASE_ENV_INIT ]  [ -f ${HBASE_CONF_DIR}/hbase-env.sh ]; then
  . ${HBASE_CONF_DIR}/hbase-env.sh
  export HBASE_ENV_INIT=true
fi
{code}

Which should ensure it doesn't get sourced multiple times - my concern for 
HBASE-7091. Should be fine... you okay with it [~nkeywal]?


was (Author: jesse_yates):
Ok, guess others are using it :) Looking through hbase-config.sh there are 
these lines:
{quote}
# Source the hbase-env.sh.  Will have JAVA_HOME defined.
# HBASE-7817 - Source the hbase-env.sh only if it has not already been done. 
HBASE_ENV_INIT keeps track of it.
if [ -z $HBASE_ENV_INIT ]  [ -f ${HBASE_CONF_DIR}/hbase-env.sh ]; then
  . ${HBASE_CONF_DIR}/hbase-env.sh
  export HBASE_ENV_INIT=true
fi
{quote}

Which should ensure it doesn't get sourced multiple times - my concern for 
HBASE-7091. Should be fine... you okay with it [~nkeywal]?

 Since HBASE-7091, HBASE_OPTS cannot be set on the command line
 --

 Key: HBASE-10081
 URL: https://issues.apache.org/jira/browse/HBASE-10081
 Project: HBase
  Issue Type: Bug
  Components: scripts
Affects Versions: 0.98.0, 0.96.0
Reporter: Nicolas Liochon
Assignee: Liu Shaohui
Priority: Minor
 Attachments: HBASE-10081-v1.diff


 Discussed in HBASE-7091.
 It's not critical, but a little bit surprising, as the comments in bin/hbase 
 doesn't say anything about this. If you create your own hbase-env then it's 
 not an issue...



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10403) Simplify offheap cache configuration

2014-01-24 Thread Nick Dimiduk (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10403?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881251#comment-13881251
 ] 

Nick Dimiduk commented on HBASE-10403:
--

Thanks for the comments [~zjushch]! I'll include them in the next patch. This 
was just a WIP I wanted to share to see if you liked the direct. So you like 
disentangling BlockCache instantiation into the static factory methods? Any 
complaint about the reflection business? Is there need for a more concrete 
factory type class, etc?

Especially now in light of HBASE-6048 resurfacing, I think it's a good idea to 
decouple the multilevel cache strategies from the implementations they wrap. 
This will be more invasive, but it would allow a user to use CombinedBlockCache 
with SlabCache via configuration.

What do you think about the idea of named configurations? IE, including some 
preset configuration by the name BucketCache offheap. Is that too much?

 Simplify offheap cache configuration
 

 Key: HBASE-10403
 URL: https://issues.apache.org/jira/browse/HBASE-10403
 Project: HBase
  Issue Type: Bug
  Components: io
Reporter: Nick Dimiduk
Assignee: Nick Dimiduk
Priority: Minor
 Attachments: HBASE-10403.0.patch


 The BucketCache (HBASE-7404) is a very nice piece of functionality which is 
 hidden behind complex configuration. Enabling it currently requires manual 
 calculation of L1 cache. It'd be nice to make this easier to use and conform 
 better with the existing heap management tools we already have.
 Turning it on currently requires explicitly setting LruBlockCache (L1) 
 instance size and IOEngine (L2) size, making sure that L1 size isn't too big 
 vs global memstore and total heap. This is further confused by 
 hbase.bucketcache.size accepting a percentage of total heap OR explicit size 
 in MB. Enabling SlabCache is slightly easier in that it just accepts whatever 
 LruBlockCache is provided.
 Turning on BucketCache using off-heap mode could look like:
 hbase-env.sh:
  Set HBASE_REGIONSERVER_OPTS:
   -Xmx5000m
   -XX:MaxDirectMemorySize=15000m
 hbase-site.xml:
  - hbase.regionserver.global.memstore.size = 0.7
  - hbase.regionserver.onheap.blockcache.size = 0.1
  - hbase.regionserver.blockcache.impl = BucketCache
  - hbase.bucketcache.ioengine = offheap
 The result being a CombinedCache instance with 500m LruBlockCache + 15000m 
 ByteBufferIOEngine running in direct mode.
 This example does a couple things (mostly for the admin):
  - knows NOT to enable SlabCache
  - s/hfile.block.cache.size/hbase.regionserver.onheap.blockcache.size/
  - maintains the validity of HBaseConfiguration's existing check that global 
 MemStore + LruBlockCache == 0.8
  - maps BucketCache into meaning a CombinedCache instance with these 
 implementations for L1 and L2.
  - Figures out appropriate values for hbase.bucketcache.size and 
 hbase.bucketcache.percentage.in.combinedcache



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Andrew Purtell (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881259#comment-13881259
 ] 

Andrew Purtell commented on HBASE-10412:


{quote}
As Andy said:
bq. We should fix this so we are not using a RPC codec for handling a WAL. That 
is surprising and wrong.

Should be ok for now to just update javadoc, but I'm worried we'll start 
rolling in custom behavior for RPC that makes it very complicated.
{quote}

I agree. 

Looks like we are splitting this into two parts.

1. Get distributed log replay working with tags using a WALCodec. (Now)

2. Redo distributed splitting a bit so it doesn't use a bare codec but uses its 
own. Maybe a RPC codec as static inner class that embeds a WAL codec? (Later) 

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker
 Fix For: 0.98.0, 0.99.0

 Attachments: HBASE-10412.patch


 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10081) Since HBASE-7091, HBASE_OPTS cannot be set on the command line

2014-01-24 Thread Jesse Yates (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10081?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881263#comment-13881263
 ] 

Jesse Yates commented on HBASE-10081:
-

It wouldn't erase it in the shell, I _think_, but rather just update it for the 
the scripts and all child processes. So, the calling shell would still have the 
variable you set.

As an example, say I have (checked locally on my Mac):
{code}
bash$ export HBASE_OPTS=-XX:+UseConcMarkSweepGC
{code}
and in my hbase-env.sh have
{code}
HBASE_OPTS=$HBASE_OPTS -XX:CustomThingy
{code}

then after running bin/start-hbase.sh, I would still just see
{code}
bash$ echo $HBASE_OPTS
-XX:+UseConcMarkSweepGC
{code}

Should be fine, yes?

 Since HBASE-7091, HBASE_OPTS cannot be set on the command line
 --

 Key: HBASE-10081
 URL: https://issues.apache.org/jira/browse/HBASE-10081
 Project: HBase
  Issue Type: Bug
  Components: scripts
Affects Versions: 0.98.0, 0.96.0
Reporter: Nicolas Liochon
Assignee: Liu Shaohui
Priority: Minor
 Attachments: HBASE-10081-v1.diff


 Discussed in HBASE-7091.
 It's not critical, but a little bit surprising, as the comments in bin/hbase 
 doesn't say anything about this. If you create your own hbase-env then it's 
 not an issue...



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10392) Correct references to hbase.regionserver.global.memstore.upperLimit

2014-01-24 Thread Nick Dimiduk (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881260#comment-13881260
 ] 

Nick Dimiduk commented on HBASE-10392:
--

I take lack of further commentary as lack of interest and acceptance of patch 
:) Will commit later today.

 Correct references to hbase.regionserver.global.memstore.upperLimit
 ---

 Key: HBASE-10392
 URL: https://issues.apache.org/jira/browse/HBASE-10392
 Project: HBase
  Issue Type: Bug
Reporter: Nick Dimiduk
Assignee: Nick Dimiduk
 Fix For: 0.99.0

 Attachments: HBASE-10392.0.patch, HBASE-10392.1.patch, 
 HBASE-10392.2.patch


 As part of the awesome new HBASE-5349, a couple references to 
 {{hbase.regionserver.global.memstore.upperLimit}} was missed. Clean those up 
 to use the new {{hbase.regionserver.global.memstore.size}} instead.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Anoop Sam John (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881265#comment-13881265
 ] 

Anoop Sam John commented on HBASE-10412:


[~jesse_yates] in case of Phoenix, I think you will need the custom 
WALCellCodec to do the RS-RS write.  Then only it will come as IndexedKeyValue 
object right? Pls correct me if I am wrong.

Yes I will update the javadoc of WALCellCodec

[~yuzhih...@gmail.com] Need to set 
conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, distributedLogReplay);  
in setupBeforeClass for starting the cluster. So I can not go with 
parameterized way.

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker
 Fix For: 0.98.0, 0.99.0

 Attachments: HBASE-10412.patch


 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Comment Edited] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Andrew Purtell (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881259#comment-13881259
 ] 

Andrew Purtell edited comment on HBASE-10412 at 1/24/14 6:31 PM:
-

{quote}
As Andy said:
bq. We should fix this so we are not using a RPC codec for handling a WAL. That 
is surprising and wrong.

Should be ok for now to just update javadoc, but I'm worried we'll start 
rolling in custom behavior for RPC that makes it very complicated.
{quote}

I agree. 

Looks like we are splitting this into two parts.

1. Get distributed log replay working with tags using a WALCodec. (Now)

2. Redo distributed replay a bit so it doesn't use a bare codec but uses its 
own. Maybe a RPC codec as static inner class that embeds a WAL codec? (Later) 


was (Author: apurtell):
{quote}
As Andy said:
bq. We should fix this so we are not using a RPC codec for handling a WAL. That 
is surprising and wrong.

Should be ok for now to just update javadoc, but I'm worried we'll start 
rolling in custom behavior for RPC that makes it very complicated.
{quote}

I agree. 

Looks like we are splitting this into two parts.

1. Get distributed log replay working with tags using a WALCodec. (Now)

2. Redo distributed splitting a bit so it doesn't use a bare codec but uses its 
own. Maybe a RPC codec as static inner class that embeds a WAL codec? (Later) 

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker
 Fix For: 0.98.0, 0.99.0

 Attachments: HBASE-10412.patch


 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Jesse Yates (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881270#comment-13881270
 ] 

Jesse Yates commented on HBASE-10412:
-

{quote}
Jesse Yates in case of Phoenix, I think you will need the custom WALCellCodec 
to do the RS-RS write. Then only it will come as IndexedKeyValue object right? 
Pls correct me if I am wrong.
{quote}

No, it needs the custom WALCellCodec to read custom kvs from the WAL. The KVs 
can then be written normally as regular updates RS-Rs

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker
 Fix For: 0.98.0, 0.99.0

 Attachments: HBASE-10412.patch


 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10408) Intermittent TestDistributedLogSplitting#testLogReplayForDisablingTable failure

2014-01-24 Thread Andrew Purtell (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10408?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881272#comment-13881272
 ] 

Andrew Purtell commented on HBASE-10408:


bq. Pardon me. There is no goto with patch v2.

Note the scare quotes.

What do you call this? 
{noformat}
continue OUTER;
{noformat}

I really don't see how my comment wasn't obvious.

 Intermittent TestDistributedLogSplitting#testLogReplayForDisablingTable 
 failure
 ---

 Key: HBASE-10408
 URL: https://issues.apache.org/jira/browse/HBASE-10408
 Project: HBase
  Issue Type: Test
Reporter: Ted Yu
Assignee: Ted Yu
 Attachments: 10408-v1.txt, 10408-v2.txt, 
 TestDistributedLogSplitting.out, 
 testDistributedLogSplitting-disablingTable.html


 Here was the assertion failure:
 {code} java.lang.AssertionError: expected:lt;1000 but was:lt;0 at 
 org.junit.Assert.fail(Assert.java:88) at 
 org.junit.Assert.failNotEquals(Assert.java:743) at 
 org.junit.Assert.assertEquals(Assert.java:118) at 
 org.junit.Assert.assertEquals(Assert.java:555) at 
 org.junit.Assert.assertEquals(Assert.java:542) at 
 org.apache.hadoop.hbase.master.TestDistributedLogSplitting.testLogReplayForDisablingTable(TestDistributedLogSplitting.java:838)
  
 {code}
 This was due to the loop starting around line 823 not picking up any edits 



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Jesse Yates (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881275#comment-13881275
 ] 

Jesse Yates commented on HBASE-10412:
-

{quote}
1. Get distributed log replay working with tags using a WALCodec. (Now)

2. Redo distributed replay a bit so it doesn't use a bare codec but uses its 
own. Maybe a RPC codec as static inner class that embeds a WAL codec? (Later) 
{quote}

I can live with that... on you Mr. Release Manager to make it happen :)

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker
 Fix For: 0.98.0, 0.99.0

 Attachments: HBASE-10412.patch


 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Andrew Purtell (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881280#comment-13881280
 ] 

Andrew Purtell commented on HBASE-10412:


bq. I can live with that... on you Mr. Release Manager to make it happen 

Wait. #2 not for 0.98.0. 

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker
 Fix For: 0.98.0, 0.99.0

 Attachments: HBASE-10412.patch


 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10408) Intermittent TestDistributedLogSplitting#testLogReplayForDisablingTable failure

2014-01-24 Thread Ted Yu (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10408?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881282#comment-13881282
 ] 

Ted Yu commented on HBASE-10408:


Previously isCarryingMeta was used as a flag to continue with the outer loop:
{code}
-  if (isCarryingMeta) {
-continue;
-  }
{code}
This flag was removed by introducing label for outer loop.

 Intermittent TestDistributedLogSplitting#testLogReplayForDisablingTable 
 failure
 ---

 Key: HBASE-10408
 URL: https://issues.apache.org/jira/browse/HBASE-10408
 Project: HBase
  Issue Type: Test
Reporter: Ted Yu
Assignee: Ted Yu
 Attachments: 10408-v1.txt, 10408-v2.txt, 
 TestDistributedLogSplitting.out, 
 testDistributedLogSplitting-disablingTable.html


 Here was the assertion failure:
 {code} java.lang.AssertionError: expected:lt;1000 but was:lt;0 at 
 org.junit.Assert.fail(Assert.java:88) at 
 org.junit.Assert.failNotEquals(Assert.java:743) at 
 org.junit.Assert.assertEquals(Assert.java:118) at 
 org.junit.Assert.assertEquals(Assert.java:555) at 
 org.junit.Assert.assertEquals(Assert.java:542) at 
 org.apache.hadoop.hbase.master.TestDistributedLogSplitting.testLogReplayForDisablingTable(TestDistributedLogSplitting.java:838)
  
 {code}
 This was due to the loop starting around line 823 not picking up any edits 



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10408) Intermittent TestDistributedLogSplitting#testLogReplayForDisablingTable failure

2014-01-24 Thread Andrew Purtell (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10408?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881288#comment-13881288
 ] 

Andrew Purtell commented on HBASE-10408:


bq. This flag was removed by introducing label for outer loop.

Yep, labeled jumps aka gotos should be avoided unless necessary in my 
opinion. I realize this is just test code but we are supposed to do code review 
here after all. No need to point out existing occurrences of this idiom in our 
code. I said unless necessary. This is a bigger change than you needed to 
make to fix the test.

 Intermittent TestDistributedLogSplitting#testLogReplayForDisablingTable 
 failure
 ---

 Key: HBASE-10408
 URL: https://issues.apache.org/jira/browse/HBASE-10408
 Project: HBase
  Issue Type: Test
Reporter: Ted Yu
Assignee: Ted Yu
 Attachments: 10408-v1.txt, 10408-v2.txt, 
 TestDistributedLogSplitting.out, 
 testDistributedLogSplitting-disablingTable.html


 Here was the assertion failure:
 {code} java.lang.AssertionError: expected:lt;1000 but was:lt;0 at 
 org.junit.Assert.fail(Assert.java:88) at 
 org.junit.Assert.failNotEquals(Assert.java:743) at 
 org.junit.Assert.assertEquals(Assert.java:118) at 
 org.junit.Assert.assertEquals(Assert.java:555) at 
 org.junit.Assert.assertEquals(Assert.java:542) at 
 org.apache.hadoop.hbase.master.TestDistributedLogSplitting.testLogReplayForDisablingTable(TestDistributedLogSplitting.java:838)
  
 {code}
 This was due to the loop starting around line 823 not picking up any edits 



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Jesse Yates (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881295#comment-13881295
 ] 

Jesse Yates commented on HBASE-10412:
-

bq. Wait. #2 not for 0.98.0. 

That's fine, but soon in the 98 series?

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker
 Fix For: 0.98.0, 0.99.0

 Attachments: HBASE-10412.patch


 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Updated] (HBASE-10408) Intermittent TestDistributedLogSplitting#testLogReplayForDisablingTable failure

2014-01-24 Thread Ted Yu (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-10408?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ted Yu updated HBASE-10408:
---

Attachment: 10408-v3.txt

See if patch v3 is better.

Thanks for the comments, Andy.

 Intermittent TestDistributedLogSplitting#testLogReplayForDisablingTable 
 failure
 ---

 Key: HBASE-10408
 URL: https://issues.apache.org/jira/browse/HBASE-10408
 Project: HBase
  Issue Type: Test
Reporter: Ted Yu
Assignee: Ted Yu
 Attachments: 10408-v1.txt, 10408-v2.txt, 10408-v3.txt, 
 TestDistributedLogSplitting.out, 
 testDistributedLogSplitting-disablingTable.html


 Here was the assertion failure:
 {code} java.lang.AssertionError: expected:lt;1000 but was:lt;0 at 
 org.junit.Assert.fail(Assert.java:88) at 
 org.junit.Assert.failNotEquals(Assert.java:743) at 
 org.junit.Assert.assertEquals(Assert.java:118) at 
 org.junit.Assert.assertEquals(Assert.java:555) at 
 org.junit.Assert.assertEquals(Assert.java:542) at 
 org.apache.hadoop.hbase.master.TestDistributedLogSplitting.testLogReplayForDisablingTable(TestDistributedLogSplitting.java:838)
  
 {code}
 This was due to the loop starting around line 823 not picking up any edits 



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Andrew Purtell (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881299#comment-13881299
 ] 

Andrew Purtell commented on HBASE-10412:


bq. That's fine, but soon in the 98 series?

See HBASE-10414 for 0.98.1.

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker
 Fix For: 0.98.0, 0.99.0

 Attachments: HBASE-10412.patch


 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Created] (HBASE-10414) Distributed replay should re-encode WAL entries with its own RPC codec

2014-01-24 Thread Andrew Purtell (JIRA)
Andrew Purtell created HBASE-10414:
--

 Summary: Distributed replay should re-encode WAL entries with its 
own RPC codec
 Key: HBASE-10414
 URL: https://issues.apache.org/jira/browse/HBASE-10414
 Project: HBase
  Issue Type: Improvement
Affects Versions: 0.98.0, 0.99.0
Reporter: Andrew Purtell
 Fix For: 0.98.1, 0.99.0


HBASE-10412 allows distributed replay to send WAL entries with tags intact 
between RegionServers by substituting a WALCodec directly for the RPC codec. We 
should instead have distributed replay handle WAL entries including tags with 
its own tag-aware RPC codec and drop the direct use of WALCodecs for that 
purpose. 



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread ramkrishna.s.vasudevan (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881304#comment-13881304
 ] 

ramkrishna.s.vasudevan commented on HBASE-10412:


+1 to have the simple and best soln for 0.98.0

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker
 Fix For: 0.98.0, 0.99.0

 Attachments: HBASE-10412.patch, HBASE-10412_V2.patch


 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Updated] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Anoop Sam John (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Anoop Sam John updated HBASE-10412:
---

Attachment: HBASE-10412_V2.patch

Patch addressing comments from Andy and Ted.

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker
 Fix For: 0.98.0, 0.99.0

 Attachments: HBASE-10412.patch, HBASE-10412_V2.patch


 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Updated] (HBASE-10413) TableSplits are not sorted by size.

2014-01-24 Thread Nick Dimiduk (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-10413?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Nick Dimiduk updated HBASE-10413:
-

Component/s: mapreduce

 TableSplits are not sorted by size.
 ---

 Key: HBASE-10413
 URL: https://issues.apache.org/jira/browse/HBASE-10413
 Project: HBase
  Issue Type: Bug
  Components: Client, mapreduce
Affects Versions: 0.96.1.1
Reporter: Lukas Nalezenec

 We had serious issue in our production today.
 InputSplits should be sorted by length but TableSplit does not contain real 
 getLength implementation:
   @Override
   public long getLength() {
 // Not clear how to obtain this... seems to be used only for sorting 
 splits
 return 0;
   }
 Can we implement this method ? 
 What is the best way ?



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10414) Distributed replay should re-encode WAL entries with its own RPC codec

2014-01-24 Thread ramkrishna.s.vasudevan (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10414?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881303#comment-13881303
 ] 

ramkrishna.s.vasudevan commented on HBASE-10414:


Will take this up for 0.98.1 along with the negotiation thing.

 Distributed replay should re-encode WAL entries with its own RPC codec
 --

 Key: HBASE-10414
 URL: https://issues.apache.org/jira/browse/HBASE-10414
 Project: HBase
  Issue Type: Improvement
Affects Versions: 0.98.0, 0.99.0
Reporter: Andrew Purtell
Assignee: ramkrishna.s.vasudevan
 Fix For: 0.98.1, 0.99.0


 HBASE-10412 allows distributed replay to send WAL entries with tags intact 
 between RegionServers by substituting a WALCodec directly for the RPC codec. 
 We should instead have distributed replay handle WAL entries including tags 
 with its own tag-aware RPC codec and drop the direct use of WALCodecs for 
 that purpose. 



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Assigned] (HBASE-10414) Distributed replay should re-encode WAL entries with its own RPC codec

2014-01-24 Thread ramkrishna.s.vasudevan (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-10414?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

ramkrishna.s.vasudevan reassigned HBASE-10414:
--

Assignee: ramkrishna.s.vasudevan

 Distributed replay should re-encode WAL entries with its own RPC codec
 --

 Key: HBASE-10414
 URL: https://issues.apache.org/jira/browse/HBASE-10414
 Project: HBase
  Issue Type: Improvement
Affects Versions: 0.98.0, 0.99.0
Reporter: Andrew Purtell
Assignee: ramkrishna.s.vasudevan
 Fix For: 0.98.1, 0.99.0


 HBASE-10412 allows distributed replay to send WAL entries with tags intact 
 between RegionServers by substituting a WALCodec directly for the RPC codec. 
 We should instead have distributed replay handle WAL entries including tags 
 with its own tag-aware RPC codec and drop the direct use of WALCodecs for 
 that purpose. 



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10408) Intermittent TestDistributedLogSplitting#testLogReplayForDisablingTable failure

2014-01-24 Thread Andrew Purtell (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10408?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881306#comment-13881306
 ] 

Andrew Purtell commented on HBASE-10408:


+1 for v3 for trunk and 0.98

 Intermittent TestDistributedLogSplitting#testLogReplayForDisablingTable 
 failure
 ---

 Key: HBASE-10408
 URL: https://issues.apache.org/jira/browse/HBASE-10408
 Project: HBase
  Issue Type: Test
Reporter: Ted Yu
Assignee: Ted Yu
 Attachments: 10408-v1.txt, 10408-v2.txt, 10408-v3.txt, 
 TestDistributedLogSplitting.out, 
 testDistributedLogSplitting-disablingTable.html


 Here was the assertion failure:
 {code} java.lang.AssertionError: expected:lt;1000 but was:lt;0 at 
 org.junit.Assert.fail(Assert.java:88) at 
 org.junit.Assert.failNotEquals(Assert.java:743) at 
 org.junit.Assert.assertEquals(Assert.java:118) at 
 org.junit.Assert.assertEquals(Assert.java:555) at 
 org.junit.Assert.assertEquals(Assert.java:542) at 
 org.apache.hadoop.hbase.master.TestDistributedLogSplitting.testLogReplayForDisablingTable(TestDistributedLogSplitting.java:838)
  
 {code}
 This was due to the loop starting around line 823 not picking up any edits 



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Created] (HBASE-10415) TestAuditLogs#testAuditAllowedStat sometimes fails in trunk

2014-01-24 Thread Ted Yu (JIRA)
Ted Yu created HBASE-10415:
--

 Summary: TestAuditLogs#testAuditAllowedStat sometimes fails in 
trunk
 Key: HBASE-10415
 URL: https://issues.apache.org/jira/browse/HBASE-10415
 Project: HBase
  Issue Type: Test
Reporter: Ted Yu


Running TestAuditLogs on Linux, I got:
{code}
testAuditAllowedStat[1](org.apache.hadoop.hdfs.server.namenode.TestAuditLogs)  
Time elapsed: 6.677 sec   FAILURE!
java.lang.AssertionError: null
at org.junit.Assert.fail(Assert.java:92)
at org.junit.Assert.assertTrue(Assert.java:43)
at org.junit.Assert.assertNotNull(Assert.java:526)
at org.junit.Assert.assertNotNull(Assert.java:537)
at 
org.apache.hadoop.hdfs.server.namenode.TestAuditLogs.verifyAuditLogsRepeat(TestAuditLogs.java:312)
at 
org.apache.hadoop.hdfs.server.namenode.TestAuditLogs.verifyAuditLogs(TestAuditLogs.java:295)
at 
org.apache.hadoop.hdfs.server.namenode.TestAuditLogs.testAuditAllowedStat(TestAuditLogs.java:163)
{code}



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Updated] (HBASE-10408) Intermittent TestDistributedLogSplitting#testLogReplayForDisablingTable failure

2014-01-24 Thread Ted Yu (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-10408?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ted Yu updated HBASE-10408:
---

Fix Version/s: 0.99.0
   0.98.0

 Intermittent TestDistributedLogSplitting#testLogReplayForDisablingTable 
 failure
 ---

 Key: HBASE-10408
 URL: https://issues.apache.org/jira/browse/HBASE-10408
 Project: HBase
  Issue Type: Test
Reporter: Ted Yu
Assignee: Ted Yu
 Fix For: 0.98.0, 0.99.0

 Attachments: 10408-v1.txt, 10408-v2.txt, 10408-v3.txt, 
 TestDistributedLogSplitting.out, 
 testDistributedLogSplitting-disablingTable.html


 Here was the assertion failure:
 {code} java.lang.AssertionError: expected:lt;1000 but was:lt;0 at 
 org.junit.Assert.fail(Assert.java:88) at 
 org.junit.Assert.failNotEquals(Assert.java:743) at 
 org.junit.Assert.assertEquals(Assert.java:118) at 
 org.junit.Assert.assertEquals(Assert.java:555) at 
 org.junit.Assert.assertEquals(Assert.java:542) at 
 org.apache.hadoop.hbase.master.TestDistributedLogSplitting.testLogReplayForDisablingTable(TestDistributedLogSplitting.java:838)
  
 {code}
 This was due to the loop starting around line 823 not picking up any edits 



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10408) Intermittent TestDistributedLogSplitting#testLogReplayForDisablingTable failure

2014-01-24 Thread Ted Yu (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10408?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881319#comment-13881319
 ] 

Ted Yu commented on HBASE-10408:


Integrated to 0.98 and trunk.

Thanks for the review, Andy.

 Intermittent TestDistributedLogSplitting#testLogReplayForDisablingTable 
 failure
 ---

 Key: HBASE-10408
 URL: https://issues.apache.org/jira/browse/HBASE-10408
 Project: HBase
  Issue Type: Test
Reporter: Ted Yu
Assignee: Ted Yu
 Fix For: 0.98.0, 0.99.0

 Attachments: 10408-v1.txt, 10408-v2.txt, 10408-v3.txt, 
 TestDistributedLogSplitting.out, 
 testDistributedLogSplitting-disablingTable.html


 Here was the assertion failure:
 {code} java.lang.AssertionError: expected:lt;1000 but was:lt;0 at 
 org.junit.Assert.fail(Assert.java:88) at 
 org.junit.Assert.failNotEquals(Assert.java:743) at 
 org.junit.Assert.assertEquals(Assert.java:118) at 
 org.junit.Assert.assertEquals(Assert.java:555) at 
 org.junit.Assert.assertEquals(Assert.java:542) at 
 org.apache.hadoop.hbase.master.TestDistributedLogSplitting.testLogReplayForDisablingTable(TestDistributedLogSplitting.java:838)
  
 {code}
 This was due to the loop starting around line 823 not picking up any edits 



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10410) On some JREs, hbase-server is failing due to timeout without any test actually timing out

2014-01-24 Thread Nicolas Liochon (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10410?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881323#comment-13881323
 ] 

Nicolas Liochon commented on HBASE-10410:
-

bq. Hmm
The syntax changed in surefire :-)

bq. surefire 2.16 seemed to result in tests not finishing?
Yeah, exactly. It seems to be an HBase regression actually, but I'm waiting for 
the 2.17 to try again. In 2.15 and previous, the bugs were in surefire itself.


 On some JREs, hbase-server is failing due to timeout without any test 
 actually timing out
 ---

 Key: HBASE-10410
 URL: https://issues.apache.org/jira/browse/HBASE-10410
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Andrew Purtell

 This is frustrating. On some JREs (Java 7) I am seeing hbase-server fail due 
 to timeout, without any test actually reporting a timeout.  This 
 StackOverflow answer talks about how JUnit  4.12-SNAPSHOT may report a 
 timeout if the test throws an InterruptedException: 
 http://stackoverflow.com/questions/17016011/junit-test-times-out-despite-executing-quickly
   Those happen at various points when shutting down the minicluster. We may 
 be letting one escape. 



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10392) Correct references to hbase.regionserver.global.memstore.upperLimit

2014-01-24 Thread Anoop Sam John (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881329#comment-13881329
 ] 

Anoop Sam John commented on HBASE-10392:


{code}
-  float globalMemstoreLimit = 
conf.getFloat(hbase.regionserver.global.memstore.upperLimit, 0.4f);
-  int gml = (int)(globalMemstoreLimit * CONVERT_TO_PERCENTAGE);
+  float globalMemstoreSize = 
conf.getFloat(hbase.regionserver.global.memstore.size, 0.4f);
+  int gml = (int)(globalMemstoreSize * CONVERT_TO_PERCENTAGE);
{code}
I was/am worried abt this change where we look at the new config alone.  What 
if a user configure hbase.regionserver.global.memstore.upperLimit (old) to 0.5 
and HConstants.HFILE_BLOCK_CACHE_SIZE_KEY to 0.4 ?  This check will not find 
out that right?

 Correct references to hbase.regionserver.global.memstore.upperLimit
 ---

 Key: HBASE-10392
 URL: https://issues.apache.org/jira/browse/HBASE-10392
 Project: HBase
  Issue Type: Bug
Reporter: Nick Dimiduk
Assignee: Nick Dimiduk
 Fix For: 0.99.0

 Attachments: HBASE-10392.0.patch, HBASE-10392.1.patch, 
 HBASE-10392.2.patch


 As part of the awesome new HBASE-5349, a couple references to 
 {{hbase.regionserver.global.memstore.upperLimit}} was missed. Clean those up 
 to use the new {{hbase.regionserver.global.memstore.size}} instead.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Ted Yu (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881333#comment-13881333
 ] 

Ted Yu commented on HBASE-10412:


{code}
+public class TestVisibilityLabelsWithDistributedLogReplay extends 
TestVisibilityLabels {
+
+  @BeforeClass
+  public static void setupBeforeClass() throws Exception {
+// setup configuration
+conf = TEST_UTIL.getConfiguration();
+conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
{code}
nit: if setting DISTRIBUTED_LOG_REPLAY_KEY is the only change from base test, 
consider reusing TestVisibilityLabels#setupBeforeClass()

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker
 Fix For: 0.98.0, 0.99.0

 Attachments: HBASE-10412.patch, HBASE-10412_V2.patch


 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Andrew Purtell (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881335#comment-13881335
 ] 

Andrew Purtell commented on HBASE-10412:


Tests should not reuse setup from other tests. That seems to be a basic 
encapsulation violation.

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker
 Fix For: 0.98.0, 0.99.0

 Attachments: HBASE-10412.patch, HBASE-10412_V2.patch


 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Updated] (HBASE-7320) Replace calls to KeyValue.getBuffer with appropropriate calls to getRowArray, getFamilyArray(), getQualifierArray, and getValueArray

2014-01-24 Thread Lars Hofhansl (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-7320?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Lars Hofhansl updated HBASE-7320:
-

Attachment: 7320-simple.txt

Here's a patch that changes all the simple calls.
(Somebody had to do the repetitive work)

All calls to getBuffer() left are of two categories:
# access a KVs complete key
# access the KV as a whole

Now we need to fix those.

 Replace calls to KeyValue.getBuffer with appropropriate calls to getRowArray, 
 getFamilyArray(), getQualifierArray, and getValueArray
 

 Key: HBASE-7320
 URL: https://issues.apache.org/jira/browse/HBASE-7320
 Project: HBase
  Issue Type: Bug
Reporter: Lars Hofhansl
Assignee: stack
 Fix For: 0.99.0

 Attachments: 7320-simple.txt


 In many places this is simple task of just replacing the method name.
 There, however, quite a few places where we assume that:
 # the entire KV is backed by a single byte array
 # the KVs key portion is backed by a single byte array
 Some of those can easily be fixed, others will need their own jiras.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Updated] (HBASE-7320) Replace calls to KeyValue.getBuffer with appropropriate calls to getRowArray, getFamilyArray(), getQualifierArray, and getValueArray

2014-01-24 Thread Lars Hofhansl (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-7320?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Lars Hofhansl updated HBASE-7320:
-

Status: Patch Available  (was: Open)

Let's see if I messed up anything.

 Replace calls to KeyValue.getBuffer with appropropriate calls to getRowArray, 
 getFamilyArray(), getQualifierArray, and getValueArray
 

 Key: HBASE-7320
 URL: https://issues.apache.org/jira/browse/HBASE-7320
 Project: HBase
  Issue Type: Bug
Reporter: Lars Hofhansl
Assignee: stack
 Fix For: 0.99.0

 Attachments: 7320-simple.txt


 In many places this is simple task of just replacing the method name.
 There, however, quite a few places where we assume that:
 # the entire KV is backed by a single byte array
 # the KVs key portion is backed by a single byte array
 Some of those can easily be fixed, others will need their own jiras.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Updated] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Anoop Sam John (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Anoop Sam John updated HBASE-10412:
---

Attachment: HBASE-10412_V3.patch

This version avoids some duplication in setupBeforeClass. 
Have to set
conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
before starting cluster.

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker
 Fix For: 0.98.0, 0.99.0

 Attachments: HBASE-10412.patch, HBASE-10412_V2.patch, 
 HBASE-10412_V3.patch


 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-7320) Replace calls to KeyValue.getBuffer with appropropriate calls to getRowArray, getFamilyArray(), getQualifierArray, and getValueArray

2014-01-24 Thread Lars Hofhansl (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-7320?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881353#comment-13881353
 ] 

Lars Hofhansl commented on HBASE-7320:
--

Looks like jenkins is taking a vacation.

 Replace calls to KeyValue.getBuffer with appropropriate calls to getRowArray, 
 getFamilyArray(), getQualifierArray, and getValueArray
 

 Key: HBASE-7320
 URL: https://issues.apache.org/jira/browse/HBASE-7320
 Project: HBase
  Issue Type: Bug
Reporter: Lars Hofhansl
Assignee: stack
 Fix For: 0.99.0

 Attachments: 7320-simple.txt


 In many places this is simple task of just replacing the method name.
 There, however, quite a few places where we assume that:
 # the entire KV is backed by a single byte array
 # the KVs key portion is backed by a single byte array
 Some of those can easily be fixed, others will need their own jiras.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Anoop Sam John (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881352#comment-13881352
 ] 

Anoop Sam John commented on HBASE-10412:


V3 avoids some duplication in test class.
bq.Tests should not reuse setup from other tests. That seems to be a basic 
encapsulation violation.
Got your point. Can ignore V3 version of patch then.

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker
 Fix For: 0.98.0, 0.99.0

 Attachments: HBASE-10412.patch, HBASE-10412_V2.patch, 
 HBASE-10412_V3.patch


 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Jeffrey Zhong (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881356#comment-13881356
 ] 

Jeffrey Zhong commented on HBASE-10412:
---

The patch looks good to me! We can combine WALEdit shipment in replay and 
replication WALEdits shipment to use the same code later. 

Could we in the future to define a tag's visibility scope so that GET/Scan code 
path can automatically strip out tags when returns to external clients?

Thanks.

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker
 Fix For: 0.98.0, 0.99.0

 Attachments: HBASE-10412.patch, HBASE-10412_V2.patch


 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Updated] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Andrew Purtell (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Andrew Purtell updated HBASE-10412:
---

Attachment: (was: HBASE-10412_V3.patch)

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker
 Fix For: 0.98.0, 0.99.0

 Attachments: HBASE-10412.patch, HBASE-10412_V2.patch


 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Andrew Purtell (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881357#comment-13881357
 ] 

Andrew Purtell commented on HBASE-10412:


bq. Got your point. Can ignore V3 version of patch then.

Ok, I need to delete it so HadoopQA will pick up V2. 

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker
 Fix For: 0.98.0, 0.99.0

 Attachments: HBASE-10412.patch, HBASE-10412_V2.patch


 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Andrew Purtell (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881362#comment-13881362
 ] 

Andrew Purtell commented on HBASE-10412:


bq. Could we in the future to define a tag's visibility scope so that GET/Scan 
code path can automatically strip out tags when returns to external clients

We had some discussion about this on HBASE-10322. The short version is per-cell 
checks introduce performance concerns. That issue has a lot more verbiage. 

With regard to replication, tag scope on a cell shares the same scope as the 
cell itself.

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker
 Fix For: 0.98.0, 0.99.0

 Attachments: HBASE-10412.patch, HBASE-10412_V2.patch


 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Andrew Purtell (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881366#comment-13881366
 ] 

Andrew Purtell commented on HBASE-10412:


Looks like HadoopQA may be on vacation. Let me test this locally. I will commit 
if good.

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker
 Fix For: 0.98.0, 0.99.0

 Attachments: HBASE-10412.patch, HBASE-10412_V2.patch


 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10413) TableSplits are not sorted by size.

2014-01-24 Thread Enis Soztutar (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881369#comment-13881369
 ] 

Enis Soztutar commented on HBASE-10413:
---

What is the issue? Is it about the MR scheduler for scheduling the tasks? 
AFAIK, getLength() has never been implemented, and MR just works fine without 
it. 

 TableSplits are not sorted by size.
 ---

 Key: HBASE-10413
 URL: https://issues.apache.org/jira/browse/HBASE-10413
 Project: HBase
  Issue Type: Bug
  Components: Client, mapreduce
Affects Versions: 0.96.1.1
Reporter: Lukas Nalezenec

 We had serious issue in our production today.
 InputSplits should be sorted by length but TableSplit does not contain real 
 getLength implementation:
   @Override
   public long getLength() {
 // Not clear how to obtain this... seems to be used only for sorting 
 splits
 return 0;
   }
 Can we implement this method ? 
 What is the best way ?



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10389) Add namespace help info in table related shell commands

2014-01-24 Thread Enis Soztutar (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10389?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881371#comment-13881371
 ] 

Enis Soztutar commented on HBASE-10389:
---

There is some documentation here: http://hbase.apache.org/book/namespace.html
We can use this as a base. 

 Add namespace help info in table related shell commands
 ---

 Key: HBASE-10389
 URL: https://issues.apache.org/jira/browse/HBASE-10389
 Project: HBase
  Issue Type: Improvement
  Components: shell
Affects Versions: 0.96.0, 0.96.1
Reporter: Jerry He
Assignee: Jerry He

 Currently in the help info of table related shell command, we don't mention 
 or give namespace as part of the table name.  
 For example, to create table:
 {code}
 hbase(main):001:0 help 'create'
 Creates a table. Pass a table name, and a set of column family
 specifications (at least one), and, optionally, table configuration.
 Column specification can be a simple string (name), or a dictionary
 (dictionaries are described below in main help output), necessarily
 including NAME attribute.
 Examples:
   hbase create 't1', {NAME = 'f1', VERSIONS = 5}
   hbase create 't1', {NAME = 'f1'}, {NAME = 'f2'}, {NAME = 'f3'}
   hbase # The above in shorthand would be the following:
   hbase create 't1', 'f1', 'f2', 'f3'
   hbase create 't1', {NAME = 'f1', VERSIONS = 1, TTL = 2592000, 
 BLOCKCACHE = true}
   hbase create 't1', {NAME = 'f1', CONFIGURATION = 
 {'hbase.hstore.blockingStoreFiles' = '10'}}
 Table configuration options can be put at the end.
 Examples:
   hbase create 't1', 'f1', SPLITS = ['10', '20', '30', '40']
   hbase create 't1', 'f1', SPLITS_FILE = 'splits.txt', OWNER = 'johndoe'
   hbase create 't1', {NAME = 'f1', VERSIONS = 5}, METADATA = { 'mykey' = 
 'myvalue' }
   hbase # Optionally pre-split the table into NUMREGIONS, using
   hbase # SPLITALGO (HexStringSplit, UniformSplit or classname)
   hbase create 't1', 'f1', {NUMREGIONS = 15, SPLITALGO = 'HexStringSplit'}
   hbase create 't1', 'f1', {NUMREGIONS = 15, SPLITALGO = 'HexStringSplit', 
 CONFIGURATION = {'hbase.hregion.scan.loadColumnFamiliesOnDemand' = 'true'}}
 You can also keep around a reference to the created table:
   hbase t1 = create 't1', 'f1'
 Which gives you a reference to the table named 't1', on which you can then
 call methods.
 {code}
 We should document the usage of namespace in these commands.
 For example:
 #namespace=foo and table qualifier=bar
 create 'foo:bar', 'fam'
 #namespace=default and table qualifier=bar
 create 'bar', 'fam'



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-7320) Replace calls to KeyValue.getBuffer with appropropriate calls to getRowArray, getFamilyArray(), getQualifierArray, and getValueArray

2014-01-24 Thread stack (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-7320?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881419#comment-13881419
 ] 

stack commented on HBASE-7320:
--

I went through trying to be careful.  Looks great to me.  If I had done it I 
would have missed loads of these... especially the likes of this:

{code}
-System.arraycopy(c.getTagsArray(), c.getTagsOffset(), 
newKV.getBuffer(),
+System.arraycopy(c.getTagsArray(), c.getTagsOffset(), 
newKV.getTagsArray(),
 newKV.getTagsOffset(), oldCellTagsLen);
{code}

Excellent [~lhofhansl]

 Replace calls to KeyValue.getBuffer with appropropriate calls to getRowArray, 
 getFamilyArray(), getQualifierArray, and getValueArray
 

 Key: HBASE-7320
 URL: https://issues.apache.org/jira/browse/HBASE-7320
 Project: HBase
  Issue Type: Bug
Reporter: Lars Hofhansl
Assignee: stack
 Fix For: 0.99.0

 Attachments: 7320-simple.txt


 In many places this is simple task of just replacing the method name.
 There, however, quite a few places where we assume that:
 # the entire KV is backed by a single byte array
 # the KVs key portion is backed by a single byte array
 Some of those can easily be fixed, others will need their own jiras.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10081) Since HBASE-7091, HBASE_OPTS cannot be set on the command line

2014-01-24 Thread stack (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10081?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881442#comment-13881442
 ] 

stack commented on HBASE-10081:
---

(Old school showing up late and coming from left field)

When did it become ok picking up config from environment?  I thought that was a 
no-no from ops 101 (inevitably this 'feature' cuts in in surprising ways).

 Since HBASE-7091, HBASE_OPTS cannot be set on the command line
 --

 Key: HBASE-10081
 URL: https://issues.apache.org/jira/browse/HBASE-10081
 Project: HBase
  Issue Type: Bug
  Components: scripts
Affects Versions: 0.98.0, 0.96.0
Reporter: Nicolas Liochon
Assignee: Liu Shaohui
Priority: Minor
 Attachments: HBASE-10081-v1.diff


 Discussed in HBASE-7091.
 It's not critical, but a little bit surprising, as the comments in bin/hbase 
 doesn't say anything about this. If you create your own hbase-env then it's 
 not an issue...



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Resolved] (HBASE-10408) Intermittent TestDistributedLogSplitting#testLogReplayForDisablingTable failure

2014-01-24 Thread Ted Yu (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-10408?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ted Yu resolved HBASE-10408.


  Resolution: Fixed
Hadoop Flags: Reviewed

 Intermittent TestDistributedLogSplitting#testLogReplayForDisablingTable 
 failure
 ---

 Key: HBASE-10408
 URL: https://issues.apache.org/jira/browse/HBASE-10408
 Project: HBase
  Issue Type: Test
Reporter: Ted Yu
Assignee: Ted Yu
 Fix For: 0.98.0, 0.99.0

 Attachments: 10408-v1.txt, 10408-v2.txt, 10408-v3.txt, 
 TestDistributedLogSplitting.out, 
 testDistributedLogSplitting-disablingTable.html


 Here was the assertion failure:
 {code} java.lang.AssertionError: expected:lt;1000 but was:lt;0 at 
 org.junit.Assert.fail(Assert.java:88) at 
 org.junit.Assert.failNotEquals(Assert.java:743) at 
 org.junit.Assert.assertEquals(Assert.java:118) at 
 org.junit.Assert.assertEquals(Assert.java:555) at 
 org.junit.Assert.assertEquals(Assert.java:542) at 
 org.apache.hadoop.hbase.master.TestDistributedLogSplitting.testLogReplayForDisablingTable(TestDistributedLogSplitting.java:838)
  
 {code}
 This was due to the loop starting around line 823 not picking up any edits 



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Created] (HBASE-10416) Improvements to the import flow

2014-01-24 Thread Vasu Mariyala (JIRA)
Vasu Mariyala created HBASE-10416:
-

 Summary: Improvements to the import flow
 Key: HBASE-10416
 URL: https://issues.apache.org/jira/browse/HBASE-10416
 Project: HBase
  Issue Type: New Feature
  Components: mapreduce
Reporter: Vasu Mariyala


Following improvements can be made to the Import logic

a) Make the import extensible (i.e., remove the filter from being a static 
member of Import and make it an instance variable of the mapper, make the 
mappers or variables of interest protected. )

b) Make sure that the Import calls filterRowKey method of the filter (Useful if 
we want to filter the data of an organization based on the row key or using 
filters like PrefixFilter). The existing test case in 
TestImportExport#testWithFilter works with this assumption but is so far 
successful because there is only one row inserted into the table.

c) Provide an option to specify the durability during the import (Specifying 
the Durability as SKIP_WAL would improve the performance of restore 
considerably.) [~lhofhansl] suggested that this should be a parameter to the 
import.

d) Some minor refactoring to avoid building a comma separated string for the 
filter args.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Updated] (HBASE-10416) Improvements to the import flow

2014-01-24 Thread Vasu Mariyala (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-10416?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Vasu Mariyala updated HBASE-10416:
--

Attachment: HBASE-10416.patch

Attaching the patch for the above mentioned issues.

 Improvements to the import flow
 ---

 Key: HBASE-10416
 URL: https://issues.apache.org/jira/browse/HBASE-10416
 Project: HBase
  Issue Type: New Feature
  Components: mapreduce
Reporter: Vasu Mariyala
 Attachments: HBASE-10416.patch


 Following improvements can be made to the Import logic
 a) Make the import extensible (i.e., remove the filter from being a static 
 member of Import and make it an instance variable of the mapper, make the 
 mappers or variables of interest protected. )
 b) Make sure that the Import calls filterRowKey method of the filter (Useful 
 if we want to filter the data of an organization based on the row key or 
 using filters like PrefixFilter). The existing test case in 
 TestImportExport#testWithFilter works with this assumption but is so far 
 successful because there is only one row inserted into the table.
 c) Provide an option to specify the durability during the import (Specifying 
 the Durability as SKIP_WAL would improve the performance of restore 
 considerably.) [~lhofhansl] suggested that this should be a parameter to the 
 import.
 d) Some minor refactoring to avoid building a comma separated string for the 
 filter args.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Updated] (HBASE-10416) Improvements to the import flow

2014-01-24 Thread Vasu Mariyala (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-10416?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Vasu Mariyala updated HBASE-10416:
--

Status: Patch Available  (was: Open)

 Improvements to the import flow
 ---

 Key: HBASE-10416
 URL: https://issues.apache.org/jira/browse/HBASE-10416
 Project: HBase
  Issue Type: New Feature
  Components: mapreduce
Reporter: Vasu Mariyala
 Attachments: HBASE-10416.patch


 Following improvements can be made to the Import logic
 a) Make the import extensible (i.e., remove the filter from being a static 
 member of Import and make it an instance variable of the mapper, make the 
 mappers or variables of interest protected. )
 b) Make sure that the Import calls filterRowKey method of the filter (Useful 
 if we want to filter the data of an organization based on the row key or 
 using filters like PrefixFilter). The existing test case in 
 TestImportExport#testWithFilter works with this assumption but is so far 
 successful because there is only one row inserted into the table.
 c) Provide an option to specify the durability during the import (Specifying 
 the Durability as SKIP_WAL would improve the performance of restore 
 considerably.) [~lhofhansl] suggested that this should be a parameter to the 
 import.
 d) Some minor refactoring to avoid building a comma separated string for the 
 filter args.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-7320) Replace calls to KeyValue.getBuffer with appropropriate calls to getRowArray, getFamilyArray(), getQualifierArray, and getValueArray

2014-01-24 Thread Lars Hofhansl (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-7320?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881472#comment-13881472
 ] 

Lars Hofhansl commented on HBASE-7320:
--

Even better, when're done many of these array copies can go away as we can just 
make a new Cell and point it to the existing array if we do not change them.

The next part is to get rid of the timestamp array API completely.

After that, I think, is to writeTo/readFrom files. The writing side might be 
simple as long we keep the format, in that case the reading side might not have 
to change at all (it is still OK to back a Cell by a single byte[])

Big parts are: ScanQueryMatcher, which dissects the KV itself, and all the 
comparators in KeyValue itself.


 Replace calls to KeyValue.getBuffer with appropropriate calls to getRowArray, 
 getFamilyArray(), getQualifierArray, and getValueArray
 

 Key: HBASE-7320
 URL: https://issues.apache.org/jira/browse/HBASE-7320
 Project: HBase
  Issue Type: Bug
Reporter: Lars Hofhansl
Assignee: stack
 Fix For: 0.99.0

 Attachments: 7320-simple.txt


 In many places this is simple task of just replacing the method name.
 There, however, quite a few places where we assume that:
 # the entire KV is backed by a single byte array
 # the KVs key portion is backed by a single byte array
 Some of those can easily be fixed, others will need their own jiras.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Updated] (HBASE-10416) Improvements to the import flow

2014-01-24 Thread Vasu Mariyala (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-10416?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Vasu Mariyala updated HBASE-10416:
--

Description: 
Following improvements can be made to the Import logic

a) Make the import extensible (i.e., remove the filter from being a static 
member of Import and make it an instance variable of the mapper, make the 
mappers or variables of interest protected. )

b) Make sure that the Import calls filterRowKey method of the filter (Useful if 
we want to filter the data of an organization based on the row key or using 
filters like PrefixFilter which filter the data in filterRowKey method rather 
than the filterKeyValue method). The existing test case in 
TestImportExport#testWithFilter works with this assumption but is so far 
successful because there is only one row inserted into the table.

c) Provide an option to specify the durability during the import (Specifying 
the Durability as SKIP_WAL would improve the performance of restore 
considerably.) [~lhofhansl] suggested that this should be a parameter to the 
import.

d) Some minor refactoring to avoid building a comma separated string for the 
filter args.

  was:
Following improvements can be made to the Import logic

a) Make the import extensible (i.e., remove the filter from being a static 
member of Import and make it an instance variable of the mapper, make the 
mappers or variables of interest protected. )

b) Make sure that the Import calls filterRowKey method of the filter (Useful if 
we want to filter the data of an organization based on the row key or using 
filters like PrefixFilter). The existing test case in 
TestImportExport#testWithFilter works with this assumption but is so far 
successful because there is only one row inserted into the table.

c) Provide an option to specify the durability during the import (Specifying 
the Durability as SKIP_WAL would improve the performance of restore 
considerably.) [~lhofhansl] suggested that this should be a parameter to the 
import.

d) Some minor refactoring to avoid building a comma separated string for the 
filter args.


 Improvements to the import flow
 ---

 Key: HBASE-10416
 URL: https://issues.apache.org/jira/browse/HBASE-10416
 Project: HBase
  Issue Type: New Feature
  Components: mapreduce
Reporter: Vasu Mariyala
 Attachments: HBASE-10416.patch


 Following improvements can be made to the Import logic
 a) Make the import extensible (i.e., remove the filter from being a static 
 member of Import and make it an instance variable of the mapper, make the 
 mappers or variables of interest protected. )
 b) Make sure that the Import calls filterRowKey method of the filter (Useful 
 if we want to filter the data of an organization based on the row key or 
 using filters like PrefixFilter which filter the data in filterRowKey method 
 rather than the filterKeyValue method). The existing test case in 
 TestImportExport#testWithFilter works with this assumption but is so far 
 successful because there is only one row inserted into the table.
 c) Provide an option to specify the durability during the import (Specifying 
 the Durability as SKIP_WAL would improve the performance of restore 
 considerably.) [~lhofhansl] suggested that this should be a parameter to the 
 import.
 d) Some minor refactoring to avoid building a comma separated string for the 
 filter args.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-7320) Replace calls to KeyValue.getBuffer with appropropriate calls to getRowArray, getFamilyArray(), getQualifierArray, and getValueArray

2014-01-24 Thread Lars Hofhansl (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-7320?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881473#comment-13881473
 ] 

Lars Hofhansl commented on HBASE-7320:
--

As for this patch, just I put it into a subtask?

 Replace calls to KeyValue.getBuffer with appropropriate calls to getRowArray, 
 getFamilyArray(), getQualifierArray, and getValueArray
 

 Key: HBASE-7320
 URL: https://issues.apache.org/jira/browse/HBASE-7320
 Project: HBase
  Issue Type: Bug
Reporter: Lars Hofhansl
Assignee: stack
 Fix For: 0.99.0

 Attachments: 7320-simple.txt


 In many places this is simple task of just replacing the method name.
 There, however, quite a few places where we assume that:
 # the entire KV is backed by a single byte array
 # the KVs key portion is backed by a single byte array
 Some of those can easily be fixed, others will need their own jiras.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Comment Edited] (HBASE-7320) Replace calls to KeyValue.getBuffer with appropropriate calls to getRowArray, getFamilyArray(), getQualifierArray, and getValueArray

2014-01-24 Thread Lars Hofhansl (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-7320?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881473#comment-13881473
 ] 

Lars Hofhansl edited comment on HBASE-7320 at 1/24/14 10:10 PM:


As for this patch, should I put it into a subtask?


was (Author: lhofhansl):
As for this patch, just I put it into a subtask?

 Replace calls to KeyValue.getBuffer with appropropriate calls to getRowArray, 
 getFamilyArray(), getQualifierArray, and getValueArray
 

 Key: HBASE-7320
 URL: https://issues.apache.org/jira/browse/HBASE-7320
 Project: HBase
  Issue Type: Bug
Reporter: Lars Hofhansl
Assignee: stack
 Fix For: 0.99.0

 Attachments: 7320-simple.txt


 In many places this is simple task of just replacing the method name.
 There, however, quite a few places where we assume that:
 # the entire KV is backed by a single byte array
 # the KVs key portion is backed by a single byte array
 Some of those can easily be fixed, others will need their own jiras.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Updated] (HBASE-10156) Fix up the HBASE-8755 slowdown when low contention

2014-01-24 Thread stack (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-10156?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

stack updated HBASE-10156:
--

Attachment: 10156v20.txt

Himanshu feedback.

 Fix up the HBASE-8755 slowdown when low contention
 --

 Key: HBASE-10156
 URL: https://issues.apache.org/jira/browse/HBASE-10156
 Project: HBase
  Issue Type: Sub-task
  Components: wal
Reporter: stack
Assignee: stack
 Attachments: 10156.txt, 10156v10.txt, 10156v11.txt, 10156v12.txt, 
 10156v12.txt, 10156v13.txt, 10156v16.txt, 10156v17.txt, 10156v18.txt, 
 10156v19.txt, 10156v2.txt, 10156v20.txt, 10156v3.txt, 10156v4.txt, 
 10156v5.txt, 10156v6.txt, 10156v7.txt, 10156v9.txt, Disrupting.java


 HBASE-8755 slows our writes when only a few clients.  Fix.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10416) Improvements to the import flow

2014-01-24 Thread Lars Hofhansl (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10416?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881496#comment-13881496
 ] 

Lars Hofhansl commented on HBASE-10416:
---

looks good to me. Thanks Vasu.

 Improvements to the import flow
 ---

 Key: HBASE-10416
 URL: https://issues.apache.org/jira/browse/HBASE-10416
 Project: HBase
  Issue Type: New Feature
  Components: mapreduce
Reporter: Vasu Mariyala
 Attachments: HBASE-10416.patch


 Following improvements can be made to the Import logic
 a) Make the import extensible (i.e., remove the filter from being a static 
 member of Import and make it an instance variable of the mapper, make the 
 mappers or variables of interest protected. )
 b) Make sure that the Import calls filterRowKey method of the filter (Useful 
 if we want to filter the data of an organization based on the row key or 
 using filters like PrefixFilter which filter the data in filterRowKey method 
 rather than the filterKeyValue method). The existing test case in 
 TestImportExport#testWithFilter works with this assumption but is so far 
 successful because there is only one row inserted into the table.
 c) Provide an option to specify the durability during the import (Specifying 
 the Durability as SKIP_WAL would improve the performance of restore 
 considerably.) [~lhofhansl] suggested that this should be a parameter to the 
 import.
 d) Some minor refactoring to avoid building a comma separated string for the 
 filter args.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10416) Improvements to the import flow

2014-01-24 Thread Ted Yu (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10416?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881508#comment-13881508
 ] 

Ted Yu commented on HBASE-10416:


Nit:
{code}
-  private static Filter instantiateFilter(Configuration conf) {
-// get the filter, if it was configured
+  public static Filter instantiateFilter(Configuration conf) {
{code}
Does instantiateFilter() need to be public ?
{code}
+  public static void flushRegionsIfNecessary(Configuration conf) throws 
IOException, InterruptedException {
{code}
Long line above.

Mind adding release notes for the new parameters ?

 Improvements to the import flow
 ---

 Key: HBASE-10416
 URL: https://issues.apache.org/jira/browse/HBASE-10416
 Project: HBase
  Issue Type: New Feature
  Components: mapreduce
Reporter: Vasu Mariyala
 Attachments: HBASE-10416.patch


 Following improvements can be made to the Import logic
 a) Make the import extensible (i.e., remove the filter from being a static 
 member of Import and make it an instance variable of the mapper, make the 
 mappers or variables of interest protected. )
 b) Make sure that the Import calls filterRowKey method of the filter (Useful 
 if we want to filter the data of an organization based on the row key or 
 using filters like PrefixFilter which filter the data in filterRowKey method 
 rather than the filterKeyValue method). The existing test case in 
 TestImportExport#testWithFilter works with this assumption but is so far 
 successful because there is only one row inserted into the table.
 c) Provide an option to specify the durability during the import (Specifying 
 the Durability as SKIP_WAL would improve the performance of restore 
 considerably.) [~lhofhansl] suggested that this should be a parameter to the 
 import.
 d) Some minor refactoring to avoid building a comma separated string for the 
 filter args.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10416) Improvements to the import flow

2014-01-24 Thread Nick Dimiduk (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10416?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881516#comment-13881516
 ] 

Nick Dimiduk commented on HBASE-10416:
--

Good work [~vasu.mariy...@gmail.com]. Have you seen my notes on HBASE-8074 and 
HBASE-8115 ?

 Improvements to the import flow
 ---

 Key: HBASE-10416
 URL: https://issues.apache.org/jira/browse/HBASE-10416
 Project: HBase
  Issue Type: New Feature
  Components: mapreduce
Reporter: Vasu Mariyala
 Attachments: HBASE-10416.patch


 Following improvements can be made to the Import logic
 a) Make the import extensible (i.e., remove the filter from being a static 
 member of Import and make it an instance variable of the mapper, make the 
 mappers or variables of interest protected. )
 b) Make sure that the Import calls filterRowKey method of the filter (Useful 
 if we want to filter the data of an organization based on the row key or 
 using filters like PrefixFilter which filter the data in filterRowKey method 
 rather than the filterKeyValue method). The existing test case in 
 TestImportExport#testWithFilter works with this assumption but is so far 
 successful because there is only one row inserted into the table.
 c) Provide an option to specify the durability during the import (Specifying 
 the Durability as SKIP_WAL would improve the performance of restore 
 considerably.) [~lhofhansl] suggested that this should be a parameter to the 
 import.
 d) Some minor refactoring to avoid building a comma separated string for the 
 filter args.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Updated] (HBASE-10412) Distributed log replay : Cell tags getting missed

2014-01-24 Thread Andrew Purtell (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-10412?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Andrew Purtell updated HBASE-10412:
---

  Resolution: Fixed
Hadoop Flags: Reviewed
  Status: Resolved  (was: Patch Available)

Local tests are good. Committed to trunk and 0.98.

 Distributed log replay : Cell tags getting missed
 -

 Key: HBASE-10412
 URL: https://issues.apache.org/jira/browse/HBASE-10412
 Project: HBase
  Issue Type: Bug
Affects Versions: 0.98.0
Reporter: Anoop Sam John
Assignee: Anoop Sam John
Priority: Blocker
 Fix For: 0.98.0, 0.99.0

 Attachments: HBASE-10412.patch, HBASE-10412_V2.patch


 This is caused by HBASE-10322. The default RPC codec KVCodec strips tags.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Created] (HBASE-10417) index is not incremented in PutSortReducer#reduce()

2014-01-24 Thread Ted Yu (JIRA)
Ted Yu created HBASE-10417:
--

 Summary: index is not incremented in PutSortReducer#reduce()
 Key: HBASE-10417
 URL: https://issues.apache.org/jira/browse/HBASE-10417
 Project: HBase
  Issue Type: Bug
Reporter: Ted Yu
Priority: Minor


Starting at line 76:
{code}
  int index = 0;
  for (KeyValue kv : map) {
context.write(row, kv);
if (index  0  index % 100 == 0)
  context.setStatus(Wrote  + index);
{code}
index is a variable inside while loop that is never incremented.
The condition index  0 cannot be true.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10383) Secure Bulk Load for 'completebulkload' fails for version 0.94.15

2014-01-24 Thread Jesse Yates (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10383?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881590#comment-13881590
 ] 

Jesse Yates commented on HBASE-10383:
-

Committed to 0.94. Thanks for the reviews and finding this bug!

 Secure Bulk Load for 'completebulkload' fails for version 0.94.15
 -

 Key: HBASE-10383
 URL: https://issues.apache.org/jira/browse/HBASE-10383
 Project: HBase
  Issue Type: Bug
  Components: Coprocessors
Affects Versions: 0.94.15
Reporter: Kashif J S
Assignee: Kashif J S
Priority: Critical
 Fix For: 0.94.17

 Attachments: 10383.txt, HBASE-10383-v2.patch, 
 hbase-10383-jyates-0.94-v0.patch


 Secure Bulk Load with kerberos enabled fails for Complete Bulk 
 LoadLoadIncrementalHfile with following exception ERROR 
 org.apache.hadoop.hbase.regionserver.HRegionServer: 
 org.apache.hadoop.hbase.ipc.HBaseRPC$UnknownProtocolException: No matching 
 handler for protocol 
 org.apache.hadoop.hbase.security.access.SecureBulkLoadProtocol in region 
 t1,,1389699438035.28bb0284d971d0676cf562efea80199b.
  at org.apache.hadoop.hbase.regionserver.HRegion.exec(HRegion.java)
  at 
 org.apache.hadoop.hbase.regionserver.HRegionServer.execCoprocessor(HRegionServer.java)
  at sun.reflect.GeneratedMethodAccessor18.invoke(Unknown Source)
  at 
 sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java)
  at java.lang.reflect.Method.invoke(Method.java)
  at 
 org.apache.hadoop.hbase.ipc.SecureRpcEngine$Server.call(SecureRpcEngine.java)
  at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java) 



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Assigned] (HBASE-10383) Secure Bulk Load for 'completebulkload' fails for version 0.94.15

2014-01-24 Thread Jesse Yates (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-10383?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jesse Yates reassigned HBASE-10383:
---

Assignee: Jesse Yates  (was: Kashif J S)

 Secure Bulk Load for 'completebulkload' fails for version 0.94.15
 -

 Key: HBASE-10383
 URL: https://issues.apache.org/jira/browse/HBASE-10383
 Project: HBase
  Issue Type: Bug
  Components: Coprocessors
Affects Versions: 0.94.15
Reporter: Kashif J S
Assignee: Jesse Yates
Priority: Critical
 Fix For: 0.94.17

 Attachments: 10383.txt, HBASE-10383-v2.patch, 
 hbase-10383-jyates-0.94-v0.patch


 Secure Bulk Load with kerberos enabled fails for Complete Bulk 
 LoadLoadIncrementalHfile with following exception ERROR 
 org.apache.hadoop.hbase.regionserver.HRegionServer: 
 org.apache.hadoop.hbase.ipc.HBaseRPC$UnknownProtocolException: No matching 
 handler for protocol 
 org.apache.hadoop.hbase.security.access.SecureBulkLoadProtocol in region 
 t1,,1389699438035.28bb0284d971d0676cf562efea80199b.
  at org.apache.hadoop.hbase.regionserver.HRegion.exec(HRegion.java)
  at 
 org.apache.hadoop.hbase.regionserver.HRegionServer.execCoprocessor(HRegionServer.java)
  at sun.reflect.GeneratedMethodAccessor18.invoke(Unknown Source)
  at 
 sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java)
  at java.lang.reflect.Method.invoke(Method.java)
  at 
 org.apache.hadoop.hbase.ipc.SecureRpcEngine$Server.call(SecureRpcEngine.java)
  at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java) 



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Resolved] (HBASE-10383) Secure Bulk Load for 'completebulkload' fails for version 0.94.15

2014-01-24 Thread Jesse Yates (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-10383?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jesse Yates resolved HBASE-10383.
-

Resolution: Fixed

 Secure Bulk Load for 'completebulkload' fails for version 0.94.15
 -

 Key: HBASE-10383
 URL: https://issues.apache.org/jira/browse/HBASE-10383
 Project: HBase
  Issue Type: Bug
  Components: Coprocessors
Affects Versions: 0.94.15
Reporter: Kashif J S
Assignee: Kashif J S
Priority: Critical
 Fix For: 0.94.17

 Attachments: 10383.txt, HBASE-10383-v2.patch, 
 hbase-10383-jyates-0.94-v0.patch


 Secure Bulk Load with kerberos enabled fails for Complete Bulk 
 LoadLoadIncrementalHfile with following exception ERROR 
 org.apache.hadoop.hbase.regionserver.HRegionServer: 
 org.apache.hadoop.hbase.ipc.HBaseRPC$UnknownProtocolException: No matching 
 handler for protocol 
 org.apache.hadoop.hbase.security.access.SecureBulkLoadProtocol in region 
 t1,,1389699438035.28bb0284d971d0676cf562efea80199b.
  at org.apache.hadoop.hbase.regionserver.HRegion.exec(HRegion.java)
  at 
 org.apache.hadoop.hbase.regionserver.HRegionServer.execCoprocessor(HRegionServer.java)
  at sun.reflect.GeneratedMethodAccessor18.invoke(Unknown Source)
  at 
 sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java)
  at java.lang.reflect.Method.invoke(Method.java)
  at 
 org.apache.hadoop.hbase.ipc.SecureRpcEngine$Server.call(SecureRpcEngine.java)
  at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java) 



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Created] (HBASE-10418) give blocks of smaller store files priority in cache

2014-01-24 Thread Sergey Shelukhin (JIRA)
Sergey Shelukhin created HBASE-10418:


 Summary: give blocks of smaller store files priority in cache
 Key: HBASE-10418
 URL: https://issues.apache.org/jira/browse/HBASE-10418
 Project: HBase
  Issue Type: Improvement
  Components: regionserver
Reporter: Sergey Shelukhin


That's just an idea at this point, I don't have a patch nor plan to make one in 
near future.
It's good for datasets that don't fit in memory especially; and if scans are 
involved. 
Scans (and gets in absence of bloom filters' help) have to read from all store 
files. Short range request will hit one block in every file.
If small files are more likely to be entirely available in memory, on average 
requests will hit less blocks from FS. 
For scans that read a lot of data, it's better to read blocks in sequence from 
a big file and blocks for small files from cache, rather than a mix of FS and 
cached blocks from different files, because the (HBase) blocks of a big file 
would be sequential in one HDFS block.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10418) give blocks of smaller store files priority in cache

2014-01-24 Thread Andrew Purtell (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10418?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881607#comment-13881607
 ] 

Andrew Purtell commented on HBASE-10418:


This presumes that smaller HFiles contain the data of interest for short scans. 
What kind of mechanism to we have in place to make that more likely than not?

Would it be better to do a bit of schema design such that small files / short 
scan data is segregated to one column family and the large files / large scan 
data to another, and then prioritize in cache by column family?

 give blocks of smaller store files priority in cache
 

 Key: HBASE-10418
 URL: https://issues.apache.org/jira/browse/HBASE-10418
 Project: HBase
  Issue Type: Improvement
  Components: regionserver
Reporter: Sergey Shelukhin

 That's just an idea at this point, I don't have a patch nor plan to make one 
 in near future.
 It's good for datasets that don't fit in memory especially; and if scans are 
 involved. 
 Scans (and gets in absence of bloom filters' help) have to read from all 
 store files. Short range request will hit one block in every file.
 If small files are more likely to be entirely available in memory, on average 
 requests will hit less blocks from FS. 
 For scans that read a lot of data, it's better to read blocks in sequence 
 from a big file and blocks for small files from cache, rather than a mix of 
 FS and cached blocks from different files, because the (HBase) blocks of a 
 big file would be sequential in one HDFS block.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Created] (HBASE-10419) Add multiget support to PerformanceEvaluation

2014-01-24 Thread Nick Dimiduk (JIRA)
Nick Dimiduk created HBASE-10419:


 Summary: Add multiget support to PerformanceEvaluation
 Key: HBASE-10419
 URL: https://issues.apache.org/jira/browse/HBASE-10419
 Project: HBase
  Issue Type: Improvement
  Components: test
Reporter: Nick Dimiduk
Assignee: Nick Dimiduk
Priority: Minor


Folks planning to use multiget may find this useful.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Updated] (HBASE-10419) Add multiget support to PerformanceEvaluation

2014-01-24 Thread Nick Dimiduk (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-10419?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Nick Dimiduk updated HBASE-10419:
-

Status: Patch Available  (was: Open)

 Add multiget support to PerformanceEvaluation
 -

 Key: HBASE-10419
 URL: https://issues.apache.org/jira/browse/HBASE-10419
 Project: HBase
  Issue Type: Improvement
  Components: test
Reporter: Nick Dimiduk
Assignee: Nick Dimiduk
Priority: Minor
 Attachments: HBASE-10419.0.patch


 Folks planning to use multiget may find this useful.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Updated] (HBASE-10419) Add multiget support to PerformanceEvaluation

2014-01-24 Thread Nick Dimiduk (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-10419?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Nick Dimiduk updated HBASE-10419:
-

Attachment: HBASE-10419.0.patch

Here's a patch adding this feature. It runs with --nomapred; haven't tried yet 
with mapreduce.

 Add multiget support to PerformanceEvaluation
 -

 Key: HBASE-10419
 URL: https://issues.apache.org/jira/browse/HBASE-10419
 Project: HBase
  Issue Type: Improvement
  Components: test
Reporter: Nick Dimiduk
Assignee: Nick Dimiduk
Priority: Minor
 Attachments: HBASE-10419.0.patch


 Folks planning to use multiget may find this useful.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10418) give blocks of smaller store files priority in cache

2014-01-24 Thread Sergey Shelukhin (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10418?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881623#comment-13881623
 ] 

Sergey Shelukhin commented on HBASE-10418:
--

Unless file key range and scan key range don't intersect

 give blocks of smaller store files priority in cache
 

 Key: HBASE-10418
 URL: https://issues.apache.org/jira/browse/HBASE-10418
 Project: HBase
  Issue Type: Improvement
  Components: regionserver
Reporter: Sergey Shelukhin

 That's just an idea at this point, I don't have a patch nor plan to make one 
 in near future.
 It's good for datasets that don't fit in memory especially; and if scans are 
 involved. 
 Scans (and gets in absence of bloom filters' help) have to read from all 
 store files. Short range request will hit one block in every file.
 If small files are more likely to be entirely available in memory, on average 
 requests will hit less blocks from FS. 
 For scans that read a lot of data, it's better to read blocks in sequence 
 from a big file and blocks for small files from cache, rather than a mix of 
 FS and cached blocks from different files, because the (HBase) blocks of a 
 big file would be sequential in one HDFS block.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (HBASE-10418) give blocks of smaller store files priority in cache

2014-01-24 Thread Sergey Shelukhin (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-10418?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881622#comment-13881622
 ] 

Sergey Shelukhin commented on HBASE-10418:
--

If you have such knowledge yes. I am talking about unknown data distribution, 
within the same table/cf for simplicity.
First, if compactions happen in normal pattern, we'll have a large file from 
major compaction and small files from flushes/minors.
If we don't know the data distribution, what is described above would be the 
expected pattern...
Specifically for scans, they cannot use bloom filters, and pretty much have to 
hit a block of each file, no matter the data distribution, right?

 give blocks of smaller store files priority in cache
 

 Key: HBASE-10418
 URL: https://issues.apache.org/jira/browse/HBASE-10418
 Project: HBase
  Issue Type: Improvement
  Components: regionserver
Reporter: Sergey Shelukhin

 That's just an idea at this point, I don't have a patch nor plan to make one 
 in near future.
 It's good for datasets that don't fit in memory especially; and if scans are 
 involved. 
 Scans (and gets in absence of bloom filters' help) have to read from all 
 store files. Short range request will hit one block in every file.
 If small files are more likely to be entirely available in memory, on average 
 requests will hit less blocks from FS. 
 For scans that read a lot of data, it's better to read blocks in sequence 
 from a big file and blocks for small files from cache, rather than a mix of 
 FS and cached blocks from different files, because the (HBase) blocks of a 
 big file would be sequential in one HDFS block.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


  1   2   >