-
Notifications
You must be signed in to change notification settings - Fork 85
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
Added storehaus-hbase #139
Merged
Merged
Changes from 8 commits
Commits
Show all changes
10 commits
Select commit
Hold shift + click to select a range
d027d2e
Added storehaus-hbase, upgraded bijection to 0.5.3
MansurAshraf fcdbdd8
added hbase dependencies
MansurAshraf c602eec
added hbase module
mav911 6c5d9dc
DRYed out the imp by moving common methods to HBaseStore trait
mav911 fee8734
changed quorum to Seq
mav911 55791ed
added support for HBaseMinicluster for testing
mav911 40a99ba
mocking hbase - second try
mav911 1677a9a
removed shutdown hook at it is causing zookeeper to hang
mav911 ecc5f3b
switched to FuturePool for blocking operations
mav911 2cbef56
removed unused imports
mav911 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
78 changes: 78 additions & 0 deletions
78
storehaus-hbase/src/main/scala/com/twitter/storehaus/hbase/HBaseByteArrayStore.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,78 @@ | ||
/* | ||
* Copyright 2013 Twitter inc. | ||
* | ||
* Licensed under the Apache License, Version 2.0 (the "License"); | ||
* you may not use this file except in compliance with the License. | ||
* You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package com.twitter.storehaus.hbase | ||
|
||
import org.apache.hadoop.hbase.client.HTablePool | ||
import com.twitter.storehaus.Store | ||
import com.twitter.util.Future | ||
import org.apache.hadoop.conf.Configuration | ||
|
||
/** | ||
* @author MansurAshraf | ||
* @since 9/8/13 | ||
*/ | ||
object HBaseByteArrayStore { | ||
def apply(quorumNames: Seq[String], | ||
table: String, | ||
columnFamily: String, | ||
column: String, | ||
createTable: Boolean, | ||
pool: HTablePool, | ||
conf: Configuration): HBaseByteArrayStore = { | ||
val store = new HBaseByteArrayStore(quorumNames, table, columnFamily, column, createTable,pool,conf) | ||
store.validateConfiguration() | ||
store.createTableIfRequired() | ||
store | ||
} | ||
|
||
def apply(quorumNames: Seq[String], | ||
table: String, | ||
columnFamily: String, | ||
column: String, | ||
createTable: Boolean): HBaseByteArrayStore = apply(quorumNames,table,columnFamily,column,createTable,new HTablePool(), new Configuration()) | ||
} | ||
|
||
class HBaseByteArrayStore(protected val quorumNames: Seq[String], | ||
protected val table: String, | ||
protected val columnFamily: String, | ||
protected val column: String, | ||
protected val createTable: Boolean, | ||
protected val pool: HTablePool, | ||
protected val conf: Configuration) extends Store[Array[Byte], Array[Byte]] with HBaseStore { | ||
|
||
/** get a single key from the store. | ||
* Prefer multiGet if you are getting more than one key at a time | ||
*/ | ||
override def get(k: Array[Byte]): Future[Option[Array[Byte]]] = { | ||
getValue(k) | ||
} | ||
|
||
/** | ||
* replace a value | ||
* Delete is the same as put((k,None)) | ||
*/ | ||
override def put(kv: (Array[Byte], Option[Array[Byte]])): Future[Unit] = { | ||
putValue(kv) | ||
} | ||
|
||
/** Close this store and release any resources. | ||
* It is undefined what happens on get/multiGet after close | ||
*/ | ||
override def close { | ||
pool.close() | ||
} | ||
} |
86 changes: 86 additions & 0 deletions
86
storehaus-hbase/src/main/scala/com/twitter/storehaus/hbase/HBaseLongStore.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,86 @@ | ||
/* | ||
* Copyright 2013 Twitter inc. | ||
* | ||
* Licensed under the Apache License, Version 2.0 (the "License"); | ||
* you may not use this file except in compliance with the License. | ||
* You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package com.twitter.storehaus.hbase | ||
|
||
import org.apache.hadoop.hbase.client.HTablePool | ||
import com.twitter.storehaus.Store | ||
import com.twitter.util.Future | ||
import com.twitter.bijection.Injection._ | ||
import org.apache.hadoop.conf.Configuration | ||
|
||
/** | ||
* @author Mansur Ashraf | ||
* @since 9/8/13 | ||
*/ | ||
object HBaseLongStore { | ||
def apply(quorumNames: Seq[String], | ||
table: String, | ||
columnFamily: String, | ||
column: String, | ||
createTable: Boolean, | ||
pool: HTablePool, | ||
conf: Configuration): HBaseLongStore = { | ||
val store = new HBaseLongStore(quorumNames, table, columnFamily, column, createTable, pool, conf) | ||
store.validateConfiguration() | ||
store.createTableIfRequired() | ||
store | ||
} | ||
|
||
def apply(quorumNames: Seq[String], | ||
table: String, | ||
columnFamily: String, | ||
column: String, | ||
createTable: Boolean): HBaseLongStore = apply(quorumNames, table, columnFamily, column, createTable, new HTablePool(), new Configuration()) | ||
} | ||
|
||
class HBaseLongStore(protected val quorumNames: Seq[String], | ||
protected val table: String, | ||
protected val columnFamily: String, | ||
protected val column: String, | ||
protected val createTable: Boolean, | ||
protected val pool: HTablePool, | ||
protected val conf: Configuration) extends Store[String, Long] with HBaseStore { | ||
|
||
/** get a single key from the store. | ||
* Prefer multiGet if you are getting more than one key at a time | ||
*/ | ||
override def get(k: String): Future[Option[Long]] = { | ||
import com.twitter.bijection.hbase.HBaseBijections._ | ||
implicit val stringInj = fromBijectionRep[String, StringBytes] | ||
implicit val LongInj = fromBijectionRep[Long, LongBytes] | ||
getValue[String, Long](k) | ||
} | ||
|
||
/** | ||
* replace a value | ||
* Delete is the same as put((k,None)) | ||
*/ | ||
override def put(kv: (String, Option[Long])): Future[Unit] = { | ||
import com.twitter.bijection.hbase.HBaseBijections._ | ||
implicit val stringInj = fromBijectionRep[String, StringBytes] | ||
implicit val LongInj = fromBijectionRep[Long, LongBytes] | ||
putValue(kv) | ||
} | ||
|
||
/** Close this store and release any resources. | ||
* It is undefined what happens on get/multiGet after close | ||
*/ | ||
override def close { | ||
pool.close() | ||
} | ||
} | ||
|
92 changes: 92 additions & 0 deletions
92
storehaus-hbase/src/main/scala/com/twitter/storehaus/hbase/HBaseStore.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,92 @@ | ||
/* | ||
* Copyright 2013 Twitter inc. | ||
* | ||
* Licensed under the Apache License, Version 2.0 (the "License"); | ||
* you may not use this file except in compliance with the License. | ||
* You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package com.twitter.storehaus.hbase | ||
|
||
import org.apache.hadoop.hbase.client._ | ||
import org.apache.hadoop.conf.Configuration | ||
import org.apache.hadoop.hbase.{HColumnDescriptor, HTableDescriptor, HBaseConfiguration} | ||
import com.twitter.bijection.hbase.HBaseBijections._ | ||
import com.twitter.bijection.Conversion._ | ||
import com.twitter.bijection.Injection | ||
import com.twitter.util.Future | ||
import scala.Some | ||
|
||
/** | ||
* @author Mansur Ashraf | ||
* @since 9/8/13 | ||
*/ | ||
trait HBaseStore { | ||
|
||
protected val quorumNames: Seq[String] | ||
protected val createTable: Boolean | ||
protected val table: String | ||
protected val columnFamily: String | ||
protected val column: String | ||
protected val pool: HTablePool | ||
protected val conf: Configuration | ||
|
||
def getHBaseAdmin: HBaseAdmin = { | ||
if (conf.get("hbase.zookeeper.quorum") == null) { | ||
conf.set("hbase.zookeeper.quorum", quorumNames.mkString(",")) | ||
} | ||
val hbaseConf = HBaseConfiguration.create(conf) | ||
new HBaseAdmin(hbaseConf) | ||
} | ||
|
||
def createTableIfRequired() { | ||
val hbaseAdmin = getHBaseAdmin | ||
if (createTable && !hbaseAdmin.tableExists(table)) { | ||
val tableDescriptor = new HTableDescriptor(table) | ||
tableDescriptor.addFamily(new HColumnDescriptor(columnFamily)) | ||
hbaseAdmin.createTable(tableDescriptor) | ||
} | ||
} | ||
|
||
def validateConfiguration() { | ||
import org.apache.commons.lang.StringUtils.isNotEmpty | ||
|
||
require(!quorumNames.isEmpty, "Zookeeper quorums are required") | ||
require(isNotEmpty(columnFamily), "column family is required") | ||
require(isNotEmpty(column), "column is required") | ||
} | ||
|
||
def getValue[K, V](key: K)(implicit keyInj: Injection[K, Array[Byte]], valueInj: Injection[V, Array[Byte]]): Future[Option[V]] = Future { | ||
val tbl = pool.getTable(table) | ||
val g = new Get(keyInj(key)) | ||
g.addColumn(columnFamily.as[StringBytes], column.as[StringBytes]) | ||
|
||
val result = tbl.get(g) | ||
val value = result.getValue(columnFamily.as[StringBytes], column.as[StringBytes]) | ||
Option(value).map(v => valueInj.invert(v).get) | ||
} | ||
|
||
def putValue[K, V](kv: (K, Option[V]))(implicit keyInj: Injection[K, Array[Byte]], valueInj: Injection[V, Array[Byte]]): Future[Unit] = { | ||
kv match { | ||
case (k, Some(v)) => Future { | ||
val p = new Put(keyInj(k)) | ||
p.add(columnFamily.as[StringBytes], column.as[StringBytes], valueInj(v)) | ||
val tbl = pool.getTable(table) | ||
tbl.put(p) | ||
} | ||
case (k, None) => Future { | ||
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. Same here; this won't start processing until someone tries to touch the future. |
||
val delete = new Delete(keyInj(k)) | ||
val tbl = pool.getTable(table) | ||
tbl.delete(delete) | ||
} | ||
} | ||
} | ||
} |
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.
worried about Future.apply here... this does call-by-name, so it actually won't start processing until the user does anything. I think we need to make a FuturePool and create a future that way, so that this computation becomes async BUT doesn't block. Thoughts?