Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[HBASE-25357] allow specifying binary row key range to pre-split regions #72

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Expand Up @@ -17,6 +17,7 @@

package org.apache.hadoop.hbase.spark

import java.nio.charset.StandardCharsets
import java.util
import java.util.concurrent.ConcurrentLinkedQueue

Expand Down Expand Up @@ -150,12 +151,10 @@ case class HBaseRelation (

def createTable() {
val numReg = parameters.get(HBaseTableCatalog.newTable).map(x => x.toInt).getOrElse(0)
val startKey = Bytes.toBytes(
parameters.get(HBaseTableCatalog.regionStart)
.getOrElse(HBaseTableCatalog.defaultRegionStart))
val endKey = Bytes.toBytes(
parameters.get(HBaseTableCatalog.regionEnd)
.getOrElse(HBaseTableCatalog.defaultRegionEnd))
val startKey = parameters.get(HBaseTableCatalog.regionStart)
.getOrElse(HBaseTableCatalog.defaultRegionStart).getBytes(StandardCharsets.ISO_8859_1)
Copy link

@wchevreuil wchevreuil Dec 4, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm not sure it is a good idea to use different encoding from the default used by Bytes util converter (StandardCharsets.UTF_8), as many pieces of hbase code would rely on the Bytes converter, comparisons may become inconsistent.

Also, why you are using a different converter here, can you elaborate better what is the issue you are having within the builtin Bytes converter?

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The spark option use string to pass parameters, not support directly passing bytes,I need pass binary row key so I have to interpreter binary bytes as ISO_8859_1 encoded String, it’s not valid UTF-8.

It’s a trick, does break backward compatibility for UTF-8 string containing characters beyond ISO_8859_1 charset, the UTF-8 string must be wrapped as explained in the JIRA issue.

I can’t figure out better way to pass bytes in spark option.

val endKey = parameters.get(HBaseTableCatalog.regionEnd)
.getOrElse(HBaseTableCatalog.defaultRegionEnd).getBytes(StandardCharsets.ISO_8859_1)
if (numReg > 3) {
val tName = TableName.valueOf(tableName)
val cfs = catalog.getColumnFamilies
Expand Down