Skip to content

Commit

Permalink
[HBASE-25357] allow specifying binary row key range to pre-split regions
Browse files Browse the repository at this point in the history
For example, the row key may start with a long integer, we can specify
ranges to pre-split regions:

```
import java.nio.charset.StandardCharsets;
import org.apache.hadoop.hbase.util.Bytes;

df.write()
  .format("org.apache.hadoop.hbase.spark")
  .option(HBaseTableCatalog.tableCatalog(), catalog)
  .option(HBaseTableCatalog.newTable(), 5)
  .option(HBaseTableCatalog.regionStart(), new String(Bytes.toBytes(0L), StandardCharsets.ISO_8859_1))
  .option(HBaseTableCatalog.regionEnd(), new String(Bytes.toBytes(2000000L), StandardCharsets.ISO_8859_1))
  .mode(SaveMode.Append)
  .save();
```
  • Loading branch information
Dieken committed Dec 4, 2020
1 parent b45d91b commit 41f2156
Showing 1 changed file with 5 additions and 6 deletions.
Original file line number Diff line number Diff line change
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)
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

0 comments on commit 41f2156

Please sign in to comment.