You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by bu...@apache.org on 2017/11/09 04:59:48 UTC
[2/9] hbase git commit: HBASE-18817 pull the hbase-spark module out
of branch-2.
http://git-wip-us.apache.org/repos/asf/hbase/blob/516d370b/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DynamicLogicExpressionSuite.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DynamicLogicExpressionSuite.scala b/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DynamicLogicExpressionSuite.scala
deleted file mode 100644
index bc833e8..0000000
--- a/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DynamicLogicExpressionSuite.scala
+++ /dev/null
@@ -1,339 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.spark
-
-import java.util
-
-import org.apache.hadoop.hbase.spark.datasources.{HBaseSparkConf, JavaBytesEncoder}
-import org.apache.hadoop.hbase.util.Bytes
-import org.apache.spark.Logging
-import org.apache.spark.sql.types._
-import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
-
-class DynamicLogicExpressionSuite extends FunSuite with
-BeforeAndAfterEach with BeforeAndAfterAll with Logging {
-
- val encoder = JavaBytesEncoder.create(HBaseSparkConf.DEFAULT_QUERY_ENCODER)
-
- test("Basic And Test") {
- val leftLogic = new LessThanLogicExpression("Col1", 0)
- leftLogic.setEncoder(encoder)
- val rightLogic = new GreaterThanLogicExpression("Col1", 1)
- rightLogic.setEncoder(encoder)
- val andLogic = new AndLogicExpression(leftLogic, rightLogic)
-
- val columnToCurrentRowValueMap = new util.HashMap[String, ByteArrayComparable]()
-
- columnToCurrentRowValueMap.put("Col1", new ByteArrayComparable(Bytes.toBytes(10)))
- val valueFromQueryValueArray = new Array[Array[Byte]](2)
- valueFromQueryValueArray(0) = encoder.encode(IntegerType, 15)
- valueFromQueryValueArray(1) = encoder.encode(IntegerType, 5)
- assert(andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10)
- valueFromQueryValueArray(1) = encoder.encode(IntegerType, 5)
- assert(!andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- valueFromQueryValueArray(0) = encoder.encode(IntegerType, 15)
- valueFromQueryValueArray(1) = encoder.encode(IntegerType, 10)
- assert(!andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- val expressionString = andLogic.toExpressionString
-
- assert(expressionString.equals("( Col1 < 0 AND Col1 > 1 )"))
-
- val builtExpression = DynamicLogicExpressionBuilder.build(expressionString, encoder)
- valueFromQueryValueArray(0) = encoder.encode(IntegerType, 15)
- valueFromQueryValueArray(1) = encoder.encode(IntegerType, 5)
- assert(builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10)
- valueFromQueryValueArray(1) = encoder.encode(IntegerType, 5)
- assert(!builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- valueFromQueryValueArray(0) = encoder.encode(IntegerType, 15)
- valueFromQueryValueArray(1) = encoder.encode(IntegerType, 10)
- assert(!builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- }
-
- test("Basic OR Test") {
- val leftLogic = new LessThanLogicExpression("Col1", 0)
- leftLogic.setEncoder(encoder)
- val rightLogic = new GreaterThanLogicExpression("Col1", 1)
- rightLogic.setEncoder(encoder)
- val OrLogic = new OrLogicExpression(leftLogic, rightLogic)
-
- val columnToCurrentRowValueMap = new util.HashMap[String, ByteArrayComparable]()
-
- columnToCurrentRowValueMap.put("Col1", new ByteArrayComparable(Bytes.toBytes(10)))
- val valueFromQueryValueArray = new Array[Array[Byte]](2)
- valueFromQueryValueArray(0) = encoder.encode(IntegerType, 15)
- valueFromQueryValueArray(1) = encoder.encode(IntegerType, 5)
- assert(OrLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10)
- valueFromQueryValueArray(1) = encoder.encode(IntegerType, 5)
- assert(OrLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- valueFromQueryValueArray(0) = encoder.encode(IntegerType, 15)
- valueFromQueryValueArray(1) = encoder.encode(IntegerType, 10)
- assert(OrLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10)
- valueFromQueryValueArray(1) = encoder.encode(IntegerType, 10)
- assert(!OrLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- val expressionString = OrLogic.toExpressionString
-
- assert(expressionString.equals("( Col1 < 0 OR Col1 > 1 )"))
-
- val builtExpression = DynamicLogicExpressionBuilder.build(expressionString, encoder)
- valueFromQueryValueArray(0) = encoder.encode(IntegerType, 15)
- valueFromQueryValueArray(1) = encoder.encode(IntegerType, 5)
- assert(builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10)
- valueFromQueryValueArray(1) = encoder.encode(IntegerType, 5)
- assert(builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- valueFromQueryValueArray(0) = encoder.encode(IntegerType, 15)
- valueFromQueryValueArray(1) = encoder.encode(IntegerType, 10)
- assert(builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10)
- valueFromQueryValueArray(1) = encoder.encode(IntegerType, 10)
- assert(!builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
- }
-
- test("Basic Command Test") {
- val greaterLogic = new GreaterThanLogicExpression("Col1", 0)
- greaterLogic.setEncoder(encoder)
- val greaterAndEqualLogic = new GreaterThanOrEqualLogicExpression("Col1", 0)
- greaterAndEqualLogic.setEncoder(encoder)
- val lessLogic = new LessThanLogicExpression("Col1", 0)
- lessLogic.setEncoder(encoder)
- val lessAndEqualLogic = new LessThanOrEqualLogicExpression("Col1", 0)
- lessAndEqualLogic.setEncoder(encoder)
- val equalLogic = new EqualLogicExpression("Col1", 0, false)
- val notEqualLogic = new EqualLogicExpression("Col1", 0, true)
- val passThrough = new PassThroughLogicExpression
-
- val columnToCurrentRowValueMap = new util.HashMap[String, ByteArrayComparable]()
- columnToCurrentRowValueMap.put("Col1", new ByteArrayComparable(Bytes.toBytes(10)))
- val valueFromQueryValueArray = new Array[Array[Byte]](1)
-
- //great than
- valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10)
- assert(!greaterLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- valueFromQueryValueArray(0) = encoder.encode(IntegerType, 20)
- assert(!greaterLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- //great than and equal
- valueFromQueryValueArray(0) = encoder.encode(IntegerType, 5)
- assert(greaterAndEqualLogic.execute(columnToCurrentRowValueMap,
- valueFromQueryValueArray))
-
- valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10)
- assert(greaterAndEqualLogic.execute(columnToCurrentRowValueMap,
- valueFromQueryValueArray))
-
- valueFromQueryValueArray(0) = encoder.encode(IntegerType, 20)
- assert(!greaterAndEqualLogic.execute(columnToCurrentRowValueMap,
- valueFromQueryValueArray))
-
- //less than
- valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10)
- assert(!lessLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- valueFromQueryValueArray(0) = encoder.encode(IntegerType, 5)
- assert(!lessLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- //less than and equal
- valueFromQueryValueArray(0) = encoder.encode(IntegerType, 20)
- assert(lessAndEqualLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- valueFromQueryValueArray(0) = encoder.encode(IntegerType, 20)
- assert(lessAndEqualLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10)
- assert(lessAndEqualLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- //equal too
- valueFromQueryValueArray(0) = Bytes.toBytes(10)
- assert(equalLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- valueFromQueryValueArray(0) = Bytes.toBytes(5)
- assert(!equalLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- //not equal too
- valueFromQueryValueArray(0) = Bytes.toBytes(10)
- assert(!notEqualLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- valueFromQueryValueArray(0) = Bytes.toBytes(5)
- assert(notEqualLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- //pass through
- valueFromQueryValueArray(0) = Bytes.toBytes(10)
- assert(passThrough.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- valueFromQueryValueArray(0) = Bytes.toBytes(5)
- assert(passThrough.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
- }
-
-
- test("Double Type") {
- val leftLogic = new LessThanLogicExpression("Col1", 0)
- leftLogic.setEncoder(encoder)
- val rightLogic = new GreaterThanLogicExpression("Col1", 1)
- rightLogic.setEncoder(encoder)
- val andLogic = new AndLogicExpression(leftLogic, rightLogic)
-
- val columnToCurrentRowValueMap = new util.HashMap[String, ByteArrayComparable]()
-
- columnToCurrentRowValueMap.put("Col1", new ByteArrayComparable(Bytes.toBytes(-4.0d)))
- val valueFromQueryValueArray = new Array[Array[Byte]](2)
- valueFromQueryValueArray(0) = encoder.encode(DoubleType, 15.0d)
- valueFromQueryValueArray(1) = encoder.encode(DoubleType, -5.0d)
- assert(andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- valueFromQueryValueArray(0) = encoder.encode(DoubleType, 10.0d)
- valueFromQueryValueArray(1) = encoder.encode(DoubleType, -1.0d)
- assert(!andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- valueFromQueryValueArray(0) = encoder.encode(DoubleType, -10.0d)
- valueFromQueryValueArray(1) = encoder.encode(DoubleType, -20.0d)
- assert(!andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- val expressionString = andLogic.toExpressionString
- // Note that here 0 and 1 is index, instead of value.
- assert(expressionString.equals("( Col1 < 0 AND Col1 > 1 )"))
-
- val builtExpression = DynamicLogicExpressionBuilder.build(expressionString, encoder)
- valueFromQueryValueArray(0) = encoder.encode(DoubleType, 15.0d)
- valueFromQueryValueArray(1) = encoder.encode(DoubleType, -5.0d)
- assert(builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- valueFromQueryValueArray(0) = encoder.encode(DoubleType, 10.0d)
- valueFromQueryValueArray(1) = encoder.encode(DoubleType, -1.0d)
- assert(!builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- valueFromQueryValueArray(0) = encoder.encode(DoubleType, -10.0d)
- valueFromQueryValueArray(1) = encoder.encode(DoubleType, -20.0d)
- assert(!builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
- }
-
- test("Float Type") {
- val leftLogic = new LessThanLogicExpression("Col1", 0)
- leftLogic.setEncoder(encoder)
- val rightLogic = new GreaterThanLogicExpression("Col1", 1)
- rightLogic.setEncoder(encoder)
- val andLogic = new AndLogicExpression(leftLogic, rightLogic)
-
- val columnToCurrentRowValueMap = new util.HashMap[String, ByteArrayComparable]()
-
- columnToCurrentRowValueMap.put("Col1", new ByteArrayComparable(Bytes.toBytes(-4.0f)))
- val valueFromQueryValueArray = new Array[Array[Byte]](2)
- valueFromQueryValueArray(0) = encoder.encode(FloatType, 15.0f)
- valueFromQueryValueArray(1) = encoder.encode(FloatType, -5.0f)
- assert(andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- valueFromQueryValueArray(0) = encoder.encode(FloatType, 10.0f)
- valueFromQueryValueArray(1) = encoder.encode(FloatType, -1.0f)
- assert(!andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- valueFromQueryValueArray(0) = encoder.encode(FloatType, -10.0f)
- valueFromQueryValueArray(1) = encoder.encode(FloatType, -20.0f)
- assert(!andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- val expressionString = andLogic.toExpressionString
- // Note that here 0 and 1 is index, instead of value.
- assert(expressionString.equals("( Col1 < 0 AND Col1 > 1 )"))
-
- val builtExpression = DynamicLogicExpressionBuilder.build(expressionString, encoder)
- valueFromQueryValueArray(0) = encoder.encode(FloatType, 15.0f)
- valueFromQueryValueArray(1) = encoder.encode(FloatType, -5.0f)
- assert(builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- valueFromQueryValueArray(0) = encoder.encode(FloatType, 10.0f)
- valueFromQueryValueArray(1) = encoder.encode(FloatType, -1.0f)
- assert(!builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- valueFromQueryValueArray(0) = encoder.encode(FloatType, -10.0f)
- valueFromQueryValueArray(1) = encoder.encode(FloatType, -20.0f)
- assert(!builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
- }
-
- test("String Type") {
- val leftLogic = new LessThanLogicExpression("Col1", 0)
- leftLogic.setEncoder(encoder)
- val rightLogic = new GreaterThanLogicExpression("Col1", 1)
- rightLogic.setEncoder(encoder)
- val andLogic = new AndLogicExpression(leftLogic, rightLogic)
-
- val columnToCurrentRowValueMap = new util.HashMap[String, ByteArrayComparable]()
-
- columnToCurrentRowValueMap.put("Col1", new ByteArrayComparable(Bytes.toBytes("row005")))
- val valueFromQueryValueArray = new Array[Array[Byte]](2)
- valueFromQueryValueArray(0) = encoder.encode(StringType, "row015")
- valueFromQueryValueArray(1) = encoder.encode(StringType, "row000")
- assert(andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- valueFromQueryValueArray(0) = encoder.encode(StringType, "row004")
- valueFromQueryValueArray(1) = encoder.encode(StringType, "row000")
- assert(!andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- valueFromQueryValueArray(0) = encoder.encode(StringType, "row020")
- valueFromQueryValueArray(1) = encoder.encode(StringType, "row010")
- assert(!andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- val expressionString = andLogic.toExpressionString
- // Note that here 0 and 1 is index, instead of value.
- assert(expressionString.equals("( Col1 < 0 AND Col1 > 1 )"))
-
- val builtExpression = DynamicLogicExpressionBuilder.build(expressionString, encoder)
- valueFromQueryValueArray(0) = encoder.encode(StringType, "row015")
- valueFromQueryValueArray(1) = encoder.encode(StringType, "row000")
- assert(builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- valueFromQueryValueArray(0) = encoder.encode(StringType, "row004")
- valueFromQueryValueArray(1) = encoder.encode(StringType, "row000")
- assert(!builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
-
- valueFromQueryValueArray(0) = encoder.encode(StringType, "row020")
- valueFromQueryValueArray(1) = encoder.encode(StringType, "row010")
- assert(!builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
- }
-
- test("Boolean Type") {
- val leftLogic = new LessThanLogicExpression("Col1", 0)
- leftLogic.setEncoder(encoder)
- val rightLogic = new GreaterThanLogicExpression("Col1", 1)
- rightLogic.setEncoder(encoder)
-
- val columnToCurrentRowValueMap = new util.HashMap[String, ByteArrayComparable]()
-
- columnToCurrentRowValueMap.put("Col1", new ByteArrayComparable(Bytes.toBytes(false)))
- val valueFromQueryValueArray = new Array[Array[Byte]](2)
- valueFromQueryValueArray(0) = encoder.encode(BooleanType, true)
- valueFromQueryValueArray(1) = encoder.encode(BooleanType, false)
- assert(leftLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
- assert(!rightLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/516d370b/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseCatalogSuite.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseCatalogSuite.scala b/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseCatalogSuite.scala
deleted file mode 100644
index 49e2f6c..0000000
--- a/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseCatalogSuite.scala
+++ /dev/null
@@ -1,111 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.spark
-
-import org.apache.hadoop.hbase.spark.datasources.{DoubleSerDes, SerDes}
-import org.apache.hadoop.hbase.util.Bytes
-import org.apache.spark.Logging
-import org.apache.spark.sql.datasources.hbase.{DataTypeParserWrapper, HBaseTableCatalog}
-import org.apache.spark.sql.types._
-import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
-
-class HBaseCatalogSuite extends FunSuite with BeforeAndAfterEach with BeforeAndAfterAll with Logging {
-
- val map = s"""MAP<int, struct<varchar:string>>"""
- val array = s"""array<struct<tinYint:tinyint>>"""
- val arrayMap = s"""MAp<int, ARRAY<double>>"""
- val catalog = s"""{
- |"table":{"namespace":"default", "name":"htable"},
- |"rowkey":"key1:key2",
- |"columns":{
- |"col1":{"cf":"rowkey", "col":"key1", "type":"string"},
- |"col2":{"cf":"rowkey", "col":"key2", "type":"double"},
- |"col3":{"cf":"cf1", "col":"col2", "type":"binary"},
- |"col4":{"cf":"cf1", "col":"col3", "type":"timestamp"},
- |"col5":{"cf":"cf1", "col":"col4", "type":"double", "serdes":"${classOf[DoubleSerDes].getName}"},
- |"col6":{"cf":"cf1", "col":"col5", "type":"$map"},
- |"col7":{"cf":"cf1", "col":"col6", "type":"$array"},
- |"col8":{"cf":"cf1", "col":"col7", "type":"$arrayMap"}
- |}
- |}""".stripMargin
- val parameters = Map(HBaseTableCatalog.tableCatalog->catalog)
- val t = HBaseTableCatalog(parameters)
-
- def checkDataType(dataTypeString: String, expectedDataType: DataType): Unit = {
- test(s"parse ${dataTypeString.replace("\n", "")}") {
- assert(DataTypeParserWrapper.parse(dataTypeString) === expectedDataType)
- }
- }
- test("basic") {
- assert(t.getField("col1").isRowKey == true)
- assert(t.getPrimaryKey == "key1")
- assert(t.getField("col3").dt == BinaryType)
- assert(t.getField("col4").dt == TimestampType)
- assert(t.getField("col5").dt == DoubleType)
- assert(t.getField("col5").serdes != None)
- assert(t.getField("col4").serdes == None)
- assert(t.getField("col1").isRowKey)
- assert(t.getField("col2").isRowKey)
- assert(!t.getField("col3").isRowKey)
- assert(t.getField("col2").length == Bytes.SIZEOF_DOUBLE)
- assert(t.getField("col1").length == -1)
- assert(t.getField("col8").length == -1)
- }
-
- checkDataType(
- map,
- t.getField("col6").dt
- )
-
- checkDataType(
- array,
- t.getField("col7").dt
- )
-
- checkDataType(
- arrayMap,
- t.getField("col8").dt
- )
-
- test("convert") {
- val m = Map("hbase.columns.mapping" ->
- "KEY_FIELD STRING :key, A_FIELD STRING c:a, B_FIELD DOUBLE c:b, C_FIELD BINARY c:c,",
- "hbase.table" -> "t1")
- val map = HBaseTableCatalog.convert(m)
- val json = map.get(HBaseTableCatalog.tableCatalog).get
- val parameters = Map(HBaseTableCatalog.tableCatalog->json)
- val t = HBaseTableCatalog(parameters)
- assert(t.getField("KEY_FIELD").isRowKey)
- assert(DataTypeParserWrapper.parse("STRING") === t.getField("A_FIELD").dt)
- assert(!t.getField("A_FIELD").isRowKey)
- assert(DataTypeParserWrapper.parse("DOUBLE") === t.getField("B_FIELD").dt)
- assert(DataTypeParserWrapper.parse("BINARY") === t.getField("C_FIELD").dt)
- }
-
- test("compatiblity") {
- val m = Map("hbase.columns.mapping" ->
- "KEY_FIELD STRING :key, A_FIELD STRING c:a, B_FIELD DOUBLE c:b, C_FIELD BINARY c:c,",
- "hbase.table" -> "t1")
- val t = HBaseTableCatalog(m)
- assert(t.getField("KEY_FIELD").isRowKey)
- assert(DataTypeParserWrapper.parse("STRING") === t.getField("A_FIELD").dt)
- assert(!t.getField("A_FIELD").isRowKey)
- assert(DataTypeParserWrapper.parse("DOUBLE") === t.getField("B_FIELD").dt)
- assert(DataTypeParserWrapper.parse("BINARY") === t.getField("C_FIELD").dt)
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/516d370b/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseConnectionCacheSuite.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseConnectionCacheSuite.scala b/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseConnectionCacheSuite.scala
deleted file mode 100644
index b3fdd4e..0000000
--- a/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseConnectionCacheSuite.scala
+++ /dev/null
@@ -1,237 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.spark
-
-import java.util.concurrent.ExecutorService
-import scala.util.Random
-
-import org.apache.hadoop.hbase.client.{BufferedMutator, Table, RegionLocator,
- Connection, BufferedMutatorParams, Admin, TableBuilder}
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.hbase.TableName
-import org.apache.spark.Logging
-import org.scalatest.FunSuite
-
-case class HBaseConnectionKeyMocker (confId: Int) extends HBaseConnectionKey (null) {
- override def hashCode: Int = {
- confId
- }
-
- override def equals(obj: Any): Boolean = {
- if(!obj.isInstanceOf[HBaseConnectionKeyMocker])
- false
- else
- confId == obj.asInstanceOf[HBaseConnectionKeyMocker].confId
- }
-}
-
-class ConnectionMocker extends Connection {
- var isClosed: Boolean = false
-
- def getRegionLocator (tableName: TableName): RegionLocator = null
- def getConfiguration: Configuration = null
- def getTable (tableName: TableName): Table = null
- def getTable(tableName: TableName, pool: ExecutorService): Table = null
- def getBufferedMutator (params: BufferedMutatorParams): BufferedMutator = null
- def getBufferedMutator (tableName: TableName): BufferedMutator = null
- def getAdmin: Admin = null
- def getTableBuilder(tableName: TableName, pool: ExecutorService): TableBuilder = null
-
- def close(): Unit = {
- if (isClosed)
- throw new IllegalStateException()
- isClosed = true
- }
-
- def isAborted: Boolean = true
- def abort(why: String, e: Throwable) = {}
-}
-
-class HBaseConnectionCacheSuite extends FunSuite with Logging {
- /*
- * These tests must be performed sequentially as they operate with an
- * unique running thread and resource.
- *
- * It looks there's no way to tell FunSuite to do so, so making those
- * test cases normal functions which are called sequentially in a single
- * test case.
- */
- test("all test cases") {
- testBasic()
- testWithPressureWithoutClose()
- testWithPressureWithClose()
- }
-
- def cleanEnv() {
- HBaseConnectionCache.connectionMap.synchronized {
- HBaseConnectionCache.connectionMap.clear()
- HBaseConnectionCache.cacheStat.numActiveConnections = 0
- HBaseConnectionCache.cacheStat.numActualConnectionsCreated = 0
- HBaseConnectionCache.cacheStat.numTotalRequests = 0
- }
- }
-
- def testBasic() {
- cleanEnv()
- HBaseConnectionCache.setTimeout(1 * 1000)
-
- val connKeyMocker1 = new HBaseConnectionKeyMocker(1)
- val connKeyMocker1a = new HBaseConnectionKeyMocker(1)
- val connKeyMocker2 = new HBaseConnectionKeyMocker(2)
-
- val c1 = HBaseConnectionCache
- .getConnection(connKeyMocker1, new ConnectionMocker)
-
- assert(HBaseConnectionCache.connectionMap.size === 1)
- assert(HBaseConnectionCache.getStat.numTotalRequests === 1)
- assert(HBaseConnectionCache.getStat.numActualConnectionsCreated === 1)
- assert(HBaseConnectionCache.getStat.numActiveConnections === 1)
-
- val c1a = HBaseConnectionCache
- .getConnection(connKeyMocker1a, new ConnectionMocker)
-
- HBaseConnectionCache.connectionMap.synchronized {
- assert(HBaseConnectionCache.connectionMap.size === 1)
- assert(HBaseConnectionCache.getStat.numTotalRequests === 2)
- assert(HBaseConnectionCache.getStat.numActualConnectionsCreated === 1)
- assert(HBaseConnectionCache.getStat.numActiveConnections === 1)
- }
-
- val c2 = HBaseConnectionCache
- .getConnection(connKeyMocker2, new ConnectionMocker)
-
- HBaseConnectionCache.connectionMap.synchronized {
- assert(HBaseConnectionCache.connectionMap.size === 2)
- assert(HBaseConnectionCache.getStat.numTotalRequests === 3)
- assert(HBaseConnectionCache.getStat.numActualConnectionsCreated === 2)
- assert(HBaseConnectionCache.getStat.numActiveConnections === 2)
- }
-
- c1.close()
- HBaseConnectionCache.connectionMap.synchronized {
- assert(HBaseConnectionCache.connectionMap.size === 2)
- assert(HBaseConnectionCache.getStat.numActiveConnections === 2)
- }
-
- c1a.close()
- HBaseConnectionCache.connectionMap.synchronized {
- assert(HBaseConnectionCache.connectionMap.size === 2)
- assert(HBaseConnectionCache.getStat.numActiveConnections === 2)
- }
-
- Thread.sleep(3 * 1000) // Leave housekeeping thread enough time
- HBaseConnectionCache.connectionMap.synchronized {
- assert(HBaseConnectionCache.connectionMap.size === 1)
- assert(HBaseConnectionCache.connectionMap.iterator.next()._1
- .asInstanceOf[HBaseConnectionKeyMocker].confId === 2)
- assert(HBaseConnectionCache.getStat.numActiveConnections === 1)
- }
-
- c2.close()
- }
-
- def testWithPressureWithoutClose() {
- cleanEnv()
-
- class TestThread extends Runnable {
- override def run() {
- for (i <- 0 to 999) {
- val c = HBaseConnectionCache.getConnection(
- new HBaseConnectionKeyMocker(Random.nextInt(10)), new ConnectionMocker)
- }
- }
- }
-
- HBaseConnectionCache.setTimeout(500)
- val threads: Array[Thread] = new Array[Thread](100)
- for (i <- 0 to 99) {
- threads.update(i, new Thread(new TestThread()))
- threads(i).run()
- }
- try {
- threads.foreach { x => x.join() }
- } catch {
- case e: InterruptedException => println(e.getMessage)
- }
-
- Thread.sleep(1000)
- HBaseConnectionCache.connectionMap.synchronized {
- assert(HBaseConnectionCache.connectionMap.size === 10)
- assert(HBaseConnectionCache.getStat.numTotalRequests === 100 * 1000)
- assert(HBaseConnectionCache.getStat.numActualConnectionsCreated === 10)
- assert(HBaseConnectionCache.getStat.numActiveConnections === 10)
-
- var totalRc : Int = 0
- HBaseConnectionCache.connectionMap.foreach {
- x => totalRc += x._2.refCount
- }
- assert(totalRc === 100 * 1000)
- HBaseConnectionCache.connectionMap.foreach {
- x => {
- x._2.refCount = 0
- x._2.timestamp = System.currentTimeMillis() - 1000
- }
- }
- }
- Thread.sleep(1000)
- assert(HBaseConnectionCache.connectionMap.size === 0)
- assert(HBaseConnectionCache.getStat.numActualConnectionsCreated === 10)
- assert(HBaseConnectionCache.getStat.numActiveConnections === 0)
- }
-
- def testWithPressureWithClose() {
- cleanEnv()
-
- class TestThread extends Runnable {
- override def run() {
- for (i <- 0 to 999) {
- val c = HBaseConnectionCache.getConnection(
- new HBaseConnectionKeyMocker(Random.nextInt(10)), new ConnectionMocker)
- Thread.`yield`()
- c.close()
- }
- }
- }
-
- HBaseConnectionCache.setTimeout(3 * 1000)
- val threads: Array[Thread] = new Array[Thread](100)
- for (i <- threads.indices) {
- threads.update(i, new Thread(new TestThread()))
- threads(i).run()
- }
- try {
- threads.foreach { x => x.join() }
- } catch {
- case e: InterruptedException => println(e.getMessage)
- }
-
- HBaseConnectionCache.connectionMap.synchronized {
- assert(HBaseConnectionCache.connectionMap.size === 10)
- assert(HBaseConnectionCache.getStat.numTotalRequests === 100 * 1000)
- assert(HBaseConnectionCache.getStat.numActualConnectionsCreated === 10)
- assert(HBaseConnectionCache.getStat.numActiveConnections === 10)
- }
-
- Thread.sleep(6 * 1000)
- HBaseConnectionCache.connectionMap.synchronized {
- assert(HBaseConnectionCache.connectionMap.size === 0)
- assert(HBaseConnectionCache.getStat.numActualConnectionsCreated === 10)
- assert(HBaseConnectionCache.getStat.numActiveConnections === 0)
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/516d370b/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseContextSuite.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseContextSuite.scala b/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseContextSuite.scala
deleted file mode 100644
index 1e1e52d..0000000
--- a/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseContextSuite.scala
+++ /dev/null
@@ -1,356 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.spark
-
-import org.apache.hadoop.hbase.client._
-import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter
-import org.apache.hadoop.hbase.util.Bytes
-import org.apache.hadoop.hbase.{ CellUtil, TableName, HBaseTestingUtility}
-import org.apache.spark.{SparkException, Logging, SparkContext}
-import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
-
-class HBaseContextSuite extends FunSuite with
-BeforeAndAfterEach with BeforeAndAfterAll with Logging {
-
- @transient var sc: SparkContext = null
- var TEST_UTIL = new HBaseTestingUtility
-
- val tableName = "t1"
- val columnFamily = "c"
-
- override def beforeAll() {
- TEST_UTIL.startMiniCluster()
- logInfo(" - minicluster started")
-
- try {
- TEST_UTIL.deleteTable(TableName.valueOf(tableName))
- } catch {
- case e: Exception =>
- logInfo(" - no table " + tableName + " found")
- }
- logInfo(" - creating table " + tableName)
- TEST_UTIL.createTable(TableName.valueOf(tableName), Bytes.toBytes(columnFamily))
- logInfo(" - created table")
-
- val envMap = Map[String,String](("Xmx", "512m"))
-
- sc = new SparkContext("local", "test", null, Nil, envMap)
- }
-
- override def afterAll() {
- logInfo("shuting down minicluster")
- TEST_UTIL.shutdownMiniCluster()
- logInfo(" - minicluster shut down")
- TEST_UTIL.cleanupTestDir()
- sc.stop()
- }
-
- test("bulkput to test HBase client") {
- val config = TEST_UTIL.getConfiguration
- val rdd = sc.parallelize(Array(
- (Bytes.toBytes("1"),
- Array((Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1")))),
- (Bytes.toBytes("2"),
- Array((Bytes.toBytes(columnFamily), Bytes.toBytes("b"), Bytes.toBytes("foo2")))),
- (Bytes.toBytes("3"),
- Array((Bytes.toBytes(columnFamily), Bytes.toBytes("c"), Bytes.toBytes("foo3")))),
- (Bytes.toBytes("4"),
- Array((Bytes.toBytes(columnFamily), Bytes.toBytes("d"), Bytes.toBytes("foo")))),
- (Bytes.toBytes("5"),
- Array((Bytes.toBytes(columnFamily), Bytes.toBytes("e"), Bytes.toBytes("bar"))))))
-
- val hbaseContext = new HBaseContext(sc, config)
- hbaseContext.bulkPut[(Array[Byte], Array[(Array[Byte], Array[Byte], Array[Byte])])](rdd,
- TableName.valueOf(tableName),
- (putRecord) => {
- val put = new Put(putRecord._1)
- putRecord._2.foreach((putValue) => put.addColumn(putValue._1, putValue._2, putValue._3))
- put
- })
-
- val connection = ConnectionFactory.createConnection(config)
- val table = connection.getTable(TableName.valueOf("t1"))
-
- try {
- val foo1 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("1"))).
- getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a"))))
- assert(foo1 == "foo1")
-
- val foo2 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("2"))).
- getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("b"))))
- assert(foo2 == "foo2")
-
- val foo3 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("3"))).
- getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("c"))))
- assert(foo3 == "foo3")
-
- val foo4 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("4"))).
- getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("d"))))
- assert(foo4 == "foo")
-
- val foo5 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("5"))).
- getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("e"))))
- assert(foo5 == "bar")
-
- } finally {
- table.close()
- connection.close()
- }
- }
-
- test("bulkDelete to test HBase client") {
- val config = TEST_UTIL.getConfiguration
- val connection = ConnectionFactory.createConnection(config)
- val table = connection.getTable(TableName.valueOf("t1"))
-
- try {
- var put = new Put(Bytes.toBytes("delete1"))
- put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1"))
- table.put(put)
- put = new Put(Bytes.toBytes("delete2"))
- put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo2"))
- table.put(put)
- put = new Put(Bytes.toBytes("delete3"))
- put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo3"))
- table.put(put)
-
- val rdd = sc.parallelize(Array(
- Bytes.toBytes("delete1"),
- Bytes.toBytes("delete3")))
-
- val hbaseContext = new HBaseContext(sc, config)
- hbaseContext.bulkDelete[Array[Byte]](rdd,
- TableName.valueOf(tableName),
- putRecord => new Delete(putRecord),
- 4)
-
- assert(table.get(new Get(Bytes.toBytes("delete1"))).
- getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")) == null)
- assert(table.get(new Get(Bytes.toBytes("delete3"))).
- getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")) == null)
- assert(Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("delete2"))).
- getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")))).equals("foo2"))
- } finally {
- table.close()
- connection.close()
- }
- }
-
- test("bulkGet to test HBase client") {
- val config = TEST_UTIL.getConfiguration
- val connection = ConnectionFactory.createConnection(config)
- val table = connection.getTable(TableName.valueOf("t1"))
-
- try {
- var put = new Put(Bytes.toBytes("get1"))
- put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1"))
- table.put(put)
- put = new Put(Bytes.toBytes("get2"))
- put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo2"))
- table.put(put)
- put = new Put(Bytes.toBytes("get3"))
- put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo3"))
- table.put(put)
- } finally {
- table.close()
- connection.close()
- }
- val rdd = sc.parallelize(Array(
- Bytes.toBytes("get1"),
- Bytes.toBytes("get2"),
- Bytes.toBytes("get3"),
- Bytes.toBytes("get4")))
- val hbaseContext = new HBaseContext(sc, config)
-
- val getRdd = hbaseContext.bulkGet[Array[Byte], String](
- TableName.valueOf(tableName),
- 2,
- rdd,
- record => {
- new Get(record)
- },
- (result: Result) => {
- if (result.listCells() != null) {
- val it = result.listCells().iterator()
- val B = new StringBuilder
-
- B.append(Bytes.toString(result.getRow) + ":")
-
- while (it.hasNext) {
- val cell = it.next()
- val q = Bytes.toString(CellUtil.cloneQualifier(cell))
- if (q.equals("counter")) {
- B.append("(" + q + "," + Bytes.toLong(CellUtil.cloneValue(cell)) + ")")
- } else {
- B.append("(" + q + "," + Bytes.toString(CellUtil.cloneValue(cell)) + ")")
- }
- }
- "" + B.toString
- } else {
- ""
- }
- })
- val getArray = getRdd.collect()
-
- assert(getArray.length == 4)
- assert(getArray.contains("get1:(a,foo1)"))
- assert(getArray.contains("get2:(a,foo2)"))
- assert(getArray.contains("get3:(a,foo3)"))
-
- }
-
- test("BulkGet failure test: bad table") {
- val config = TEST_UTIL.getConfiguration
-
- val rdd = sc.parallelize(Array(
- Bytes.toBytes("get1"),
- Bytes.toBytes("get2"),
- Bytes.toBytes("get3"),
- Bytes.toBytes("get4")))
- val hbaseContext = new HBaseContext(sc, config)
-
- intercept[SparkException] {
- try {
- val getRdd = hbaseContext.bulkGet[Array[Byte], String](
- TableName.valueOf("badTableName"),
- 2,
- rdd,
- record => {
- new Get(record)
- },
- (result: Result) => "1")
-
- getRdd.collect()
-
- fail("We should have failed and not reached this line")
- } catch {
- case ex: SparkException => {
- assert(
- ex.getMessage.contains(
- "org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException"))
- throw ex
- }
- }
- }
- }
-
- test("BulkGet failure test: bad column") {
-
- val config = TEST_UTIL.getConfiguration
- val connection = ConnectionFactory.createConnection(config)
- val table = connection.getTable(TableName.valueOf("t1"))
-
- try {
- var put = new Put(Bytes.toBytes("get1"))
- put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1"))
- table.put(put)
- put = new Put(Bytes.toBytes("get2"))
- put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo2"))
- table.put(put)
- put = new Put(Bytes.toBytes("get3"))
- put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo3"))
- table.put(put)
- } finally {
- table.close()
- connection.close()
- }
-
- val rdd = sc.parallelize(Array(
- Bytes.toBytes("get1"),
- Bytes.toBytes("get2"),
- Bytes.toBytes("get3"),
- Bytes.toBytes("get4")))
- val hbaseContext = new HBaseContext(sc, config)
-
- val getRdd = hbaseContext.bulkGet[Array[Byte], String](
- TableName.valueOf(tableName),
- 2,
- rdd,
- record => {
- new Get(record)
- },
- (result: Result) => {
- if (result.listCells() != null) {
- val cellValue = result.getColumnLatestCell(
- Bytes.toBytes("c"), Bytes.toBytes("bad_column"))
- if (cellValue == null) "null" else "bad"
- } else "noValue"
- })
- var nullCounter = 0
- var noValueCounter = 0
- getRdd.collect().foreach(r => {
- if ("null".equals(r)) nullCounter += 1
- else if ("noValue".equals(r)) noValueCounter += 1
- })
- assert(nullCounter == 3)
- assert(noValueCounter == 1)
- }
-
- test("distributedScan to test HBase client") {
- val config = TEST_UTIL.getConfiguration
- val connection = ConnectionFactory.createConnection(config)
- val table = connection.getTable(TableName.valueOf("t1"))
-
- try {
- var put = new Put(Bytes.toBytes("scan1"))
- put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1"))
- table.put(put)
- put = new Put(Bytes.toBytes("scan2"))
- put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo2"))
- table.put(put)
- put = new Put(Bytes.toBytes("scan2"))
- put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("b"), Bytes.toBytes("foo-2"))
- table.put(put)
- put = new Put(Bytes.toBytes("scan3"))
- put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo3"))
- table.put(put)
- put = new Put(Bytes.toBytes("scan4"))
- put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo3"))
- table.put(put)
- put = new Put(Bytes.toBytes("scan5"))
- put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo3"))
- table.put(put)
- } finally {
- table.close()
- connection.close()
- }
-
- val hbaseContext = new HBaseContext(sc, config)
-
- val scan = new Scan()
- val filter = new FirstKeyOnlyFilter()
- scan.setCaching(100)
- scan.setStartRow(Bytes.toBytes("scan2"))
- scan.setStopRow(Bytes.toBytes("scan4_"))
- scan.setFilter(filter)
-
- val scanRdd = hbaseContext.hbaseRDD(TableName.valueOf(tableName), scan)
-
- try {
- val scanList = scanRdd.map(r => r._1.copyBytes()).collect()
- assert(scanList.length == 3)
- var cnt = 0
- scanRdd.map(r => r._2.listCells().size()).collect().foreach(l => {
- cnt += l
- })
- // the number of cells returned would be 4 without the Filter
- assert(cnt == 3);
- } catch {
- case ex: Exception => ex.printStackTrace()
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/516d370b/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseDStreamFunctionsSuite.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseDStreamFunctionsSuite.scala b/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseDStreamFunctionsSuite.scala
deleted file mode 100644
index e6767ae..0000000
--- a/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseDStreamFunctionsSuite.scala
+++ /dev/null
@@ -1,142 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.spark
-
-import org.apache.hadoop.hbase.client._
-import org.apache.hadoop.hbase.util.Bytes
-import org.apache.hadoop.hbase.{CellUtil, TableName, HBaseTestingUtility}
-import org.apache.spark.rdd.RDD
-import org.apache.spark.streaming.{Milliseconds, StreamingContext}
-import org.apache.spark.{SparkContext, Logging}
-import org.apache.hadoop.hbase.spark.HBaseDStreamFunctions._
-import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
-
-import scala.collection.mutable
-
-class HBaseDStreamFunctionsSuite extends FunSuite with
-BeforeAndAfterEach with BeforeAndAfterAll with Logging {
- @transient var sc: SparkContext = null
-
- var TEST_UTIL: HBaseTestingUtility = new HBaseTestingUtility
-
- val tableName = "t1"
- val columnFamily = "c"
-
- override def beforeAll() {
-
- TEST_UTIL.startMiniCluster()
-
- logInfo(" - minicluster started")
- try
- TEST_UTIL.deleteTable(TableName.valueOf(tableName))
- catch {
- case e: Exception => logInfo(" - no table " + tableName + " found")
-
- }
- logInfo(" - creating table " + tableName)
- TEST_UTIL.createTable(TableName.valueOf(tableName), Bytes.toBytes(columnFamily))
- logInfo(" - created table")
-
- sc = new SparkContext("local", "test")
- }
-
- override def afterAll() {
- TEST_UTIL.deleteTable(TableName.valueOf(tableName))
- TEST_UTIL.shutdownMiniCluster()
- sc.stop()
- }
-
- test("bulkput to test HBase client") {
- val config = TEST_UTIL.getConfiguration
- val rdd1 = sc.parallelize(Array(
- (Bytes.toBytes("1"),
- Array((Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1")))),
- (Bytes.toBytes("2"),
- Array((Bytes.toBytes(columnFamily), Bytes.toBytes("b"), Bytes.toBytes("foo2")))),
- (Bytes.toBytes("3"),
- Array((Bytes.toBytes(columnFamily), Bytes.toBytes("c"), Bytes.toBytes("foo3"))))))
-
- val rdd2 = sc.parallelize(Array(
- (Bytes.toBytes("4"),
- Array((Bytes.toBytes(columnFamily), Bytes.toBytes("d"), Bytes.toBytes("foo")))),
- (Bytes.toBytes("5"),
- Array((Bytes.toBytes(columnFamily), Bytes.toBytes("e"), Bytes.toBytes("bar"))))))
-
- var isFinished = false
-
- val hbaseContext = new HBaseContext(sc, config)
- val ssc = new StreamingContext(sc, Milliseconds(200))
-
- val queue = mutable.Queue[RDD[(Array[Byte], Array[(Array[Byte],
- Array[Byte], Array[Byte])])]]()
- queue += rdd1
- queue += rdd2
- val dStream = ssc.queueStream(queue)
-
- dStream.hbaseBulkPut(
- hbaseContext,
- TableName.valueOf(tableName),
- (putRecord) => {
- val put = new Put(putRecord._1)
- putRecord._2.foreach((putValue) => put.addColumn(putValue._1, putValue._2, putValue._3))
- put
- })
-
- dStream.foreachRDD(rdd => {
- if (rdd.count() == 0) {
- isFinished = true
- }
- })
-
- ssc.start()
-
- while (!isFinished) {
- Thread.sleep(100)
- }
-
- ssc.stop(true, true)
-
- val connection = ConnectionFactory.createConnection(config)
- val table = connection.getTable(TableName.valueOf("t1"))
-
- try {
- val foo1 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("1"))).
- getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a"))))
- assert(foo1 == "foo1")
-
- val foo2 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("2"))).
- getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("b"))))
- assert(foo2 == "foo2")
-
- val foo3 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("3"))).
- getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("c"))))
- assert(foo3 == "foo3")
-
- val foo4 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("4"))).
- getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("d"))))
- assert(foo4 == "foo")
-
- val foo5 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("5"))).
- getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("e"))))
- assert(foo5 == "bar")
- } finally {
- table.close()
- connection.close()
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/516d370b/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseRDDFunctionsSuite.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseRDDFunctionsSuite.scala b/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseRDDFunctionsSuite.scala
deleted file mode 100644
index 89148c3..0000000
--- a/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseRDDFunctionsSuite.scala
+++ /dev/null
@@ -1,398 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.spark
-
-import org.apache.hadoop.hbase.client._
-import org.apache.hadoop.hbase.util.Bytes
-import org.apache.hadoop.hbase.{CellUtil, TableName, HBaseTestingUtility}
-import org.apache.hadoop.hbase.spark.HBaseRDDFunctions._
-import org.apache.spark.{Logging, SparkContext}
-import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
-
-import scala.collection.mutable
-
-class HBaseRDDFunctionsSuite extends FunSuite with
-BeforeAndAfterEach with BeforeAndAfterAll with Logging {
- @transient var sc: SparkContext = null
- var TEST_UTIL: HBaseTestingUtility = new HBaseTestingUtility
-
- val tableName = "t1"
- val columnFamily = "c"
-
- override def beforeAll() {
-
- TEST_UTIL.startMiniCluster
-
- logInfo(" - minicluster started")
- try
- TEST_UTIL.deleteTable(TableName.valueOf(tableName))
- catch {
- case e: Exception => logInfo(" - no table " + tableName + " found")
-
- }
- logInfo(" - creating table " + tableName)
- TEST_UTIL.createTable(TableName.valueOf(tableName), Bytes.toBytes(columnFamily))
- logInfo(" - created table")
-
- sc = new SparkContext("local", "test")
- }
-
- override def afterAll() {
- TEST_UTIL.deleteTable(TableName.valueOf(tableName))
- logInfo("shuting down minicluster")
- TEST_UTIL.shutdownMiniCluster()
-
- sc.stop()
- }
-
- test("bulkput to test HBase client") {
- val config = TEST_UTIL.getConfiguration
- val rdd = sc.parallelize(Array(
- (Bytes.toBytes("1"),
- Array((Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1")))),
- (Bytes.toBytes("2"),
- Array((Bytes.toBytes(columnFamily), Bytes.toBytes("b"), Bytes.toBytes("foo2")))),
- (Bytes.toBytes("3"),
- Array((Bytes.toBytes(columnFamily), Bytes.toBytes("c"), Bytes.toBytes("foo3")))),
- (Bytes.toBytes("4"),
- Array((Bytes.toBytes(columnFamily), Bytes.toBytes("d"), Bytes.toBytes("foo")))),
- (Bytes.toBytes("5"),
- Array((Bytes.toBytes(columnFamily), Bytes.toBytes("e"), Bytes.toBytes("bar"))))))
-
- val hbaseContext = new HBaseContext(sc, config)
-
- rdd.hbaseBulkPut(
- hbaseContext,
- TableName.valueOf(tableName),
- (putRecord) => {
- val put = new Put(putRecord._1)
- putRecord._2.foreach((putValue) => put.addColumn(putValue._1, putValue._2, putValue._3))
- put
- })
-
- val connection = ConnectionFactory.createConnection(config)
- val table = connection.getTable(TableName.valueOf("t1"))
-
- try {
- val foo1 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("1"))).
- getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a"))))
- assert(foo1 == "foo1")
-
- val foo2 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("2"))).
- getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("b"))))
- assert(foo2 == "foo2")
-
- val foo3 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("3"))).
- getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("c"))))
- assert(foo3 == "foo3")
-
- val foo4 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("4"))).
- getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("d"))))
- assert(foo4 == "foo")
-
- val foo5 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("5"))).
- getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("e"))))
- assert(foo5 == "bar")
- } finally {
- table.close()
- connection.close()
- }
- }
-
- test("bulkDelete to test HBase client") {
- val config = TEST_UTIL.getConfiguration
- val connection = ConnectionFactory.createConnection(config)
- val table = connection.getTable(TableName.valueOf("t1"))
-
- try {
- var put = new Put(Bytes.toBytes("delete1"))
- put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1"))
- table.put(put)
- put = new Put(Bytes.toBytes("delete2"))
- put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo2"))
- table.put(put)
- put = new Put(Bytes.toBytes("delete3"))
- put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo3"))
- table.put(put)
-
- val rdd = sc.parallelize(Array(
- Bytes.toBytes("delete1"),
- Bytes.toBytes("delete3")))
-
- val hbaseContext = new HBaseContext(sc, config)
-
- rdd.hbaseBulkDelete(hbaseContext,
- TableName.valueOf(tableName),
- putRecord => new Delete(putRecord),
- 4)
-
- assert(table.get(new Get(Bytes.toBytes("delete1"))).
- getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")) == null)
- assert(table.get(new Get(Bytes.toBytes("delete3"))).
- getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")) == null)
- assert(Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("delete2"))).
- getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")))).equals("foo2"))
- } finally {
- table.close()
- connection.close()
- }
-
- }
-
- test("bulkGet to test HBase client") {
- val config = TEST_UTIL.getConfiguration
- val connection = ConnectionFactory.createConnection(config)
- val table = connection.getTable(TableName.valueOf("t1"))
-
- try {
- var put = new Put(Bytes.toBytes("get1"))
- put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1"))
- table.put(put)
- put = new Put(Bytes.toBytes("get2"))
- put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo2"))
- table.put(put)
- put = new Put(Bytes.toBytes("get3"))
- put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo3"))
- table.put(put)
- } finally {
- table.close()
- connection.close()
- }
-
- val rdd = sc.parallelize(Array(
- Bytes.toBytes("get1"),
- Bytes.toBytes("get2"),
- Bytes.toBytes("get3"),
- Bytes.toBytes("get4")))
- val hbaseContext = new HBaseContext(sc, config)
-
- //Get with custom convert logic
- val getRdd = rdd.hbaseBulkGet[String](hbaseContext, TableName.valueOf(tableName), 2,
- record => {
- new Get(record)
- },
- (result: Result) => {
- if (result.listCells() != null) {
- val it = result.listCells().iterator()
- val B = new StringBuilder
-
- B.append(Bytes.toString(result.getRow) + ":")
-
- while (it.hasNext) {
- val cell = it.next
- val q = Bytes.toString(CellUtil.cloneQualifier(cell))
- if (q.equals("counter")) {
- B.append("(" + q + "," + Bytes.toLong(CellUtil.cloneValue(cell)) + ")")
- } else {
- B.append("(" + q + "," + Bytes.toString(CellUtil.cloneValue(cell)) + ")")
- }
- }
- "" + B.toString
- } else {
- ""
- }
- })
-
- val getArray = getRdd.collect()
-
- assert(getArray.length == 4)
- assert(getArray.contains("get1:(a,foo1)"))
- assert(getArray.contains("get2:(a,foo2)"))
- assert(getArray.contains("get3:(a,foo3)"))
- }
-
- test("bulkGet default converter to test HBase client") {
- val config = TEST_UTIL.getConfiguration
- val connection = ConnectionFactory.createConnection(config)
- val table = connection.getTable(TableName.valueOf("t1"))
-
- try {
- var put = new Put(Bytes.toBytes("get1"))
- put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1"))
- table.put(put)
- put = new Put(Bytes.toBytes("get2"))
- put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo2"))
- table.put(put)
- put = new Put(Bytes.toBytes("get3"))
- put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo3"))
- table.put(put)
- } finally {
- table.close()
- connection.close()
- }
-
- val rdd = sc.parallelize(Array(
- Bytes.toBytes("get1"),
- Bytes.toBytes("get2"),
- Bytes.toBytes("get3"),
- Bytes.toBytes("get4")))
- val hbaseContext = new HBaseContext(sc, config)
-
- val getRdd = rdd.hbaseBulkGet(hbaseContext, TableName.valueOf("t1"), 2,
- record => {
- new Get(record)
- }).map((row) => {
- if (row != null && row._2.listCells() != null) {
- val it = row._2.listCells().iterator()
- val B = new StringBuilder
-
- B.append(Bytes.toString(row._2.getRow) + ":")
-
- while (it.hasNext) {
- val cell = it.next
- val q = Bytes.toString(CellUtil.cloneQualifier(cell))
- if (q.equals("counter")) {
- B.append("(" + q + "," + Bytes.toLong(CellUtil.cloneValue(cell)) + ")")
- } else {
- B.append("(" + q + "," + Bytes.toString(CellUtil.cloneValue(cell)) + ")")
- }
- }
- "" + B.toString
- } else {
- ""
- }})
-
- val getArray = getRdd.collect()
-
- assert(getArray.length == 4)
- assert(getArray.contains("get1:(a,foo1)"))
- assert(getArray.contains("get2:(a,foo2)"))
- assert(getArray.contains("get3:(a,foo3)"))
- }
-
- test("foreachPartition with puts to test HBase client") {
- val config = TEST_UTIL.getConfiguration
- val rdd = sc.parallelize(Array(
- (Bytes.toBytes("1foreach"),
- Array((Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1")))),
- (Bytes.toBytes("2foreach"),
- Array((Bytes.toBytes(columnFamily), Bytes.toBytes("b"), Bytes.toBytes("foo2")))),
- (Bytes.toBytes("3foreach"),
- Array((Bytes.toBytes(columnFamily), Bytes.toBytes("c"), Bytes.toBytes("foo3")))),
- (Bytes.toBytes("4foreach"),
- Array((Bytes.toBytes(columnFamily), Bytes.toBytes("d"), Bytes.toBytes("foo")))),
- (Bytes.toBytes("5foreach"),
- Array((Bytes.toBytes(columnFamily), Bytes.toBytes("e"), Bytes.toBytes("bar"))))))
-
- val hbaseContext = new HBaseContext(sc, config)
-
- rdd.hbaseForeachPartition(hbaseContext, (it, conn) => {
- val bufferedMutator = conn.getBufferedMutator(TableName.valueOf("t1"))
- it.foreach((putRecord) => {
- val put = new Put(putRecord._1)
- putRecord._2.foreach((putValue) => put.addColumn(putValue._1, putValue._2, putValue._3))
- bufferedMutator.mutate(put)
- })
- bufferedMutator.flush()
- bufferedMutator.close()
- })
-
- val connection = ConnectionFactory.createConnection(config)
- val table = connection.getTable(TableName.valueOf("t1"))
-
- try {
- val foo1 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("1foreach"))).
- getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a"))))
- assert(foo1 == "foo1")
-
- val foo2 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("2foreach"))).
- getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("b"))))
- assert(foo2 == "foo2")
-
- val foo3 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("3foreach"))).
- getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("c"))))
- assert(foo3 == "foo3")
-
- val foo4 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("4foreach"))).
- getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("d"))))
- assert(foo4 == "foo")
-
- val foo5 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("5"))).
- getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("e"))))
- assert(foo5 == "bar")
- } finally {
- table.close()
- connection.close()
- }
- }
-
- test("mapPartitions with Get from test HBase client") {
- val config = TEST_UTIL.getConfiguration
- val connection = ConnectionFactory.createConnection(config)
- val table = connection.getTable(TableName.valueOf("t1"))
-
- try {
- var put = new Put(Bytes.toBytes("get1"))
- put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1"))
- table.put(put)
- put = new Put(Bytes.toBytes("get2"))
- put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo2"))
- table.put(put)
- put = new Put(Bytes.toBytes("get3"))
- put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo3"))
- table.put(put)
- } finally {
- table.close()
- connection.close()
- }
-
- val rdd = sc.parallelize(Array(
- Bytes.toBytes("get1"),
- Bytes.toBytes("get2"),
- Bytes.toBytes("get3"),
- Bytes.toBytes("get4")))
- val hbaseContext = new HBaseContext(sc, config)
-
- //Get with custom convert logic
- val getRdd = rdd.hbaseMapPartitions(hbaseContext, (it, conn) => {
- val table = conn.getTable(TableName.valueOf("t1"))
- var res = mutable.MutableList[String]()
-
- it.foreach(r => {
- val get = new Get(r)
- val result = table.get(get)
- if (result.listCells != null) {
- val it = result.listCells().iterator()
- val B = new StringBuilder
-
- B.append(Bytes.toString(result.getRow) + ":")
-
- while (it.hasNext) {
- val cell = it.next()
- val q = Bytes.toString(CellUtil.cloneQualifier(cell))
- if (q.equals("counter")) {
- B.append("(" + q + "," + Bytes.toLong(CellUtil.cloneValue(cell)) + ")")
- } else {
- B.append("(" + q + "," + Bytes.toString(CellUtil.cloneValue(cell)) + ")")
- }
- }
- res += "" + B.toString
- } else {
- res += ""
- }
- })
- res.iterator
- })
-
- val getArray = getRdd.collect()
-
- assert(getArray.length == 4)
- assert(getArray.contains("get1:(a,foo1)"))
- assert(getArray.contains("get2:(a,foo2)"))
- assert(getArray.contains("get3:(a,foo3)"))
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/516d370b/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseTestSource.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseTestSource.scala b/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseTestSource.scala
deleted file mode 100644
index ccb4625..0000000
--- a/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseTestSource.scala
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.spark
-
-import org.apache.hadoop.hbase.spark.datasources.HBaseSparkConf
-import org.apache.spark.SparkEnv
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.{Row, SQLContext}
-import org.apache.spark.sql.sources._
-import org.apache.spark.sql.types._
-
-class HBaseTestSource extends RelationProvider {
- override def createRelation(
- sqlContext: SQLContext,
- parameters: Map[String, String]): BaseRelation = {
- DummyScan(
- parameters("cacheSize").toInt,
- parameters("batchNum").toInt,
- parameters("blockCacheingEnable").toBoolean,
- parameters("rowNum").toInt)(sqlContext)
- }
-}
-
-case class DummyScan(
- cacheSize: Int,
- batchNum: Int,
- blockCachingEnable: Boolean,
- rowNum: Int)(@transient val sqlContext: SQLContext)
- extends BaseRelation with TableScan {
- private def sparkConf = SparkEnv.get.conf
- override def schema: StructType =
- StructType(StructField("i", IntegerType, nullable = false) :: Nil)
-
- override def buildScan(): RDD[Row] = sqlContext.sparkContext.parallelize(0 until rowNum)
- .map(Row(_))
- .map{ x =>
- if (sparkConf.getInt(HBaseSparkConf.QUERY_BATCHSIZE,
- -1) != batchNum ||
- sparkConf.getInt(HBaseSparkConf.QUERY_CACHEDROWS,
- -1) != cacheSize ||
- sparkConf.getBoolean(HBaseSparkConf.QUERY_CACHEBLOCKS,
- false) != blockCachingEnable) {
- throw new Exception("HBase Spark configuration cannot be set properly")
- }
- x
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/516d370b/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/PartitionFilterSuite.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/PartitionFilterSuite.scala b/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/PartitionFilterSuite.scala
deleted file mode 100644
index f47a319..0000000
--- a/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/PartitionFilterSuite.scala
+++ /dev/null
@@ -1,523 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.spark
-
-import org.apache.hadoop.hbase.spark.datasources.HBaseSparkConf
-import org.apache.hadoop.hbase.{TableName, HBaseTestingUtility}
-import org.apache.spark.sql.datasources.hbase.HBaseTableCatalog
-import org.apache.spark.sql.{DataFrame, SQLContext}
-import org.apache.spark.{SparkConf, SparkContext, Logging}
-import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
-
-case class FilterRangeRecord(
- intCol0: Int,
- boolCol1: Boolean,
- doubleCol2: Double,
- floatCol3: Float,
- intCol4: Int,
- longCol5: Long,
- shortCol6: Short,
- stringCol7: String,
- byteCol8: Byte)
-
-object FilterRangeRecord {
- def apply(i: Int): FilterRangeRecord = {
- FilterRangeRecord(if (i % 2 == 0) i else -i,
- i % 2 == 0,
- if (i % 2 == 0) i.toDouble else -i.toDouble,
- i.toFloat,
- if (i % 2 == 0) i else -i,
- i.toLong,
- i.toShort,
- s"String$i extra",
- i.toByte)
- }
-}
-
-class PartitionFilterSuite extends FunSuite with
- BeforeAndAfterEach with BeforeAndAfterAll with Logging {
- @transient var sc: SparkContext = null
- var TEST_UTIL: HBaseTestingUtility = new HBaseTestingUtility
-
- var sqlContext: SQLContext = null
- var df: DataFrame = null
-
- def withCatalog(cat: String): DataFrame = {
- sqlContext
- .read
- .options(Map(HBaseTableCatalog.tableCatalog -> cat))
- .format("org.apache.hadoop.hbase.spark")
- .load()
- }
-
- override def beforeAll() {
-
- TEST_UTIL.startMiniCluster
- val sparkConf = new SparkConf
- sparkConf.set(HBaseSparkConf.QUERY_CACHEBLOCKS, "true")
- sparkConf.set(HBaseSparkConf.QUERY_BATCHSIZE, "100")
- sparkConf.set(HBaseSparkConf.QUERY_CACHEDROWS, "100")
-
- sc = new SparkContext("local", "test", sparkConf)
- new HBaseContext(sc, TEST_UTIL.getConfiguration)
- sqlContext = new SQLContext(sc)
- }
-
- override def afterAll() {
- logInfo("shutting down minicluster")
- TEST_UTIL.shutdownMiniCluster()
-
- sc.stop()
- }
-
- override def beforeEach(): Unit = {
- DefaultSourceStaticUtils.lastFiveExecutionRules.clear()
- }
-
- // The original raw data used for construct result set without going through
- // data frame logic. It is used to verify the result set retrieved from data frame logic.
- val rawResult = (0 until 32).map { i =>
- FilterRangeRecord(i)
- }
-
- def collectToSet[T](df: DataFrame): Set[T] = {
- df.collect().map(_.getAs[T](0)).toSet
- }
- val catalog = s"""{
- |"table":{"namespace":"default", "name":"rangeTable"},
- |"rowkey":"key",
- |"columns":{
- |"intCol0":{"cf":"rowkey", "col":"key", "type":"int"},
- |"boolCol1":{"cf":"cf1", "col":"boolCol1", "type":"boolean"},
- |"doubleCol2":{"cf":"cf2", "col":"doubleCol2", "type":"double"},
- |"floatCol3":{"cf":"cf3", "col":"floatCol3", "type":"float"},
- |"intCol4":{"cf":"cf4", "col":"intCol4", "type":"int"},
- |"longCol5":{"cf":"cf5", "col":"longCol5", "type":"bigint"},
- |"shortCol6":{"cf":"cf6", "col":"shortCol6", "type":"smallint"},
- |"stringCol7":{"cf":"cf7", "col":"stringCol7", "type":"string"},
- |"byteCol8":{"cf":"cf8", "col":"byteCol8", "type":"tinyint"}
- |}
- |}""".stripMargin
-
- test("populate rangeTable") {
- val sql = sqlContext
- import sql.implicits._
-
- sc.parallelize(rawResult).toDF.write.options(
- Map(HBaseTableCatalog.tableCatalog -> catalog, HBaseTableCatalog.newTable -> "5"))
- .format("org.apache.hadoop.hbase.spark")
- .save()
- }
- test("rangeTable full query") {
- val df = withCatalog(catalog)
- df.show
- assert(df.count() === 32)
- }
-
- /**
- *expected result: only showing top 20 rows
- *+-------+
- *|intCol0|
- *+-------+
- *| -31 |
- *| -29 |
- *| -27 |
- *| -25 |
- *| -23 |
- *| -21 |
- *| -19 |
- *| -17 |
- *| -15 |
- *| -13 |
- *| -11 |
- *| -9 |
- *| -7 |
- *| -5 |
- *| -3 |
- *| -1 |
- *+---- +
- */
- test("rangeTable rowkey less than 0") {
- val sql = sqlContext
- import sql.implicits._
- val df = withCatalog(catalog)
- val s = df.filter($"intCol0" < 0).select($"intCol0")
- s.show
- // filter results without going through dataframe
- val expected = rawResult.filter(_.intCol0 < 0).map(_.intCol0).toSet
- // filter results going through dataframe
- val result = collectToSet[Int](s)
- assert(expected === result)
- }
-
- /**
- *expected result: only showing top 20 rows
- *+-------+
- *|intCol4|
- *+-------+
- *| -31 |
- *| -29 |
- *| -27 |
- *| -25 |
- *| -23 |
- *| -21 |
- *| -19 |
- *| -17 |
- *| -15 |
- *| -13 |
- *| -11 |
- *| -9 |
- *| -7 |
- *| -5 |
- *| -3 |
- *| -1 |
- *+-------+
- */
- test("rangeTable int col less than 0") {
- val sql = sqlContext
- import sql.implicits._
- val df = withCatalog(catalog)
- val s = df.filter($"intCol4" < 0).select($"intCol4")
- s.show
- // filter results without going through dataframe
- val expected = rawResult.filter(_.intCol4 < 0).map(_.intCol4).toSet
- // filter results going through dataframe
- val result = collectToSet[Int](s)
- assert(expected === result)
- }
-
- /**
- *expected result: only showing top 20 rows
- *+-----------+
- *| doubleCol2|
- *+-----------+
- *| 0.0 |
- *| 2.0 |
- *|-31.0 |
- *|-29.0 |
- *|-27.0 |
- *|-25.0 |
- *|-23.0 |
- *|-21.0 |
- *|-19.0 |
- *|-17.0 |
- *|-15.0 |
- *|-13.0 |
- *|-11.0 |
- *| -9.0 |
- *| -7.0 |
- *| -5.0 |
- *| -3.0 |
- *| -1.0 |
- *+-----------+
- */
- test("rangeTable double col less than 0") {
- val sql = sqlContext
- import sql.implicits._
- val df = withCatalog(catalog)
- val s = df.filter($"doubleCol2" < 3.0).select($"doubleCol2")
- s.show
- // filter results without going through dataframe
- val expected = rawResult.filter(_.doubleCol2 < 3.0).map(_.doubleCol2).toSet
- // filter results going through dataframe
- val result = collectToSet[Double](s)
- assert(expected === result)
- }
-
- /**
- * expected result: only showing top 20 rows
- *+-------+
- *|intCol0|
- *+-------+
- *| -31 |
- *| -29 |
- *| -27 |
- *| -25 |
- *| -23 |
- *| -21 |
- *| -19 |
- *| -17 |
- *| -15 |
- *| -13 |
- *| -11 |
- *+-------+
- *
- */
- test("rangeTable lessequal than -10") {
- val sql = sqlContext
- import sql.implicits._
- val df = withCatalog(catalog)
- val s = df.filter($"intCol0" <= -10).select($"intCol0")
- s.show
- // filter results without going through dataframe
- val expected = rawResult.filter(_.intCol0 <= -10).map(_.intCol0).toSet
- // filter results going through dataframe
- val result = collectToSet[Int](s)
- assert(expected === result)
- }
-
- /**
- *expected result: only showing top 20 rows
- *+-------+
- *|intCol0|
- *+----+
- *| -31 |
- *| -29 |
- *| -27 |
- *| -25 |
- *| -23 |
- *| -21 |
- *| -19 |
- *| -17 |
- *| -15 |
- *| -13 |
- *| -11 |
- *| -9 |
- *+-------+
- */
- test("rangeTable lessequal than -9") {
- val sql = sqlContext
- import sql.implicits._
- val df = withCatalog(catalog)
- val s = df.filter($"intCol0" <= -9).select($"intCol0")
- s.show
- // filter results without going through dataframe
- val expected = rawResult.filter(_.intCol0 <= -9).map(_.intCol0).toSet
- // filter results going through dataframe
- val result = collectToSet[Int](s)
- assert(expected === result)
- }
-
- /**
- *expected result: only showing top 20 rows
- *+-------+
- *|intCol0|
- *+-------+
- *| 0 |
- *| 2 |
- *| 4 |
- *| 6 |
- *| 8 |
- *| 10 |
- *| 12 |
- *| 14 |
- *| 16 |
- *| 18 |
- *| 20 |
- *| 22 |
- *| 24 |
- *| 26 |
- *| 28 |
- *| 30 |
- *| -9 |
- *| -7 |
- *| -5 |
- *| -3 |
- *+-------+
- */
- test("rangeTable greaterequal than -9") {
- val sql = sqlContext
- import sql.implicits._
- val df = withCatalog(catalog)
- val s = df.filter($"intCol0" >= -9).select($"intCol0")
- s.show
- // filter results without going through dataframe
- val expected = rawResult.filter(_.intCol0 >= -9).map(_.intCol0).toSet
- // filter results going through dataframe
- val result = collectToSet[Int](s)
- assert(expected === result)
- }
-
- /**
- *expected result: only showing top 20 rows
- *+-------+
- *|intCol0|
- *+-------+
- *| 0 |
- *| 2 |
- *| 4 |
- *| 6 |
- *| 8 |
- *| 10 |
- *| 12 |
- *| 14 |
- *| 16 |
- *| 18 |
- *| 20 |
- *| 22 |
- *| 24 |
- *| 26 |
- *| 28 |
- *| 30 |
- *+-------+
- */
- test("rangeTable greaterequal than 0") {
- val sql = sqlContext
- import sql.implicits._
- val df = withCatalog(catalog)
- val s = df.filter($"intCol0" >= 0).select($"intCol0")
- s.show
- // filter results without going through dataframe
- val expected = rawResult.filter(_.intCol0 >= 0).map(_.intCol0).toSet
- // filter results going through dataframe
- val result = collectToSet[Int](s)
- assert(expected === result)
- }
-
- /**
- *expected result: only showing top 20 rows
- *+-------+
- *|intCol0|
- *+-------+
- *| 12 |
- *| 14 |
- *| 16 |
- *| 18 |
- *| 20 |
- *| 22 |
- *| 24 |
- *| 26 |
- *| 28 |
- *| 30 |
- *+-------+
- */
- test("rangeTable greater than 10") {
- val sql = sqlContext
- import sql.implicits._
- val df = withCatalog(catalog)
- val s = df.filter($"intCol0" > 10).select($"intCol0")
- s.show
- // filter results without going through dataframe
- val expected = rawResult.filter(_.intCol0 > 10).map(_.intCol0).toSet
- // filter results going through dataframe
- val result = collectToSet[Int](s)
- assert(expected === result)
- }
-
- /**
- *expected result: only showing top 20 rows
- *+-------+
- *|intCol0|
- *+-------+
- *| 0 |
- *| 2 |
- *| 4 |
- *| 6 |
- *| 8 |
- *| 10 |
- *| -9 |
- *| -7 |
- *| -5 |
- *| -3 |
- *| -1 |
- *+-------+
- */
- test("rangeTable and") {
- val sql = sqlContext
- import sql.implicits._
- val df = withCatalog(catalog)
- val s = df.filter($"intCol0" > -10 && $"intCol0" <= 10).select($"intCol0")
- s.show
- // filter results without going through dataframe
- val expected = rawResult.filter(x => x.intCol0 > -10 && x.intCol0 <= 10 ).map(_.intCol0).toSet
- // filter results going through dataframe
- val result = collectToSet[Int](s)
- assert(expected === result)
- }
-
- /**
- *expected result: only showing top 20 rows
- *+-------+
- *|intCol0|
- *+-------+
- *| 12 |
- *| 14 |
- *| 16 |
- *| 18 |
- *| 20 |
- *| 22 |
- *| 24 |
- *| 26 |
- *| 28 |
- *| 30 |
- *| -31 |
- *| -29 |
- *| -27 |
- *| -25 |
- *| -23 |
- *| -21 |
- *| -19 |
- *| -17 |
- *| -15 |
- *| -13 |
- *+-------+
- */
-
- test("or") {
- val sql = sqlContext
- import sql.implicits._
- val df = withCatalog(catalog)
- val s = df.filter($"intCol0" <= -10 || $"intCol0" > 10).select($"intCol0")
- s.show
- // filter results without going through dataframe
- val expected = rawResult.filter(x => x.intCol0 <= -10 || x.intCol0 > 10).map(_.intCol0).toSet
- // filter results going through dataframe
- val result = collectToSet[Int](s)
- assert(expected === result)
- }
-
- /**
- *expected result: only showing top 20 rows
- *+-------+
- *|intCol0|
- *+-------+
- *| 0 |
- *| 2 |
- *| 4 |
- *| 6 |
- *| 8 |
- *| 10 |
- *| 12 |
- *| 14 |
- *| 16 |
- *| 18 |
- *| 20 |
- *| 22 |
- *| 24 |
- *| 26 |
- *| 28 |
- *| 30 |
- *| -31 |
- *| -29 |
- *| -27 |
- *| -25 |
- *+-------+
- */
- test("rangeTable all") {
- val sql = sqlContext
- import sql.implicits._
- val df = withCatalog(catalog)
- val s = df.filter($"intCol0" >= -100).select($"intCol0")
- s.show
- // filter results without going through dataframe
- val expected = rawResult.filter(_.intCol0 >= -100).map(_.intCol0).toSet
- // filter results going through dataframe
- val result = collectToSet[Int](s)
- assert(expected === result)
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/516d370b/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e4a040f..a64d56c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -85,11 +85,9 @@
<module>hbase-checkstyle</module>
<module>hbase-external-blockcache</module>
<module>hbase-shaded</module>
- <module>hbase-spark</module>
<module>hbase-archetypes</module>
<module>hbase-metrics-api</module>
<module>hbase-metrics</module>
- <module>hbase-spark-it</module>
<module>hbase-backup</module>
</modules>
<!--Add apache snapshots in case we want to use unreleased versions of plugins:
@@ -998,38 +996,6 @@
</configuration>
</execution>
<execution>
- <id>banned-scala</id>
- <goals>
- <goal>enforce</goal>
- </goals>
- <configuration>
- <rules>
- <bannedDependencies>
- <excludes>
- <exclude>org.scala-lang:scala-library</exclude>
- </excludes>
- <message>We don't allow Scala outside of the hbase-spark module, see HBASE-13992.</message>
- </bannedDependencies>
- </rules>
- </configuration>
- </execution>
- <execution>
- <id>banned-hbase-spark</id>
- <goals>
- <goal>enforce</goal>
- </goals>
- <configuration>
- <rules>
- <bannedDependencies>
- <excludes>
- <exclude>org.apache.hbase:hbase-spark</exclude>
- </excludes>
- <message>We don't allow other modules to depend on hbase-spark, see HBASE-13992.</message>
- </bannedDependencies>
- </rules>
- </configuration>
- </execution>
- <execution>
<id>check-aggregate-license</id>
<!-- must check after LICENSE is built at 'generate-resources' -->
<phase>process-resources</phase>
@@ -3305,7 +3271,7 @@
<exclude>**/protobuf/*</exclude>
<exclude>**/*.scala</exclude>
</sourceFileExcludes>
- <excludePackageNames>org.apache.hadoop.hbase.tmpl.common:com.google.protobuf:org.apache.hadoop.hbase.spark:org.apache.hadoop.hbase.generated*</excludePackageNames>
+ <excludePackageNames>org.apache.hadoop.hbase.tmpl.common:com.google.protobuf:org.apache.hadoop.hbase.generated*</excludePackageNames>
<show>private</show> <!-- (shows all classes and members) -->
<quiet>true</quiet>
<linksource>true</linksource>
@@ -3348,7 +3314,7 @@
<exclude>**/protobuf/*</exclude>
<exclude>**/*.scala</exclude>
</sourceFileExcludes>
- <excludePackageNames>org.apache.hadoop.hbase.tmpl.common:com.google.protobuf:org.apache.hadoop.hbase.spark:org.apache.hadoop.hbase.generated*</excludePackageNames>
+ <excludePackageNames>org.apache.hadoop.hbase.tmpl.common:com.google.protobuf:org.apache.hadoop.hbase.generated*</excludePackageNames>
<show>private</show> <!-- (shows all classes and members) -->
<quiet>true</quiet>
<linksource>true</linksource>