-
Notifications
You must be signed in to change notification settings - Fork 3
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
[feat] Add support to use hbase online store #126
Merged
Merged
Changes from 12 commits
Commits
Show all changes
14 commits
Select commit
Hold shift + click to select a range
ad454e1
Add spark changes to use hbase api
shydefoo 2528ffe
Fix connection issues
shydefoo 773eac1
Fix linting
shydefoo fd3030a
Add configuration for hbase api
shydefoo cdfad3d
Set platform to linux/amd64
shydefoo 73dfd8e
Update application.yaml to include hbase
shydefoo acd54c2
Refator BigTableSinkRelation to use updated classes
shydefoo f685315
Fix issue due to difference in bigtable and hbase response
shydefoo 0054a14
Fix linting
shydefoo 38f7bc7
Clean up comments
shydefoo 691c8d1
Fix application yaml
shydefoo 8855d0a
Merge branch 'bayu/hbase' into hbase-poc
shydefoo a0e44a3
Add option for hbase for stream ingestion jobs
shydefoo fa28a59
Fix linting
shydefoo File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -4,7 +4,13 @@ import com.google.cloud.bigtable.hbase.BigtableConfiguration | |
import dev.caraml.spark.serialization.Serializer | ||
import dev.caraml.spark.utils.StringUtils | ||
import org.apache.hadoop.conf.Configuration | ||
import org.apache.hadoop.hbase.client.Put | ||
import org.apache.hadoop.hbase.client.{ | ||
Admin, | ||
ColumnFamilyDescriptorBuilder, | ||
Connection, | ||
Put, | ||
TableDescriptorBuilder | ||
} | ||
import org.apache.hadoop.hbase.mapred.TableOutputFormat | ||
import org.apache.hadoop.hbase.{HColumnDescriptor, HTableDescriptor, TableName} | ||
import org.apache.hadoop.mapred.JobConf | ||
|
@@ -30,42 +36,49 @@ class BigTableSinkRelation( | |
|
||
override def schema: StructType = ??? | ||
|
||
def getConnection(hadoopConfig: Configuration): Connection = { | ||
BigtableConfiguration.connect(hadoopConfig) | ||
} | ||
|
||
def createTable(): Unit = { | ||
val btConn = BigtableConfiguration.connect(hadoopConfig) | ||
val btConn = getConnection(hadoopConfig) | ||
try { | ||
val admin = btConn.getAdmin | ||
|
||
val table = if (!admin.isTableAvailable(TableName.valueOf(tableName))) { | ||
val t = new HTableDescriptor(TableName.valueOf(tableName)) | ||
val metadataCF = new HColumnDescriptor(metadataColumnFamily) | ||
t.addFamily(metadataCF) | ||
t | ||
val tableBuilder = TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName)) | ||
val cf = ColumnFamilyDescriptorBuilder.of(metadataColumnFamily) | ||
tableBuilder.setColumnFamily(cf) | ||
val table = tableBuilder.build() | ||
table | ||
} else { | ||
admin.getTableDescriptor(TableName.valueOf(tableName)) | ||
val t = btConn.getTable(TableName.valueOf(tableName)) | ||
t.getDescriptor() | ||
} | ||
|
||
val featuresCF = new HColumnDescriptor(config.namespace) | ||
val featuresCFBuilder = ColumnFamilyDescriptorBuilder.newBuilder(config.namespace.getBytes) | ||
if (config.maxAge > 0) { | ||
featuresCF.setTimeToLive(config.maxAge.toInt) | ||
featuresCFBuilder.setTimeToLive(config.maxAge.toInt) | ||
} | ||
featuresCFBuilder.setMaxVersions(1) | ||
val featuresCF = featuresCFBuilder.build() | ||
|
||
featuresCF.setMaxVersions(1) | ||
|
||
// TODO: Set compression type for column family | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. seems this comment can be removed, or is it expected for future notes? |
||
val tdb = TableDescriptorBuilder.newBuilder(table) | ||
if (!table.getColumnFamilyNames.contains(config.namespace.getBytes)) { | ||
table.addFamily(featuresCF) | ||
|
||
tdb.setColumnFamily(featuresCF) | ||
val t = tdb.build() | ||
if (!admin.isTableAvailable(table.getTableName)) { | ||
admin.createTable(table) | ||
admin.createTable(t) | ||
} else { | ||
admin.modifyTable(table) | ||
admin.modifyTable(t) | ||
} | ||
} else if ( | ||
config.maxAge > 0 && table | ||
.getColumnFamily(config.namespace.getBytes) | ||
.getTimeToLive != featuresCF.getTimeToLive | ||
) { | ||
table.modifyFamily(featuresCF) | ||
admin.modifyTable(table) | ||
tdb.modifyColumnFamily(featuresCF) | ||
admin.modifyTable(tdb.build()) | ||
} | ||
} finally { | ||
btConn.close() | ||
|
@@ -115,7 +128,7 @@ class BigTableSinkRelation( | |
val qualifier = "avro".getBytes | ||
put.addColumn(metadataColumnFamily.getBytes, qualifier, schema.asInstanceOf[String].getBytes) | ||
|
||
val btConn = BigtableConfiguration.connect(hadoopConfig) | ||
val btConn = getConnection(hadoopConfig) | ||
try { | ||
val table = btConn.getTable(TableName.valueOf(tableName)) | ||
table.checkAndPut( | ||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
17 changes: 17 additions & 0 deletions
17
caraml-store-spark/src/main/scala/dev/caraml/spark/stores/bigtable/HbaseSinkRelation.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,17 @@ | ||
package dev.caraml.spark.stores.bigtable | ||
|
||
import dev.caraml.spark.serialization.Serializer | ||
import org.apache.hadoop.conf.Configuration | ||
import org.apache.hadoop.hbase.client.{Connection, ConnectionFactory} | ||
import org.apache.spark.sql.SQLContext | ||
|
||
class HbaseSinkRelation( | ||
sqlContext: SQLContext, | ||
serializer: Serializer, | ||
config: SparkBigtableConfig, | ||
hadoopConfig: Configuration | ||
) extends BigTableSinkRelation(sqlContext, serializer, config, hadoopConfig) { | ||
override def getConnection(hadoopConfig: Configuration): Connection = { | ||
ConnectionFactory.createConnection(hadoopConfig) | ||
} | ||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
is it expected for redis and redisCluster commented?