|
18 | 18 | package org.apache.hadoop.hbase.spark |
19 | 19 |
|
20 | 20 | import org.apache.hadoop.fs.{FileSystem, Path} |
21 | | -import org.apache.hadoop.hbase.client.{Get, ConnectionFactory} |
| 21 | +import org.apache.hadoop.hbase.client.{ConnectionFactory, Get} |
22 | 22 | import org.apache.hadoop.hbase.io.hfile.{CacheConfig, HFile} |
23 | 23 | import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles |
24 | | -import org.apache.hadoop.hbase.{HConstants, CellUtil, HBaseTestingUtility, TableName} |
| 24 | +import org.apache.hadoop.hbase.{CellUtil, HBaseTestingUtility, HConstants, TableName} |
25 | 25 | import org.apache.hadoop.hbase.util.Bytes |
26 | 26 | import org.apache.hadoop.hbase.spark.HBaseRDDFunctions._ |
27 | 27 | import org.apache.spark.SparkContext |
28 | 28 | import org.junit.rules.TemporaryFolder |
29 | 29 | import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} |
30 | 30 |
|
| 31 | +import java.io.File |
| 32 | +import java.net.URI |
| 33 | +import java.nio.file.Files |
| 34 | + |
31 | 35 | class BulkLoadSuite extends FunSuite with |
32 | 36 | BeforeAndAfterEach with BeforeAndAfterAll with Logging { |
33 | 37 | @transient var sc: SparkContext = null |
@@ -65,6 +69,61 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { |
65 | 69 | sc.stop() |
66 | 70 | } |
67 | 71 |
|
| 72 | + test ("Staging dir: Test usage of staging dir on a separate filesystem") { |
| 73 | + val config = TEST_UTIL.getConfiguration |
| 74 | + |
| 75 | + logInfo(" - creating table " + tableName) |
| 76 | + TEST_UTIL.createTable(TableName.valueOf(tableName), |
| 77 | + Array(Bytes.toBytes(columnFamily1), Bytes.toBytes(columnFamily2))) |
| 78 | + |
| 79 | + // Test creates rdd with 2 column families and |
| 80 | + // write those to hfiles on local filesystem |
| 81 | + // using bulkLoad functionality. We don't check the load functionality |
| 82 | + // due the limitations of the HBase Minicluster |
| 83 | + |
| 84 | + val rdd = sc.parallelize(Array( |
| 85 | + (Bytes.toBytes("1"), |
| 86 | + (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo1"))), |
| 87 | + (Bytes.toBytes("2"), |
| 88 | + (Bytes.toBytes(columnFamily2), Bytes.toBytes("b"), Bytes.toBytes("bar.2"))))) |
| 89 | + |
| 90 | + val hbaseContext = new HBaseContext(sc, config) |
| 91 | + val uri = Files.createTempDirectory("tmpDirPrefix").toUri |
| 92 | + val stagingUri = new URI(uri + "staging_dir") |
| 93 | + val stagingFolder = new File(stagingUri) |
| 94 | + val fs = new Path(stagingUri.toString).getFileSystem(config) |
| 95 | + try { |
| 96 | + hbaseContext.bulkLoad[(Array[Byte], (Array[Byte], Array[Byte], Array[Byte]))](rdd, |
| 97 | + TableName.valueOf(tableName), |
| 98 | + t => { |
| 99 | + val rowKey = t._1 |
| 100 | + val family: Array[Byte] = t._2._1 |
| 101 | + val qualifier = t._2._2 |
| 102 | + val value: Array[Byte] = t._2._3 |
| 103 | + |
| 104 | + val keyFamilyQualifier = new KeyFamilyQualifier(rowKey, family, qualifier) |
| 105 | + |
| 106 | + Seq((keyFamilyQualifier, value)).iterator |
| 107 | + }, |
| 108 | + stagingUri.toString) |
| 109 | + |
| 110 | + assert(fs.listStatus(new Path(stagingFolder.getPath)).length == 2) |
| 111 | + |
| 112 | + } finally { |
| 113 | + val admin = ConnectionFactory.createConnection(config).getAdmin |
| 114 | + try { |
| 115 | + admin.disableTable(TableName.valueOf(tableName)) |
| 116 | + admin.deleteTable(TableName.valueOf(tableName)) |
| 117 | + } finally { |
| 118 | + admin.close() |
| 119 | + } |
| 120 | + fs.delete(new Path(stagingFolder.getPath), true) |
| 121 | + |
| 122 | + testFolder.delete() |
| 123 | + |
| 124 | + } |
| 125 | + } |
| 126 | + |
68 | 127 | test("Wide Row Bulk Load: Test multi family and multi column tests " + |
69 | 128 | "with all default HFile Configs.") { |
70 | 129 | val config = TEST_UTIL.getConfiguration |
|
0 commit comments