├── src ├── NOTICE ├── main │ ├── scala │ │ └── com │ │ │ └── gravity │ │ │ ├── hadoop │ │ │ ├── Implicits.scala │ │ │ ├── RichFileSystem.scala │ │ │ └── hdfs.scala │ │ │ └── hbase │ │ │ ├── mapreduce.scala │ │ │ ├── schema │ │ │ ├── DeleteOp.scala │ │ │ ├── IncrementOp.scala │ │ │ ├── PutOp.scala │ │ │ ├── ResultCaching.scala │ │ │ ├── HRow.scala │ │ │ ├── OpBase.scala │ │ │ ├── DeserializedResult.scala │ │ │ ├── QueryFilter.scala │ │ │ ├── ScanQuery.scala │ │ │ ├── Query.scala │ │ │ ├── Schema.scala │ │ │ ├── Serialization.scala │ │ │ ├── HbaseTable.scala │ │ │ └── Query2.scala │ │ │ └── schema.scala │ └── java │ │ └── com │ │ └── gravity │ │ └── hadoop │ │ └── GravityTableOutputFormat.java └── test │ └── scala │ └── com │ └── gravity │ └── hbase │ └── schema │ ├── HPasteTestCase.scala │ ├── ExampleSchema.scala │ └── WebCrawlSchemaTest.scala ├── .gitignore ├── pom.xml ├── LICENSE └── README.markdown /src/NOTICE: -------------------------------------------------------------------------------- 1 | This product includes software developed by Gravity.com 2 | (http://www.gravity.com/). -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | # use glob syntax. 2 | syntax: glob 3 | *.ser 4 | *.class 5 | *~ 6 | *.bak 7 | #*.off 8 | *.old 9 | 10 | # vim 11 | tags 12 | .swp 13 | 14 | #jrebel files - usually local dirs 15 | rebel.xml 16 | 17 | # eclipse conf file 18 | .settings 19 | .classpath 20 | .project 21 | .manager 22 | .scala_dependencies 23 | 24 | # idea 25 | .idea 26 | *.iml 27 | *.ipr 28 | *.iws 29 | atlassian-ide-plugin.xml 30 | 31 | # building 32 | target 33 | build 34 | null 35 | tmp* 36 | dist 37 | test-output 38 | build.log 39 | 40 | # other scm 41 | .svn 42 | .CVS 43 | .hg* 44 | 45 | # switch to regexp syntax. 46 | # syntax: regexp 47 | # ^\.pc/ 48 | 49 | #output not in target directory 50 | build.log 51 | 52 | 53 | #Apple Stuff 54 | .DS_Store 55 | 56 | release.properties 57 | pom.xml.releaseBackup -------------------------------------------------------------------------------- /src/main/scala/com/gravity/hadoop/Implicits.scala: -------------------------------------------------------------------------------- 1 | /** Licensed to Gravity.com under one 2 | * or more contributor license agreements. See the NOTICE file 3 | * distributed with this work for additional information 4 | * regarding copyright ownership. Gravity.com licenses this file 5 | * to you under the Apache License, Version 2.0 (the 6 | * "License"); you may not use this file except in compliance 7 | * with the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.gravity.hadoop 19 | 20 | import org.apache.hadoop.fs.FileSystem 21 | 22 | object Implicits { 23 | implicit def asRichFileSystem(fs: FileSystem) = new RichFileSystem(fs) 24 | } -------------------------------------------------------------------------------- /src/main/scala/com/gravity/hbase/mapreduce.scala: -------------------------------------------------------------------------------- 1 | /** Licensed to Gravity.com under one 2 | * or more contributor license agreements. See the NOTICE file 3 | * distributed with this work for additional information 4 | * regarding copyright ownership. Gravity.com licenses this file 5 | * to you under the Apache License, Version 2.0 (the 6 | * "License"); you may not use this file except in compliance 7 | * with the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.gravity.hbase 19 | 20 | import java.util.Random 21 | 22 | package object mapreduce { 23 | type MapperFunc[MK,MV,MOK,MOV, S <: SettingsBase] = (HMapContext[MK,MV,MOK,MOV,S]) => Unit 24 | 25 | type ReducerFunc[MOK,MOV,ROK,ROV, S<:SettingsBase] = (HReduceContext[MOK,MOV,ROK,ROV,S]) => Unit 26 | 27 | def genTmpFile = "/tmp/htemp-" + new Random().nextInt(Int.MaxValue) 28 | } -------------------------------------------------------------------------------- /src/test/scala/com/gravity/hbase/schema/HPasteTestCase.scala: -------------------------------------------------------------------------------- 1 | package com.gravity.hbase.schema 2 | 3 | import junit.framework.TestCase 4 | import org.apache.hadoop.hbase.util.Bytes 5 | import org.apache.hadoop.hbase.HBaseTestingUtility 6 | import scala.collection.mutable.{SynchronizedSet, HashSet} 7 | 8 | /* )\._.,--....,'``. 9 | .b--. /; _.. \ _\ (`._ ,. 10 | `=,-,-'~~~ `----(,_..'--(,_..'`-.;.' */ 11 | 12 | /** 13 | * This sets up the testing cluster. 14 | * We don't support auto table creation (and all the dangerous thereof), so we manually use the Hbase API to create our test tables. 15 | */ 16 | object LocalCluster { 17 | val htest = new HBaseTestingUtility() 18 | htest.startMiniCluster() 19 | 20 | def getTestConfiguration = htest.getConfiguration 21 | 22 | private val alreadyInittedTables = new HashSet[String] with SynchronizedSet[String] 23 | 24 | def initializeSchema(schema:Schema) { 25 | schema.tables.foreach { 26 | table => 27 | if(!alreadyInittedTables.exists(_ == table.tableName)){ 28 | htest.createTable(Bytes.toBytes(table.tableName), table.familyBytes.toArray) 29 | alreadyInittedTables += table.tableName 30 | } 31 | } 32 | } 33 | } 34 | 35 | class HPasteTestCase(schema:Schema) extends TestCase { 36 | 37 | LocalCluster.initializeSchema(schema) 38 | } -------------------------------------------------------------------------------- /src/main/scala/com/gravity/hbase/schema/DeleteOp.scala: -------------------------------------------------------------------------------- 1 | package com.gravity.hbase.schema 2 | 3 | import scala.collection.mutable.Buffer 4 | import org.apache.hadoop.hbase.client.Delete 5 | import scala.collection.Set 6 | 7 | /* )\._.,--....,'``. 8 | .b--. /; _.. \ _\ (`._ ,. 9 | `=,-,-'~~~ `----(,_..'--(,_..'`-.;.' */ 10 | 11 | /** 12 | * A deletion operation. If nothing is specified but a key, will delete the whole row. 13 | * If a family is specified, will just delete the values in 14 | * that family. 15 | * @param table 16 | * @param key 17 | * @param previous 18 | * @tparam T 19 | * @tparam R 20 | */ 21 | class DeleteOp[T <: HbaseTable[T, R, _], R](table: HbaseTable[T, R, _], key: Array[Byte], previous: Buffer[OpBase[T, R]] = Buffer[OpBase[T, R]]()) extends OpBase[T, R](table, key, previous) { 22 | val delete = new Delete(key) 23 | 24 | def +(that: OpBase[T, R]) = new DeleteOp(table,key, previous ++ that.previous) 25 | 26 | 27 | def family[F, K, V](family: (T) => ColumnFamily[T, R, F, K, V]) = { 28 | val fam = family(table.pops) 29 | delete.deleteFamily(fam.familyBytes) 30 | this 31 | } 32 | 33 | def values[F, K, V](family: (T) => ColumnFamily[T, R, F, K, V], qualifiers: Set[K]) = { 34 | val fam = family(table.pops) 35 | for (q <- qualifiers) { 36 | delete.deleteColumns(fam.familyBytes, fam.keyConverter.toBytes(q)) 37 | } 38 | this 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /src/main/scala/com/gravity/hbase/schema/IncrementOp.scala: -------------------------------------------------------------------------------- 1 | package com.gravity.hbase.schema 2 | 3 | import scala.collection.mutable.Buffer 4 | import org.apache.hadoop.hbase.client.Increment 5 | import scala.collection.Map 6 | 7 | /* )\._.,--....,'``. 8 | .b--. /; _.. \ _\ (`._ ,. 9 | `=,-,-'~~~ `----(,_..'--(,_..'`-.;.' */ 10 | 11 | /** 12 | * An increment operation -- can increment multiple columns in a single go. 13 | * @param table 14 | * @param key 15 | * @param previous 16 | * @tparam T 17 | * @tparam R 18 | */ 19 | class IncrementOp[T <: HbaseTable[T, R, _], R](table: HbaseTable[T, R, _], key: Array[Byte], previous: Buffer[OpBase[T, R]] = Buffer[OpBase[T, R]]()) extends OpBase[T, R](table, key, previous) { 20 | val increment = new Increment(key) 21 | increment.setWriteToWAL(false) 22 | 23 | def +(that: OpBase[T, R]) = new IncrementOp(table,key, previous ++ that.previous) 24 | 25 | 26 | def value[F, K, Long](column: (T) => Column[T, R, F, K, Long], value: java.lang.Long) = { 27 | val col = column(table.pops) 28 | increment.addColumn(col.familyBytes, col.columnBytes, value) 29 | this 30 | } 31 | 32 | def valueMap[F, K, Long](family: (T) => ColumnFamily[T, R, F, K, Long], values: Map[K, Long]) = { 33 | val fam = family(table.pops) 34 | for ((key, value) <- values) { 35 | increment.addColumn(fam.familyBytes, fam.keyConverter.toBytes(key), value.asInstanceOf[java.lang.Long]) 36 | } 37 | this 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /src/main/scala/com/gravity/hadoop/RichFileSystem.scala: -------------------------------------------------------------------------------- 1 | /** Licensed to Gravity.com under one 2 | * or more contributor license agreements. See the NOTICE file 3 | * distributed with this work for additional information 4 | * regarding copyright ownership. Gravity.com licenses this file 5 | * to you under the Apache License, Version 2.0 (the 6 | * "License"); you may not use this file except in compliance 7 | * with the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.gravity.hadoop 19 | 20 | import scala.collection.JavaConversions._ 21 | import org.apache.hadoop.fs.{PathFilter, Path, FileSystem} 22 | import java.io.SequenceInputStream 23 | 24 | class RichFileSystem(fs: FileSystem) { 25 | /** 26 | * Opens and streams the Hadoop part files in the given directory, so you 27 | * can treat them as a single merged file. 28 | */ 29 | def openParts(dir: Path): SequenceInputStream = { 30 | val parts = fs.listStatus(dir, new PathFilter { 31 | override def accept(path: Path) = path.getName.startsWith("part-") 32 | }).toIterator 33 | val inputStreams = parts map (fileStatus => fs.open(fileStatus.getPath)) 34 | new SequenceInputStream(inputStreams) 35 | } 36 | } -------------------------------------------------------------------------------- /src/main/scala/com/gravity/hbase/schema/PutOp.scala: -------------------------------------------------------------------------------- 1 | package com.gravity.hbase.schema 2 | 3 | import scala.collection.mutable.Buffer 4 | import org.apache.hadoop.hbase.client.Put 5 | import org.joda.time.DateTime 6 | import scala.collection.Map 7 | 8 | /* )\._.,--....,'``. 9 | .b--. /; _.. \ _\ (`._ ,. 10 | `=,-,-'~~~ `----(,_..'--(,_..'`-.;.' */ 11 | 12 | /** 13 | * A Put operation. Can work across multiple columns or entire column families treated as Maps. 14 | * @param table 15 | * @param key 16 | * @param previous 17 | * @param writeToWAL 18 | * @tparam T 19 | * @tparam R 20 | */ 21 | class PutOp[T <: HbaseTable[T, R, _], R](table: HbaseTable[T, R, _], key: Array[Byte], previous: Buffer[OpBase[T, R]] = Buffer[OpBase[T, R]](), writeToWAL: Boolean = true) extends OpBase[T, R](table, key, previous) { 22 | val put = new Put(key) 23 | put.setWriteToWAL(writeToWAL) 24 | 25 | 26 | def +(that: OpBase[T, R]) = new PutOp(table,key, previous ++ that.previous, writeToWAL) 27 | 28 | def value[F, K, V](column: (T) => Column[T, R, F, K, V], value: V, timeStamp: DateTime = null) = { 29 | val col = column(table.asInstanceOf[T]) 30 | if (timeStamp == null) { 31 | put.add(col.familyBytes, col.columnBytes, col.valueConverter.toBytes(value)) 32 | } else { 33 | put.add(col.familyBytes, col.columnBytes, timeStamp.getMillis, col.valueConverter.toBytes(value)) 34 | } 35 | this 36 | } 37 | 38 | def valueMap[F, K, V](family: (T) => ColumnFamily[T, R, F, K, V], values: Map[K, V]) = { 39 | val fam = family(table.pops) 40 | for ((key, value) <- values) { 41 | put.add(fam.familyBytes, fam.keyConverter.toBytes(key), fam.valueConverter.toBytes(value)) 42 | } 43 | this 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /src/main/scala/com/gravity/hbase/schema/ResultCaching.scala: -------------------------------------------------------------------------------- 1 | /** Licensed to Gravity.com under one 2 | * or more contributor license agreements. See the NOTICE file 3 | * distributed with this work for additional information 4 | * regarding copyright ownership. Gravity.com licenses this file 5 | * to you under the Apache License, Version 2.0 (the 6 | * "License"); you may not use this file except in compliance 7 | * with the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.gravity.hbase.schema 19 | 20 | import org.apache.hadoop.hbase.client.{Get, Scan} 21 | 22 | /* )\._.,--....,'``. 23 | .b--. /; _.. \ _\ (`._ ,. 24 | `=,-,-'~~~ `----(,_..'--(,_..'`-.;.' */ 25 | 26 | 27 | case class ScanCachePolicy(ttlMinutes: Int) 28 | 29 | /** 30 | * An interface that can be injected into an HbaseTable implementation that supports caching 31 | * of queries. 32 | * @tparam T 33 | * @tparam R 34 | * @tparam RR 35 | */ 36 | trait QueryResultCache[T <: HbaseTable[T, R, RR], R, RR <: HRow[T,R]] { 37 | 38 | def getScanResult(key: Scan): Option[Seq[RR]] 39 | 40 | def putScanResult(key: Scan, value: Seq[RR], ttl: Int) 41 | 42 | def getResult(key: Get): Option[RR] 43 | 44 | def putResult(key: Get, value: RR, ttl: Int) 45 | } 46 | 47 | /** 48 | * The default implementation of QueryResultCache. Will do nothing. 49 | * @tparam T 50 | * @tparam R 51 | * @tparam RR 52 | */ 53 | class NoOpCache[T <: HbaseTable[T, R,RR], R, RR <: HRow[T,R]] extends QueryResultCache[T, R, RR] { 54 | 55 | override def getScanResult(key: Scan): Option[Seq[RR]] = None 56 | 57 | override def putScanResult(key: Scan, value: Seq[RR], ttl: Int) {} 58 | 59 | override def getResult(key: Get): Option[RR] = None 60 | 61 | 62 | override def putResult(key: Get, value: RR, ttl: Int) {} 63 | } 64 | -------------------------------------------------------------------------------- /src/main/scala/com/gravity/hbase/schema/HRow.scala: -------------------------------------------------------------------------------- 1 | package com.gravity.hbase.schema 2 | 3 | import scala.collection.JavaConversions._ 4 | 5 | /* )\._.,--....,'``. 6 | .b--. /; _.. \ _\ (`._ ,. 7 | `=,-,-'~~~ `----(,_..'--(,_..'`-.;.' */ 8 | 9 | 10 | /** 11 | * Standard base class for all Row objects. 12 | * Inside of a *Row object, it is good to use lazy val and def as opposed to val. 13 | * Because HRow objects are now the first-class instantiation of a query result, and because 14 | * they are the type cached in Ehcache, they are good places to cache values. 15 | * @param result the raw [[org.apache.hadoop.hbase.client.Result]] returned from the `hbase` [[org.apache.hadoop.hbase.client.Get]] 16 | * @param table the underlying [[com.gravity.hbase.schema.HbaseTable]] 17 | * @tparam T the source [[com.gravity.hbase.schema.HbaseTable]] this result came from 18 | * @tparam R the `type` of the table's rowid 19 | * 20 | */ 21 | abstract class HRow[T <: HbaseTable[T, R, _], R](result: DeserializedResult, table: HbaseTable[T, R, _]) extends QueryResult[T, R](result, table, table.tableName) { 22 | 23 | def prettyPrint() {println(prettyFormat())} 24 | 25 | def prettyPrintNoValues() {println(prettyFormatNoValues())} 26 | 27 | def size = { 28 | var _size = 0 29 | for (i <- 0 until result.values.length) { 30 | val familyMap = result.values(i) 31 | if (familyMap != null) { 32 | _size += familyMap.values.size 33 | } 34 | } 35 | _size 36 | } 37 | 38 | def prettyFormatNoValues() = { 39 | val sb = new StringBuilder() 40 | sb.append("Row Key: " + result.rowid + " (" + result.values.size + " families)" + "\n") 41 | for (i <- 0 until result.values.length) { 42 | val familyMap = result.values(i) 43 | if (familyMap != null) { 44 | val family = table.familyByIndex(i) 45 | sb.append("\tFamily: " + family.familyName + " (" + familyMap.values.size + " items)\n") 46 | } 47 | } 48 | sb.toString 49 | } 50 | 51 | 52 | def prettyFormat() = { 53 | val sb = new StringBuilder() 54 | sb.append("Row Key: " + result.rowid + " (" + result.values.size + " families)" + "\n") 55 | for (i <- 0 until result.values.length) { 56 | val familyMap = result.values(i) 57 | if (familyMap != null) { 58 | val family = table.familyByIndex(i) 59 | sb.append("\tFamily: " + family.familyName + " (" + familyMap.values.size + " items)\n") 60 | for ((key, value) <- familyMap) { 61 | sb.append("\t\tColumn: " + key + "\n") 62 | sb.append("\t\t\tValue: " + value + "\n") 63 | sb.append("\t\t\tTimestamp: " + result.columnTimestampByNameAsDate(family, key) + "\n") 64 | } 65 | 66 | } 67 | } 68 | sb.toString 69 | } 70 | } 71 | -------------------------------------------------------------------------------- /src/main/scala/com/gravity/hbase/schema/OpBase.scala: -------------------------------------------------------------------------------- 1 | package com.gravity.hbase.schema 2 | 3 | import scala.collection.mutable.Buffer 4 | import org.apache.hadoop.io.Writable 5 | import org.apache.hadoop.hbase.client.{Increment, Delete, Put} 6 | import scala.collection.JavaConversions._ 7 | 8 | /* )\._.,--....,'``. 9 | .b--. /; _.. \ _\ (`._ ,. 10 | `=,-,-'~~~ `----(,_..'--(,_..'`-.;.' */ 11 | 12 | /** 13 | * An individual data modification operation (put, increment, or delete usually) 14 | * These operations are chained together by the client, and then executed in bulk. 15 | * @param table 16 | * @param key 17 | * @param previous 18 | * @tparam T 19 | * @tparam R 20 | */ 21 | abstract class OpBase[T <: HbaseTable[T, R, _], R](val table: HbaseTable[T, R, _], key: Array[Byte], val previous: Buffer[OpBase[T, R]] = Buffer[OpBase[T, R]]()) { 22 | 23 | previous += this 24 | 25 | def +(that:OpBase[T,R]) : OpBase[T,R] 26 | 27 | def put(key: R, writeToWAL: Boolean = true) = { 28 | val po = new PutOp(table, table.rowKeyConverter.toBytes(key), previous, writeToWAL) 29 | po 30 | } 31 | 32 | def increment(key: R) = { 33 | val inc = new IncrementOp(table, table.rowKeyConverter.toBytes(key), previous) 34 | inc 35 | } 36 | 37 | def delete(key: R) = { 38 | val del = new DeleteOp(table, table.rowKeyConverter.toBytes(key), previous) 39 | del 40 | } 41 | 42 | def size = previous.size 43 | 44 | def getOperations: Iterable[Writable] = { 45 | val calls = Buffer[Writable]() 46 | previous.foreach { 47 | case put: PutOp[T, R] => { 48 | calls += put.put 49 | } 50 | case delete: DeleteOp[T, R] => { 51 | calls += delete.delete 52 | } 53 | case increment: IncrementOp[T, R] => { 54 | calls += increment.increment 55 | } 56 | } 57 | 58 | calls 59 | 60 | } 61 | 62 | /** 63 | * This is an experimental call that utilizes a shared instance of a table to flush writes. 64 | */ 65 | def executeBuffered(tableName: String = table.tableName) { 66 | 67 | val (deletes, puts, increments) = prepareOperations 68 | 69 | if (deletes.size == 0 && puts.size == 0 && increments.size == 0) { 70 | } else { 71 | table.withBufferedTable(tableName) { 72 | bufferTable => 73 | if (puts.size > 0) { 74 | bufferTable.put(puts) 75 | } 76 | if (deletes.size > 0) { 77 | bufferTable.delete(deletes) 78 | } 79 | if (increments.size > 0) { 80 | increments.foreach { 81 | increment => 82 | bufferTable.increment(increment) 83 | } 84 | } 85 | } 86 | } 87 | 88 | } 89 | 90 | def prepareOperations = { 91 | val puts = Buffer[Put]() 92 | val deletes = Buffer[Delete]() 93 | val increments = Buffer[Increment]() 94 | 95 | previous.foreach { 96 | case put: PutOp[T, R] => { 97 | if (!put.put.isEmpty) { 98 | puts += put.put 99 | } 100 | } 101 | case delete: DeleteOp[T, R] => { 102 | deletes += delete.delete 103 | } 104 | case increment: IncrementOp[T, R] => { 105 | increments += increment.increment 106 | } 107 | } 108 | 109 | (deletes, puts, increments) 110 | } 111 | 112 | def execute(tableName: String = table.tableName) = { 113 | val (deletes, puts, increments) = prepareOperations 114 | 115 | if (deletes.size == 0 && puts.size == 0 && increments.size == 0) { 116 | //No need to do anything if there are no real operations to execute 117 | } else { 118 | table.withTable(tableName) { 119 | table => 120 | if (puts.size > 0) { 121 | table.put(puts) 122 | //IN THEORY, the operations will happen in order. If not, break this into two different batched calls for deletes and puts 123 | } 124 | if (deletes.size > 0) { 125 | table.delete(deletes) 126 | } 127 | if (increments.size > 0) { 128 | increments.foreach(increment => table.increment(increment)) 129 | } 130 | } 131 | } 132 | 133 | 134 | OpsResult(0, puts.size, increments.size) 135 | } 136 | } 137 | 138 | case class OpsResult(numDeletes: Int, numPuts: Int, numIncrements: Int) 139 | -------------------------------------------------------------------------------- /src/main/scala/com/gravity/hbase/schema/DeserializedResult.scala: -------------------------------------------------------------------------------- 1 | package com.gravity.hbase.schema 2 | 3 | import org.joda.time.DateTime 4 | import scala.collection.mutable.Buffer 5 | 6 | /* )\._.,--....,'``. 7 | .b--. /; _.. \ _\ (`._ ,. 8 | `=,-,-'~~~ `----(,_..'--(,_..'`-.;.' */ 9 | 10 | /** 11 | * The container for the result values deserialized from Hbase. 12 | * @param rowid 13 | * @param famCount 14 | */ 15 | case class DeserializedResult(rowid: AnyRef, famCount: Int) { 16 | 17 | def isEmpty = values.size == 0 18 | 19 | def getRow[R]() = rowid.asInstanceOf[R] 20 | 21 | 22 | def familyValueMap[K, V](fam: ColumnFamily[_, _, _, _, _]) = { 23 | val famMap = family(fam) 24 | if (famMap != null) { 25 | famMap.asInstanceOf[java.util.HashMap[K, V]] 26 | } else { 27 | new java.util.HashMap[K, V]() 28 | } 29 | } 30 | 31 | def familyKeySet[K](fam: ColumnFamily[_, _, _, _, _]) = { 32 | val famMap = family(fam) 33 | if (famMap != null) { 34 | famMap.keySet.asInstanceOf[java.util.Set[K]] 35 | } else { 36 | new java.util.HashSet[K]() 37 | } 38 | } 39 | 40 | def family(family: ColumnFamily[_, _, _, _, _]) = { 41 | values(family.index) 42 | } 43 | 44 | def familyOf(column: Column[_, _, _, _, _]) = family(column.family) 45 | 46 | def familyMap(fam: ColumnFamily[_, _, _, _, _]) = family(fam) 47 | 48 | def hasColumn(column: Column[_, _, _, _, _]) = { 49 | val valueMap = familyOf(column) 50 | if (valueMap == null || valueMap.size == 0) false else true 51 | } 52 | 53 | def columnValue(fam: ColumnFamily[_, _, _, _, _], columnName: AnyRef) = { 54 | val valueMap = family(fam) 55 | if (valueMap == null) { 56 | null 57 | } else { 58 | valueMap.get(columnName) 59 | } 60 | } 61 | 62 | def columnTimestamp(fam: ColumnFamily[_, _, _, _, _], columnName: AnyRef) = { 63 | val res = timestampLookaside(fam.index) 64 | if (res != null) { 65 | val colRes = res.get(columnName) 66 | colRes 67 | } 68 | else { 69 | 0l 70 | } 71 | } 72 | 73 | 74 | def columnTimestampAsDate(column: Column[_, _, _, _, _]) = { 75 | val cts = columnTimestamp(column.family, column.columnNameRef) 76 | if (cts > 0) { 77 | new DateTime(cts) 78 | } else { 79 | null 80 | } 81 | } 82 | 83 | def columnTimestampByName(fam: ColumnFamily[_, _, _, _, _], columnName: AnyRef) = { 84 | val cts = columnTimestamp(fam, columnName) 85 | cts 86 | } 87 | 88 | def columnTimestampByNameAsDate(fam: ColumnFamily[_, _, _, _, _], columnName: AnyRef) = { 89 | val cts = columnTimestamp(fam, columnName) 90 | if (cts > 0) { 91 | new DateTime(cts) 92 | } 93 | else { 94 | null 95 | } 96 | } 97 | 98 | 99 | def columnValueSpecific(column: Column[_, _, _, _, _]) = { 100 | columnValue(column.family, column.columnNameRef) 101 | } 102 | 103 | 104 | var values = new Array[java.util.HashMap[AnyRef, AnyRef]](famCount) 105 | 106 | var timestampLookaside = new Array[java.util.HashMap[AnyRef, Long]](famCount) 107 | 108 | 109 | /**This is a map whose key is the family type, and whose values are maps of column keys to columnvalues paired with their timestamps */ 110 | // val values = new java.util.HashMap[ColumnFamily[_, _, _, _, _], java.util.HashMap[AnyRef, AnyRef]]() 111 | 112 | // val timestampLookaside = new java.util.HashMap[ColumnFamily[_, _, _, _, _], java.util.HashMap[AnyRef, Long]]() 113 | 114 | def add(family: ColumnFamily[_, _, _, _, _], qualifier: AnyRef, value: AnyRef, timeStamp: Long) { 115 | var map = values(family.index) 116 | if (map == null) { 117 | map = new java.util.HashMap[AnyRef, AnyRef]() 118 | values(family.index) = map 119 | } 120 | map.put(qualifier, value) 121 | 122 | var tsMap = timestampLookaside(family.index) 123 | if (tsMap == null) { 124 | tsMap = new java.util.HashMap[AnyRef, Long]() 125 | timestampLookaside(family.index) = tsMap 126 | } 127 | tsMap.put(qualifier, timeStamp) 128 | //Add timestamp lookaside 129 | } 130 | 131 | var errorBuffer: Buffer[(Array[Byte], Array[Byte], Array[Byte], Long)] = _ 132 | 133 | def addErrorBuffer(family: Array[Byte], qualifier: Array[Byte], value: Array[Byte], timestamp: Long) { 134 | if (errorBuffer == null) { 135 | errorBuffer = Buffer() 136 | } 137 | errorBuffer.append((family, qualifier, value, timestamp)) 138 | } 139 | 140 | def hasErrors = (errorBuffer != null) 141 | } 142 | -------------------------------------------------------------------------------- /src/main/scala/com/gravity/hbase/schema/QueryFilter.scala: -------------------------------------------------------------------------------- 1 | /** Licensed to Gravity.com under one 2 | * or more contributor license agreements. See the NOTICE file 3 | * distributed with this work for additional information 4 | * regarding copyright ownership. Gravity.com licenses this file 5 | * to you under the Apache License, Version 2.0 (the 6 | * "License"); you may not use this file except in compliance 7 | * with the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.gravity.hbase.schema 19 | 20 | import org.apache.hadoop.hbase.client._ 21 | import org.apache.hadoop.hbase.util._ 22 | import scala.collection.JavaConversions._ 23 | import org.apache.hadoop.conf.Configuration 24 | import java.io._ 25 | import org.apache.hadoop.io.{BytesWritable, Writable} 26 | import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp 27 | import scala.collection._ 28 | import java.util.NavigableSet 29 | import scala.collection.mutable.Buffer 30 | import org.joda.time.DateTime 31 | import org.apache.hadoop.hbase.filter._ 32 | import org.apache.hadoop.hbase.filter.FilterList.Operator 33 | /* )\._.,--....,'``. 34 | .b--. /; _.. \ _\ (`._ ,. 35 | `=,-,-'~~~ `----(,_..'--(,_..'`-.;.' */ 36 | 37 | @deprecated("Use query2 and filter()") 38 | class QueryFilter[T <: HbaseTable[T,R,_],R](table:HbaseTable[T,R,_]) { 39 | var currentFilter : FilterList = new FilterList(Operator.MUST_PASS_ALL) 40 | 41 | def and = { 42 | if(currentFilter == null) { 43 | currentFilter = new FilterList(Operator.MUST_PASS_ALL) 44 | }else { 45 | val encompassingFilter = new FilterList(Operator.MUST_PASS_ALL) 46 | encompassingFilter.addFilter(currentFilter) 47 | currentFilter = encompassingFilter 48 | } 49 | this 50 | } 51 | 52 | def or = { 53 | if(currentFilter == null) { 54 | currentFilter = new FilterList(Operator.MUST_PASS_ONE) 55 | }else { 56 | val encompassingFilter = new FilterList(Operator.MUST_PASS_ONE) 57 | encompassingFilter.addFilter(currentFilter) 58 | currentFilter = encompassingFilter 59 | } 60 | this 61 | } 62 | 63 | def lessThanColumnKey[F,K,V](family: (T) => ColumnFamily[T,R,F,K,V], value:K)(implicit k:ByteConverter[K]) = { 64 | val valueFilter = new QualifierFilter(CompareOp.LESS_OR_EQUAL, new BinaryComparator(k.toBytes(value))) 65 | val familyFilter = new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(family(table.pops).familyBytes)) 66 | val andFilter = new FilterList(Operator.MUST_PASS_ALL) 67 | andFilter.addFilter(familyFilter) 68 | andFilter.addFilter(valueFilter) 69 | currentFilter.addFilter(familyFilter) 70 | this 71 | } 72 | 73 | def greaterThanColumnKey[F,K,V](family: (T) => ColumnFamily[T,R,F,K,V], value:K)(implicit k:ByteConverter[K]) = { 74 | val andFilter = new FilterList(Operator.MUST_PASS_ALL) 75 | val familyFilter = new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(family(table.pops).familyBytes)) 76 | val valueFilter = new QualifierFilter(CompareOp.GREATER_OR_EQUAL, new BinaryComparator(k.toBytes(value))) 77 | andFilter.addFilter(familyFilter) 78 | andFilter.addFilter(valueFilter) 79 | currentFilter.addFilter(familyFilter) 80 | this 81 | } 82 | 83 | 84 | def betweenColumnKeys[F,K,V](family: (T) => ColumnFamily[T,R,F,K,V], lower: K, upper: K)(implicit f:ByteConverter[K]) = { 85 | val familyFilter = new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(family(table.pops).familyBytes)) 86 | val begin = new QualifierFilter(CompareOp.GREATER_OR_EQUAL, new BinaryComparator(f.toBytes(lower))) 87 | val end = new QualifierFilter(CompareOp.LESS_OR_EQUAL, new BinaryComparator(f.toBytes(upper))) 88 | val filterList = new FilterList(Operator.MUST_PASS_ALL) 89 | filterList.addFilter(familyFilter) 90 | filterList.addFilter(begin) 91 | filterList.addFilter(end) 92 | currentFilter.addFilter(filterList) 93 | 94 | this 95 | } 96 | 97 | def allOfFamilies[F](familyList: ((T) => ColumnFamily[T,R,F,_,_])*) = { 98 | val filterList = new FilterList(Operator.MUST_PASS_ONE) 99 | for(family <- familyList) { 100 | val familyFilter = new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(family(table.pops).familyBytes)) 101 | filterList.addFilter(familyFilter) 102 | } 103 | currentFilter.addFilter(filterList) 104 | this 105 | } 106 | 107 | 108 | 109 | } 110 | 111 | 112 | -------------------------------------------------------------------------------- /src/main/scala/com/gravity/hadoop/hdfs.scala: -------------------------------------------------------------------------------- 1 | package com.gravity.hadoop 2 | 3 | import org.apache.hadoop.conf.Configuration 4 | import org.apache.hadoop.fs.{PathFilter, Path, FileSystem} 5 | import org.apache.hadoop.io.{SequenceFile, Writable} 6 | import java.io._ 7 | import scala.collection.mutable.Buffer 8 | 9 | /** 10 | * Convenience methods for reading and writing files to and from hdfs. 11 | */ 12 | package object hdfs { 13 | implicit def asRichFileSystem(fs: FileSystem) = new RichFileSystem(fs) 14 | 15 | 16 | /** 17 | * Gives you a file writer into the local cluster hdfs instance 18 | * @param relpath The relative path 19 | * @param recreateIfPresent If true, will delete the file if it already exists 20 | * @param work A function that works with the output. The output will be closed when this function goes out of scope. 21 | * @return 22 | */ 23 | def withHdfsWriter(fs: FileSystem, relpath: String, recreateIfPresent: Boolean = true)(work: (BufferedWriter) => Unit) { 24 | val path = new Path(relpath) 25 | val fileSystem = fs 26 | if (recreateIfPresent) { 27 | if (fileSystem.exists(path)) { 28 | fileSystem.delete(path) 29 | } 30 | } 31 | val output = new BufferedWriter(new OutputStreamWriter(fileSystem.create(path))) 32 | try { 33 | work(output) 34 | } finally { 35 | output.close() 36 | } 37 | } 38 | 39 | def perPartSequenceFileKV[K <: Writable, V <: Writable](fs: FileSystem, relpath: String, conf: Configuration,fileBeginsWith:String="part-")(key: K, value: V)(line: (K, V) => Unit) { 40 | val glob = new Path(relpath) 41 | 42 | val files = fs.listStatus(glob, new PathFilter { 43 | override def accept(path: Path) = path.getName.startsWith(fileBeginsWith) 44 | }) 45 | 46 | for (file <- files) { 47 | perSequenceFileKV(fs, file.getPath.toString, conf)(key, value)(line) 48 | } 49 | } 50 | 51 | def perSequenceFileKV[K <: Writable, V <: Writable](fs: FileSystem, relpath: String, conf: Configuration)(key: K, value: V)(line: (K, V) => Unit) { 52 | val reader = new SequenceFile.Reader(fs, new Path(relpath), conf) 53 | 54 | try { 55 | while (reader.next(key, value)) { 56 | line(key, value) 57 | } 58 | } finally { 59 | reader.close() 60 | } 61 | } 62 | 63 | 64 | /** 65 | * Allows you to work with a reader opened into an hdfs file on the test cluster. 66 | * @param relpath The path to the file 67 | * @param work The work you will do 68 | * @tparam A If you want to return a value after the work, here it is. 69 | * @return 70 | */ 71 | def withHdfsReader[A](fs: FileSystem, relpath: String)(work: (BufferedReader) => A): A = { 72 | val path = new Path(relpath) 73 | val input = new BufferedReader(new InputStreamReader(fs.open(path))) 74 | 75 | try { 76 | work(input) 77 | } finally { 78 | input.close() 79 | } 80 | } 81 | 82 | def withHdfsDirectoryReader[A](fs: FileSystem, relpath: String)(work: (BufferedReader) => A): A = { 83 | val path = new Path(relpath) 84 | val input = new BufferedReader(new InputStreamReader(new RichFileSystem(fs).openParts(path))) 85 | try { 86 | work(input) 87 | } finally { 88 | input.close() 89 | } 90 | } 91 | 92 | 93 | /** 94 | * Reads a file into a buffer, allowing you to decide what's in the buffer depending on the output of the linereader function 95 | * @param relpath Path to local hdfs buffer 96 | * @param linereader Function to return an element in the buffer, given the line fo the file 97 | * @tparam A 98 | * @return 99 | */ 100 | def perHdfsLineToSeq[A](fs: FileSystem, relpath: String)(linereader: (String) => A): Seq[A] = { 101 | val result = Buffer[A]() 102 | withHdfsReader(fs, relpath) { 103 | input => 104 | var done = false 105 | while (!done) { 106 | val line = input.readLine() 107 | if (line == null) { 108 | done = true 109 | } else { 110 | result += linereader(line) 111 | } 112 | } 113 | } 114 | result.toSeq 115 | } 116 | 117 | 118 | /** 119 | * Reads a file line by line. If you want to have the results in a buffer, use perHdfsLineToSeq 120 | * @param relpath 121 | * @param linereader 122 | * @tparam A 123 | * @return 124 | */ 125 | def perHdfsLine[A](fs: FileSystem, relpath: String)(linereader: (String) => Unit) { 126 | withHdfsReader(fs, relpath) { 127 | input => 128 | var done = false 129 | while (!done) { 130 | val line = input.readLine() 131 | if (line == null) { 132 | done = true 133 | } else { 134 | linereader(line) 135 | } 136 | } 137 | } 138 | } 139 | 140 | /** 141 | * For each line in a directory of files 142 | * @param relpath Path to files (or glob path) 143 | * @param linereader Will be invoked once per line with a string representation 144 | * @return Bupkiss 145 | */ 146 | def perHdfsDirectoryLine(fs: FileSystem, relpath: String)(linereader: (String) => Unit) { 147 | withHdfsDirectoryReader(fs, relpath) { 148 | input => 149 | var done = false 150 | while (!done) { 151 | val line = input.readLine() 152 | if (line == null) { 153 | done = true 154 | } else { 155 | linereader(line) 156 | } 157 | } 158 | } 159 | } 160 | } -------------------------------------------------------------------------------- /src/main/scala/com/gravity/hbase/schema/ScanQuery.scala: -------------------------------------------------------------------------------- 1 | /** Licensed to Gravity.com under one 2 | * or more contributor license agreements. See the NOTICE file 3 | * distributed with this work for additional information 4 | * regarding copyright ownership. Gravity.com licenses this file 5 | * to you under the Apache License, Version 2.0 (the 6 | * "License"); you may not use this file except in compliance 7 | * with the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.gravity.hbase.schema 19 | 20 | import org.apache.hadoop.hbase.client._ 21 | import org.apache.hadoop.hbase.util._ 22 | import scala.collection.JavaConversions._ 23 | import org.apache.hadoop.conf.Configuration 24 | import java.io._ 25 | import org.apache.hadoop.io.{BytesWritable, Writable} 26 | import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp 27 | import org.apache.hadoop.hbase.filter.{Filter, FilterList, SingleColumnValueFilter} 28 | import scala.collection._ 29 | import java.util.NavigableSet 30 | import scala.collection.mutable.Buffer 31 | import org.joda.time.DateTime 32 | 33 | /* )\._.,--....,'``. 34 | .b--. /; _.. \ _\ (`._ ,. 35 | `=,-,-'~~~ `----(,_..'--(,_..'`-.;.' */ 36 | 37 | class ScanQuery[T <: HbaseTable[T, R,RR], R, RR <: HRow[T,R]](table: HbaseTable[T, R,RR]) { 38 | val scan = new Scan() 39 | scan.setCaching(100) 40 | scan.setMaxVersions(1) 41 | 42 | val filterBuffer = scala.collection.mutable.Buffer[Filter]() 43 | 44 | def executeWithCaching(operator: FilterList.Operator = FilterList.Operator.MUST_PASS_ALL, ttl: Int = 30): Seq[RR] = { 45 | completeScanner(operator) 46 | val results = table.cache.getScanResult(scan) match { 47 | case Some(result) => { 48 | result 49 | } 50 | case None => { 51 | val results = scala.collection.mutable.Buffer[RR]() 52 | table.withTable() { 53 | htable => 54 | val scanner = htable.getScanner(scan) 55 | try { 56 | for (result <- scanner) { 57 | results += table.buildRow(result) 58 | } 59 | table.cache.putScanResult(scan, results.toSeq, ttl) 60 | results 61 | } finally { 62 | scanner.close() 63 | } 64 | } 65 | } 66 | } 67 | 68 | results 69 | } 70 | 71 | def execute(handler: (RR) => Unit, operator: FilterList.Operator = FilterList.Operator.MUST_PASS_ALL) { 72 | table.withTable() { 73 | htable => 74 | completeScanner(operator) 75 | val scanner = htable.getScanner(scan) 76 | 77 | try { 78 | for (result <- scanner) { 79 | handler(table.buildRow(result)) 80 | } 81 | } finally { 82 | scanner.close() 83 | } 84 | } 85 | } 86 | 87 | /* 88 | Prepares the scanner for use by chaining the filters together. Should be called immediately before passing the scanner to the table. 89 | */ 90 | def completeScanner(operator: FilterList.Operator = FilterList.Operator.MUST_PASS_ALL) { 91 | if (filterBuffer.size > 0) { 92 | val filterList = new FilterList(operator) 93 | filterBuffer.foreach {filter => filterList.addFilter(filter)} 94 | scan.setFilter(filterList) 95 | } 96 | } 97 | 98 | def executeToSeq[I](handler: (RR) => I, operator: FilterList.Operator = FilterList.Operator.MUST_PASS_ALL): Seq[I] = { 99 | val results = Buffer[I]() 100 | 101 | table.withTable() { 102 | htable => 103 | completeScanner(operator) 104 | val scanner = htable.getScanner(scan) 105 | 106 | try { 107 | for (result <- scanner; if (result != null)) { 108 | results += handler(table.buildRow(result)) 109 | } 110 | } finally { 111 | scanner.close() 112 | } 113 | } 114 | 115 | results.toSeq 116 | } 117 | 118 | def withFamily[F, K, V](family: FamilyExtractor[T, R, F, K, V]) = { 119 | val fam = family(table.pops) 120 | scan.addFamily(fam.familyBytes) 121 | this 122 | } 123 | 124 | def withColumn[F, K, V](column: ColumnExtractor[T, R, F, K, V]) = { 125 | val col = column(table.pops) 126 | scan.addColumn(col.familyBytes, col.columnBytes) 127 | 128 | this 129 | } 130 | 131 | def withFilter(f: () => Filter) = { 132 | filterBuffer.add(f()) 133 | this 134 | } 135 | 136 | def addFilter(filter: Filter) = withFilter(() => filter) 137 | 138 | def withColumnOp[F, K, V](column: ColumnExtractor[T, R, F, K, V], compareOp: CompareOp, value: Option[V], excludeIfNull: Boolean)(implicit c: ByteConverter[V]) = { 139 | val col = column(table.pops) 140 | val filter = new SingleColumnValueFilter( 141 | col.familyBytes, 142 | col.columnBytes, 143 | compareOp, 144 | value match {case Some(v) => c.toBytes(v); case None => new Array[Byte](0)} 145 | ) 146 | filter.setFilterIfMissing(excludeIfNull) 147 | filterBuffer += filter 148 | this 149 | } 150 | 151 | def withStartKey[R](key: R)(implicit c: ByteConverter[R]) = {scan.setStartRow(c.toBytes(key)); this} 152 | 153 | def withEndKey[R](key: R)(implicit c: ByteConverter[R]) = {scan.setStopRow(c.toBytes(key)); this} 154 | 155 | def withCaching(rowsToCache: Int) = {scan.setCaching(rowsToCache); this;} 156 | } 157 | 158 | 159 | -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4.0.0 4 | com.gravity 5 | gravity-hpaste 6 | 0.1.13-SNAPSHOT 7 | jar 8 | hpaste 9 | http://github.com/GravityLabs/HPaste 10 | Scala mapper for hbase 11 | 2011 12 | 13 | Gravity 14 | http://www.gravity.com/ 15 | 16 | 17 | 18 | org.sonatype.oss 19 | oss-parent 20 | 7 21 | 22 | 23 | 24 | 25 | Lemmsjid 26 | Chris Bissell 27 | chris@gravity.com 28 | http://github.com/Lemmsjid 29 | Gravity 30 | http://www.gravity.com/ 31 | 32 | developer 33 | architect 34 | 35 | -8 36 | 37 | 38 | erraggy 39 | Robbie Coleman 40 | robbie@gravity.com 41 | http://robbie.robnrob.com/ 42 | Gravity 43 | http://www.gravity.com/ 44 | 45 | developer 46 | 47 | -8 48 | 49 | http://1.gravatar.com/avatar/dc77b368ec1f077dcc4aca3b9c003d2d 50 | 51 | 52 | 53 | 54 | scm:git:git@github.com:GravityLabs/HPaste.git 55 | scm:git:git@github.com:GravityLabs/HPaste.git 56 | scm:git:git@github.com:GravityLabs/HPaste.git 57 | 58 | 59 | 60 | Apache 2 61 | http://www.apache.org/licenses/LICENSE-2.0.txt 62 | repo 63 | A business-friendly OSS license 64 | 65 | 66 | 67 | 68 | 1.5 69 | 1.5 70 | UTF-8 71 | 2.9.1 72 | 73 | 74 | 75 | 76 | joda-time 77 | joda-time 78 | 1.6.1 79 | 80 | 81 | org.apache.hbase 82 | hbase 83 | 0.90.4 84 | 85 | 86 | org.slf4j 87 | slf4j-log4j12 88 | 89 | 90 | log4j 91 | log4j 92 | 93 | 94 | org.jruby 95 | jruby-complete 96 | 97 | 98 | 99 | 100 | org.apache.hbase 101 | hbase 102 | 0.90.4 103 | test-jar 104 | test 105 | 106 | 107 | org.jruby 108 | jruby-complete 109 | 110 | 111 | 112 | 113 | org.apache.hadoop 114 | hadoop-test 115 | 0.20.2 116 | test 117 | 118 | 119 | org.scala-lang 120 | scala-library 121 | ${scala.version} 122 | 123 | 124 | 125 | 126 | junit 127 | junit 128 | 4.8.1 129 | test 130 | 131 | 132 | 133 | 134 | 135 | 136 | org.apache.maven.plugins 137 | maven-source-plugin 138 | 2.1.2 139 | 140 | 141 | 142 | jar 143 | 144 | 145 | 146 | 147 | 148 | org.scala-tools 149 | maven-scala-plugin 150 | 2.15.0 151 | 152 | 153 | 154 | compile 155 | testCompile 156 | 157 | 158 | 159 | -make:transitive 160 | -dependencyfile 161 | ${project.build.directory}/.scala_dependencies 162 | 163 | 164 | 165 | 166 | 167 | 168 | org.apache.maven.plugins 169 | maven-surefire-plugin 170 | 2.6 171 | 172 | -Xmx1024m 173 | false 174 | true 175 | 176 | **/*Test.* 177 | 178 | 179 | **/*IT.* 180 | 181 | 182 | 183 | 184 | org.apache.maven.plugins 185 | maven-release-plugin 186 | 2.2.2 187 | 188 | 189 | 190 | 191 | 192 | 193 | release-sign-artifacts 194 | 195 | 196 | performRelease 197 | true 198 | 199 | 200 | 201 | 202 | 203 | org.apache.maven.plugins 204 | maven-gpg-plugin 205 | 1.4 206 | 207 | 208 | sign-artifacts 209 | verify 210 | 211 | sign 212 | 213 | 214 | 215 | 216 | 217 | 218 | 219 | 220 | 221 | 222 | -------------------------------------------------------------------------------- /src/main/java/com/gravity/hadoop/GravityTableOutputFormat.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2009 The Apache Software Foundation 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | package com.gravity.hadoop; 21 | 22 | import org.apache.commons.logging.Log; 23 | import org.apache.commons.logging.LogFactory; 24 | import org.apache.hadoop.conf.Configurable; 25 | import org.apache.hadoop.conf.Configuration; 26 | import org.apache.hadoop.hbase.HBaseConfiguration; 27 | import org.apache.hadoop.hbase.HConstants; 28 | import org.apache.hadoop.hbase.client.Delete; 29 | import org.apache.hadoop.hbase.client.HTable; 30 | import org.apache.hadoop.hbase.client.Put; 31 | import org.apache.hadoop.hbase.mapreduce.TableOutputCommitter; 32 | import org.apache.hadoop.hbase.zookeeper.ZKUtil; 33 | import org.apache.hadoop.io.Writable; 34 | import org.apache.hadoop.mapreduce.*; 35 | 36 | import java.io.IOException; 37 | 38 | /** 39 | * This is called the GravityTableOutputFormat because it does not call 40 | * HConnectionManager.deleteAllConnections(true); 41 | * when the job is complete (which makes it impossible to unit test workflows) 42 | * 43 | * Convert Map/Reduce output and write it to an HBase table. The KEY is ignored 44 | * while the output value must be either a {@link Put} or a 45 | * {@link Delete} instance. 46 | * 47 | * @param The type of the key. Ignored in this class. 48 | */ 49 | public class GravityTableOutputFormat extends OutputFormat 50 | implements Configurable { 51 | 52 | private final Log LOG = LogFactory.getLog(GravityTableOutputFormat.class); 53 | 54 | /** Job parameter that specifies the output table. */ 55 | public static final String OUTPUT_TABLE = "hbase.mapred.outputtable"; 56 | 57 | /** 58 | * Optional job parameter to specify a peer cluster. 59 | * Used specifying remote cluster when copying between hbase clusters (the 60 | * source is picked up from hbase-site.xml). 61 | */ 62 | public static final String QUORUM_ADDRESS = "hbase.mapred.output.quorum"; 63 | 64 | /** Optional specification of the rs class name of the peer cluster */ 65 | public static final String 66 | REGION_SERVER_CLASS = "hbase.mapred.output.rs.class"; 67 | /** Optional specification of the rs impl name of the peer cluster */ 68 | public static final String 69 | REGION_SERVER_IMPL = "hbase.mapred.output.rs.impl"; 70 | 71 | 72 | /** The configuration. */ 73 | private Configuration conf = null; 74 | 75 | private HTable table; 76 | 77 | /** 78 | * Writes the reducer output to an HBase table. 79 | * 80 | * @param The type of the key. 81 | */ 82 | protected static class TableRecordWriter 83 | extends RecordWriter { 84 | 85 | /** The table to write to. */ 86 | private HTable table; 87 | 88 | /** 89 | * Instantiate a TableRecordWriter with the HBase HClient for writing. 90 | * 91 | * @param table The table to write to. 92 | */ 93 | public TableRecordWriter(HTable table) { 94 | this.table = table; 95 | } 96 | 97 | /** 98 | * Closes the writer, in this case flush table commits. 99 | * 100 | * @param context The context. 101 | * @throws IOException When closing the writer fails. 102 | * @see org.apache.hadoop.mapreduce.RecordWriter#close(org.apache.hadoop.mapreduce.TaskAttemptContext) 103 | */ 104 | @Override 105 | public void close(TaskAttemptContext context) 106 | throws IOException { 107 | table.flushCommits(); 108 | // The following call will shutdown all connections to the cluster from 109 | // this JVM. It will close out our zk session otherwise zk wil log 110 | // expired sessions rather than closed ones. If any other HTable instance 111 | // running in this JVM, this next call will cause it damage. Presumption 112 | // is that the above this.table is only instance. 113 | 114 | // HConnectionManager.deleteAllConnections(true); 115 | 116 | } 117 | 118 | /** 119 | * Writes a key/value pair into the table. 120 | * 121 | * @param key The key. 122 | * @param value The value. 123 | * @throws IOException When writing fails. 124 | * @see org.apache.hadoop.mapreduce.RecordWriter#write(java.lang.Object, java.lang.Object) 125 | */ 126 | @Override 127 | public void write(KEY key, Writable value) 128 | throws IOException { 129 | if (value instanceof Put) { 130 | ((Put)value).setWriteToWAL(false); 131 | this.table.put(new Put((Put)value)); 132 | 133 | } 134 | else if (value instanceof Delete) this.table.delete(new Delete((Delete)value)); 135 | else throw new IOException("Pass a Delete or a Put"); 136 | } 137 | } 138 | 139 | /** 140 | * Creates a new record writer. 141 | * 142 | * @param context The current task context. 143 | * @return The newly created writer instance. 144 | * @throws IOException When creating the writer fails. 145 | * @throws InterruptedException When the jobs is cancelled. 146 | * @see org.apache.hadoop.mapreduce.lib.output.FileOutputFormat#getRecordWriter(org.apache.hadoop.mapreduce.TaskAttemptContext) 147 | */ 148 | @Override 149 | public RecordWriter getRecordWriter( 150 | TaskAttemptContext context) 151 | throws IOException, InterruptedException { 152 | return new TableRecordWriter(this.table); 153 | } 154 | 155 | /** 156 | * Checks if the output target exists. 157 | * 158 | * @param context The current context. 159 | * @throws IOException When the check fails. 160 | * @throws InterruptedException When the job is aborted. 161 | * @see org.apache.hadoop.mapreduce.OutputFormat#checkOutputSpecs(org.apache.hadoop.mapreduce.JobContext) 162 | */ 163 | @Override 164 | public void checkOutputSpecs(JobContext context) throws IOException, 165 | InterruptedException { 166 | // TODO Check if the table exists? 167 | 168 | } 169 | 170 | /** 171 | * Returns the output committer. 172 | * 173 | * @param context The current context. 174 | * @return The committer. 175 | * @throws IOException When creating the committer fails. 176 | * @throws InterruptedException When the job is aborted. 177 | * @see org.apache.hadoop.mapreduce.OutputFormat#getOutputCommitter(org.apache.hadoop.mapreduce.TaskAttemptContext) 178 | */ 179 | @Override 180 | public OutputCommitter getOutputCommitter(TaskAttemptContext context) 181 | throws IOException, InterruptedException { 182 | return new TableOutputCommitter(); 183 | } 184 | 185 | public Configuration getConf() { 186 | return conf; 187 | } 188 | 189 | @Override 190 | public void setConf(Configuration otherConf) { 191 | // otherConf.set("hbase.client.keyvalue.maxsize","104857600"); 192 | this.conf = HBaseConfiguration.create(otherConf); 193 | String tableName = this.conf.get(OUTPUT_TABLE); 194 | String address = this.conf.get(QUORUM_ADDRESS); 195 | String serverClass = this.conf.get(REGION_SERVER_CLASS); 196 | String serverImpl = this.conf.get(REGION_SERVER_IMPL); 197 | 198 | try { 199 | if (address != null) { 200 | ZKUtil.applyClusterKeyToConf(this.conf, address); 201 | } 202 | if (serverClass != null) { 203 | this.conf.set(HConstants.REGION_SERVER_CLASS, serverClass); 204 | this.conf.set(HConstants.REGION_SERVER_IMPL, serverImpl); 205 | } 206 | this.table = new HTable(this.conf, tableName); 207 | // this.table.setWriteBufferSize(1024 * 1024 * 12); 208 | this.table.setAutoFlush(false); 209 | LOG.info("Created table instance for " + tableName); 210 | } catch(IOException e) { 211 | LOG.error(e); 212 | } 213 | } 214 | } 215 | -------------------------------------------------------------------------------- /src/main/scala/com/gravity/hbase/schema.scala: -------------------------------------------------------------------------------- 1 | /** Licensed to Gravity.com under one 2 | * or more contributor license agreements. See the NOTICE file 3 | * distributed with this work for additional information 4 | * regarding copyright ownership. Gravity.com licenses this file 5 | * to you under the Apache License, Version 2.0 (the 6 | * "License"); you may not use this file except in compliance 7 | * with the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.gravity.hbase 19 | 20 | import org.apache.hadoop.hbase.util.Bytes 21 | import scala.collection._ 22 | import org.joda.time.{DateMidnight, DateTime} 23 | import org.apache.hadoop.io.BytesWritable 24 | import java.io._ 25 | 26 | /* )\._.,--....,'``. 27 | .b--. /; _.. \ _\ (`._ ,. 28 | `=,-,-'~~~ `----(,_..'--(,_..'`-.;.' */ 29 | 30 | class AnyNotSupportedException() extends Exception("Any not supported") 31 | 32 | trait AnyConverterSignal 33 | 34 | /** 35 | * This is the standard set of types that can be auto converted into hbase values (they work as families, columns, and values) 36 | */ 37 | package object schema { 38 | 39 | def toBytesWritable[T](item: T)(implicit c: ByteConverter[T]) = { 40 | c.toBytesWritable(item) 41 | } 42 | 43 | def fromBytesWritable[T](bytes: BytesWritable)(implicit c: ByteConverter[T]) = { 44 | c.fromBytesWritable(bytes) 45 | } 46 | 47 | 48 | type FamilyExtractor[T <: HbaseTable[T, R, _], R, F, K, V] = (T) => ColumnFamily[T, R, F, K, V] 49 | type ColumnExtractor[T <: HbaseTable[T, R, _], R, F, K, V] = (T) => Column[T, R, F, K, V] 50 | 51 | implicit object AnyConverter extends ByteConverter[Any] with AnyConverterSignal { 52 | override def toBytes(t: Any) = throw new AnyNotSupportedException() 53 | 54 | override def fromBytes(bytes: Array[Byte], offset: Int, length: Int) = throw new AnyNotSupportedException() 55 | } 56 | 57 | implicit object StringConverter extends ByteConverter[String] { 58 | override def toBytes(t: String) = Bytes.toBytes(t) 59 | 60 | override def fromBytes(bytes: Array[Byte], offset: Int, length: Int) = Bytes.toString(bytes, offset, length) 61 | } 62 | 63 | implicit object StringSeqConverter extends SeqConverter[String] 64 | 65 | implicit object StringSetConverter extends SetConverter[String] 66 | 67 | 68 | implicit object IntConverter extends ByteConverter[Int] { 69 | override def toBytes(t: Int) = Bytes.toBytes(t) 70 | 71 | override def fromBytes(bytes: Array[Byte], offset: Int, length: Int) = Bytes.toInt(bytes, offset, length) 72 | } 73 | 74 | implicit object IntSeqConverter extends SeqConverter[Int] 75 | 76 | implicit object IntSetConverter extends SetConverter[Int] 77 | 78 | implicit object ShortConverter extends ByteConverter[Short] { 79 | override def toBytes(t: Short) = Bytes.toBytes(t) 80 | 81 | override def fromBytes(bytes: Array[Byte], offset: Int, length: Int) = Bytes.toShort(bytes, offset, length) 82 | } 83 | 84 | implicit object ShortSeqConverter extends SeqConverter[Short] 85 | 86 | implicit object ShortSetConverter extends SetConverter[Short] 87 | 88 | implicit object BooleanConverter extends ByteConverter[Boolean] { 89 | override def toBytes(t: Boolean) = Bytes.toBytes(t) 90 | 91 | override def fromBytes(bytes: Array[Byte], offset: Int, length: Int) = { 92 | (bytes(offset) != 0) 93 | } 94 | } 95 | 96 | implicit object BooleanSeqConverter extends SeqConverter[Boolean] 97 | 98 | implicit object BooleanSetConverter extends SetConverter[Boolean] 99 | 100 | implicit object LongConverter extends ByteConverter[Long] { 101 | override def toBytes(t: Long) = Bytes.toBytes(t) 102 | 103 | override def fromBytes(bytes: Array[Byte], offset: Int, length: Int) = Bytes.toLong(bytes, offset, length) 104 | } 105 | 106 | implicit object LongSeqConverter extends SeqConverter[Long] 107 | 108 | implicit object LongSetConverter extends SetConverter[Long] 109 | 110 | implicit object DoubleConverter extends ByteConverter[Double] { 111 | override def toBytes(t: Double) = Bytes.toBytes(t) 112 | 113 | override def fromBytes(bytes: Array[Byte], offset: Int, length: Int) = Bytes.toDouble(bytes, offset) 114 | } 115 | 116 | implicit object DoubleSeqConverter extends SeqConverter[Double] 117 | 118 | implicit object DoubleSetConverter extends SetConverter[Double] 119 | 120 | 121 | implicit object FloatConverter extends ByteConverter[Float] { 122 | override def toBytes(t: Float) = Bytes.toBytes(t) 123 | 124 | override def fromBytes(bytes: Array[Byte], offset: Int, length: Int) = Bytes.toFloat(bytes, offset) 125 | } 126 | 127 | implicit object FloatSeqConverter extends SeqConverter[Float] 128 | 129 | implicit object FloatSetConverter extends SetConverter[Float] 130 | 131 | 132 | implicit object CommaSetConverter extends ByteConverter[CommaSet] { 133 | val SPLITTER = ",".r 134 | 135 | override def toBytes(t: CommaSet) = Bytes.toBytes(t.items.mkString(",")) 136 | 137 | override def fromBytes(bytes: Array[Byte], offset: Int, length: Int) = new CommaSet(SPLITTER.split(Bytes.toString(bytes, offset, length)).toSet) 138 | } 139 | 140 | implicit object CommaSetSeqConverter extends SeqConverter[CommaSet] 141 | 142 | implicit object CommaSetSetConverter extends SetConverter[CommaSet] 143 | 144 | 145 | implicit object YearDayConverter extends ByteConverter[YearDay] { 146 | val SPLITTER = "_".r 147 | 148 | override def toBytes(t: YearDay) = Bytes.toBytes(t.year.toString + "_" + t.day.toString) 149 | 150 | override def fromBytes(bytes: Array[Byte], offset: Int, length: Int) = { 151 | val strRep = Bytes.toString(bytes, offset, length) 152 | val strRepSpl = SPLITTER.split(strRep) 153 | val year = strRepSpl(0).toInt 154 | val day = strRepSpl(1).toInt 155 | YearDay(year, day) 156 | } 157 | } 158 | 159 | implicit object YearDaySeqConverter extends SeqConverter[YearDay] 160 | 161 | implicit object YearDaySetConverter extends SetConverter[YearDay] 162 | 163 | implicit object DateMidnightConverter extends ComplexByteConverter[DateMidnight] { 164 | override def write(dm: DateMidnight, output: PrimitiveOutputStream) { 165 | output.writeLong(dm.getMillis) 166 | } 167 | 168 | override def read(input: PrimitiveInputStream) = new DateMidnight(input.readLong()) 169 | 170 | 171 | def apply(year: Int, day: Int) = new DateMidnight().withYear(year).withDayOfYear(day) 172 | } 173 | 174 | implicit object DateTimeConverter extends ComplexByteConverter[DateTime] { 175 | override def write(dm: DateTime, output: PrimitiveOutputStream) { 176 | output.writeLong(dm.getMillis) 177 | } 178 | 179 | override def read(input: PrimitiveInputStream) = new DateTime(input.readLong()) 180 | } 181 | 182 | implicit object DateTimeSeqConverter extends SeqConverter[DateTime] 183 | 184 | implicit object DateTimeSetConverter extends SetConverter[DateTime] 185 | 186 | implicit object DateMidnightSeqConverter extends SeqConverter[DateMidnight] 187 | 188 | implicit object DateMidnightSetConverter extends SetConverter[DateMidnight] 189 | 190 | implicit object StringLongMap extends MapConverter[String, Long] 191 | implicit object ImmutableStringLongMap extends ImmutableMapConverter[String, Long] 192 | implicit object MutableStringLongMap extends MutableMapConverter[String, Long] 193 | 194 | implicit object StringStringMap extends MapConverter[String,String] 195 | implicit object MutableStringStringMap extends MutableMapConverter[String,String] 196 | implicit object ImmutableStringStringMap extends ImmutableMapConverter[String,String] 197 | 198 | /* 199 | Helper function to make byte arrays out of arbitrary values. 200 | */ 201 | def makeBytes(writer: (PrimitiveOutputStream) => Unit): Array[Byte] = { 202 | val bos = new ByteArrayOutputStream() 203 | val dataOutput = new PrimitiveOutputStream(bos) 204 | writer(dataOutput) 205 | bos.toByteArray 206 | } 207 | 208 | def makeWritable(writer: (PrimitiveOutputStream) => Unit): BytesWritable = new BytesWritable(makeBytes(writer)) 209 | 210 | def readBytes[T](bytes: Array[Byte])(reader: (PrimitiveInputStream) => T): T = { 211 | val bis = new ByteArrayInputStream(bytes) 212 | val dis = new PrimitiveInputStream(bis) 213 | val results = reader(dis) 214 | dis.close() 215 | results 216 | } 217 | 218 | def readWritable[T](bytesWritable: BytesWritable)(reader: (PrimitiveInputStream) => T): T = readBytes(bytesWritable.getBytes)(reader) 219 | 220 | } -------------------------------------------------------------------------------- /src/main/scala/com/gravity/hbase/schema/Query.scala: -------------------------------------------------------------------------------- 1 | /** Licensed to Gravity.com under one 2 | * or more contributor license agreements. See the NOTICE file 3 | * distributed with this work for additional information 4 | * regarding copyright ownership. Gravity.com licenses this file 5 | * to you under the Apache License, Version 2.0 (the 6 | * "License"); you may not use this file except in compliance 7 | * with the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.gravity.hbase.schema 19 | 20 | import org.apache.hadoop.hbase.client._ 21 | import org.apache.hadoop.hbase.util._ 22 | import scala.collection.JavaConversions._ 23 | import org.apache.hadoop.conf.Configuration 24 | import java.io._ 25 | import org.apache.hadoop.io.{BytesWritable, Writable} 26 | import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp 27 | import org.apache.hadoop.hbase.filter.{Filter, FilterList, SingleColumnValueFilter} 28 | import scala.collection._ 29 | import java.util.NavigableSet 30 | import scala.collection.mutable.Buffer 31 | import org.joda.time.DateTime 32 | 33 | /* )\._.,--....,'``. 34 | .b--. /; _.. \ _\ (`._ ,. 35 | `=,-,-'~~~ `----(,_..'--(,_..'`-.;.' */ 36 | 37 | @deprecated 38 | class Query[T <: HbaseTable[T, R,RR], R, RR <: HRow[T,R]](table: HbaseTable[T, R, RR]) { 39 | 40 | val keys = Buffer[Array[Byte]]() 41 | val families = Buffer[Array[Byte]]() 42 | val columns = Buffer[(Array[Byte], Array[Byte])]() 43 | 44 | 45 | def withKey(key: R)= { 46 | keys += table.rowKeyConverter.toBytes(key) 47 | this 48 | } 49 | 50 | def withKeys(keys: Set[R]) = { 51 | for (key <- keys) { 52 | withKey(key) 53 | } 54 | this 55 | } 56 | 57 | def withColumnFamily[F, K, V](family: (T) => ColumnFamily[T, R, F, K, V]): Query[T, R, RR] = { 58 | val fam = family(table.pops) 59 | families += fam.familyBytes 60 | this 61 | } 62 | 63 | def withColumn[F, K, V](family: (T) => ColumnFamily[T, R, F, K, V], columnName: K): Query[T, R, RR] = { 64 | val fam = family(table.pops) 65 | columns += (fam.familyBytes -> fam.keyConverter.toBytes(columnName)) 66 | this 67 | } 68 | 69 | def withColumn[F, K, V](column: (T) => Column[T, R, F, K, V]): Query[T, R, RR] = { 70 | val col = column(table.pops) 71 | columns += (col.familyBytes -> col.columnBytes) 72 | this 73 | } 74 | 75 | def single(tableName: String = table.tableName, ttl: Int = 30, skipCache: Boolean = true) = singleOption(tableName, ttl, skipCache, false).get 76 | 77 | 78 | def singleOption(tableName: String = table.tableName, ttl: Int = 30, skipCache: Boolean = true, noneOnEmpty: Boolean = true): Option[RR] = { 79 | require(keys.size == 1, "Calling single() with more than one key") 80 | val get = new Get(keys.head) 81 | get.setMaxVersions(1) 82 | for (family <- families) { 83 | get.addFamily(family) 84 | } 85 | for ((columnFamily, column) <- columns) { 86 | get.addColumn(columnFamily, column) 87 | } 88 | 89 | val fromCache = if (skipCache) None else table.cache.getResult(get) 90 | 91 | fromCache match { 92 | case Some(result) => Some(result) 93 | case None => { 94 | table.withTableOption(tableName) { 95 | case Some(htable) => { 96 | val result = htable.get(get) 97 | if (noneOnEmpty && result.isEmpty) { 98 | None 99 | } else { 100 | val qr = table.buildRow(result) 101 | if (!skipCache && !result.isEmpty) table.cache.putResult(get, qr, ttl) 102 | Some(qr) 103 | } 104 | } 105 | case None => None 106 | } 107 | } 108 | } 109 | 110 | } 111 | 112 | def execute(tableName: String = table.tableName, ttl: Int = 30, skipCache: Boolean = true): Seq[RR] = { 113 | if (keys.isEmpty) return Seq.empty[RR] // no keys..? nothing to see here... move along... move along. 114 | 115 | val results = Buffer[RR]() // buffer for storing all results retrieved 116 | 117 | // if we are utilizing cache, we'll need to be able to recall the `Get' later to use as the cache key 118 | val getsByKey = if (skipCache) mutable.Map.empty[String, Get] else mutable.Map[String, Get]() 119 | 120 | if (!skipCache) getsByKey.sizeHint(keys.size) // perf optimization 121 | 122 | // buffer for all `Get's that really need to be gotten 123 | val cacheMisses = Buffer[Get]() 124 | 125 | val gets = buildGetsAndCheckCache(skipCache) { 126 | case (get: Get, key: Array[Byte]) => if (!skipCache) getsByKey.put(new String(key), get) 127 | } { 128 | case (qropt: Option[RR], get: Get) => if (!skipCache) { 129 | qropt match { 130 | case Some(result) => results += result // got it! place it in our result buffer 131 | case None => cacheMisses += get // missed it! place the get in the buffer 132 | } 133 | } 134 | } 135 | 136 | // identify what still needs to be `Get'ed ;-} 137 | val hbaseGets = if (skipCache) gets else cacheMisses 138 | 139 | if (!hbaseGets.isEmpty) { 140 | // only do this if we have something to do 141 | table.withTable(tableName) { 142 | htable => 143 | htable.get(hbaseGets).foreach(res => { 144 | if (res != null && !res.isEmpty) { 145 | // ignore empty results 146 | val qr = table.buildRow(res) // construct query result 147 | 148 | // now is where we need to retrive the 'get' used for this result so that we can 149 | // pass this 'get' as the key for our local cache 150 | if (!skipCache) table.cache.putResult(getsByKey(new String(res.getRow)), qr, ttl) 151 | results += qr // place it in our result buffer 152 | } 153 | }) 154 | } 155 | } 156 | 157 | results.toSeq // DONE! 158 | } 159 | 160 | def executeMap(tableName: String = table.tableName, ttl: Int = 30, skipCache: Boolean = true): Map[R, RR] = { 161 | if (keys.isEmpty) return Map.empty[R, RR] // don't get all started with nothing to do 162 | 163 | // init our result map and give it a hint of the # of keys we have 164 | val resultMap = mutable.Map[R, RR]() 165 | resultMap.sizeHint(keys.size) // perf optimization 166 | 167 | // if we are utilizing cache, we'll need to be able to recall the `Get' later to use as the cache key 168 | val getsByKey = if (skipCache) mutable.Map.empty[String, Get] else mutable.Map[String, Get]() 169 | 170 | if (!skipCache) getsByKey.sizeHint(keys.size) // perf optimization 171 | 172 | // buffer for all `Get's that really need to be gotten 173 | val cacheMisses = Buffer[Get]() 174 | 175 | val gets = buildGetsAndCheckCache(skipCache) { 176 | case (get: Get, key: Array[Byte]) => if (!skipCache) getsByKey.put(new String(key), get) 177 | } { 178 | case (qropt: Option[RR], get: Get) => if (!skipCache) { 179 | qropt match { 180 | case Some(result) => resultMap.put(result.rowid, result) // got it! place it in our result map 181 | case None => cacheMisses += get // missed it! place the get in the buffer 182 | } 183 | } 184 | } 185 | 186 | // identify what still needs to be `Get'ed ;-} 187 | val hbaseGets = if (skipCache) gets else cacheMisses 188 | 189 | if (!hbaseGets.isEmpty) { 190 | // only do this if we have something to do 191 | table.withTable(tableName) { 192 | htable => 193 | htable.get(hbaseGets).foreach(res => { 194 | if (res != null && !res.isEmpty) { 195 | // ignore empty results 196 | val qr = table.buildRow(res) // construct query result 197 | 198 | // now is where we need to retrive the 'get' used for this result so that we can 199 | // pass this 'get' as the key for our local cache 200 | if (!skipCache) table.cache.putResult(getsByKey(new String(res.getRow)), qr, ttl) 201 | resultMap(qr.rowid) = qr // place it in our result map 202 | } 203 | }) 204 | } 205 | } 206 | 207 | resultMap // DONE! 208 | } 209 | 210 | private def buildGetsAndCheckCache(skipCache: Boolean)(receiveGetAndKey: (Get, Array[Byte]) => Unit = (get, key) => {})(receiveCachedResult: (Option[RR], Get) => Unit = (qr, get) => {}): Seq[Get] = { 211 | if (keys.isEmpty) return Seq.empty[Get] // no keys..? nothing to see here... move along... move along. 212 | 213 | val gets = Buffer[Get]() // buffer for the raw `Get's 214 | 215 | for (key <- keys) { 216 | val get = new Get(key) 217 | gets += get 218 | receiveGetAndKey(get, key) 219 | } 220 | 221 | // since the families and columns will be identical for all `Get's, only build them once 222 | val firstGet = gets(0) 223 | 224 | // add all families to the first `Get' 225 | for (family <- families) { 226 | firstGet.addFamily(family) 227 | } 228 | // add all columns to the first `Get' 229 | for ((columnFamily, column) <- columns) { 230 | firstGet.addColumn(columnFamily, column) 231 | } 232 | 233 | var pastFirst = false 234 | for (get <- gets) { 235 | if (pastFirst) { 236 | // we want to skip the first `Get' as it already has families/columns 237 | 238 | // for all subsequent `Get's, we will build their familyMap from the first `Get' 239 | firstGet.getFamilyMap.foreach((kv: (Array[Byte], NavigableSet[Array[Byte]])) => { 240 | get.getFamilyMap.put(kv._1, kv._2) 241 | }) 242 | } else { 243 | pastFirst = true 244 | } 245 | 246 | // try the cache with this filled in get 247 | if (!skipCache) receiveCachedResult(table.cache.getResult(get), get) 248 | } 249 | 250 | gets 251 | } 252 | 253 | } 254 | -------------------------------------------------------------------------------- /src/test/scala/com/gravity/hbase/schema/ExampleSchema.scala: -------------------------------------------------------------------------------- 1 | /** Licensed to Gravity.com under one 2 | * or more contributor license agreements. See the NOTICE file 3 | * distributed with this work for additional information 4 | * regarding copyright ownership. Gravity.com licenses this file 5 | * to you under the Apache License, Version 2.0 (the 6 | * "License"); you may not use this file except in compliance 7 | * with the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.gravity.hbase.schema 19 | 20 | import org.junit.Assert._ 21 | import scala.collection._ 22 | import org.junit._ 23 | import org.joda.time.DateTime 24 | import CustomTypes._ 25 | 26 | /* )\._.,--....,'``. 27 | .b--. /; _.. \ _\ (`._ ,. 28 | `=,-,-'~~~ `----(,_..'--(,_..'`-.;.' */ 29 | 30 | /** 31 | * This test is intended to simultaneously test the library and show how you put together your own schema. 32 | */ 33 | 34 | /** 35 | * CUSTOM TYPES 36 | */ 37 | case class Kitten(name:String, age:Int, height:Double) 38 | 39 | case class PageUrl(url:String) 40 | 41 | /** 42 | * CUSTOM SERIALIZERS 43 | * These are serializers for custom types. When you create your own serializers, which is common, it's useful to put them 44 | * in their own object definition. Then, when you need the serializers in client code, make sure you import the object. For 45 | * the below, you'd do 46 | * import com.gravity.hbase.schema.CustomTypes._ 47 | */ 48 | object CustomTypes { 49 | 50 | implicit object PageUrlConverter extends ComplexByteConverter[PageUrl] { 51 | override def write(url:PageUrl, output:PrimitiveOutputStream) { 52 | output.writeUTF(url.url) 53 | } 54 | override def read(input:PrimitiveInputStream) = { 55 | PageUrl(input.readUTF()) 56 | } 57 | } 58 | 59 | implicit object KittenConverter extends ComplexByteConverter[Kitten] { 60 | override def write(kitten:Kitten, output:PrimitiveOutputStream) { 61 | output.writeUTF(kitten.name) 62 | output.writeInt(kitten.age) 63 | output.writeDouble(kitten.height) 64 | } 65 | 66 | override def read(input:PrimitiveInputStream) = { 67 | Kitten(input.readUTF(), input.readInt(), input.readDouble()) 68 | } 69 | } 70 | 71 | implicit object KittenSeqConverter extends SeqConverter[Kitten] 72 | } 73 | 74 | 75 | object ExampleSchema extends Schema { 76 | 77 | 78 | //There should only be one HBaseConfiguration object per process. You'll probably want to manage that 79 | //instance yourself, so this library expects a reference to that instance. It's implicitly injected into 80 | //the code, so the most convenient place to put it is right after you declare your Schema. 81 | implicit val conf = LocalCluster.getTestConfiguration 82 | 83 | //A table definition, where the row keys are Strings 84 | class ExampleTable extends HbaseTable[ExampleTable,String, ExampleTableRow](tableName = "schema_example",rowKeyClass=classOf[String], tableConfig = HbaseTableConfig(maxFileSizeInBytes=1073741824)) 85 | { 86 | def rowBuilder(result:DeserializedResult) = new ExampleTableRow(this,result) 87 | 88 | val meta = family[String, String, Any]("meta") 89 | //Column family definition 90 | //Inside meta, assume a column called title whose value is a string 91 | val title = column(meta, "title", classOf[String]) 92 | //Inside meta, assume a column called url whose value is a string 93 | val url = column(meta, "url", classOf[String]) 94 | //Inside meta, assume a column called views whose value is a string 95 | val views = column(meta, "views", classOf[Long]) 96 | //A column called date whose value is a Joda DateTime 97 | val creationDate = column(meta, "date", classOf[DateTime]) 98 | 99 | //A column called viewsArr whose value is a sequence of strings 100 | val viewsArr = column(meta,"viewsArr", classOf[Seq[String]]) 101 | //A column called viewsMap whose value is a map of String to Long 102 | val viewsMap = column(meta,"viewsMap", classOf[Map[String,Long]]) 103 | 104 | //A column family called views whose column names are Strings and values are Longs. Can be treated as a Map 105 | val viewCounts = family[String, String, Long]("views") 106 | 107 | //A column family called views whose column names are YearDay instances and whose values are Longs 108 | val viewCountsByDay = family[String, YearDay, Long]("viewsByDay") 109 | 110 | //A column family called kittens whose column values are the custom Kitten type 111 | val kittens = family[String,String,Kitten]("kittens") 112 | } 113 | 114 | class ExampleTableRow(table:ExampleTable,result:DeserializedResult) extends HRow[ExampleTable,String](result,table) 115 | 116 | //Register the table (DON'T FORGET TO DO THIS :) ) 117 | val ExampleTable = table(new ExampleTable) 118 | 119 | } 120 | 121 | 122 | 123 | 124 | /** 125 | * This test is intended to simultaneously test the library and show how you put together your own schema. 126 | */ 127 | class ExampleSchemaTest extends HPasteTestCase(ExampleSchema) { 128 | 129 | /** 130 | * Test that a complex custom type can be added and retrieved from a table as a Map 131 | */ 132 | @Test def testComplexCustomType() { 133 | val kittens = Map("Suki" -> Kitten("Suki",9,8.6), "Efrem" -> Kitten("Efrem",8,6.8), "Rory" -> Kitten("Rory",9,9.6),"Scout"->Kitten("Scout",8,12.3)) 134 | 135 | ExampleSchema.ExampleTable.put("Chris").valueMap(_.kittens,kittens).execute() 136 | 137 | val result = ExampleSchema.ExampleTable.query.withKey("Chris").withColumnFamily(_.kittens).single() 138 | val kittens2 = result.family(_.kittens) 139 | 140 | Assert.assertEquals(kittens,kittens2) 141 | } 142 | 143 | /** 144 | * Test that the create script looks right 145 | */ 146 | @Test def testCreateScript() { 147 | val createScript = """create 'schema_example', {NAME => 'meta', VERSIONS => 1},{NAME => 'views', VERSIONS => 1},{NAME => 'viewsByDay', VERSIONS => 1},{NAME => 'kittens', VERSIONS => 1} 148 | alter 'schema_example', {METHOD => 'table_att', MAX_FILESIZE => '1073741824'}""" 149 | 150 | val create = ExampleSchema.ExampleTable.createScript() 151 | println(create) 152 | Assert.assertEquals(createScript,create) 153 | } 154 | 155 | /** 156 | * Test the creation of a multi column alter script 157 | */ 158 | @Test def testAlterScript() { 159 | val expected = """flush 'schema_example' 160 | disable 'schema_example' 161 | alter 'schema_example', {NAME => 'kittens', VERSIONS => 1},{NAME => 'views', VERSIONS => 1} 162 | alter 'schema_example', {METHOD => 'table_att', MAX_FILESIZE => '1073741824'} 163 | enable 'schema_example'""" 164 | 165 | val alter = ExampleSchema.ExampleTable.alterScript(families=ExampleSchema.ExampleTable.kittens :: ExampleSchema.ExampleTable.viewCounts :: Nil) 166 | 167 | Assert.assertEquals(expected,alter) 168 | } 169 | 170 | /** 171 | * Helper method 172 | */ 173 | def dumpViewMap(key: String) { 174 | val dayViewsRes = ExampleSchema.ExampleTable.query.withKey(key).withColumnFamily(_.viewCountsByDay).withColumn(_.views).withColumn(_.title).single() 175 | 176 | val dayViewsMap = dayViewsRes.family(_.viewCountsByDay) 177 | 178 | for ((yearDay, views) <- dayViewsMap) { 179 | println("Got yearday " + yearDay + " with views " + views) 180 | } 181 | } 182 | 183 | @Test def testMaps() { 184 | val viewMap = Map("Chris"->50l, "Fred" -> 100l) 185 | ExampleSchema.ExampleTable 186 | .put("MapTest").value(_.viewsMap,viewMap) 187 | .execute() 188 | 189 | val res = ExampleSchema.ExampleTable.query.withKey("MapTest").withColumn(_.viewsMap).single() 190 | val returnedMap = res.column(_.viewsMap).get 191 | 192 | Assert.assertEquals(returnedMap,viewMap) 193 | } 194 | 195 | @Test def testSeqs() { 196 | ExampleSchema.ExampleTable 197 | .put("SeqTest").value(_.viewsArr, Seq("Chris","Fred","Bill")) 198 | .execute() 199 | 200 | val res = ExampleSchema.ExampleTable.query.withKey("SeqTest").withColumn(_.viewsArr).single() 201 | 202 | val resSeq = res.column(_.viewsArr).get 203 | Assert.assertEquals(resSeq(0),"Chris") 204 | Assert.assertEquals(resSeq(1),"Fred") 205 | Assert.assertEquals(resSeq(2),"Bill") 206 | } 207 | 208 | /** 209 | * This test does a bunch of operations without asserts, it's here to play around with the data. 210 | */ 211 | @Test def testPut() { 212 | ExampleSchema.ExampleTable 213 | .put("Chris").value(_.title, "My Life, My Times") 214 | .put("Joe").value(_.title, "Joe's Life and Times") 215 | .put("Fred").value(_.viewsArr,Seq("Chris","Bissell")) 216 | .increment("Chris").value(_.views, 10l) 217 | .put("Chris").valueMap(_.viewCountsByDay, Map(YearDay(2011,16)->60l, YearDay(2011,17)->50l)) 218 | .put("Fred").value(_.viewsMap, mutable.Map("Chris"->50l,"Bissell"->100l)) 219 | .execute() 220 | 221 | val arrRes = ExampleSchema.ExampleTable.query.withKey("Fred").withColumn(_.viewsArr).withColumn(_.viewsMap).single() 222 | 223 | val arr = arrRes.column(_.viewsArr) 224 | 225 | arr.get.foreach(println) 226 | 227 | val arrMap = arrRes.column(_.viewsMap) 228 | arrMap.get.foreach((tuple: (String, Long)) => println(tuple._1 + " views " + tuple._2)) 229 | 230 | val id = "Bill" 231 | 232 | ExampleSchema.ExampleTable.put(id).value(_.title, "My kittens").execute() 233 | 234 | ExampleSchema.ExampleTable.put(id).valueMap(_.viewCounts, Map("Today" -> 61l, "Yesterday" -> 86l)).execute() 235 | 236 | val dayMap = Map( 237 | YearDay(2011, 63) -> 64l, 238 | YearDay(2011, 64) -> 66l, 239 | YearDay(2011, 65) -> 67l 240 | ) 241 | 242 | 243 | ExampleSchema.ExampleTable.put(id).valueMap(_.viewCountsByDay, dayMap).execute() 244 | 245 | println("Dumping after map insert") 246 | dumpViewMap(id) 247 | 248 | ExampleSchema.ExampleTable.increment(id).valueMap(_.viewCountsByDay, dayMap).execute() 249 | 250 | println("Dumping after increment") 251 | dumpViewMap(id) 252 | 253 | ExampleSchema.ExampleTable.delete(id).family(_.viewCountsByDay).execute() 254 | println("Dumping after delete") 255 | dumpViewMap(id) 256 | 257 | val views = ExampleSchema.ExampleTable.query.withKey("Chris").withColumn(_.views).single().column(_.views) 258 | 259 | val myviewqueryresult = ExampleSchema.ExampleTable.query.withKey("Chris").withColumn(_.views).single() 260 | 261 | 262 | println("Views: " + views.get) 263 | } 264 | 265 | @Test def testWithKeys() { 266 | ExampleSchema.ExampleTable.put("Robbie").value(_.title, "My Bros, My Probs") 267 | .put("Ronnie").value(_.title, "My Weights, My Muskellz").execute() 268 | 269 | val bros = ExampleSchema.ExampleTable.query.withKeys(Set("Robbie", "Ronnie")).withColumnFamily(_.meta).execute() 270 | 271 | if (bros.isEmpty) fail("Failed to retrieve the data we just put!") 272 | 273 | for (bro <- bros) { 274 | bro.column(_.title) match { 275 | case Some(title) => println("%nBro: %s; title: %s".format(bro.rowid, title)) 276 | case None => fail("FAILED TO GET TITLE!") 277 | 278 | } 279 | } 280 | 281 | } 282 | 283 | 284 | 285 | } 286 | 287 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | Apache License 2 | Version 2.0, January 2004 3 | http://www.apache.org/licenses/ 4 | 5 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION 6 | 7 | 1. Definitions. 8 | 9 | "License" shall mean the terms and conditions for use, reproduction, 10 | and distribution as defined by Sections 1 through 9 of this document. 11 | 12 | "Licensor" shall mean the copyright owner or entity authorized by 13 | the copyright owner that is granting the License. 14 | 15 | "Legal Entity" shall mean the union of the acting entity and all 16 | other entities that control, are controlled by, or are under common 17 | control with that entity. For the purposes of this definition, 18 | "control" means (i) the power, direct or indirect, to cause the 19 | direction or management of such entity, whether by contract or 20 | otherwise, or (ii) ownership of fifty percent (50%) or more of the 21 | outstanding shares, or (iii) beneficial ownership of such entity. 22 | 23 | "You" (or "Your") shall mean an individual or Legal Entity 24 | exercising permissions granted by this License. 25 | 26 | "Source" form shall mean the preferred form for making modifications, 27 | including but not limited to software source code, documentation 28 | source, and configuration files. 29 | 30 | "Object" form shall mean any form resulting from mechanical 31 | transformation or translation of a Source form, including but 32 | not limited to compiled object code, generated documentation, 33 | and conversions to other media types. 34 | 35 | "Work" shall mean the work of authorship, whether in Source or 36 | Object form, made available under the License, as indicated by a 37 | copyright notice that is included in or attached to the work 38 | (an example is provided in the Appendix below). 39 | 40 | "Derivative Works" shall mean any work, whether in Source or Object 41 | form, that is based on (or derived from) the Work and for which the 42 | editorial revisions, annotations, elaborations, or other modifications 43 | represent, as a whole, an original work of authorship. For the purposes 44 | of this License, Derivative Works shall not include works that remain 45 | separable from, or merely link (or bind by name) to the interfaces of, 46 | the Work and Derivative Works thereof. 47 | 48 | "Contribution" shall mean any work of authorship, including 49 | the original version of the Work and any modifications or additions 50 | to that Work or Derivative Works thereof, that is intentionally 51 | submitted to Licensor for inclusion in the Work by the copyright owner 52 | or by an individual or Legal Entity authorized to submit on behalf of 53 | the copyright owner. For the purposes of this definition, "submitted" 54 | means any form of electronic, verbal, or written communication sent 55 | to the Licensor or its representatives, including but not limited to 56 | communication on electronic mailing lists, source code control systems, 57 | and issue tracking systems that are managed by, or on behalf of, the 58 | Licensor for the purpose of discussing and improving the Work, but 59 | excluding communication that is conspicuously marked or otherwise 60 | designated in writing by the copyright owner as "Not a Contribution." 61 | 62 | "Contributor" shall mean Licensor and any individual or Legal Entity 63 | on behalf of whom a Contribution has been received by Licensor and 64 | subsequently incorporated within the Work. 65 | 66 | 2. Grant of Copyright License. Subject to the terms and conditions of 67 | this License, each Contributor hereby grants to You a perpetual, 68 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 69 | copyright license to reproduce, prepare Derivative Works of, 70 | publicly display, publicly perform, sublicense, and distribute the 71 | Work and such Derivative Works in Source or Object form. 72 | 73 | 3. Grant of Patent License. Subject to the terms and conditions of 74 | this License, each Contributor hereby grants to You a perpetual, 75 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 76 | (except as stated in this section) patent license to make, have made, 77 | use, offer to sell, sell, import, and otherwise transfer the Work, 78 | where such license applies only to those patent claims licensable 79 | by such Contributor that are necessarily infringed by their 80 | Contribution(s) alone or by combination of their Contribution(s) 81 | with the Work to which such Contribution(s) was submitted. If You 82 | institute patent litigation against any entity (including a 83 | cross-claim or counterclaim in a lawsuit) alleging that the Work 84 | or a Contribution incorporated within the Work constitutes direct 85 | or contributory patent infringement, then any patent licenses 86 | granted to You under this License for that Work shall terminate 87 | as of the date such litigation is filed. 88 | 89 | 4. Redistribution. You may reproduce and distribute copies of the 90 | Work or Derivative Works thereof in any medium, with or without 91 | modifications, and in Source or Object form, provided that You 92 | meet the following conditions: 93 | 94 | (a) You must give any other recipients of the Work or 95 | Derivative Works a copy of this License; and 96 | 97 | (b) You must cause any modified files to carry prominent notices 98 | stating that You changed the files; and 99 | 100 | (c) You must retain, in the Source form of any Derivative Works 101 | that You distribute, all copyright, patent, trademark, and 102 | attribution notices from the Source form of the Work, 103 | excluding those notices that do not pertain to any part of 104 | the Derivative Works; and 105 | 106 | (d) If the Work includes a "NOTICE" text file as part of its 107 | distribution, then any Derivative Works that You distribute must 108 | include a readable copy of the attribution notices contained 109 | within such NOTICE file, excluding those notices that do not 110 | pertain to any part of the Derivative Works, in at least one 111 | of the following places: within a NOTICE text file distributed 112 | as part of the Derivative Works; within the Source form or 113 | documentation, if provided along with the Derivative Works; or, 114 | within a display generated by the Derivative Works, if and 115 | wherever such third-party notices normally appear. The contents 116 | of the NOTICE file are for informational purposes only and 117 | do not modify the License. You may add Your own attribution 118 | notices within Derivative Works that You distribute, alongside 119 | or as an addendum to the NOTICE text from the Work, provided 120 | that such additional attribution notices cannot be construed 121 | as modifying the License. 122 | 123 | You may add Your own copyright statement to Your modifications and 124 | may provide additional or different license terms and conditions 125 | for use, reproduction, or distribution of Your modifications, or 126 | for any such Derivative Works as a whole, provided Your use, 127 | reproduction, and distribution of the Work otherwise complies with 128 | the conditions stated in this License. 129 | 130 | 5. Submission of Contributions. Unless You explicitly state otherwise, 131 | any Contribution intentionally submitted for inclusion in the Work 132 | by You to the Licensor shall be under the terms and conditions of 133 | this License, without any additional terms or conditions. 134 | Notwithstanding the above, nothing herein shall supersede or modify 135 | the terms of any separate license agreement you may have executed 136 | with Licensor regarding such Contributions. 137 | 138 | 6. Trademarks. This License does not grant permission to use the trade 139 | names, trademarks, service marks, or product names of the Licensor, 140 | except as required for reasonable and customary use in describing the 141 | origin of the Work and reproducing the content of the NOTICE file. 142 | 143 | 7. Disclaimer of Warranty. Unless required by applicable law or 144 | agreed to in writing, Licensor provides the Work (and each 145 | Contributor provides its Contributions) on an "AS IS" BASIS, 146 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 147 | implied, including, without limitation, any warranties or conditions 148 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A 149 | PARTICULAR PURPOSE. You are solely responsible for determining the 150 | appropriateness of using or redistributing the Work and assume any 151 | risks associated with Your exercise of permissions under this License. 152 | 153 | 8. Limitation of Liability. In no event and under no legal theory, 154 | whether in tort (including negligence), contract, or otherwise, 155 | unless required by applicable law (such as deliberate and grossly 156 | negligent acts) or agreed to in writing, shall any Contributor be 157 | liable to You for damages, including any direct, indirect, special, 158 | incidental, or consequential damages of any character arising as a 159 | result of this License or out of the use or inability to use the 160 | Work (including but not limited to damages for loss of goodwill, 161 | work stoppage, computer failure or malfunction, or any and all 162 | other commercial damages or losses), even if such Contributor 163 | has been advised of the possibility of such damages. 164 | 165 | 9. Accepting Warranty or Additional Liability. While redistributing 166 | the Work or Derivative Works thereof, You may choose to offer, 167 | and charge a fee for, acceptance of support, warranty, indemnity, 168 | or other liability obligations and/or rights consistent with this 169 | License. However, in accepting such obligations, You may act only 170 | on Your own behalf and on Your sole responsibility, not on behalf 171 | of any other Contributor, and only if You agree to indemnify, 172 | defend, and hold each Contributor harmless for any liability 173 | incurred by, or claims asserted against, such Contributor by reason 174 | of your accepting any such warranty or additional liability. 175 | 176 | END OF TERMS AND CONDITIONS 177 | 178 | APPENDIX: How to apply the Apache License to your work. 179 | 180 | To apply the Apache License to your work, attach the following 181 | boilerplate notice, with the fields enclosed by brackets "[]" 182 | replaced with your own identifying information. (Don't include 183 | the brackets!) The text should be enclosed in the appropriate 184 | comment syntax for the file format. We also recommend that a 185 | file or class name and description of purpose be included on the 186 | same "printed page" as the copyright notice for easier 187 | identification within third-party archives. 188 | 189 | Copyright 2011 Gravity Inc. 190 | 191 | Licensed under the Apache License, Version 2.0 (the "License"); 192 | you may not use this file except in compliance with the License. 193 | You may obtain a copy of the License at 194 | 195 | http://www.apache.org/licenses/LICENSE-2.0 196 | 197 | Unless required by applicable law or agreed to in writing, software 198 | distributed under the License is distributed on an "AS IS" BASIS, 199 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 200 | See the License for the specific language governing permissions and 201 | limitations under the License. -------------------------------------------------------------------------------- /src/main/scala/com/gravity/hbase/schema/Schema.scala: -------------------------------------------------------------------------------- 1 | /**Licensed to Gravity.com under one 2 | * or more contributor license agreements. See the NOTICE file 3 | * distributed with this work for additional information 4 | * regarding copyright ownership. Gravity.com licenses this file 5 | * to you under the Apache License, Version 2.0 (the 6 | * "License"); you may not use this file except in compliance 7 | * with the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.gravity.hbase.schema 19 | 20 | import org.apache.hadoop.hbase.client._ 21 | import org.apache.hadoop.hbase.util._ 22 | import scala.collection.JavaConversions._ 23 | import org.apache.hadoop.conf.Configuration 24 | import java.io._ 25 | import org.apache.hadoop.io.Writable 26 | import scala.collection._ 27 | import mutable.{ArrayBuffer, Buffer} 28 | import org.joda.time.DateTime 29 | import com.gravity.hbase.schema._ 30 | import java.math.BigInteger 31 | import java.nio.ByteBuffer 32 | import org.apache.commons.lang.ArrayUtils 33 | import java.util.{Arrays, HashMap} 34 | import org.apache.hadoop.hbase.util.Bytes.ByteArrayComparator 35 | import org.apache.hadoop.hbase.{HColumnDescriptor, KeyValue} 36 | import com.gravity.hbase.{AnyConverterSignal, AnyNotSupportedException} 37 | 38 | /* )\._.,--....,'``. 39 | .b--. /; _.. \ _\ (`._ ,. 40 | `=,-,-'~~~ `----(,_..'--(,_..'`-.;.' */ 41 | 42 | 43 | /**When a query comes back, there are a bucket of column families and columns to retrieve. This class retrieves them. 44 | * 45 | * @tparam T the source [[com.gravity.hbase.schema.HbaseTable]] this result came from 46 | * @tparam R the `type` of the table's rowid 47 | * 48 | * @param result the raw [[org.apache.hadoop.hbase.client.Result]] returned from the `hbase` [[org.apache.hadoop.hbase.client.Get]] 49 | * @param table the underlying [[com.gravity.hbase.schema.HbaseTable]] 50 | * @param tableName the name of the actual table 51 | */ 52 | class QueryResult[T <: HbaseTable[T, R, _], R](val result: DeserializedResult, val table: HbaseTable[T, R, _], val tableName: String) extends Serializable { 53 | 54 | 55 | /**This is a convenience method to allow consumers to check 56 | * if a column has a value present in the result without 57 | * invoking the deserialization of the value 58 | * 59 | * @tparam F the type of the column family name 60 | * @tparam K the type of the column family qualifier 61 | * @tparam V the type of the column family value 62 | * 63 | * @param column the underlying table's column `val` 64 | * 65 | * @return `true` if the column value is present and otherwise `false` 66 | */ 67 | def isColumnPresent[F, K, V](column: (T) => Column[T, R, F, K, V]): Boolean = { 68 | val co = column(table.pops) 69 | result.hasColumn(co) 70 | } 71 | 72 | /**Extracts and deserializes the value of the `column` specified 73 | * 74 | * @tparam F the type of the column family name 75 | * @tparam K the type of the column family qualifier 76 | * @tparam V the type of the column family value 77 | * 78 | * @param column the underlying table's column `val` 79 | * 80 | * @return `Some` value of type `V` if the column value is present, otherwise `None` 81 | * 82 | * @note if there is no explicitly defined `val` for the desired column, use `columnFromFamily` 83 | */ 84 | def column[F, K, V](column: (T) => Column[T, R, F, K, V]) = { 85 | val co = column(table.pops) 86 | val colVal = result.columnValueSpecific(co) 87 | if (colVal == null) { 88 | None 89 | } 90 | else { 91 | Some[V](colVal.asInstanceOf[V]) 92 | } 93 | } 94 | 95 | /**Extracts and deserializes the value of the `family` + `columnName` specified 96 | * 97 | * @tparam F the type of the column family name 98 | * @tparam K the type of the column family qualifier 99 | * @tparam V the type of the column family value 100 | * 101 | * @param family the underlying table's family `val` 102 | * @param columnName value of the desired column's qualifier 103 | * 104 | * @return `Some` value of type `V` if the column value is present, otherwise `None` 105 | */ 106 | def columnFromFamily[F, K, V](family: (T) => ColumnFamily[T, R, F, K, V], columnName: K) = { 107 | val fam = family(table.pops) 108 | val colVal = result.columnValue(fam, columnName.asInstanceOf[AnyRef]) 109 | if (colVal == null) { 110 | None 111 | } 112 | else { 113 | Some[V](colVal.asInstanceOf[V]) 114 | } 115 | } 116 | 117 | /**Extracts and deserializes the Timestamp of the `family` + `columnName` specified 118 | * 119 | * @tparam F the type of the column family name 120 | * @tparam K the type of the column family qualifier 121 | * @tparam V the type of the column family value 122 | * 123 | * @param family the underlying table's family `val` 124 | * @param columnName value of the desired column's qualifier 125 | * 126 | * @return `Some` [[org.joda.time.DateTime]] if the column value is present, otherwise `None` 127 | */ 128 | def columnFromFamilyTimestamp[F, K, V](family: (T) => ColumnFamily[T, R, F, K, V], columnName: K) = { 129 | val fam = family(table.pops) 130 | val colVal = result.columnTimestampByNameAsDate(fam, columnName.asInstanceOf[AnyRef]) 131 | if (colVal == null) { 132 | None 133 | } 134 | else { 135 | Some(colVal) 136 | } 137 | } 138 | 139 | /**Extracts column timestamp of the specified `column` 140 | * 141 | * @tparam F the type of the column family name 142 | * @tparam K the type of the column family qualifier 143 | * @tparam V the type of the column family value 144 | * 145 | * @param column the underlying table's column `val` 146 | * 147 | * @return `Some` [[org.joda.time.DateTime]] if the column value is present, otherwise `None` 148 | */ 149 | def columnTimestamp[F, K, V](column: (T) => Column[T, R, F, K, V]): Option[DateTime] = { 150 | val co = column(table.pops) 151 | val res = result.columnTimestampAsDate(co) 152 | if (res == null) { 153 | None 154 | } 155 | else { 156 | Some(res) 157 | } 158 | } 159 | 160 | /**Extracts most recent column timestamp of the specified `family` 161 | * 162 | * @tparam F the type of the column family name 163 | * @tparam K the type of the column family qualifier 164 | * @tparam V the type of the column family value 165 | * 166 | * @param family the underlying table's family `val` 167 | * 168 | * @return `Some` [[org.joda.time.DateTime]] if at least one column value is present, otherwise `None` 169 | */ 170 | def familyLatestTimestamp[F, K, V](family: (T) => ColumnFamily[T, R, F, K, V]): Option[DateTime] = { 171 | val fam = family(table.pops) 172 | val familyPairs = result.familyMap(fam) 173 | if (familyPairs != null) { 174 | var ts = -1l 175 | for (kv <- familyPairs) { 176 | val tsn = result.columnTimestampByName(fam, kv._1) 177 | if (tsn > ts) ts = tsn 178 | } 179 | if (ts > 0) { 180 | Some(new DateTime(ts)) 181 | } 182 | else { 183 | None 184 | } 185 | 186 | } else { 187 | None 188 | } 189 | } 190 | 191 | /**Extracts and deserializes the entire family as a `Map[K, V]` 192 | * 193 | * @tparam F the type of the column family name 194 | * @tparam K the type of the column family qualifier 195 | * @tparam V the type of the column family value 196 | * 197 | * @param family the underlying table's family `val` 198 | * 199 | */ 200 | def family[F, K, V](family: (T) => ColumnFamily[T, R, F, K, V]): Map[K, V] = { 201 | val fm = family(table.pops) 202 | result.familyValueMap[K, V](fm) 203 | 204 | } 205 | 206 | /**Extracts and deserializes only the keys (qualifiers) of the family as a `Set[K]` 207 | * 208 | * @tparam F the type of the column family name 209 | * @tparam K the type of the column family qualifier 210 | * 211 | * @param family the underlying table's family `val` 212 | * 213 | */ 214 | def familyKeySet[F, K](family: (T) => ColumnFamily[T, R, F, K, _]): Set[K] = { 215 | val fm = family(table.pops) 216 | result.familyKeySet[K](fm) 217 | } 218 | 219 | /**The row identifier deserialized as type `R` 220 | * 221 | */ 222 | def rowid = result.getRow[R]() 223 | 224 | def getTableName = tableName 225 | } 226 | 227 | /** 228 | * A query for setting up a scanner across the whole table or key subsets. 229 | * There is a lot of room for expansion in this class -- caching parameters, scanner specs, key-only, etc. 230 | */ 231 | 232 | 233 | 234 | 235 | 236 | 237 | 238 | 239 | 240 | 241 | 242 | /** 243 | * A query for retrieving values. It works somewhat differently than the data modification operations, in that you do the following: 244 | * 1. Specify one or more keys 245 | * 2. Specify columns and families to scan in for ALL the specified keys 246 | * 247 | * In other words there's no concept of having multiple rows fetched with different columns for each row (that seems to be a rare use-case and 248 | * would make the API very complex). 249 | */ 250 | 251 | trait KeyValueConvertible[F, K, V] { 252 | val familyConverter: ByteConverter[F] 253 | val keyConverter: ByteConverter[K] 254 | val valueConverter: ByteConverter[V] 255 | 256 | def keyToBytes(key: K) = keyConverter.toBytes(key) 257 | 258 | def valueToBytes(value: V) = valueConverter.toBytes(value) 259 | 260 | def keyToBytesUnsafe(key: AnyRef) = keyConverter.toBytes(key.asInstanceOf[K]) 261 | 262 | def valueToBytesUnsafe(value: AnyRef) = valueConverter.toBytes(value.asInstanceOf[V]) 263 | 264 | def keyFromBytesUnsafe(bytes: Array[Byte]) = keyConverter.fromBytes(bytes).asInstanceOf[AnyRef] 265 | 266 | def valueFromBytesUnsafe(bytes: Array[Byte]) = valueConverter.fromBytes(bytes).asInstanceOf[AnyRef] 267 | 268 | def family: ColumnFamily[_, _, _, _, _] 269 | } 270 | 271 | /** 272 | * Represents the specification of a Column Family 273 | */ 274 | class ColumnFamily[T <: HbaseTable[T, R, _], R, F, K, V](val table: HbaseTable[T, R, _], val familyName: F, val compressed: Boolean = false, val versions: Int = 1, val index: Int, val ttlInSeconds: Int = HColumnDescriptor.DEFAULT_TTL)(implicit c: ByteConverter[F], d: ByteConverter[K], e: ByteConverter[V]) extends KeyValueConvertible[F, K, V] { 275 | val familyConverter = c 276 | val keyConverter = d 277 | val valueConverter = e 278 | val familyBytes = c.toBytes(familyName) 279 | 280 | 281 | def family = this 282 | } 283 | 284 | /** 285 | * Represents the specification of a Column. 286 | */ 287 | class Column[T <: HbaseTable[T, R, _], R, F, K, V](table: HbaseTable[T, R, _], columnFamily: ColumnFamily[T, R, F, K, _], val columnName: K, val columnIndex: Int)(implicit fc: ByteConverter[F], kc: ByteConverter[K], kv: ByteConverter[V]) extends KeyValueConvertible[F, K, V] { 288 | val columnBytes = kc.toBytes(columnName) 289 | val familyBytes = columnFamily.familyBytes 290 | val columnNameRef = columnName.asInstanceOf[AnyRef] 291 | 292 | val familyConverter = fc 293 | val keyConverter = kc 294 | val valueConverter = kv 295 | 296 | def getQualifier: K = columnName 297 | 298 | def family = columnFamily.asInstanceOf[ColumnFamily[_, _, _, _, _]] 299 | 300 | 301 | } 302 | 303 | trait Schema { 304 | val tables = scala.collection.mutable.Set[HbaseTable[_, _, _]]() 305 | 306 | def table[T <: HbaseTable[T, _, _]](table: T) = { 307 | tables += table 308 | table.init() 309 | table 310 | } 311 | 312 | 313 | } 314 | 315 | case class YearDay(year: Int, day: Int) 316 | 317 | case class CommaSet(items: Set[String]) { 318 | def mkString: String = items.mkString 319 | 320 | def mkString(sep: String): String = items.mkString(sep) 321 | 322 | def mkString(start: String, sep: String, end: String): String = items.mkString(start, sep, end) 323 | } 324 | 325 | object CommaSet { 326 | val empty = CommaSet(Set.empty[String]) 327 | 328 | def apply(items: String*): CommaSet = CommaSet(items.toSet) 329 | } 330 | 331 | -------------------------------------------------------------------------------- /src/main/scala/com/gravity/hbase/schema/Serialization.scala: -------------------------------------------------------------------------------- 1 | /** Licensed to Gravity.com under one 2 | * or more contributor license agreements. See the NOTICE file 3 | * distributed with this work for additional information 4 | * regarding copyright ownership. Gravity.com licenses this file 5 | * to you under the Apache License, Version 2.0 (the 6 | * "License"); you may not use this file except in compliance 7 | * with the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.gravity.hbase.schema 19 | 20 | 21 | import org.apache.hadoop.hbase.client._ 22 | import org.apache.hadoop.hbase.util._ 23 | import scala.collection.JavaConversions._ 24 | import org.apache.hadoop.conf.Configuration 25 | import java.io._ 26 | import org.apache.hadoop.io.{BytesWritable, Writable} 27 | import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp 28 | import org.apache.hadoop.hbase.filter.{Filter, FilterList, SingleColumnValueFilter} 29 | import scala.collection._ 30 | import org.joda.time.DateTime 31 | import scala.collection.mutable.{ListBuffer, Buffer} 32 | import java.util.{HashMap, NavigableSet} 33 | 34 | /* )\._.,--....,'``. 35 | .b--. /; _.. \ _\ (`._ ,. 36 | `=,-,-'~~~ `----(,_..'--(,_..'`-.;.' */ 37 | 38 | /** Expresses an input stream that can read ordered primitives from a binary input, and can also use the ByteConverter[T] interface to read serializable objects. 39 | * 40 | */ 41 | class PrimitiveInputStream(input: InputStream) extends DataInputStream(input) { 42 | /** 43 | * Read an object, assuming the existence of a ComplexByteConverter[T] implementation 44 | * The byte converter is stateless and should be therefore defined somewhere as an implicit object 45 | */ 46 | def readObj[T](implicit c: ComplexByteConverter[T]) = { 47 | c.read(this) 48 | } 49 | 50 | def skipLong() {this.skipBytes(8)} 51 | 52 | //WORK IN PROGRESS 53 | def readRow[T <: HbaseTable[T, R, RR], R, RR <: HRow[T, R]](table: HbaseTable[T, R, RR]) = { 54 | val rowBytesLength = readInt() 55 | val rowBytes = new Array[Byte](rowBytesLength) 56 | read(rowBytes) 57 | val rowId = table.rowKeyConverter.fromBytes(rowBytes) 58 | val ds = DeserializedResult(rowId.asInstanceOf[AnyRef], table.families.length) 59 | 60 | val famCount = readInt() 61 | 62 | for (i <- 0 until famCount) { 63 | val fam = table.familyByIndex(i) 64 | val kvLength = readInt() 65 | 66 | for (ii <- 0 until kvLength) { 67 | val isTypedColumn = readBoolean 68 | val converter = if (isTypedColumn) { 69 | val colIdx = readInt 70 | val col = table.columnByIndex(colIdx) 71 | col 72 | } else { 73 | fam 74 | } 75 | 76 | val keyLength = readInt 77 | val keyBytes = new Array[Byte](keyLength) 78 | read(keyBytes) 79 | val valueLength = readInt 80 | val valueBytes = new Array[Byte](valueLength) 81 | read(valueBytes) 82 | val key = converter.keyFromBytesUnsafe(keyBytes) 83 | val value = converter.valueFromBytesUnsafe(valueBytes) 84 | ds.add(fam,key,value,0l) 85 | } 86 | } 87 | table.rowBuilder(ds) 88 | 89 | } 90 | } 91 | 92 | /** Expresses an output stream that can write ordered primitives into a binary output, and can also use the ByteConverter[T] interface to write serializable objects. 93 | */ 94 | class PrimitiveOutputStream(output: OutputStream) extends DataOutputStream(output) { 95 | 96 | //WORK IN PROGRESS 97 | def writeRow[T <: HbaseTable[T, R, RR], R, RR <: HRow[T, R]](table: HbaseTable[T,R,RR],row: RR) { 98 | 99 | //Serialize row id 100 | val rowIdBytes = row.table.rowKeyConverter.toBytes(row.rowid) 101 | writeInt(rowIdBytes.length) 102 | write(rowIdBytes) 103 | 104 | //Write number of families 105 | writeInt(row.result.values.length) 106 | 107 | var idx = 0 108 | while (idx < row.result.values.length) { 109 | val family = row.result.values(idx) 110 | val colFam = row.table.familyByIndex(idx) 111 | if(family == null) { 112 | writeInt(0) 113 | }else { 114 | writeInt(family.size()) 115 | family.foreach { 116 | case (colKey: AnyRef, colVal: AnyRef) => 117 | //See if it's a strongly typed column 118 | val converters: KeyValueConvertible[_, _, _] = row.table.columnsByName.get(colKey) match { 119 | case Some(col) => { 120 | writeBoolean(true) 121 | writeInt(col.columnIndex) 122 | col 123 | 124 | } 125 | case None => { 126 | writeBoolean(false) 127 | colFam 128 | } 129 | } 130 | 131 | val keyBytes = converters.keyToBytesUnsafe(colKey) 132 | writeInt(keyBytes.length) 133 | write(keyBytes) 134 | val valBytes = converters.valueToBytesUnsafe(colVal) 135 | writeInt(valBytes.length) 136 | write(valBytes) 137 | } 138 | 139 | } 140 | idx += 1 141 | } 142 | } 143 | 144 | 145 | /** 146 | * Write an object, assuming the existence of a ComplexByteConverter[T] implementation. 147 | * The byte converter is stateless and should be therefore defined somewhere as an implicit object 148 | */ 149 | def writeObj[T](obj: T)(implicit c: ComplexByteConverter[T]) { 150 | c.write(obj, this) 151 | } 152 | } 153 | 154 | /** 155 | * Class to be implemented by custom converters 156 | */ 157 | abstract class ByteConverter[T] { 158 | def toBytes(t: T): Array[Byte] 159 | 160 | 161 | def fromBytes(bytes: Array[Byte]): T = fromBytes(bytes, 0, bytes.length) 162 | 163 | def fromBytes(bytes: Array[Byte], offset: Int, length: Int): T 164 | 165 | def fromByteString(str: String): T = { 166 | fromBytes(Bytes.toBytesBinary(str)) 167 | } 168 | 169 | def toByteString(item: T) = { 170 | Bytes.toStringBinary(toBytes(item)) 171 | } 172 | 173 | def toBytesWritable(t: T): BytesWritable = { 174 | new BytesWritable(toBytes(t)) 175 | } 176 | 177 | def fromBytesWritable(bytes: BytesWritable): T = { 178 | fromBytes(bytes.getBytes) 179 | } 180 | } 181 | 182 | /** 183 | * Simple high performance conversions from complex types to bytes 184 | */ 185 | abstract class ComplexByteConverter[T] extends ByteConverter[T] { 186 | override def toBytes(t: T): Array[Byte] = { 187 | val bos = new ByteArrayOutputStream() 188 | 189 | val dout = new PrimitiveOutputStream(bos) 190 | write(t, dout) 191 | 192 | bos.toByteArray 193 | } 194 | 195 | def write(data: T, output: PrimitiveOutputStream) 196 | 197 | override def fromBytes(bytes: Array[Byte], offset: Int, length: Int): T = { 198 | val din = new PrimitiveInputStream(new ByteArrayInputStream(bytes, offset, length)) 199 | read(din) 200 | } 201 | 202 | override def fromBytes(bytes: Array[Byte]): T = { 203 | val din = new PrimitiveInputStream(new ByteArrayInputStream(bytes)) 204 | read(din) 205 | } 206 | 207 | def read(input: PrimitiveInputStream): T 208 | 209 | def safeReadField[A](input: PrimitiveInputStream)(readField: (PrimitiveInputStream)=>A, valueOnFail: A): A = { 210 | if (input.available() < 1) return valueOnFail 211 | 212 | try { 213 | readField(input) 214 | } 215 | catch { 216 | case _: IOException => valueOnFail 217 | } 218 | } 219 | } 220 | 221 | trait MapStream[K,V] { 222 | val c : ByteConverter[K] 223 | val d : ByteConverter[V] 224 | 225 | def writeMap(map:Map[K,V], output: PrimitiveOutputStream) { 226 | val length = map.size 227 | output.writeInt(length) 228 | 229 | for ((k, v) <- map) { 230 | val keyBytes = c.toBytes(k) 231 | val valBytes = d.toBytes(v) 232 | output.writeInt(keyBytes.length) 233 | output.write(keyBytes) 234 | output.writeInt(valBytes.length) 235 | output.write(valBytes) 236 | } 237 | 238 | } 239 | 240 | def readMap(input:PrimitiveInputStream) : Array[(K,V)] = { 241 | val length = input.readInt() 242 | val kvarr = Array.ofDim[(K, V)](length) 243 | 244 | var i = 0 245 | while (i < length) { 246 | val keyLength = input.readInt 247 | val keyArr = new Array[Byte](keyLength) 248 | input.read(keyArr) 249 | val key = c.fromBytes(keyArr) 250 | 251 | val valLength = input.readInt 252 | val valArr = new Array[Byte](valLength) 253 | input.read(valArr) 254 | val value = d.fromBytes(valArr) 255 | 256 | kvarr(i) = (key -> value) 257 | i = i + 1 258 | } 259 | kvarr 260 | } 261 | } 262 | 263 | class ImmutableMapConverter[K, V](implicit val c: ByteConverter[K],val d: ByteConverter[V]) extends ComplexByteConverter[scala.collection.immutable.Map[K, V]] with MapStream[K,V] { 264 | override def write(map: scala.collection.immutable.Map[K, V], output: PrimitiveOutputStream) { 265 | writeMap(map,output) 266 | } 267 | 268 | override def read(input: PrimitiveInputStream) = { 269 | val kvarr = readMap(input) 270 | scala.collection.immutable.Map[K, V](kvarr: _*) 271 | } 272 | } 273 | 274 | class MutableMapConverter[K, V](implicit val c: ByteConverter[K],val d: ByteConverter[V]) extends ComplexByteConverter[scala.collection.mutable.Map[K, V]] with MapStream[K,V] { 275 | override def write(map: scala.collection.mutable.Map[K, V], output: PrimitiveOutputStream) { 276 | writeMap(map,output) 277 | } 278 | 279 | override def read(input: PrimitiveInputStream) = { 280 | val kvarr = readMap(input) 281 | scala.collection.mutable.Map[K, V](kvarr: _*) 282 | } 283 | } 284 | 285 | 286 | class MapConverter[K, V](implicit val c: ByteConverter[K],val d: ByteConverter[V]) extends ComplexByteConverter[Map[K, V]] with MapStream[K,V] { 287 | override def write(map: Map[K, V], output: PrimitiveOutputStream) { 288 | writeMap(map,output) 289 | } 290 | 291 | override def read(input: PrimitiveInputStream) = { 292 | val kvarr = readMap(input) 293 | Map[K, V](kvarr: _*) 294 | } 295 | } 296 | 297 | 298 | class MutableSetConverter[T](implicit c: ByteConverter[T]) extends ComplexByteConverter[scala.collection.mutable.Set[T]] with CollStream[T] { 299 | 300 | override def write(set: scala.collection.mutable.Set[T], output: PrimitiveOutputStream) { 301 | writeColl(set, set.size, output, c) 302 | } 303 | 304 | override def read(input: PrimitiveInputStream): scala.collection.mutable.Set[T] = { 305 | scala.collection.mutable.Set(readColl(input, c):_*) 306 | } 307 | } 308 | 309 | 310 | class ImmutableSetConverter[T](implicit c: ByteConverter[T]) extends ComplexByteConverter[scala.collection.immutable.Set[T]] with CollStream[T] { 311 | 312 | override def write(set: scala.collection.immutable.Set[T], output: PrimitiveOutputStream) { 313 | writeColl(set, set.size, output, c) 314 | } 315 | 316 | override def read(input: PrimitiveInputStream): scala.collection.immutable.Set[T] = { 317 | readColl(input, c).toSet 318 | } 319 | } 320 | 321 | 322 | class SetConverter[T](implicit c: ByteConverter[T]) extends ComplexByteConverter[Set[T]] with CollStream[T] { 323 | 324 | override def write(set: Set[T], output: PrimitiveOutputStream) { 325 | writeColl(set, set.size, output, c) 326 | } 327 | 328 | override def read(input: PrimitiveInputStream): Set[T] = { 329 | readColl(input, c).toSet 330 | } 331 | } 332 | 333 | 334 | class SeqConverter[T](implicit c: ByteConverter[T]) extends ComplexByteConverter[Seq[T]] with CollStream[T] { 335 | override def write(seq: Seq[T], output: PrimitiveOutputStream) { 336 | writeColl(seq, seq.length, output, c) 337 | } 338 | 339 | override def read(input: PrimitiveInputStream) = readColl(input, c).toSeq 340 | } 341 | 342 | class BufferConverter[T](implicit c: ByteConverter[T]) extends ComplexByteConverter[Buffer[T]] with CollStream[T] { 343 | override def write(buf: Buffer[T], output: PrimitiveOutputStream) { 344 | writeBuf(buf, output) 345 | } 346 | 347 | def writeBuf(buf: Buffer[T], output: PrimitiveOutputStream) { 348 | writeColl(buf, buf.length, output, c) 349 | } 350 | 351 | override def read(input: PrimitiveInputStream) = readColl(input, c) 352 | } 353 | 354 | trait CollStream[T] { 355 | 356 | def writeColl(items: Iterable[T], length: Int, output: PrimitiveOutputStream, c: ByteConverter[T]) { 357 | 358 | output.writeInt(length) 359 | 360 | val iter = items.iterator 361 | while (iter.hasNext) { 362 | val t = iter.next() 363 | val bytes = c.toBytes(t) 364 | output.writeInt(bytes.length) 365 | output.write(bytes) 366 | } 367 | } 368 | 369 | def readColl(input: PrimitiveInputStream, c: ByteConverter[T]): Buffer[T] = { 370 | val length = input.readInt() 371 | val cpx = if (c.isInstanceOf[ComplexByteConverter[T]]) c.asInstanceOf[ComplexByteConverter[T]] else null 372 | 373 | var i = 0 374 | val buff = Buffer[T]() 375 | while (i < length) { 376 | val arrLength = input.readInt() 377 | if (cpx != null) { 378 | buff += cpx.read(input) 379 | } else { 380 | val arr = new Array[Byte](arrLength) 381 | input.read(arr) 382 | buff += c.fromBytes(arr) 383 | } 384 | i = i + 1 385 | } 386 | 387 | buff 388 | } 389 | } 390 | 391 | -------------------------------------------------------------------------------- /src/test/scala/com/gravity/hbase/schema/WebCrawlSchemaTest.scala: -------------------------------------------------------------------------------- 1 | package com.gravity.hbase.schema 2 | 3 | import com.gravity.hbase.mapreduce._ 4 | import java.lang.String 5 | import org.joda.time.{DateMidnight, DateTime} 6 | import com.gravity.hbase.mapreduce.{HMapReduceTask, HJob} 7 | import java.net.URL 8 | import com.gravity.hbase.schema._ 9 | import scala.collection._ 10 | import com.gravity.hbase.schema.WebCrawlingSchema.WebPageRow 11 | import org.junit.{Assert, Test} 12 | import org.apache.hadoop.io.Text 13 | 14 | /* )\._.,--....,'``. 15 | .b--. /; _.. \ _\ (`._ ,. 16 | `=,-,-'~~~ `----(,_..'--(,_..'`-.;.' */ 17 | 18 | object WebCrawlingSchema extends Schema { 19 | 20 | 21 | implicit val conf = LocalCluster.getTestConfiguration 22 | 23 | class WebTable extends HbaseTable[WebTable, String, WebPageRow](tableName = "pages", rowKeyClass = classOf[String]) { 24 | def rowBuilder(result: DeserializedResult) = new WebPageRow(this, result) 25 | 26 | val meta = family[String, String, Any]("meta") 27 | val title = column(meta, "title", classOf[String]) 28 | val lastCrawled = column(meta, "lastCrawled", classOf[DateTime]) 29 | 30 | val content = family[String, String, Any]("text", compressed = true) 31 | val article = column(content, "article", classOf[String]) 32 | val attributes = column(content, "attrs", classOf[Map[String, String]]) 33 | 34 | val searchMetrics = family[String, DateMidnight, Long]("searchesByDay") 35 | 36 | 37 | } 38 | 39 | class WebPageRow(table: WebTable, result: DeserializedResult) extends HRow[WebTable, String](result, table) { 40 | def domain = new URL(rowid).getAuthority 41 | } 42 | 43 | val WebTable = table(new WebTable) 44 | 45 | 46 | class SiteMetricsTable extends HbaseTable[SiteMetricsTable, String, SiteMetricsRow](tableName = "site-metrics", rowKeyClass = classOf[String]) { 47 | def rowBuilder(result: DeserializedResult) = new SiteMetricsRow(this, result) 48 | 49 | val meta = family[String, String, Any]("meta") 50 | val name = column(meta, "name", classOf[String]) 51 | 52 | val searchMetrics = family[String, DateMidnight, Long]("searchesByDay") 53 | } 54 | 55 | class SiteMetricsRow(table: SiteMetricsTable, result: DeserializedResult) extends HRow[SiteMetricsTable, String](result, table) 56 | 57 | val Sites = table(new SiteMetricsTable) 58 | 59 | } 60 | 61 | class WebContentSequencingJob extends HJob[NoSettings]("Content to sequence file by url", 62 | HMapTask( 63 | HTaskID("Sequencing Task"), 64 | HTaskConfigs(), 65 | HIO( 66 | HTableInput(WebCrawlingSchema.WebTable), 67 | HSequenceOutput[Text,Text]("/user/hpaste/reports/contentsequence") 68 | ), 69 | new FromTableMapper(WebCrawlingSchema.WebTable, classOf[Text], classOf[Text]) { 70 | def map() { 71 | val webpage = row 72 | val title = webpage.column(_.title).getOrElse("") 73 | val text = webpage.column(_.article).getOrElse("") 74 | if (title.length > 0 && text.length > 0) { 75 | write(new Text(title), new Text(text)) 76 | } 77 | } 78 | } 79 | ) 80 | ) 81 | 82 | 83 | class WebSearchAggregationJob extends HJob[NoSettings]("Aggregate web searches by site", 84 | HMapReduceTask( 85 | HTaskID("Aggregation task"), 86 | HTaskConfigs(), 87 | HIO( 88 | HTableInput(WebCrawlingSchema.WebTable), 89 | HTableOutput(WebCrawlingSchema.Sites) 90 | ), 91 | new FromTableBinaryMapperFx(WebCrawlingSchema.WebTable) { 92 | val webPage = row 93 | val domain = new URL(webPage.rowid).getAuthority 94 | ctr("Sites for domain" + domain) 95 | 96 | val dates = webPage.family(_.searchMetrics) 97 | 98 | for ((dateOfSearches, searchCount) <- dates) { 99 | val keyOutput = makeWritable { 100 | keyWriter => 101 | keyWriter.writeUTF(domain) 102 | keyWriter.writeObj(dateOfSearches) 103 | } 104 | val valueOutput = makeWritable { 105 | valueWriter => 106 | valueWriter.writeLong(searchCount) 107 | } 108 | ctr("Dated metrics written for domain " + domain) 109 | write(keyOutput, valueOutput) 110 | } 111 | }, 112 | new ToTableBinaryReducerFx(WebCrawlingSchema.Sites) { 113 | val (domain, dateOfSearches) = readKey { 114 | keyInput => 115 | (keyInput.readUTF(), keyInput.readObj[DateMidnight]) 116 | } 117 | 118 | var totalCounts = 0l 119 | 120 | perValue { 121 | valueInput => 122 | totalCounts += valueInput.readLong 123 | } 124 | 125 | 126 | write( 127 | WebCrawlingSchema.Sites.put(domain).valueMap(_.searchMetrics, Map(dateOfSearches -> totalCounts)) 128 | ) 129 | } 130 | ) 131 | ) 132 | 133 | class WebTablePagesBySiteJob extends HJob[NoSettings]("Get articles by site", 134 | HMapReduceTask( 135 | HTaskID("Articles by Site"), 136 | HTaskConfigs(), 137 | HIO( 138 | HTableInput(WebCrawlingSchema.WebTable), 139 | HPathOutput("/reports/wordcount") 140 | ), 141 | new FromTableBinaryMapperFx(WebCrawlingSchema.WebTable) { 142 | val webPage: WebPageRow = row //For illustrative purposes we're specifying the type here, no need to 143 | val domain = row.domain //We've added a convenience method to WebPageRow to extract the domain for us 144 | 145 | write( 146 | {keyOutput => keyOutput.writeUTF(domain)}, //This writes out the domain as the key 147 | {valueOutput => valueOutput.writeRow(WebCrawlingSchema.WebTable, webPage)} //This writes the entire value of the row out 148 | ) 149 | }, 150 | new BinaryToTextReducerFx { 151 | val domain = readKey(_.readUTF()) //This allows you to read out the key 152 | 153 | perValue { 154 | valueInput => 155 | val webPage: WebPageRow = valueInput.readRow(WebCrawlingSchema.WebTable) //Now you can read out the entire WebPageRow object from the value stream 156 | ctr("Pages for domain " + domain) 157 | writeln(domain + "\t" + webPage.column(_.title).getOrElse("No Title")) //This is a convenience function that writes a line to the text output 158 | } 159 | } 160 | ) 161 | ) 162 | 163 | class WebCrawlSchemaTest extends HPasteTestCase(WebCrawlingSchema) { 164 | 165 | @Test def testWebTablePutsAndGets() { 166 | WebCrawlingSchema.WebTable 167 | .put("http://mycrawledsite.com/crawledpage.html") 168 | .value(_.title, "My Crawled Page Title") 169 | .value(_.lastCrawled, new DateTime()) 170 | .value(_.article, "Jonsie went to the store. She didn't notice the spinning of the Earth, nor did the Earth notice the expansion of the Universe.") 171 | .value(_.attributes, Map("foo" -> "bar", "custom" -> "data")) 172 | .valueMap(_.searchMetrics, Map(new DateMidnight(2011, 6, 5) -> 3l, new DateMidnight(2011, 6, 4) -> 34l)) 173 | .execute() 174 | 175 | 176 | WebCrawlingSchema.WebTable.query2.withKey("http://mycrawledsite.com/crawledpage.html") 177 | .withColumns(_.title, _.lastCrawled) 178 | .withFamilies(_.searchMetrics) 179 | .singleOption() match { 180 | case Some(pageRow) => { 181 | println("Title: " + pageRow.column(_.title).getOrElse("No Title")) 182 | println("Crawled on: " + pageRow.column(_.lastCrawled).getOrElse(new DateTime())) 183 | 184 | pageRow.family(_.searchMetrics).foreach { 185 | case (date: DateMidnight, views: Long) => 186 | println("Got " + views + " views on date " + date.toString("MM-dd-yyyy")) 187 | } 188 | //Do something with title and crawled date... 189 | } 190 | case None => { 191 | println("Row not found") 192 | } 193 | } 194 | } 195 | 196 | @Test def testOpBaseAddition() { 197 | val url1 = "http://mycrawledsite.com/opbaseaddition.html" 198 | val url2 = "http://mycrawledsite.com/opbaseaddition2.html" 199 | 200 | val op1 = WebCrawlingSchema.WebTable.put(url1).value(_.title,"Addition1") 201 | op1.value(_.article,"How stop blop blop?") 202 | val op2 = WebCrawlingSchema.WebTable.put(url2).value(_.title,"Addition2") 203 | op2.value(_.article,"How now, brown cow") 204 | 205 | val res = (op1 + op2).execute() 206 | 207 | val results = WebCrawlingSchema.WebTable.query2.withKeys(Set(url1,url2)).executeMap() 208 | 209 | Assert.assertEquals("Addition1",results(url1).column(_.title).get) 210 | Assert.assertEquals("How stop blop blop?",results(url1).column(_.article).get) 211 | Assert.assertEquals("Addition2",results(url2).column(_.title).get) 212 | Assert.assertEquals("How now, brown cow",results(url2).column(_.article).get) 213 | 214 | } 215 | 216 | @Test def testAggregationMRJob() { 217 | WebCrawlingSchema.WebTable 218 | .put("http://mycrawledsite.com/crawledpage2.html") 219 | .value(_.title, "My Crawled Page Title") 220 | .value(_.lastCrawled, new DateTime()) 221 | .value(_.article, "Jonsie went to the store. She didn't notice the spinning of the Earth, nor did the Earth notice the expansion of the Universe.") 222 | .value(_.attributes, Map("foo" -> "bar", "custom" -> "data")) 223 | .valueMap(_.searchMetrics, Map(new DateMidnight(2011, 6, 5) -> 3l, new DateMidnight(2011, 6, 4) -> 34l)) 224 | .put("http://mycrawledsite.com/crawledpage3.html") 225 | .value(_.title, "My Crawled Page Title") 226 | .value(_.lastCrawled, new DateTime()) 227 | .value(_.article, "Jonsie went to the store. She didn't notice the spinning of the Earth, nor did the Earth notice the expansion of the Universe.") 228 | .value(_.attributes, Map("foo" -> "bar", "custom" -> "data")) 229 | .valueMap(_.searchMetrics, Map(new DateMidnight(2011, 6, 5) -> 3l, new DateMidnight(2011, 6, 4) -> 34l)) 230 | .put("http://mycrawledsite.com/crawledpage4.html") 231 | .value(_.title, "My Crawled Page Title") 232 | .value(_.lastCrawled, new DateTime()) 233 | .value(_.article, "Jonsie went to the store. She didn't notice the spinning of the Earth, nor did the Earth notice the expansion of the Universe.") 234 | .value(_.attributes, Map("foo" -> "bar", "custom" -> "data")) 235 | .valueMap(_.searchMetrics, Map(new DateMidnight(2011, 6, 5) -> 3l, new DateMidnight(2011, 6, 4) -> 34l)) 236 | .execute() 237 | 238 | 239 | new WebSearchAggregationJob().run(Settings.None, LocalCluster.getTestConfiguration) 240 | 241 | WebCrawlingSchema.Sites.query2.withKey("mycrawledsite.com").singleOption() match { 242 | case Some(siteRow) => { 243 | siteRow.family(_.searchMetrics).foreach {println} 244 | } 245 | case None => { 246 | println("Didn't find the site, strange!") 247 | } 248 | } 249 | } 250 | 251 | @Test def testPagesBySiteJob() { 252 | WebCrawlingSchema.WebTable 253 | .put("http://mycrawledsite2.com/crawledpage2.html") 254 | .value(_.title, "My Crawled Page Title2") 255 | .valueMap(_.searchMetrics, Map(new DateMidnight(2011, 6, 5) -> 3l, new DateMidnight(2011, 6, 4) -> 34l)) 256 | .put("http://mycrawledsite2.com/crawledpage3.html") 257 | .value(_.title, "My Crawled Page Title3") 258 | .valueMap(_.searchMetrics, Map(new DateMidnight(2011, 6, 5) -> 3l, new DateMidnight(2011, 6, 4) -> 34l)) 259 | .put("http://mycrawledsite3.com/crawledpage4.html") 260 | .value(_.title, "My Crawled Page Title4") 261 | .valueMap(_.searchMetrics, Map(new DateMidnight(2011, 6, 5) -> 3l, new DateMidnight(2011, 6, 4) -> 34l)) 262 | .put("http://mycrawledsite3.com/crawledpage4.html") 263 | .value(_.title, "My Crawled Page Title5") 264 | .valueMap(_.searchMetrics, Map(new DateMidnight(2011, 6, 5) -> 3l, new DateMidnight(2011, 6, 4) -> 34l)) 265 | .put("http://mycrawledsite3.com/crawledpage4.html") 266 | .value(_.title, "My Crawled Page Title6") 267 | .valueMap(_.searchMetrics, Map(new DateMidnight(2011, 6, 5) -> 3l, new DateMidnight(2011, 6, 4) -> 34l)) 268 | .execute() 269 | 270 | new WebTablePagesBySiteJob().run(Settings.None, LocalCluster.getTestConfiguration) 271 | } 272 | 273 | @Test def testBeginValueFiltration() { 274 | val site1 = "http://filtersite1.com/" 275 | val site2 = "http://filtersite2.com/" 276 | 277 | val articles = Map("About Cats" -> "aboutcats.html", "About Dogs" -> "aboutdogs.html", "About Orangutans" -> "aboutorangutans.html", "Interview with a Kitten" -> "interview.html") 278 | 279 | for { 280 | (title, shortUrl) <- articles 281 | longUrl = site1 + shortUrl 282 | longUrl2 = site2 + shortUrl 283 | } { 284 | WebCrawlingSchema.WebTable.put(longUrl).value(_.title, longUrl).execute() 285 | WebCrawlingSchema.WebTable.put(longUrl2).value(_.title, longUrl2).execute() 286 | } 287 | 288 | val results = WebCrawlingSchema.WebTable.query2.filter(_.or(_.columnValueMustContain(_.title,site1))).scanToIterable(itm=>itm) 289 | Assert.assertTrue(results.size == 4) 290 | 291 | val results2 = WebCrawlingSchema.WebTable.query2.filter(_.or(_.columnValueMustContain(_.title,site2))).scanToIterable(itm=>itm) 292 | Assert.assertTrue(results2.size == 4) 293 | 294 | } 295 | 296 | @Test def testNoOpExecute() { 297 | WebCrawlingSchema.WebTable.put("Hello").execute() 298 | } 299 | 300 | @Test def testDeletion() { 301 | WebCrawlingSchema.WebTable.put("http://hithere.com/yo").value(_.title,"Hi, this will be deleted").execute() 302 | WebCrawlingSchema.WebTable.delete("http://hithere.com/yo").execute() 303 | 304 | WebCrawlingSchema.WebTable.query2.withKey("http://hithere.com/yo").singleOption() match { 305 | case Some(result) => { 306 | Assert.fail("Deletion did not go through") 307 | } 308 | case None => { 309 | } 310 | } 311 | } 312 | 313 | @Test def testScanBatching() { 314 | WebCrawlingSchema.WebTable.put("http://batching.com/article1").value(_.title,"Batch Title 1").value(_.article,"Content 1").execute() 315 | WebCrawlingSchema.WebTable.put("http://batching.com/article2").value(_.title,"Batch Title 2").value(_.article,"Content 2").execute() 316 | WebCrawlingSchema.WebTable.put("http://batching.com/article3").value(_.title,"Batch Title 3").value(_.article,"Content 3").execute() 317 | WebCrawlingSchema.WebTable.put("http://batching.com/article4").value(_.title,"Batch Title 4").value(_.article,"Content 4").execute() 318 | WebCrawlingSchema.WebTable.query2.withBatchSize(1).scan({page=> 319 | page.prettyPrintNoValues() 320 | Assert.assertTrue(page.size <= 1) 321 | }) 322 | 323 | WebCrawlingSchema.WebTable.query2.withBatchSize(2).scan({page=> 324 | page.prettyPrintNoValues() 325 | Assert.assertTrue(page.size <= 2) 326 | }) 327 | 328 | } 329 | 330 | @Test def testContentSequencing() { 331 | val domain = "http://sequencing.com/" 332 | WebCrawlingSchema.WebTable.put(domain + "article1").value(_.title,"Batch Title 1").value(_.article,"Content 1").execute() 333 | WebCrawlingSchema.WebTable.put(domain + "article2").value(_.title,"Batch Title 2").value(_.article,"Content 2").execute() 334 | WebCrawlingSchema.WebTable.put(domain + "article3").value(_.title,"Batch Title 3").value(_.article,"Content 3").execute() 335 | WebCrawlingSchema.WebTable.put(domain + "article4").value(_.title,"Batch Title 4").value(_.article,"Content 4").execute() 336 | new WebContentSequencingJob().run(Settings.None,LocalCluster.getTestConfiguration) 337 | } 338 | } -------------------------------------------------------------------------------- /src/main/scala/com/gravity/hbase/schema/HbaseTable.scala: -------------------------------------------------------------------------------- 1 | package com.gravity.hbase.schema 2 | 3 | import java.nio.ByteBuffer 4 | import scala.collection.mutable.ArrayBuffer 5 | import scala.collection._ 6 | import org.apache.commons.lang.ArrayUtils 7 | import org.apache.hadoop.hbase.util.Bytes 8 | import com.gravity.hbase.AnyConverterSignal 9 | import org.apache.hadoop.hbase.util.Bytes.ByteArrayComparator 10 | import java.io.IOException 11 | import org.apache.hadoop.conf.Configuration 12 | import java.util.Arrays 13 | import org.apache.hadoop.hbase.{HColumnDescriptor, KeyValue} 14 | import scala.Int 15 | import org.apache.hadoop.hbase.client._ 16 | 17 | /* )\._.,--....,'``. 18 | .b--. /; _.. \ _\ (`._ ,. 19 | `=,-,-'~~~ `----(,_..'--(,_..'`-.;.' */ 20 | 21 | /** 22 | * Represents the structural configuration for a table 23 | * @param maxFileSizeInBytes 24 | */ 25 | case class HbaseTableConfig( 26 | maxFileSizeInBytes:Long = -1, 27 | memstoreFlushSizeInBytes:Long = -1, 28 | tablePoolSize:Int = 5 29 | ) 30 | 31 | object HbaseTable { 32 | def defaultConfig = HbaseTableConfig() 33 | } 34 | 35 | /** 36 | * Represents a Table. Expects an instance of HBaseConfiguration to be present. 37 | * A parameter-type T should be the actual table that is implementing this one 38 | * (this is to allow syntactic sugar for easily specifying columns during 39 | * queries). 40 | * A parameter-type R should be the type of the key for the table. 41 | * @param tableName 42 | * @param cache 43 | * @param rowKeyClass 44 | * @param logSchemaInconsistencies 45 | * @param conf 46 | * @param keyConverter 47 | * @tparam T 48 | * @tparam R 49 | * @tparam RR 50 | */ 51 | abstract class HbaseTable[T <: HbaseTable[T, R, RR], R, RR <: HRow[T, R]](val tableName: String, var cache: QueryResultCache[T, R, RR] = new NoOpCache[T, R, RR](), rowKeyClass: Class[R], logSchemaInconsistencies: Boolean = false, tableConfig:HbaseTableConfig = HbaseTable.defaultConfig)(implicit conf: Configuration, keyConverter: ByteConverter[R]) { 52 | 53 | 54 | def rowBuilder(result: DeserializedResult): RR 55 | 56 | val rowKeyConverter = keyConverter 57 | 58 | /**Provides the client with an instance of the superclass this table was defined against. */ 59 | def pops = this.asInstanceOf[T] 60 | 61 | /**A method injected by the super class that will build a strongly-typed row object. */ 62 | def buildRow(result: Result): RR = { 63 | rowBuilder(convertResult(result)) 64 | } 65 | 66 | /**A pool of table objects with AutoFlush set to true */ 67 | val tablePool = new HTablePool(conf, tableConfig.tablePoolSize) 68 | 69 | /**A pool of table objects with AutoFlush set to false --therefore usable for asynchronous write buffering */ 70 | val bufferTablePool = new HTablePool(conf, 1, new HTableInterfaceFactory { 71 | def createHTableInterface(config: Configuration, tableName: Array[Byte]): HTableInterface = { 72 | val table = new HTable(conf, tableName) 73 | table.setWriteBufferSize(2000000L) 74 | table.setAutoFlush(false) 75 | table 76 | } 77 | 78 | def releaseHTableInterface(table: HTableInterface) { 79 | try { 80 | table.close() 81 | } catch { 82 | case ex: IOException => throw new RuntimeException(ex) 83 | } 84 | } 85 | }) 86 | 87 | 88 | @volatile var famLookup: Array[Array[Byte]] = null 89 | @volatile var colFamLookup: Array[Array[Byte]] = null 90 | @volatile var famIdx: IndexedSeq[KeyValueConvertible[_, _, _]] = null 91 | @volatile var colFamIdx: IndexedSeq[KeyValueConvertible[_, _, _]] = null 92 | 93 | val bc = new ByteArrayComparator() 94 | 95 | implicit val o = new math.Ordering[Array[Byte]] { 96 | def compare(a: Array[Byte], b: Array[Byte]): Int = { 97 | bc.compare(a, b) 98 | } 99 | } 100 | 101 | 102 | /**Looks up a KeyValueConvertible by the family and column bytes provided. 103 | * Because of the rules of the system, the lookup goes as follows: 104 | * 1. Find a column first. If you find a column first, it means there is a strongly-typed column defined. 105 | * 2. If no column, then find the family. 106 | * 107 | */ 108 | def converterByBytes(famBytes: Array[Byte], colBytes: Array[Byte]): KeyValueConvertible[_, _, _] = { 109 | 110 | if (colFamLookup.length == 0 || famLookup.length == 0) { 111 | throw new RuntimeException("Attempting to lookup 0 length columns and families--HBaseTable is corrupt") 112 | } 113 | 114 | val fullKey = ArrayUtils.addAll(famBytes, colBytes) 115 | val resIdx = Arrays.binarySearch(colFamLookup, fullKey, bc) 116 | if (resIdx > -1) { 117 | colFamIdx(resIdx) 118 | } else { 119 | val resFamIdx = Arrays.binarySearch(famLookup, famBytes, bc) 120 | if (resFamIdx > -1) { 121 | famIdx(resFamIdx) 122 | } 123 | else { 124 | null 125 | } 126 | } 127 | 128 | 129 | } 130 | 131 | /**Converts a result to a DeserializedObject. A conservative implementation that is slower than convertResultRaw but will always be more stable against 132 | * binary changes to Hbase's KeyValue format. 133 | */ 134 | def convertResult(result: Result) = { 135 | if (result.isEmpty) { 136 | throw new RuntimeException("Attempting to deserialize an empty result. If you want to handle the eventuality of an empty result, call singleOption() instead of single()") 137 | } 138 | val keyValues = result.raw() 139 | val buff = result.getBytes.get() 140 | 141 | val rowId = keyConverter.fromBytes(buff, keyValues(0).getRowOffset, keyValues(0).getRowLength).asInstanceOf[AnyRef] 142 | 143 | val ds = DeserializedResult(rowId, families.size) 144 | 145 | var itr = 0 146 | 147 | while (itr < keyValues.length) { 148 | val kv = keyValues(itr) 149 | val family = kv.getFamily 150 | val key = kv.getQualifier 151 | try { 152 | val c = converterByBytes(family, key) 153 | if (c == null) { 154 | if (logSchemaInconsistencies) { 155 | println("Table: " + tableName + " : Null Converter : " + Bytes.toString(kv.getFamily)) 156 | } 157 | } 158 | else if (!c.keyConverter.isInstanceOf[AnyConverterSignal] && !c.valueConverter.isInstanceOf[AnyConverterSignal]) { 159 | val f = c.family 160 | val k = c.keyConverter.fromBytes(buff, kv.getQualifierOffset, kv.getQualifierLength).asInstanceOf[AnyRef] 161 | val r = c.valueConverter.fromBytes(buff, kv.getValueOffset, kv.getValueLength).asInstanceOf[AnyRef] 162 | val ts = kv.getTimestamp 163 | 164 | ds.add(f, k, r, ts) 165 | } else { 166 | if (logSchemaInconsistencies) { 167 | println("Table: " + tableName + " : Any Converter : " + Bytes.toString(kv.getFamily)) 168 | } 169 | //TODO: Just like AnyNotSupportException, add a counter here because this means a column was removed, but the data is still in the database. 170 | } 171 | } finally { 172 | itr = itr + 1 173 | } 174 | 175 | } 176 | ds 177 | } 178 | 179 | /** 180 | * 181 | * @param result 182 | * @return 183 | */ 184 | def convertResultRaw(result: Result) = { 185 | 186 | 187 | val bytes = result.getBytes() 188 | val buf = bytes.get() 189 | var offset = bytes.getOffset 190 | val finalOffset = bytes.getSize + offset 191 | var row: Array[Byte] = null 192 | var ds: DeserializedResult = null 193 | 194 | while (offset < finalOffset) { 195 | val keyLength = Bytes.toInt(buf, offset) 196 | offset = offset + Bytes.SIZEOF_INT 197 | 198 | val keyOffset = offset + KeyValue.ROW_OFFSET 199 | val rowLength = Bytes.toShort(buf, keyOffset) 200 | val familyOffset = offset + KeyValue.ROW_OFFSET + Bytes.SIZEOF_SHORT + rowLength + Bytes.SIZEOF_BYTE 201 | val familyLength = buf(familyOffset - 1) 202 | val family = new Array[Byte](familyLength) 203 | System.arraycopy(buf, familyOffset, family, 0, familyLength) 204 | 205 | val qualifierOffset = familyOffset + familyLength 206 | val qualifierLength = keyLength - (KeyValue.KEY_INFRASTRUCTURE_SIZE + rowLength + familyLength) 207 | val key = new Array[Byte](qualifierLength) 208 | System.arraycopy(buf, qualifierOffset, key, 0, qualifierLength) 209 | 210 | val valueOffset = keyOffset + keyLength 211 | val valueLength = Bytes.toInt(buf, offset + Bytes.SIZEOF_INT) 212 | val value = new Array[Byte](valueLength) 213 | System.arraycopy(buf, valueOffset, value, 0, valueLength) 214 | 215 | val tsOffset = keyOffset + keyLength - KeyValue.TIMESTAMP_TYPE_SIZE 216 | val ts = Bytes.toLong(buf, tsOffset) 217 | 218 | if (row == null) { 219 | val rowOffset = keyOffset + Bytes.SIZEOF_SHORT 220 | row = new Array[Byte](rowLength) 221 | System.arraycopy(buf, rowOffset, row, 0, rowLength) 222 | val rowId = keyConverter.fromBytes(result.getRow).asInstanceOf[AnyRef] 223 | ds = DeserializedResult(rowId, families.size) 224 | } 225 | 226 | try { 227 | val c = converterByBytes(family, key) 228 | val f = c.family 229 | val k = c.keyConverter.fromBytes(key).asInstanceOf[AnyRef] 230 | val r = c.valueConverter.fromBytes(value).asInstanceOf[AnyRef] 231 | println("Adding value " + r) 232 | ds.add(f, k, r, ts) 233 | } catch { 234 | case ex: Exception => { 235 | println("Adding error buffer") 236 | ds.addErrorBuffer(family, key, value, ts) 237 | } 238 | } 239 | 240 | offset = offset + keyLength 241 | } 242 | ds 243 | } 244 | 245 | 246 | def familyBytes = families.map(family => family.familyBytes) 247 | 248 | def familyByIndex(idx: Int) = familyArray(idx) 249 | 250 | lazy val familyArray = { 251 | val arr = new Array[ColumnFamily[_, _, _, _, _]](families.length) 252 | families.foreach { 253 | fam => 254 | arr(fam.index) = fam 255 | } 256 | arr 257 | } 258 | 259 | def columnByIndex(idx: Int) = columnArray(idx) 260 | 261 | lazy val columnArray = { 262 | val arr = new Array[Column[_, _, _, _, _]](columns.length) 263 | columns.foreach {col => arr(col.columnIndex) = col} 264 | arr 265 | } 266 | 267 | 268 | //alter 'articles', NAME => 'html', VERSIONS =>1, COMPRESSION=>'lzo' 269 | 270 | /** 271 | * Generates a creation script for the table, based on the column families and table config. 272 | * @param tableNameOverride 273 | * @return 274 | */ 275 | def createScript(tableNameOverride: String = tableName) = { 276 | var create = "create '" + tableNameOverride + "', " 277 | create += (for (family <- families) yield { 278 | familyDef(family) 279 | }).mkString(",") 280 | 281 | create += alterTableAttributesScripts(tableNameOverride) 282 | 283 | create 284 | } 285 | 286 | def alterTableAttributesScripts(tableName:String) = { 287 | var alterScript = "" 288 | if(tableConfig.memstoreFlushSizeInBytes > -1) { 289 | alterScript += alterTableAttributeScript(tableName, "MEMSTORE_FLUSHSIZE", tableConfig.memstoreFlushSizeInBytes.toString) 290 | } 291 | if(tableConfig.maxFileSizeInBytes > -1) { 292 | alterScript += alterTableAttributeScript(tableName, "MAX_FILESIZE", tableConfig.maxFileSizeInBytes.toString) 293 | } 294 | alterScript 295 | } 296 | 297 | def alterTableAttributeScript(tableName:String, attributeName:String, value:String) = { 298 | "\nalter '" + tableName + "', {METHOD => 'table_att', "+attributeName+" => '" + value + "'}" 299 | } 300 | 301 | def deleteScript(tableNameOverride: String = tableName) = { 302 | val delete = "disable '" + tableNameOverride + "'\n" 303 | 304 | delete + "delete '" + tableNameOverride + "'" 305 | } 306 | 307 | /** 308 | * Generates a production-friendly alter script (flush, disable, alter, enable) 309 | * @param tableNameOverride 310 | * @param families 311 | * @return 312 | */ 313 | def alterScript(tableNameOverride: String = tableName, families: Seq[ColumnFamily[T, _, _, _, _]] = families) = { 314 | 315 | var alter = "flush '" + tableNameOverride + "'\n" 316 | alter += "disable '" + tableNameOverride + "'\n" 317 | alter += "alter '" + tableNameOverride + "', " 318 | alter += (for (family <- families) yield { 319 | familyDef(family) 320 | }).mkString(",") 321 | 322 | alter += alterTableAttributesScripts(tableNameOverride) 323 | alter += "\nenable '" + tableNameOverride + "'" 324 | alter 325 | } 326 | 327 | def familyDef(family: ColumnFamily[T, _, _, _, _]) = { 328 | val compression = if (family.compressed) ", COMPRESSION=>'lzo'" else "" 329 | val ttl = if (family.ttlInSeconds < HColumnDescriptor.DEFAULT_TTL) ", TTL=>'" + family.ttlInSeconds + "'" else "" 330 | "{NAME => '%s', VERSIONS => %d%s%s}".format(Bytes.toString(family.familyBytes), family.versions, compression, ttl) 331 | } 332 | 333 | 334 | def getTable(name: String) = tablePool.getTable(name) 335 | 336 | def getBufferedTable(name: String) = bufferTablePool.getTable(name) 337 | 338 | private val columns = ArrayBuffer[Column[T, R, _, _, _]]() 339 | val families = ArrayBuffer[ColumnFamily[T, R, _, _, _]]() 340 | 341 | val columnsByName = mutable.Map[AnyRef, Column[T, R, _, _, _]]() 342 | 343 | private val columnsByBytes = mutable.Map[ByteBuffer, KeyValueConvertible[_, _, _]]() 344 | private val familiesByBytes = mutable.Map[ByteBuffer, KeyValueConvertible[_, _, _]]() 345 | 346 | var columnIdx = 0 347 | 348 | def column[F, K, V](columnFamily: ColumnFamily[T, R, F, K, _], columnName: K, valueClass: Class[V])(implicit fc: ByteConverter[F], kc: ByteConverter[K], kv: ByteConverter[V]) = { 349 | val c = new Column[T, R, F, K, V](this, columnFamily, columnName, columnIdx) 350 | columns += c 351 | 352 | val famBytes = columnFamily.familyBytes 353 | val colBytes = c.columnBytes 354 | val fullKey = ArrayUtils.addAll(famBytes, colBytes) 355 | val bufferKey = ByteBuffer.wrap(fullKey) 356 | 357 | columnsByName.put(columnName.asInstanceOf[AnyRef], c) 358 | columnsByBytes.put(bufferKey, c) 359 | columnIdx = columnIdx + 1 360 | c 361 | } 362 | 363 | var familyIdx = 0 364 | 365 | def family[F, K, V](familyName: F, compressed: Boolean = false, versions: Int = 1, rowTtlInSeconds: Int = Int.MaxValue)(implicit c: ByteConverter[F], d: ByteConverter[K], e: ByteConverter[V]) = { 366 | val family = new ColumnFamily[T, R, F, K, V](this, familyName, compressed, versions, familyIdx, rowTtlInSeconds) 367 | familyIdx = familyIdx + 1 368 | families += family 369 | familiesByBytes.put(ByteBuffer.wrap(family.familyBytes), family) 370 | family 371 | } 372 | 373 | def getTableOption(name: String) = { 374 | try { 375 | Some(getTable(name)) 376 | } catch { 377 | case e: Exception => None 378 | } 379 | } 380 | 381 | 382 | def withTableOption[Q](name: String)(work: (Option[HTableInterface]) => Q): Q = { 383 | val table = getTableOption(name) 384 | try { 385 | work(table) 386 | } finally { 387 | table foreach (tbl => tablePool.putTable(tbl)) 388 | } 389 | } 390 | 391 | def withBufferedTable[Q](mytableName: String = tableName)(work: (HTableInterface) => Q): Q = { 392 | val table = getBufferedTable(mytableName) 393 | try { 394 | work(table) 395 | } finally { 396 | bufferTablePool.putTable(table) 397 | } 398 | } 399 | 400 | def withTable[Q](mytableName: String = tableName)(funct: (HTableInterface) => Q): Q = { 401 | withTableOption(mytableName) { 402 | case Some(table) => { 403 | funct(table) 404 | } 405 | case None => throw new RuntimeException("Table " + tableName + " does not exist") 406 | } 407 | } 408 | 409 | @deprecated("Use query2 instead, it is a generic interface for gets or scans") 410 | def scan = new ScanQuery(this) 411 | 412 | @deprecated("Use query2 instead") 413 | def query = new Query(this) 414 | 415 | def query2 = new Query2(this) 416 | 417 | def put(key: R, writeToWAL: Boolean = true) = new PutOp(this, keyConverter.toBytes(key)) 418 | 419 | def delete(key: R) = new DeleteOp(this, keyConverter.toBytes(key)) 420 | 421 | def increment(key: R) = new IncrementOp(this, keyConverter.toBytes(key)) 422 | 423 | 424 | def init() { 425 | famLookup = Array.ofDim[Array[Byte]](families.size) 426 | for ((fam, idx) <- families.zipWithIndex) { 427 | famLookup(idx) = fam.familyBytes 428 | } 429 | Arrays.sort(famLookup, bc) 430 | famIdx = families.sortBy(_.familyBytes).toIndexedSeq 431 | 432 | colFamLookup = Array.ofDim[Array[Byte]](columns.size) 433 | for ((col, idx) <- columns.zipWithIndex) { 434 | colFamLookup(idx) = ArrayUtils.addAll(col.familyBytes, col.columnBytes) 435 | } 436 | Arrays.sort(colFamLookup, bc) 437 | colFamIdx = columns.sortBy(col => ArrayUtils.addAll(col.familyBytes, col.columnBytes)).toIndexedSeq 438 | } 439 | 440 | } 441 | -------------------------------------------------------------------------------- /README.markdown: -------------------------------------------------------------------------------- 1 | # Welcome to HPaste! 2 | 3 | ### What is HPaste? 4 | 5 | HPaste unlocks the rich functionality of HBase for a Scala audience. In so doing, it attempts to achieve the following goals: 6 | 7 | * Provide a strong, clear syntax for querying and filtration 8 | * Perform as fast as possible while maintaining idiomatic Scala client code -- the abstractions should not show up in a profiler! 9 | * Re-articulate HBase's data structures rather than force it into an ORM-style atmosphere. 10 | * A rich set of base classes for writing MapReduce jobs in hadoop against HBase tables. 11 | * Provide a maximum amount of code re-use between general Hbase client usage, and operation from within a MapReduce job. 12 | * Use Scala's type system to its advantage--the compiler should verify the integrity of the schema. 13 | * Be a verbose DSL--minimize boilerplate code, but be human readable! 14 | 15 | ### What isn't HPaste? 16 | 17 | You'll notice that HPaste has a lot of convenience classes for MapReduce jobs. This is to make it painless to use your domain objects and tables in the context of MR jobs. HPaste has no aspirations to replace tuple-based frameworks like Pig or Cascading (both of which we use for complex log parsing). HPaste is intended to hug the Hadoop MapReduce API very closely, building convenience functions where necessary, without abstracting too far away from the base concept. 18 | 19 | The goal of HPaste's MapReduce support is to allow you to build rich functionality into your Table and Row objects, and make it be painless to have those tables and rows participate in MapReduce jobs. Oftentimes in HBase you have a combination of OLTP style operations (client gets data, client serves data), and OLAP style operations (pivot one table around a particular piece of data, and output to another table). That is where HPaste comes in handy, because there is often an impedance in Pig and/or Cascading between HBase-friendly binary data serialized objects and the tuple framework that makes those libraries so awesome to use for ad-hoc log-style data. 20 | 21 | (It is a mini-goal of HPaste to integrate into Cascading's tuple framework.) 22 | 23 | ## Project Status 24 | This project is currently actively developed and maintained. It is used in a large production codebase in high-throughput, memory-intensive scenarios, and has many months of bug fixes under its belt. Because it already has a great deal of code utilizing it, there will not be many breaking changes to the API. Instead what we usually do is provide an upgraded API that sits next to the old API, then deprecate the old one. 25 | 26 | ## Installation 27 | 28 | This project uses [Maven](http://maven.apache.org/ "Apache Maven"). To use HPaste in your own maven project, simply add it as a dependency: 29 | 30 | ```xml 31 | 32 | com.gravity 33 | gravity-hpaste 34 | 0.1.11 35 | 36 | ``` 37 | 38 | ## Quickstart 39 | 40 | Here's some quick code examples to give you a sense of what you're getting into. All of the examples in the sections below come from the HPaste unit tests. Specifically the file [WebCrawlSchemaTest.scala](https://github.com/GravityLabs/HPaste/blob/master/src/test/scala/com/gravity/hbase/schema/WebCrawlSchemaTest.scala). If you go to that file and follow along with the explanations below, things will make more sense. 41 | 42 | #### Creating a WebTable 43 | The classic case for HBase and BigTable is crawling and storing web pages. You need to define a WebTable for your crawling. The below defines a table called WebTable, with a String key. 44 | 45 | * It has a column family called "meta", that holds columns with String keys and Any value-type. 46 | * It then specifies that the "meta" family holds a "title" column, a "lastCrawled" column, and a "url" column. 47 | * A second column family holds content and has the compressed flag to true. 48 | * It has a family "content" for storing content. The "article" column is for storing the main page content. 49 | * The Attributes column is a map where you can atomically store values keyed by a string. 50 | * SearchMetrics is a column family that contains searches your users have made that have sent them to that page, organized by day. 51 | 52 | ```scala 53 | class WebTable extends HbaseTable[WebTable, String, WebPageRow](tableName = "pages", rowKeyClass = classOf[String]) { 54 | def rowBuilder(result: DeserializedResult) = new WebPageRow(this, result) 55 | 56 | val meta = family[String, String, Any]("meta") 57 | val title = column(meta, "title", classOf[String]) 58 | val lastCrawled = column(meta, "lastCrawled", classOf[DateTime]) 59 | 60 | val content = family[String, String, Any]("text", compressed = true) 61 | val article = column(content, "article", classOf[String]) 62 | val attributes = column(content, "attrs", classOf[Map[String, String]]) 63 | 64 | val searchMetrics = family[String, DateMidnight, Long]("searchesByDay") 65 | 66 | 67 | } 68 | 69 | class WebPageRow(table: WebTable, result: DeserializedResult) extends HRow[WebTable, String](result, table) 70 | 71 | val WebTable = table(new WebTable) 72 | } 73 | ``` 74 | 75 | #### Putting values into the WebTable 76 | 77 | Now, suppose you're crawling a website. The below will create a row with the values specified. When you call value(), the first argument is a function that points to the column you specified in the above WebTable schema. This dips into DSL-land. 78 | 79 | ```scala 80 | WebCrawlingSchema.WebTable 81 | .put("http://mycrawledsite.com/crawledpage.html") 82 | .value(_.title, "My Crawled Page Title") 83 | .value(_.lastCrawled, new DateTime()) 84 | .value(_.article, "Jonsie went to the store. She didn't notice the spinning of the Earth, nor did the Earth notice the expansion of the Universe.") 85 | .value(_.attributes, Map("foo" -> "bar", "custom" -> "data")) 86 | .valueMap(_.searchMetrics, Map(new DateMidnight(2011, 6, 5) -> 3l, new DateMidnight(2011, 6, 4) -> 34l)) 87 | .execute() 88 | ``` 89 | 90 | #### Querying values out of the WebTable 91 | 92 | Let's get the above page out of the WebTable. Let's say we just want the title of the page and when it was last crawled. The withColumns() call tells HBase to only fetch those columns. It takes a series of functions that return the column values you specified in the WebTable, so you get compile-time checking on that. 93 | 94 | ```scala 95 | WebCrawlingSchema.WebTable.query2.withKey("http://mycrawledsite.com/crawledpage.html") 96 | .withColumns(_.title, _.lastCrawled) 97 | .withFamilies(_.searchMetrics) 98 | .singleOption() match { 99 | case Some(pageRow) => { 100 | println("Title: " + pageRow.column(_.title).getOrElse("No Title")) 101 | println("Crawled on: " + pageRow.column(_.lastCrawled).getOrElse(new DateTime())) 102 | 103 | pageRow.family(_.searchMetrics).foreach { 104 | case (date: DateMidnight, views: Long) => 105 | println("Got " + views + " views on date " + date.toString("MM-dd-yyyy")) 106 | } 107 | //Do something with title and crawled date... 108 | } 109 | case None => { 110 | println("Row not found") 111 | } 112 | } 113 | ``` 114 | 115 | The result you get back is an instance of the row class you specified against the WebTable: WebPageRow. When you get a WebPageRow back from a query, a scan, or a map reduce job, you can fetch the columns out via the column() call. If you asked for a column family that can be treated as a Map (a Column Family that does not have columns specified in it), then you can retrieve the map via the family() call. 116 | 117 | #### Aggregating values via MapReduce jobs 118 | 119 | HPaste contains support for low-level MapReduce operations. What we mean by low-level is that there are not many layers of abstraction on top of a basic MR job--instead, we focus on making it easy to create and manage table inputs and outputs and serialize binary data between them. 120 | 121 | Let's say we have a lot of pages crawled, and we, as a search engine, have searches people have performed for those pages. We now want to roll those into per-site totals. Let's create a table next to our WebTable called the SiteMetricsTable: 122 | 123 | ```scala 124 | class SiteMetricsTable extends HbaseTable[SiteMetricsTable, String, SiteMetricsRow](tableName = "site-metrics", rowKeyClass = classOf[String]) { 125 | def rowBuilder(result: DeserializedResult) = new SiteMetricsRow(this, result) 126 | 127 | val meta = family[String, String, Any]("meta") 128 | val name = column(meta, "name", classOf[String]) 129 | 130 | val searchMetrics = family[String, DateMidnight, Long]("searchesByDay") 131 | } 132 | 133 | class SiteMetricsRow(table: SiteMetricsTable, result: DeserializedResult) extends HRow[SiteMetricsTable, String](result, table) 134 | 135 | val Sites = table(new SiteMetricsTable) 136 | 137 | ``` 138 | 139 | Now we'll make a MapReduce job that scans the WebPages table, aggregates its metrics, and writes them to the SiteMetricsTable: 140 | 141 | ```scala 142 | class WebSearchAggregationJob extends HJob[NoSettings]("Aggregate web searches by site", 143 | HMapReduceTask( 144 | HTaskID("Aggregation task"), 145 | HTaskConfigs(), 146 | HIO( 147 | HTableInput(WebCrawlingSchema.WebTable), 148 | HTableOutput(WebCrawlingSchema.Sites) 149 | ), 150 | new FromTableBinaryMapperFx(WebCrawlingSchema.WebTable) { 151 | val webPage = row 152 | val domain = new URL(webPage.rowid).getAuthority 153 | ctr("Sites for domain" + domain) 154 | 155 | val dates = webPage.family(_.searchMetrics) 156 | 157 | for((dateOfSearches,searchCount) <- dates) { 158 | val keyOutput = makeWritable{keyWriter=> 159 | keyWriter.writeUTF(domain) 160 | keyWriter.writeObj(dateOfSearches) 161 | } 162 | val valueOutput = makeWritable{valueWriter=> 163 | valueWriter.writeLong(searchCount) 164 | } 165 | ctr("Dated metrics written for domain " + domain) 166 | write(keyOutput, valueOutput) 167 | } 168 | }, 169 | new ToTableBinaryReducerFx(WebCrawlingSchema.Sites) { 170 | val (domain, dateOfSearches) = readKey{keyInput=> 171 | (keyInput.readUTF(), keyInput.readObj[DateMidnight]) 172 | } 173 | 174 | var totalCounts = 0l 175 | 176 | perValue{valueInput=> 177 | totalCounts += valueInput.readLong 178 | } 179 | 180 | 181 | write( 182 | WebCrawlingSchema.Sites.put(domain).valueMap(_.searchMetrics,Map(dateOfSearches->totalCounts)) 183 | ) 184 | } 185 | ) 186 | ) 187 | 188 | ``` 189 | The above is a self-contained MapReduce job that is ready to go. We can execute the above job via (where the Configuration object is the one relevant to your clsuter): 190 | 191 | ```scala 192 | new WebSearchAggregationJob().run(Settings.None, LocalCluster.getTestConfiguration) 193 | ``` 194 | 195 | All of the above examples are part of the HPaste unit tests, so it should be easy to use them to set up your own system. 196 | 197 | ## Features not covered in the Quickstart 198 | There are many features not included in the Quickstart that exist in the codebase. 199 | 200 | * Complex type serialization 201 | * Chaining map reduce jobs 202 | * Scanners and the filtration DSL 203 | * Settings classes that can be injected into MapReduce jobs 204 | 205 | # Building and Testing HPaste 206 | This project is put together with Maven. In theory you should be able to build and run the project's tests via: 207 | 208 | `` 209 | mvn test 210 | `` 211 | 212 | The tests will create a temporary hbase cluster, create temporary tables, and run map reduce jobs against those tables. The unit tests are the best way to encounter HPaste, because they are constantly added to and perform live operations against a real cluster, so there's no smoke-and-mirrors. 213 | 214 | # More In Depth 215 | 216 | ## Defining a Schema 217 | 218 | Following the HBase structure, first you define a Schema, then Tables, then Column Families, then (optionally) Columns. Below is an example schema that contains a single Table definition. The table will be called "schema_example" in HBase. It will expect its row keys to be Strings. 219 | 220 | ```scala 221 | object ExampleSchema extends Schema { 222 | 223 | 224 | //There should only be one HBaseConfiguration object per process. You'll probably want to manage that 225 | //instance yourself, so this library expects a reference to that instance. It's implicitly injected into 226 | //the code, so the most convenient place to put it is right after you declare your Schema. 227 | implicit val conf = LocalCluster.getTestConfiguration 228 | 229 | //A table definition, where the row keys are Strings 230 | class ExampleTable extends HbaseTable[ExampleTable,String, ExampleTableRow](tableName = "schema_example",rowKeyClass=classOf[String]) 231 | { 232 | def rowBuilder(result:DeserializedResult) = new ExampleTableRow(this,result) 233 | 234 | val meta = family[String, String, Any]("meta") 235 | //Column family definition 236 | //Inside meta, assume a column called title whose value is a string 237 | val title = column(meta, "title", classOf[String]) 238 | //Inside meta, assume a column called url whose value is a string 239 | val url = column(meta, "url", classOf[String]) 240 | //Inside meta, assume a column called views whose value is a string 241 | val views = column(meta, "views", classOf[Long]) 242 | //A column called date whose value is a Joda DateTime 243 | val creationDate = column(meta, "date", classOf[DateTime]) 244 | 245 | //A column called viewsArr whose value is a sequence of strings 246 | val viewsArr = column(meta,"viewsArr", classOf[Seq[String]]) 247 | //A column called viewsMap whose value is a map of String to Long 248 | val viewsMap = column(meta,"viewsMap", classOf[Map[String,Long]]) 249 | 250 | //A column family called views whose column names are Strings and values are Longs. Can be treated as a Map 251 | val viewCounts = family[String, String, Long]("views") 252 | 253 | //A column family called views whose column names are YearDay instances and whose values are Longs 254 | val viewCountsByDay = family[String, YearDay, Long]("viewsByDay") 255 | 256 | //A column family called kittens whose column values are the custom Kitten type 257 | val kittens = family[String,String,Kitten]("kittens") 258 | } 259 | 260 | class ExampleTableRow(table:ExampleTable,result:DeserializedResult) extends HRow[ExampleTable,String](result,table) 261 | 262 | //Register the table (DON'T FORGET TO DO THIS :) ) 263 | val ExampleTable = table(new ExampleTable) 264 | 265 | } 266 | 267 | ``` 268 | 269 | The above table has a column family called meta, and several strongly-typed columns in the "meta" family. It then creates several column families that do not have column definitions under them ("views" and "viewsByDay"). The reason for this is that in HBase you tend to create two different types of column families: in one scenario, you create a column family that contains a set of columns that resemble columns in an RDBMS: each column is "typed" and has a unique identity. In the second scenario, you create a column family that resembles a Map: you dynamically add key-value pairs to this family. In this second scenario, you don't know what the columns are ahead of time--you'll be adding and removing columns on the fly. HPaste supports both models. 270 | 271 | ## Lifecycle Management 272 | 273 | Because HBase prefers to have a single instance of the Configuration object for connection management purposes, HPaste does not manage any Configuration lifecycle. When you specify your schema, you need to have an implicit instance of Configuration in scope. 274 | 275 | ## Table Creation Scripts 276 | If you have an existing table in HBase with the same name and families, you can get started. If you don't, you can now call: 277 | ```scala 278 | val create = ExampleSchema.ExampleTable.createScript() 279 | println(create) 280 | ``` 281 | Paste the results into your hbase shell to create the table. 282 | 283 | ## Data Manipulation 284 | HPaste supports GETS, PUTS, and INCREMENTS. You can batch multiple operations together, or do them serially. 285 | 286 | ### Column Valued Operations 287 | Column valued operations are ops that work against a particular column. Rather like a RDBMS. 288 | 289 | The following example chains a series of operations together. We will put a title against Chris, a title against Joe, and increment Chris' "views" column by 10. If the columns do not exist, they will be lazily created as per HBase convention. 290 | 291 | ```scala 292 | ExampleSchema.ExampleTable 293 | .put("Chris").value(_.title, "My Life, My Times") 294 | .put("Joe").value(_.title, "Joe's Life and Times") 295 | .increment("Chris").value(_.views, 10l) 296 | .execute() 297 | ``` 298 | 299 | When you call execute(), the operations you chained together will be executed in the following order: DELETES, then PUTS, then INCREMENTS. We decided to order the operations in that way because that is how they will generally be ordered in normal use cases, and also because HBase cannot guarantee ordering in batch-operations in which increments are mixed in with deletes and puts. 300 | 301 | If you need to enforce your own ordering, you should break your statements apart by calling execute() when you want to flush operations to HBase. 302 | 303 | ### Column Family Operations 304 | 305 | Often times you will define a column family where the columns themselves hold data (as columns are dynamically created in HBase). 306 | 307 | Here's the "views" column we specified in the example schema. The type parameters tell the system that we expect this column family to have a string-valued family name, a string-valued column name, and a Long value. 308 | 309 | ```scala 310 | val viewCounts = family[String, String, Long]("views") 311 | ``` 312 | 313 | Because Column Families treated in this fashion resemble the Map construct most closely, that's what we support. The following will put two columns titled "Today" and "Yesterday" into the Example Table under the key 1346: 314 | 315 | ```scala 316 | ExampleSchema.ExampleTable.put(1346l).valueMap(_.viewCounts, Map("Today" -> 61l, "Yesterday" -> 86l)).execute 317 | ``` 318 | 319 | ### Serialization: Creating your own types 320 | 321 | For serialization, HPaste uses Scala's implicit parameter system, which is very handy for creating an extensible conversion library. All of HPaste's built-in serializers are contained in a package object called "schema". Here's the serializer for Strings: 322 | 323 | ```scala 324 | implicit object StringConverter extends ByteConverter[String] { 325 | override def toBytes(t: String) = Bytes.toBytes(t) 326 | override def fromBytes(bytes: Array[Byte]) = Bytes.toString(bytes) 327 | } 328 | ``` 329 | 330 | To create your own serializer, follow the above pattern. Implement a ByteConverter of the type you want, and declare it as an implicit object. Any time you use a type that requires that serializer, make sure the namespace is in scope (for example, HPaste wants you to have "import com.gravity.hbase.schema._" at the top of your client code, so that all the standard serializers get imported). 331 | 332 | #### Complex Type Serialization : Strings (slow, big, but readable) 333 | If you want to create your own custom type, the easiest way to do so is to make it a string value under the covers. For example, there's a built in type in HPaste called YearDay: 334 | 335 | ```scala 336 | case class YearDay(year: Int, day: Int) 337 | ``` 338 | 339 | The serializer for it is located in the "schema" package object. All it does is convert the YearDay object to a string, then serialize out the String as bytes using the Hbase Bytes helper object. 340 | 341 | ```scala 342 | implicit object YearDayConverter extends ByteConverter[YearDay] { 343 | val SPLITTER = "_".r 344 | override def toBytes(t:YearDay) = Bytes.toBytes(t.year.toString + "_" + t.day.toString) 345 | override def fromBytes(bytes:Array[Byte]) = { 346 | val strRep = Bytes.toString(bytes) 347 | val strRepSpl = SPLITTER.split(strRep) 348 | val year = strRepSpl(0).toInt 349 | val day = strRepSpl(1).toInt 350 | YearDay(year,day) 351 | } 352 | } 353 | ``` 354 | This means that YearDay objects will be human readable when you look at them in HBase, and also it means it's mentally easier to reason about the layout of the object when you're extending it. 355 | 356 | That having been said, string-based serialization is the lowest common denominator when it comes to speed and size of data. The next section describes custom byte serialization. 357 | 358 | #### Complex Type Serialization : Byte Arrays (fast, small, unreadable) 359 | Because ByteConverter (and by proxy HBase) just needs byte arrays, you can extend the class with any serializer framework you want (Thrift, Avro, etc.). 360 | 361 | HPaste does have some extra support for the simplest, most performant, and most dangerous Java serialization framework: DataInputStream and DataOutputStream. To use this, you can inherit from ComplexByteConverter. This will provide a DataInputStream when bytes need to be serialized to your type, and a DataOutputStream when it's time for your object to become a byte array. 362 | 363 | Let's say you have a class Kitten: 364 | 365 | ```scala 366 | case class Kitten(name:String, age:Int, height:Double) 367 | ``` 368 | 369 | The following defines a serializer for Kitten: 370 | 371 | ```scala 372 | implicit object KittenConverter extends ComplexByteConverter[Kitten] { 373 | override def write(kitten:Kitten, output:DataOutputStream) { 374 | output.writeUTF(kitten.name) 375 | output.writeInt(kitten.age) 376 | output.writeDouble(kitten.height) 377 | } 378 | 379 | override def read(input:DataInputStream) = { 380 | Kitten(input.readUTF(), input.readInt(), input.readDouble()) 381 | } 382 | } 383 | ``` 384 | 385 | WARNING: There are a lot of production issues with using raw serialization this way. They mainly revolve around versioning. What if you want to change the layout of Kitten later? The above method of serializing uses no metadata. The bytestream is just a bunch of undifferentiated primitives. If you, for example, add a new Int value before the Name, the deserializer will, when it encounters older objects, read the first bytes of the Name as the Int. In that situation you'll need to write a tool that massages the old Kitten byte-layout into the new Kitten byte-layout. Better serialization frameworks do a better job of handling this situation. HPaste will soon support Avro out of the box. That having been said, as long as you're mindful about what you're doing, DataInput and DataOutput are blazingly fast and leave a tiny data footprint--if you confine yourself to relative immutable type definitions, you can go far with this model. 386 | 387 | ### Sequences, Maps, and Sets 388 | 389 | Once you have a serializer for a type, you can (fairly) easily define a serializer that lets you define a Set, Map, or Seq of that object. The below assumes you already have a ByteConverter for a Kitten object, and will now support column values that are sequences. 390 | 391 | ```scala 392 | implicit object KittenSeqConverter extends SeqConverter[Kitten] 393 | ``` 394 | 395 | NOTE: The above way of defining a sequence is clumsy, agreed. Need to refactor. 396 | 397 | 398 | ### Mixing Modification Operations 399 | 400 | You can mix column-valued and column family operations into the same call. 401 | 402 | ```scala 403 | ExampleSchema.ExampleTable 404 | .put("Chris").value(_.title, "My Life, My Times") 405 | .put("Joe").value(_.title, "Joe's Life and Times") 406 | .put("Fred").value(_.viewsArr,Seq("Chris","Bissell")) 407 | .increment("Chris").value(_.views, 10l) 408 | .put("Chris").valueMap(_.viewCountsByDay, Map(YearDay(2011,16)->60l, YearDay(2011,17)->50l)) 409 | .execute() 410 | ``` 411 | 412 | # Data Retrieval 413 | 414 | 415 | 1. Creating a specification for your Get operation, in which you decide what rows to get back, what columns and column families to get back. 416 | 417 | 2. When you issue the retrieval operation and get the results back, extracting the results into useful data. 418 | 419 | Assuming the examples under the DATA MANIPULATION section, the following will retrieve a Map of String to Long out of the Example Table: 420 | 421 | ```scala 422 | val dayViewsRes = ExampleSchema.ExampleTable.query.withKey(key).withColumnFamily(_.viewCountsByDay).single() 423 | val dayViewsMap = dayViewsRes.family(_.viewCountsByDay) 424 | ``` 425 | 426 | # More on MapReduce support 427 | 428 | ## Row Serialization between Mappers and Reducers 429 | 430 | HRows can be serialized between Mappers and Reducers, using the writeRow and readRow methods of PrimitiveOutputStream and PrimitiveInputStream. 431 | 432 | This is often the least performant way of passing data between mappers and reducers (depending on how big your rows are), but also involves the least amount of code. 433 | 434 | Below is a job that assumes the same WebCrawling schema in the Quickstart. Its goal is to write output files that show page by the site they're contained in. We've added a convenience function to the WebPageRow to extract the domain: 435 | 436 | ```scala 437 | class WebPageRow(table: WebTable, result: DeserializedResult) extends HRow[WebTable, String](result, table) { 438 | def domain = new URL(rowid).getAuthority 439 | } 440 | ``` 441 | 442 | Here is the job. 443 | 444 | ```scala 445 | class WebTablePagesBySiteJob extends HJob[NoSettings]("Get articles by site", 446 | HMapReduceTask( 447 | HTaskID("Articles by Site"), 448 | HTaskConfigs(), 449 | HIO( 450 | HTableInput(WebCrawlingSchema.WebTable), 451 | HPathOutput("/reports/wordcount") 452 | ), 453 | new FromTableBinaryMapperFx(WebCrawlingSchema.WebTable) { 454 | val webPage : WebPageRow = row //For illustrative purposes we're specifying the type here, no need to 455 | val domain = row.domain //We've added a convenience method to WebPageRow to extract the domain for us 456 | 457 | write( 458 | {keyOutput=>keyOutput.writeUTF(domain)}, //This writes out the domain as the key 459 | {valueOutput=>valueOutput.writeRow(WebCrawlingSchema.WebTable,webPage)} //This writes the entire value of the row out 460 | ) 461 | }, 462 | new BinaryToTextReducerFx { 463 | val domain = readKey(_.readUTF()) //This allows you to read out the key 464 | 465 | perValue{valueInput=> 466 | val webPage : WebPageRow = valueInput.readRow(WebCrawlingSchema.WebTable) //Now you can read out the entire WebPageRow object from the value stream 467 | ctr("Pages for domain " + domain) 468 | writeln(domain + "\t" + webPage.column(_.title).getOrElse("No Title")) //This is a convenience function that writes a line to the text output 469 | } 470 | } 471 | ) 472 | ) 473 | 474 | ``` 475 | This job is part of HPaste's unit tests, so you can see it in context at [WebCrawlSchemaTest.scala](https://github.com/GravityLabs/HPaste/blob/master/src/test/scala/com/gravity/hbase/schema/WebCrawlSchemaTest.scala). 476 | 477 | # Developers 478 | 479 | This library was developed by [Chris Bissell](https://github.com/Lemmsjid "Lemmsjid"), with many contributions, both philosophical and code, by [Robbie Coleman](https://github.com/erraggy "erraggy"). -------------------------------------------------------------------------------- /src/main/scala/com/gravity/hbase/schema/Query2.scala: -------------------------------------------------------------------------------- 1 | /**Licensed to Gravity.com under one 2 | * or more contributor license agreements. See the NOTICE file 3 | * distributed with this work for additional information 4 | * regarding copyright ownership. Gravity.com licenses this file 5 | * to you under the Apache License, Version 2.0 (the 6 | * "License"); you may not use this file except in compliance 7 | * with the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.gravity.hbase.schema 19 | 20 | import org.apache.hadoop.hbase.client._ 21 | import org.apache.hadoop.hbase.util._ 22 | import scala.collection.JavaConversions._ 23 | import org.apache.hadoop.conf.Configuration 24 | import java.io._ 25 | import org.apache.hadoop.io.{BytesWritable, Writable} 26 | import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp 27 | import scala.collection._ 28 | import java.util.NavigableSet 29 | import scala.collection.mutable.Buffer 30 | import org.apache.hadoop.hbase.filter.FilterList.Operator 31 | import org.joda.time.{ReadableInstant, DateTime} 32 | import org.apache.hadoop.hbase.filter._ 33 | 34 | /* )\._.,--....,'``. 35 | .b--. /; _.. \ _\ (`._ ,. 36 | `=,-,-'~~~ `----(,_..'--(,_..'`-.;.' */ 37 | 38 | 39 | /** 40 | * Expresses a scan, get, or batched get against hbase. Which one it becomes depends on what 41 | * calls you make. If you specify withKey() it will 42 | * become a Get, withKeys() will make into a batched get, and no keys at all will make it a Scan. 43 | * 44 | * @tparam T the table to work with 45 | * @tparam R the row key type 46 | * @tparam RR the row result type 47 | * @param table the instance of the table to work with 48 | */ 49 | class Query2[T <: HbaseTable[T, R, RR], R, RR <: HRow[T, R]](val table: HbaseTable[T, R, RR]) { 50 | 51 | def filter(filterFx: ((FilterBuilder) => Unit)*) = { 52 | val fb = new FilterBuilder(true) 53 | for (fx <- filterFx) { 54 | fx(fb) 55 | } 56 | currentFilter = fb.coreList 57 | this 58 | } 59 | 60 | def filterOr(filterFx: ((FilterBuilder) => Unit)*) = { 61 | val fb = new FilterBuilder(false) 62 | for (fx <- filterFx) { 63 | fx(fb) 64 | } 65 | currentFilter = fb.coreList 66 | this 67 | 68 | } 69 | 70 | 71 | class FilterBuilder(and: Boolean) { 72 | var coreList: FilterList = if (and) new FilterList(Operator.MUST_PASS_ALL) else new FilterList(Operator.MUST_PASS_ONE) 73 | val clauseBuilder = new ClauseBuilder() 74 | 75 | private def addFilter(filter: FilterList) { 76 | //coreList = filter 77 | coreList.addFilter(filter) 78 | } 79 | 80 | 81 | def or(clauses: ((ClauseBuilder) => Option[Filter])*) = { 82 | val orFilter = new FilterList(FilterList.Operator.MUST_PASS_ONE) 83 | for (ctx <- clauses) { 84 | val filter = ctx(clauseBuilder) 85 | if (filter.isDefined) { 86 | orFilter.addFilter(filter.get) 87 | } 88 | } 89 | if (orFilter.getFilters().size() > 0) { 90 | addFilter(orFilter) 91 | } 92 | this 93 | } 94 | 95 | def and(clauses: ((ClauseBuilder) => Option[Filter])*) = { 96 | val andFilter = new FilterList(FilterList.Operator.MUST_PASS_ALL) 97 | for (cfx <- clauses) { 98 | val filter = cfx(clauseBuilder) 99 | if (filter.isDefined) { 100 | andFilter.addFilter(filter.get) 101 | } 102 | } 103 | if (andFilter.getFilters.size() > 0) { 104 | addFilter(andFilter) 105 | } 106 | this 107 | } 108 | 109 | } 110 | 111 | class ClauseBuilder() { 112 | 113 | def columnValueMustStartWith[F, K, V](column: (T) => Column[T, R, F, K, String], prefix: String) = { 114 | val c = column(table.pops) 115 | val prefixFilter = new BinaryPrefixComparator(Bytes.toBytes(prefix)) 116 | val vc = new SingleColumnValueFilter(c.familyBytes, c.columnBytes, CompareOp.EQUAL, prefixFilter) 117 | Some(vc) 118 | } 119 | 120 | 121 | def noClause = None 122 | 123 | def columnValueMustContain[F, K, V](column: (T) => Column[T, R, F, K, String], substr: String) = { 124 | val c = column(table.pops) 125 | val substrFilter = new SubstringComparator(substr) 126 | val vc = new SingleColumnValueFilter(c.familyBytes, c.columnBytes, CompareOp.EQUAL, substrFilter) 127 | Some(vc) 128 | } 129 | 130 | /** 131 | * Untested 132 | */ 133 | def whereFamilyHasKeyGreaterThan[F, K](family: (T) => ColumnFamily[T, R, F, K, _], key: K) = { 134 | val f = family(table.pops) 135 | val fl = new FilterList(Operator.MUST_PASS_ALL) 136 | val ts = new QualifierFilter(CompareOp.GREATER_OR_EQUAL, new BinaryComparator(f.keyConverter.toBytes(key))) 137 | val ff = new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(f.familyBytes)) 138 | fl.addFilter(ts) 139 | fl.addFilter(ff) 140 | val sk = new SkipFilter(fl) 141 | Some(sk) 142 | } 143 | 144 | def columnValueMustPassRegex[F, K, V](column: (T) => Column[T, R, F, K, String], regex: String) = { 145 | val c = column(table.pops) 146 | val regexFilter = new RegexStringComparator(regex) 147 | val vc = new SingleColumnValueFilter(c.familyBytes, c.columnBytes, CompareOp.EQUAL, regexFilter) 148 | Some(vc) 149 | } 150 | 151 | 152 | def columnValueMustNotContain[F, K, V](column: (T) => Column[T, R, F, K, String], substr: String) = { 153 | val c = column(table.pops) 154 | val substrFilter = new SubstringComparator(substr) 155 | val vc = new SingleColumnValueFilter(c.familyBytes, c.columnBytes, CompareOp.NOT_EQUAL, substrFilter) 156 | Some(vc) 157 | } 158 | 159 | 160 | def maxRowsPerServer(rowsize: Int): Option[Filter] = { 161 | val pageFilter = new PageFilter(rowsize) 162 | Some(pageFilter) 163 | } 164 | 165 | def columnValueMustEqual[F, K, V](column: (T) => Column[T, R, F, K, V], value: V) = { 166 | val c = column(table.pops) 167 | val vc = new SingleColumnValueFilter(c.familyBytes, c.columnBytes, CompareOp.EQUAL, c.valueConverter.toBytes(value)) 168 | vc.setFilterIfMissing(true) 169 | vc.setLatestVersionOnly(true) 170 | Some(vc) 171 | } 172 | 173 | def columnValueMustBeGreaterThan[F, K, V](column: (T) => Column[T, R, F, K, V], value: V) = { 174 | val c = column(table.pops) 175 | val vc = new SingleColumnValueFilter(c.familyBytes, c.columnBytes, CompareOp.GREATER, c.valueConverter.toBytes(value)) 176 | vc.setFilterIfMissing(true) 177 | vc.setLatestVersionOnly(true) 178 | Some(vc) 179 | } 180 | 181 | def columnValueMustBeLessThan[F, K, V](column: (T) => Column[T, R, F, K, V], value: V) = { 182 | val c = column(table.pops) 183 | val vc = new SingleColumnValueFilter(c.familyBytes, c.columnBytes, CompareOp.LESS, c.valueConverter.toBytes(value)) 184 | vc.setFilterIfMissing(true) 185 | vc.setLatestVersionOnly(true) 186 | Some(vc) 187 | } 188 | 189 | 190 | def columnValueMustBePresent[F, K, V](column: (T) => Column[T, R, F, K, V]) = { 191 | val c = column(table.pops) 192 | val vc = new SingleColumnValueFilter(c.familyBytes, c.columnBytes, CompareOp.NOT_EQUAL, Bytes.toBytes(0)) 193 | vc.setFilterIfMissing(true) 194 | vc.setLatestVersionOnly(true) 195 | Some(new SkipFilter(vc)) 196 | } 197 | 198 | def lessThanColumnKey[F, K, V](family: (T) => ColumnFamily[T, R, F, K, V], value: K) = { 199 | val fam = family(table.pops) 200 | val valueFilter = new QualifierFilter(CompareOp.LESS_OR_EQUAL, new BinaryComparator(fam.keyConverter.toBytes(value))) 201 | val familyFilter = new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(family(table.pops).familyBytes)) 202 | val andFilter = new FilterList(Operator.MUST_PASS_ALL) 203 | andFilter.addFilter(familyFilter) 204 | andFilter.addFilter(valueFilter) 205 | Some(andFilter) 206 | } 207 | 208 | def greaterThanColumnKey[F, K, V](family: (T) => ColumnFamily[T, R, F, K, V], value: K) = { 209 | val fam = family(table.pops) 210 | val andFilter = new FilterList(Operator.MUST_PASS_ALL) 211 | val familyFilter = new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(fam.familyBytes)) 212 | val valueFilter = new QualifierFilter(CompareOp.GREATER_OR_EQUAL, new BinaryComparator(fam.keyConverter.toBytes(value))) 213 | andFilter.addFilter(familyFilter) 214 | andFilter.addFilter(valueFilter) 215 | Some(andFilter) 216 | } 217 | 218 | // def columnFamily[F,K,V](family: (T) => ColumnFamily[T,R,F,K,V])(implicit c: ByteConverter[F]): Query[T,R] = { 219 | // val familyFilter = new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(family(table.pops).familyBytes)) 220 | // currentFilter.addFilter(familyFilter) 221 | // this 222 | // } 223 | 224 | 225 | def whereColumnMustExist[F, K, _](column: (T) => Column[T, R, F, K, _]) = { 226 | val c = column(table.pops) 227 | val valFilter = new SingleColumnValueExcludeFilter(c.familyBytes, c.columnBytes, CompareOp.NOT_EQUAL, new Array[Byte](0)) 228 | valFilter.setFilterIfMissing(true) 229 | Some(valFilter) 230 | } 231 | 232 | 233 | def betweenColumnKeys[F, K, V](family: (T) => ColumnFamily[T, R, F, K, V], lower: K, upper: K) = { 234 | val fam = family(table.pops) 235 | val familyFilter = new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(fam.familyBytes)) 236 | val begin = new QualifierFilter(CompareOp.GREATER_OR_EQUAL, new BinaryComparator(fam.keyConverter.toBytes(lower))) 237 | val end = new QualifierFilter(CompareOp.LESS_OR_EQUAL, new BinaryComparator(fam.keyConverter.toBytes(upper))) 238 | 239 | 240 | val filterList = new FilterList(Operator.MUST_PASS_ALL) 241 | filterList.addFilter(familyFilter) 242 | filterList.addFilter(begin) 243 | filterList.addFilter(end) 244 | Some(filterList) 245 | } 246 | 247 | def inFamily[F](family: (T) => ColumnFamily[T, R, F, _, _]) = { 248 | val fam = family(table.pops) 249 | val ff = new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(fam.familyBytes)) 250 | Some(ff) 251 | } 252 | 253 | def allInFamilies[F](familyList: ((T) => ColumnFamily[T, R, F, _, _])*) = { 254 | val filterList = new FilterList(Operator.MUST_PASS_ONE) 255 | for (family <- familyList) { 256 | val familyFilter = new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(family(table.pops).familyBytes)) 257 | filterList.addFilter(familyFilter) 258 | } 259 | Some(filterList) 260 | } 261 | } 262 | 263 | 264 | val keys = Buffer[Array[Byte]]() 265 | val families = Buffer[Array[Byte]]() 266 | val columns = Buffer[(Array[Byte], Array[Byte])]() 267 | var currentFilter: FilterList = _ 268 | // new FilterList(Operator.MUST_PASS_ALL) 269 | var startRowBytes: Array[Byte] = null 270 | var endRowBytes: Array[Byte] = null 271 | var batchSize = -1 272 | 273 | /**The key to fetch (this makes it into a Get request against hbase) */ 274 | def withKey(key: R) = { 275 | keys += table.rowKeyConverter.toBytes(key) 276 | this 277 | } 278 | 279 | /**Multiple keys to fetch (this makes it into a multi-Get request against hbase) */ 280 | def withKeys(keys: Set[R]) = { 281 | for (key <- keys) { 282 | withKey(key) 283 | } 284 | this 285 | } 286 | 287 | 288 | def and = { 289 | if (currentFilter == null) { 290 | currentFilter = new FilterList(Operator.MUST_PASS_ALL) 291 | } else { 292 | val encompassingFilter = new FilterList(Operator.MUST_PASS_ALL) 293 | encompassingFilter.addFilter(currentFilter) 294 | currentFilter = encompassingFilter 295 | } 296 | this 297 | } 298 | 299 | def or = { 300 | if (currentFilter == null) { 301 | currentFilter = new FilterList(Operator.MUST_PASS_ONE) 302 | } else { 303 | val encompassingFilter = new FilterList(Operator.MUST_PASS_ONE) 304 | encompassingFilter.addFilter(currentFilter) 305 | currentFilter = encompassingFilter 306 | } 307 | this 308 | } 309 | 310 | def columnValueMustEqual[F, K, V](column: (T) => Column[T, R, F, K, V], value: V) = { 311 | val c = column(table.pops) 312 | val vc = new SingleColumnValueFilter(c.familyBytes, c.columnBytes, CompareOp.EQUAL, c.valueConverter.toBytes(value)) 313 | vc.setFilterIfMissing(true) 314 | vc.setLatestVersionOnly(true) 315 | currentFilter.addFilter(vc) 316 | this 317 | } 318 | 319 | def lessThanColumnKey[F, K, V](family: (T) => ColumnFamily[T, R, F, K, V], value: K) = { 320 | val fam = family(table.pops) 321 | val valueFilter = new QualifierFilter(CompareOp.LESS_OR_EQUAL, new BinaryComparator(fam.keyConverter.toBytes(value))) 322 | val familyFilter = new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(family(table.pops).familyBytes)) 323 | val andFilter = new FilterList(Operator.MUST_PASS_ALL) 324 | andFilter.addFilter(familyFilter) 325 | andFilter.addFilter(valueFilter) 326 | currentFilter.addFilter(andFilter) 327 | this 328 | } 329 | 330 | def greaterThanColumnKey[F, K, V](family: (T) => ColumnFamily[T, R, F, K, V], value: K) = { 331 | val fam = family(table.pops) 332 | val andFilter = new FilterList(Operator.MUST_PASS_ALL) 333 | val familyFilter = new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(fam.familyBytes)) 334 | val valueFilter = new QualifierFilter(CompareOp.GREATER_OR_EQUAL, new BinaryComparator(fam.keyConverter.toBytes(value))) 335 | andFilter.addFilter(familyFilter) 336 | andFilter.addFilter(valueFilter) 337 | currentFilter.addFilter(andFilter) 338 | this 339 | } 340 | 341 | // def columnFamily[F,K,V](family: (T) => ColumnFamily[T,R,F,K,V])(implicit c: ByteConverter[F]): Query[T,R] = { 342 | // val familyFilter = new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(family(table.pops).familyBytes)) 343 | // currentFilter.addFilter(familyFilter) 344 | // this 345 | // } 346 | 347 | 348 | def betweenColumnKeys[F, K, V](family: (T) => ColumnFamily[T, R, F, K, V], lower: K, upper: K) = { 349 | val fam = family(table.pops) 350 | val familyFilter = new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(fam.familyBytes)) 351 | val begin = new QualifierFilter(CompareOp.GREATER_OR_EQUAL, new BinaryComparator(fam.keyConverter.toBytes(lower))) 352 | val end = new QualifierFilter(CompareOp.LESS_OR_EQUAL, new BinaryComparator(fam.keyConverter.toBytes(upper))) 353 | val filterList = new FilterList(Operator.MUST_PASS_ALL) 354 | filterList.addFilter(familyFilter) 355 | filterList.addFilter(begin) 356 | filterList.addFilter(end) 357 | currentFilter.addFilter(filterList) 358 | 359 | this 360 | } 361 | 362 | def allInFamilies[F](familyList: ((T) => ColumnFamily[T, R, F, _, _])*) = { 363 | val filterList = new FilterList(Operator.MUST_PASS_ONE) 364 | for (family <- familyList) { 365 | val familyFilter = new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(family(table.pops).familyBytes)) 366 | filterList.addFilter(familyFilter) 367 | } 368 | currentFilter.addFilter(filterList) 369 | this 370 | } 371 | 372 | def withFamilies[F](familyList: ((T) => ColumnFamily[T, R, F, _, _])*) = { 373 | for (family <- familyList) { 374 | val fam = family(table.pops) 375 | families += fam.familyBytes 376 | } 377 | this 378 | } 379 | 380 | def withColumn[F, K, V](family: (T) => ColumnFamily[T, R, F, K, V], columnName: K) = { 381 | val fam = family(table.pops) 382 | columns += (fam.familyBytes -> fam.keyConverter.toBytes(columnName)) 383 | this 384 | } 385 | 386 | def withColumn[F, K, V](column: (T) => Column[T, R, F, K, V]) = { 387 | val col = column(table.pops) 388 | columns += (col.familyBytes -> col.columnBytes) 389 | this 390 | } 391 | 392 | def withColumns[F, K, V](columnList: ((T) => Column[T, R, F, _, _])*) = { 393 | for (column <- columnList) { 394 | val col = column(table.pops) 395 | columns += (col.familyBytes -> col.columnBytes) 396 | } 397 | this 398 | } 399 | 400 | var startTime: Long = Long.MinValue 401 | var endTime: Long = Long.MaxValue 402 | 403 | def betweenDates(start: ReadableInstant, end: ReadableInstant) = { 404 | startTime = start.getMillis 405 | endTime = end.getMillis 406 | this 407 | } 408 | 409 | def afterDate(start: ReadableInstant) = { 410 | startTime = start.getMillis 411 | this 412 | } 413 | 414 | def untilDate(end: ReadableInstant) = { 415 | endTime = end.getMillis 416 | this 417 | } 418 | 419 | def single(tableName: String = table.tableName, ttl: Int = 30, skipCache: Boolean = true) = singleOption(tableName, ttl, skipCache, false).get 420 | 421 | def singleOption(tableName: String = table.tableName, ttl: Int = 30, skipCache: Boolean = true, noneOnEmpty: Boolean = true): Option[RR] = { 422 | require(keys.size == 1, "Calling single() with more than one key") 423 | require(keys.size >= 1, "Calling a Get operation with no keys specified") 424 | val get = new Get(keys.head) 425 | get.setMaxVersions(1) 426 | 427 | if (startTime != Long.MinValue || endTime != Long.MaxValue) { 428 | get.setTimeRange(startTime, endTime) 429 | } 430 | 431 | 432 | for (family <- families) { 433 | get.addFamily(family) 434 | } 435 | for ((columnFamily, column) <- columns) { 436 | get.addColumn(columnFamily, column) 437 | } 438 | if (currentFilter != null && currentFilter.getFilters.size() > 0) { 439 | get.setFilter(currentFilter) 440 | } 441 | 442 | val fromCache = if (skipCache) None else table.cache.getResult(get) 443 | 444 | fromCache match { 445 | case Some(result) => Some(result) 446 | case None => { 447 | table.withTableOption(tableName) { 448 | case Some(htable) => { 449 | val result = htable.get(get) 450 | if (noneOnEmpty && result.isEmpty) { 451 | None 452 | } else { 453 | val qr = table.buildRow(result) 454 | if (!skipCache && !result.isEmpty) table.cache.putResult(get, qr, ttl) 455 | Some(qr) 456 | } 457 | } 458 | case None => None 459 | } 460 | } 461 | } 462 | 463 | } 464 | 465 | def execute(tableName: String = table.tableName, ttl: Int = 30, skipCache: Boolean = true): Seq[RR] = { 466 | if (keys.isEmpty) return Seq.empty[RR] // no keys..? nothing to see here... move along... move along. 467 | require(!keys.isEmpty, "execute assumes that you have called withKeys() or withKey(). If you are trying to do a scan, you should call Scan()") 468 | 469 | val results = Buffer[RR]() // buffer for storing all results retrieved 470 | 471 | // if we are utilizing cache, we'll need to be able to recall the `Get' later to use as the cache key 472 | val getsByKey = if (skipCache) mutable.Map.empty[String, Get] else mutable.Map[String, Get]() 473 | 474 | if (!skipCache) getsByKey.sizeHint(keys.size) // perf optimization 475 | 476 | // buffer for all `Get's that really need to be gotten 477 | val cacheMisses = Buffer[Get]() 478 | 479 | val gets = buildGetsAndCheckCache(skipCache) { 480 | case (get: Get, key: Array[Byte]) => if (!skipCache) getsByKey.put(new String(key), get) 481 | } { 482 | case (qropt: Option[RR], get: Get) => if (!skipCache) { 483 | qropt match { 484 | case Some(result) => results += result // got it! place it in our result buffer 485 | case None => cacheMisses += get // missed it! place the get in the buffer 486 | } 487 | } 488 | } 489 | 490 | // identify what still needs to be `Get'ed ;-} 491 | val hbaseGets = if (skipCache) gets else cacheMisses 492 | 493 | if (!hbaseGets.isEmpty) { 494 | // only do this if we have something to do 495 | table.withTable(tableName) { 496 | htable => 497 | htable.get(hbaseGets).foreach(res => { 498 | if (res != null && !res.isEmpty) { 499 | // ignore empty results 500 | val qr = table.buildRow(res) // construct query result 501 | 502 | // now is where we need to retrive the 'get' used for this result so that we can 503 | // pass this 'get' as the key for our local cache 504 | if (!skipCache) table.cache.putResult(getsByKey(new String(res.getRow)), qr, ttl) 505 | results += qr // place it in our result buffer 506 | } 507 | }) 508 | } 509 | } 510 | 511 | results.toSeq // DONE! 512 | } 513 | 514 | def executeMap(tableName: String = table.tableName, ttl: Int = 30, skipCache: Boolean = true): Map[R, RR] = { 515 | if (keys.isEmpty) return Map.empty[R, RR] // don't get all started with nothing to do 516 | 517 | // init our result map and give it a hint of the # of keys we have 518 | val resultMap = mutable.Map[R, RR]() 519 | resultMap.sizeHint(keys.size) // perf optimization 520 | 521 | // if we are utilizing cache, we'll need to be able to recall the `Get' later to use as the cache key 522 | val getsByKey = if (skipCache) mutable.Map.empty[String, Get] else mutable.Map[String, Get]() 523 | 524 | if (!skipCache) getsByKey.sizeHint(keys.size) // perf optimization 525 | 526 | // buffer for all `Get's that really need to be gotten 527 | val cacheMisses = Buffer[Get]() 528 | 529 | val gets = buildGetsAndCheckCache(skipCache) { 530 | case (get: Get, key: Array[Byte]) => if (!skipCache) getsByKey.put(new String(key), get) 531 | } { 532 | case (qropt: Option[RR], get: Get) => if (!skipCache) { 533 | qropt match { 534 | case Some(result) => resultMap.put(result.rowid, result) // got it! place it in our result map 535 | case None => cacheMisses += get // missed it! place the get in the buffer 536 | } 537 | } 538 | } 539 | 540 | // identify what still needs to be `Get'ed ;-} 541 | val hbaseGets = if (skipCache) gets else cacheMisses 542 | 543 | if (!hbaseGets.isEmpty) { 544 | // only do this if we have something to do 545 | table.withTable(tableName) { 546 | htable => 547 | htable.get(hbaseGets).foreach(res => { 548 | if (res != null && !res.isEmpty) { 549 | // ignore empty results 550 | val qr = table.buildRow(res) // construct query result 551 | 552 | // now is where we need to retrive the 'get' used for this result so that we can 553 | // pass this 'get' as the key for our local cache 554 | if (!skipCache) table.cache.putResult(getsByKey(new String(res.getRow)), qr, ttl) 555 | resultMap(qr.rowid) = qr // place it in our result map 556 | } 557 | }) 558 | } 559 | } 560 | 561 | resultMap // DONE! 562 | } 563 | 564 | 565 | private def buildGetsAndCheckCache(skipCache: Boolean)(receiveGetAndKey: (Get, Array[Byte]) => Unit = (get, key) => {})(receiveCachedResult: (Option[RR], Get) => Unit = (qr, get) => {}): Seq[Get] = { 566 | if (keys.isEmpty) return Seq.empty[Get] // no keys..? nothing to see here... move along... move along. 567 | 568 | val gets = Buffer[Get]() // buffer for the raw `Get's 569 | 570 | for (key <- keys) { 571 | val get = new Get(key) 572 | if (startTime != Long.MinValue || endTime != Long.MaxValue) { 573 | get.setTimeRange(startTime, endTime) 574 | } 575 | 576 | 577 | gets += get 578 | receiveGetAndKey(get, key) 579 | } 580 | 581 | // since the families and columns will be identical for all `Get's, only build them once 582 | val firstGet = gets(0) 583 | 584 | // add all families to the first `Get' 585 | for (family <- families) { 586 | firstGet.addFamily(family) 587 | } 588 | // add all columns to the first `Get' 589 | for ((columnFamily, column) <- columns) { 590 | firstGet.addColumn(columnFamily, column) 591 | } 592 | if (currentFilter != null && currentFilter.getFilters.size() > 0) { 593 | firstGet.setFilter(currentFilter) 594 | } 595 | 596 | 597 | var pastFirst = false 598 | for (get <- gets) { 599 | if (pastFirst) { 600 | // we want to skip the first `Get' as it already has families/columns 601 | 602 | // for all subsequent `Get's, we will build their familyMap from the first `Get' 603 | firstGet.getFamilyMap.foreach((kv: (Array[Byte], NavigableSet[Array[Byte]])) => { 604 | get.getFamilyMap.put(kv._1, kv._2) 605 | }) 606 | if (currentFilter != null && currentFilter.getFilters.size() > 0) { 607 | get.setFilter(currentFilter) 608 | } 609 | } else { 610 | pastFirst = true 611 | } 612 | 613 | // try the cache with this filled in get 614 | if (!skipCache) receiveCachedResult(table.cache.getResult(get), get) 615 | } 616 | 617 | gets 618 | } 619 | 620 | 621 | def withStartRow(row: R) = { 622 | startRowBytes = table.rowKeyConverter.toBytes(row) 623 | this 624 | } 625 | 626 | def withEndRow(row: R) = { 627 | endRowBytes = table.rowKeyConverter.toBytes(row) 628 | this 629 | } 630 | 631 | def withBatchSize(size: Int) = { 632 | batchSize = size 633 | this 634 | } 635 | 636 | def makeScanner(maxVersions: Int = 1, cacheBlocks: Boolean = true, cacheSize: Int = 100) = { 637 | require(keys.size == 0, "A scanner should not specify keys, use singleOption or execute or executeMap") 638 | val scan = new Scan() 639 | scan.setMaxVersions(maxVersions) 640 | scan.setCaching(cacheSize) 641 | scan.setCacheBlocks(cacheBlocks) 642 | 643 | if (batchSize > -1) { 644 | scan.setBatch(batchSize) 645 | } 646 | 647 | if (startTime != Long.MinValue || endTime != Long.MaxValue) { 648 | scan.setTimeRange(startTime, endTime) 649 | } 650 | 651 | if (startRowBytes != null) { 652 | scan.setStartRow(startRowBytes) 653 | } 654 | if (endRowBytes != null) { 655 | scan.setStopRow(endRowBytes) 656 | } 657 | 658 | for (family <- families) { 659 | scan.addFamily(family) 660 | } 661 | for (column <- columns) { 662 | scan.addColumn(column._1, column._2) 663 | } 664 | 665 | if (currentFilter != null && currentFilter.getFilters.size > 0) { 666 | scan.setFilter(currentFilter) 667 | } 668 | 669 | scan 670 | } 671 | 672 | def scan(handler: (RR) => Unit, maxVersions: Int = 1, cacheBlocks: Boolean = true, cacheSize: Int = 100, useLocalCache: Boolean = false, localTTL: Int = 30) { 673 | val scan = makeScanner(maxVersions, cacheBlocks, cacheSize) 674 | 675 | val results = if (useLocalCache) Buffer[RR]() else Buffer.empty[RR] 676 | 677 | def cacheHandler(rr: RR) { 678 | if (useLocalCache) results += rr 679 | } 680 | 681 | def cacheComplete() { 682 | if (useLocalCache && !results.isEmpty) table.cache.putScanResult(scan, results.toSeq, localTTL) 683 | } 684 | 685 | val whatWeGetFromCache = if (useLocalCache) table.cache.getScanResult(scan) else None 686 | 687 | whatWeGetFromCache match { 688 | case Some(result) => { 689 | println("cache hit against key " + scan.toString) 690 | result.foreach(handler) 691 | } 692 | case None => { 693 | 694 | table.withTable() { 695 | htable => 696 | 697 | val scanner = htable.getScanner(scan) 698 | 699 | try { 700 | var done = false 701 | while (!done) { 702 | val result = scanner.next() 703 | if (result != null) { 704 | val rr = table.buildRow(result) 705 | cacheHandler(rr) 706 | handler(rr) 707 | } else { 708 | done = true 709 | } 710 | } 711 | } finally { 712 | cacheComplete() 713 | scanner.close() 714 | } 715 | } 716 | } 717 | } 718 | 719 | } 720 | 721 | def scanToIterable[I](handler: (RR) => I, maxVersions: Int = 1, cacheBlocks: Boolean = true, cacheSize: Int = 100, useLocalCache: Boolean = false, localTTL: Int = 30) = { 722 | val scan = makeScanner(maxVersions, cacheBlocks, cacheSize) 723 | 724 | val results = if (useLocalCache) Buffer[RR]() else Buffer.empty[RR] 725 | 726 | def cacheHandler(rr: RR) { 727 | if (useLocalCache) results += rr 728 | } 729 | 730 | def cacheComplete() { 731 | if (useLocalCache && !results.isEmpty) table.cache.putScanResult(scan, results.toSeq, localTTL) 732 | } 733 | 734 | val whatWeGetFromCache = if (useLocalCache) table.cache.getScanResult(scan) else None 735 | 736 | val results2 = whatWeGetFromCache match { 737 | case Some(rrs) => rrs.map(rr => handler(rr)) 738 | case None => { 739 | val runResults = table.withTable() { 740 | htable => 741 | val scanner = htable.getScanner(scan) 742 | try { 743 | for (result <- scanner; if (result != null)) yield { 744 | val rr = table.buildRow(result) 745 | cacheHandler(rr) 746 | handler(rr) 747 | } 748 | } finally { 749 | cacheComplete() 750 | scanner.close() 751 | } 752 | } 753 | 754 | runResults 755 | } 756 | } 757 | results2 758 | } 759 | 760 | trait Stopable extends Throwable 761 | 762 | object YouCanStopNow extends Stopable 763 | 764 | /**Similar to the scan method but if your handler returns false, it will stop scanning. 765 | * 766 | */ 767 | def scanUntil(handler: (RR) => Boolean, maxVersions: Int = 1, cacheBlocks: Boolean = true, cacheSize: Int = 100) { 768 | table.withTable() { 769 | htable => 770 | val scan = makeScanner(maxVersions, cacheBlocks, cacheSize) 771 | 772 | val scanner = htable.getScanner(scan) 773 | 774 | try { 775 | for (result <- scanner) { 776 | if (!handler(table.buildRow(result))) throw YouCanStopNow 777 | } 778 | } catch { 779 | case _: Stopable => // nothing to see here... move along. move along. 780 | } finally { 781 | scanner.close() 782 | } 783 | } 784 | } 785 | 786 | 787 | } 788 | 789 | object Query2 { 790 | def p(depth: Int = 1, msg: Any) { 791 | println(("\t" * depth) + msg) 792 | } 793 | 794 | def printFilter(depth: Int, f: Filter) { 795 | p(depth, "Filter All Remaining: " + f.filterAllRemaining()) 796 | p(depth, "Has Filter Row: " + f.hasFilterRow) 797 | p(depth, "To String: " + f.toString) 798 | if (f.isInstanceOf[FilterList]) { 799 | val fl = f.asInstanceOf[FilterList] 800 | p(depth, "Operator: " + fl.getOperator()) 801 | fl.getFilters.foreach(sf => printFilter(depth + 1, sf)) 802 | } 803 | } 804 | 805 | } 806 | --------------------------------------------------------------------------------