[
https://issues.apache.org/jira/browse/HBASE-9549?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Nick Dimiduk updated HBASE-9549:
--------------------------------
Attachment: HBASE-9549.01.patch
This patch includes a correction in the way rest/RowSpec builds a get from the
query URI. It also is more pedantic about interpreting the results of a call to
KeyValue#parseColumn. It will throw exceptions in most places when an invalid
column is encountered. It also updates the documentation in Thrift to be more
clear about API differences between that and the Java API.
> KeyValue#parseColumn(byte[]) does not handle empty qualifier
> ------------------------------------------------------------
>
> Key: HBASE-9549
> URL: https://issues.apache.org/jira/browse/HBASE-9549
> Project: HBase
> Issue Type: Bug
> Components: mapreduce, REST, Thrift, util
> Affects Versions: 0.95.2
> Reporter: Nick Dimiduk
> Assignee: Nick Dimiduk
> Priority: Minor
> Fix For: 0.98.0, 0.96.1
>
> Attachments: HBASE-9549.00.patch, HBASE-9549.01.patch
>
>
> HTable allows a user to interact directly with a KeyValue with an empty
> qualifier, yet {{KeyValue#parseColumn(byte[])}} treats this as a reference to
> a column family. No qualifier delimiter and an empty qualifier are treated as
> the same:
> {code}
> if (index == -1) {
> // If no delimiter, return array of size 1
> return new byte [][] { c };
> } else if(index == c.length - 1) {
> // Only a family, return array size 1
> byte [] family = new byte[c.length-1];
> System.arraycopy(c, 0, family, 0, family.length);
> return new byte [][] { family };
> }
> ...
> {code}
> This inconsistency breaks external interfaces which depend on
> {{parseColumn}}, for instance, the shell:
> {noformat}
> # shell interactions with KV with an empty qualifier
> hbase(main):001:0> create 'foo', 'f1'
> 0 row(s) in 1.4130 seconds
> => Hbase::Table - foo
> hbase(main):002:0> put 'foo', 'rk1', 'f1:', 'empty?'
> 0 row(s) in 0.0750 seconds # <= put works
> hbase(main):003:0> put 'foo', 'rk1', 'f1:bar', 'value'
> 0 row(s) in 0.0070 seconds
> # attempt to retrieve just the kv with empty qualifier
> hbase(main):004:0> get 'foo', 'rk1', 'f1:'
> COLUMN CELL
> f1: timestamp=1379363480020,
> value=empty?
> f1:bar timestamp=1379363546360,
> value=value
> 2 row(s) in 0.0360 seconds # <= returns more than expected!
> hbase(main):005:0> get 'foo', 'rk1', 'f1'
> COLUMN CELL
> f1: timestamp=1379363480020,
> value=empty?
> f1:bar timestamp=1379363546360,
> value=value
> 2 row(s) in 0.0120 seconds
> hbase(main):006:0> delete 'foo', 'rk1', 'f1:'
> 0 row(s) in 0.0290 seconds # <= delete works
> hbase(main):007:0> get 'foo', 'rk1', 'f1:'
> COLUMN CELL
> f1:bar timestamp=1379363546360,
> value=value
> 1 row(s) in 0.0260 seconds
> hbase(main):008:0> get 'foo', 'rk1', 'f1'
> COLUMN CELL
> f1:bar timestamp=1379363546360,
> value=value
> 1 row(s) in 0.0080 seconds
> # restore the empty qual kv for HTable test
> hbase(main):011:0> put 'foo', 'rk1', 'f1:', 'empty?'
> 0 row(s) in 0.0950 seconds
> hbase(main):010:0> get 'foo', 'rk1', 'f1:'
> COLUMN CELL
> f1: timestamp=1379365262555,
> value=empty?
> f1:bar timestamp=1379365134135,
> value=value
> 2 row(s) in 0.0290 seconds
> hbase(main):011:0> get 'foo', 'rk1', 'f1'
> COLUMN CELL
> f1: timestamp=1379365262555,
> value=empty?
> f1:bar timestamp=1379365134135,
> value=value
> 2 row(s) in 0.0080 seconds
> hbase(main):012:0> hconf = org.apache.hadoop.hbase.HBaseConfiguration.create()
> => #<Java::OrgApacheHadoopConf::Configuration:0x208e2fb5>
> hbase(main):013:0> t = org.apache.hadoop.hbase.client.HTable.new(hconf,'foo')
> => #<Java::OrgApacheHadoopHbaseClient::HTable:0x437d51a6>
> # create a Get requesting the empty qualifier only, works
> hbase(main):014:0> g1 =
> org.apache.hadoop.hbase.client.Get.new(org.apache.hadoop.hbase.util.Bytes.toBytes('rk1'))
> => #<Java::OrgApacheHadoopHbaseClient::Get:0x796523ab>
> hbase(main):015:0>
> g1.addColumn(org.apache.hadoop.hbase.util.Bytes.toBytes('f1'), nil)
> => #<Java::OrgApacheHadoopHbaseClient::Get:0x796523ab>
> hbase(main):016:0> t.get(g1).toString()
> => "keyvalues={rk1/f1:/1379365262555/Put/vlen=6/mvcc=0}"
> # create a Get requesting the whole family, works
> hbase(main):017:0> g2 =
> org.apache.hadoop.hbase.client.Get.new(org.apache.hadoop.hbase.util.Bytes.toBytes('rk1'))
> => #<Java::OrgApacheHadoopHbaseClient::Get:0x52e5376a>
> hbase(main):018:0>
> g2.addFamily(org.apache.hadoop.hbase.util.Bytes.toBytes('f1'))
> => #<Java::OrgApacheHadoopHbaseClient::Get:0x52e5376a>
> hbase(main):019:0> t.get(g2).toString()
> => "keyvalues={rk1/f1:/1379365262555/Put/vlen=6/mvcc=0,
> rk1/f1:bar/1379365134135/Put/vlen=5/mvcc=0}"
> {noformat}
--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira