├── .gitignore ├── LICENSE ├── README.md ├── bin ├── hbase-sql └── pyspark-hbase ├── conf └── hspark.properties ├── doc └── SparkSQLOnHBase_v2.2.docx ├── examples ├── README.md ├── sales1m.csv.tar.gz ├── spark-hbase-demo1.md ├── spark-hbase-demo2.md ├── spark-hbase-demo3.md └── teacher1k.csv ├── pom.xml ├── python └── pyspark_hbase │ ├── __init__.py │ ├── shell.py │ └── sql │ ├── __init__.py │ └── context.py └── src ├── main ├── java │ └── org │ │ └── apache │ │ └── spark │ │ └── sql │ │ └── hbase │ │ ├── CheckDirProtos.java │ │ └── util │ │ └── AdditionalComparatorsProtos.java ├── protobuf │ ├── AdditionalComparators.proto │ └── CheckDirService.proto └── scala │ └── org │ └── apache │ └── spark │ └── sql │ ├── catalyst │ ├── NotPusher.scala │ └── expressions │ │ └── PartialPredicateOperations.scala │ ├── hbase │ ├── CheckDirEndPointImpl.scala │ ├── HBaseCatalog.scala │ ├── HBaseCriticalPoint.scala │ ├── HBaseCustomFilter.scala │ ├── HBasePartition.scala │ ├── HBasePartitioner.scala │ ├── HBaseRelation.scala │ ├── HBaseSQLCliDriver.scala │ ├── HBaseSQLConf.scala │ ├── HBaseSQLReaderRDD.scala │ ├── HBaseSerializer.scala │ ├── HBaseShuffledRDD.scala │ ├── HBaseSparkSession.scala │ ├── HadoopReader.scala │ ├── IndexMappable.scala │ ├── ScanPredClassifier.scala │ ├── SparkSqlRegionObserver.scala │ ├── execution │ │ ├── HBaseSQLTableScan.scala │ │ ├── HBaseStrategies.scala │ │ └── hbaseCommands.scala │ ├── package.scala │ ├── types │ │ └── HBaseBytesType.scala │ └── util │ │ ├── DataTypeUtils.scala │ │ ├── HBaseKVHelper.scala │ │ ├── Util.scala │ │ ├── bytesUtils.scala │ │ └── comparators.scala │ └── types │ ├── PartialOrderingDataType.scala │ └── RangeType.scala └── test ├── java └── org │ └── apache │ └── spark │ └── sql │ └── hbase │ └── api │ └── java │ └── JavaAPISuite.java ├── resources ├── 131_regions.txt ├── cf.txt ├── joinTable1.txt ├── joinTable2.txt ├── joinTable3.txt ├── joinTable4.txt ├── loadData.txt ├── loadNullableData.txt ├── log4j.properties ├── onecoljoin1.txt ├── onecoljoin2.txt ├── people.txt ├── splitLoadData.txt ├── splitLoadData1.txt ├── store_sales.txt ├── store_sales_stringformat.txt ├── teacher.txt ├── testTable.txt └── users.parquet │ ├── .part-r-00001.gz.parquet.crc │ ├── .part-r-00002.gz.parquet.crc │ ├── _SUCCESS │ ├── _common_metadata │ ├── _metadata │ ├── part-r-00001.gz.parquet │ └── part-r-00002.gz.parquet └── scala └── org └── apache └── spark └── sql └── hbase ├── BytesUtilsSuite.scala ├── CriticalPointsTestSuite.scala ├── HBaseAdditionalQuerySuite.scala ├── HBaseAdvancedSQLQuerySuite.scala ├── HBaseAggregateQueriesSuite.scala ├── HBaseBasicOperationSuite.scala ├── HBaseBasicQueriesSuite.scala ├── HBaseBulkLoadIntoTableSuite.scala ├── HBaseCFTestSuite.scala ├── HBaseCatalogTestSuite.scala ├── HBaseInsertTableSuite.scala ├── HBasePartitionerSuite.scala ├── HBaseSQLQuerySuite.scala ├── HBaseTpcMiniTestSuite.scala ├── HBaseTpcStringFormatMiniTestSuite.scala ├── TestBase.scala ├── TestBaseWithNonSplitData.scala ├── TestBaseWithSplitData.scala ├── TestData.scala └── TestHbase.scala /.gitignore: -------------------------------------------------------------------------------- 1 | *.iml 2 | .idea/ 3 | target/ 4 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | ## Heracles: Fast SQL on HBase using SparkSQL 2 | 3 | **Note:** The original project name is "`HSpark`" which is requested to rename to the current name due to trade mark concerns by Apache foundation. 4 | 5 | Apache HBase is a distributed Key-Value store of data on HDFS. It is modeled after Google’s Big Table, and provides APIs to query the data. The data is organized, partitioned and distributed by its “row keys”. Per partition, the data is further physically partitioned by “column families” that specify collections of “columns” of data. The data model is for wide and sparse tables where columns are dynamic and may well be sparse. 6 | 7 | Although HBase is a very useful big data store, its access mechanism is very primitive and only through client-side APIs, Map/Reduce interfaces and interactive shells. SQL accesses to HBase data are available through Map/Reduce or interfaces mechanisms such as Apache Hive and Impala, or some “native” SQL technologies like Apache Phoenix. While the former is usually cheaper to implement and use, their latencies and efficiencies often cannot compare favorably with the latter and are often suitable only for offline analysis. The latter category, in contrast, often performs better and qualifies more as online engines; they are often on top of purpose-built execution engines. 8 | 9 | Currently Spark supports queries against HBase data through HBase’s Map/Reduce interface (i.e., TableInputFormat). Spark SQL supports use of Hive data, which theoretically should be able to support HBase data access, out-of-box, through HBase’s Map/Reduce interface and therefore falls into the first category of the “SQL on HBase” technologies. 10 | 11 | We believe, as a unified big data processing engine, Spark is in good position to provide better HBase support. 12 | 13 | ## Online Documentation 14 | 15 | Online documentation is in the doc folder. 16 | 17 | ## Requirements 18 | 19 | This version of 2.2.0 requires Spark 2.2.0. 20 | 21 | ## Building Spark HBase 22 | 23 | Spark HBase is built using [Apache Maven](http://maven.apache.org/). 24 | 25 | I. Clone the Heracles project from GitHub 26 | ``` 27 | $ git clone https://github.com/bomeng/Heracles.git 28 | ``` 29 | or 30 | ``` 31 | $ git clone git@github.com:bomeng/Heracles.git 32 | ``` 33 | 34 | II. Go to the root of the source tree 35 | ``` 36 | $ cd Heracles 37 | ``` 38 | 39 | III. Build the project 40 | Build without testing 41 | ``` 42 | $ mvn -DskipTests clean install 43 | ``` 44 | Or, build with testing. It will run test suites against a HBase minicluster. 45 | ``` 46 | $ mvn clean install 47 | ``` 48 | 49 | ## Coprocessor 50 | 51 | Currently, HBase coprocessor is not supported in this release. 52 | 53 | 54 | ## Interactive Scala Shell 55 | 56 | The shell will connect to a local HBase master. You need to configure the HBase's hbase-env.sh file under "conf" folder by adding hspark.jar to its classpath. 57 | 58 | ``` 59 | export HBASE_CLASSPATH=/hspark-2.2.0.jar 60 | ``` 61 | 62 | You may need to set the JAVA_HOME in the hbase-env.sh as well. Follow the instruction of configuring HBase to its proper settings (e.g. hbase-site.xml etc), after that, you can start HBase by following command: 63 | 64 | ``` 65 | start-hbase.sh 66 | ``` 67 | 68 | Then, the easiest way to start using Spark HBase is through the Scala shell: 69 | ``` 70 | ./bin/hbase-sql 71 | ``` 72 | 73 | ## Python Shell 74 | 75 | First, add the spark-hbase jar to the SPARK_CLASSPATH in the $SPARK_HOME/conf directory, as follows: 76 | ``` 77 | SPARK_CLASSPATH=$SPARK_CLASSPATH:/spark-hbase-root-dir/target/Heracles-2.2.0.jar 78 | ``` 79 | Then go to the spark-hbase installation directory and issue 80 | ``` 81 | ./bin/pyspark-hbase 82 | ``` 83 | A successful message is as follows: 84 | ``` 85 | You are using Heracles !!! 86 | HBaseSQLContext available as hsqlContext. 87 | ``` 88 | To run a python script, the PYTHONPATH environment should be set to the "python" directory of the Spark-HBase installation. For example, 89 | ``` 90 | export PYTHONPATH=/root-of-Heracles/python 91 | ``` 92 | 93 | Note that the shell commands are not included in the Zip file of the Spark release. They are for developers' use only for this version of 2.2.0. Instead, users can use "$SPARK_HOME/bin/spark-shell --packages Heracles/Heracles:2.2.0" for SQL shell or "$SPARK_HOME/bin/pyspark --packages Heracles/Heracles:2.2.0" for Pythin shell. 94 | 95 | ## Running Tests 96 | 97 | Testing first requires [building Spark HBase](#building-spark-hbase). Once Spark HBase is built ... 98 | 99 | Run all test suites from Maven: 100 | ``` 101 | mvn -Phbase,hadoop-2.4 test 102 | ``` 103 | Run a single test suite from Maven, for example: 104 | ``` 105 | mvn -Phbase,hadoop-2.4 test -DwildcardSuites=org.apache.spark.sql.hbase.BasicQueriesSuite 106 | ``` 107 | ## IDE Setup 108 | 109 | We use IntelliJ IDEA for Spark HBase development. You can get the community edition for free and install the JetBrains Scala plugin from Preferences > Plugins. 110 | 111 | To import the current Spark HBase project for IntelliJ: 112 | 113 | 1. Download IntelliJ and install the Scala plug-in for IntelliJ. You may also need to install Maven plug-in for IntelliJ. 114 | 2. Go to "File -> Import Project", locate the Spark HBase source directory, and select "Maven Project". 115 | 3. In the Import Wizard, select "Import Maven projects automatically" and leave other settings at their default. 116 | 4. Make sure some specific profiles are enabled. Select corresponding Hadoop version, "maven3" and also "hbase" in order to get dependencies. 117 | 5. Leave other settings at their default and you should be able to start your development. 118 | 6. When you run the scala test, sometimes you will get out of memory exception. You can increase your VM memory usage by the following setting, for example: 119 | 120 | ``` 121 | -XX:MaxPermSize=512m -Xmx3072m 122 | ``` 123 | 124 | You can also make those setting to be the default by setting to the "Defaults -> ScalaTest". 125 | 126 | ## Configuration 127 | 128 | Please refer to the [Configuration guide](http://spark.apache.org/docs/latest/configuration.html) 129 | in the online documentation for an overview on how to configure Spark. 130 | 131 | For HBase 1.2, it is recommended to use higher "open files" and "max user processes" ulimit values. A typical value is 65536(64K). 132 | -------------------------------------------------------------------------------- /bin/hbase-sql: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one or more 5 | # contributor license agreements. See the NOTICE file distributed with 6 | # this work for additional information regarding copyright ownership. 7 | # The ASF licenses this file to You under the Apache License, Version 2.0 8 | # (the "License"); you may not use this file except in compliance with 9 | # the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | # 19 | 20 | # 21 | # Shell script for starting the Spark SQL for HBase CLI 22 | 23 | # Enter posix mode for bash 24 | set -o posix 25 | 26 | export CLASS="org.apache.spark.sql.hbase.HBaseSQLCliDriver" 27 | 28 | # Figure out where Spark is installed 29 | if [ -z "$SPARK_HOME" ]; then 30 | echo "\$SPARK_HOME is not set" 31 | fi 32 | 33 | export FWDIR=$SPARK_HOME 34 | export HSPARK_SOURCE="$(cd "`dirname "$0"`"/..; pwd)" 35 | export JAR="$HSPARK_SOURCE/target/hspark-2.2.0.jar" 36 | 37 | function usage { 38 | if [ -n "$1" ]; then 39 | echo "$1" 40 | fi 41 | echo "Usage: ./bin/hbase-sql [options] [cli option]" 42 | pattern="usage" 43 | pattern+="\|Spark assembly has been built with hbase" 44 | pattern+="\|NOTE: SPARK_PREPEND_CLASSES is set" 45 | pattern+="\|Spark Command: " 46 | pattern+="\|--help" 47 | pattern+="\|=======" 48 | 49 | "$FWDIR"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 50 | echo 51 | echo "CLI options:" 52 | "$FWDIR"/bin/spark-class "$CLASS" --help 2>&1 | grep -v "$pattern" 1>&2 53 | exit "$2" 54 | } 55 | export -f usage 56 | 57 | if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then 58 | usage "" 0 59 | fi 60 | 61 | exec "${SPARK_HOME}"/bin/spark-submit --class "$CLASS" "$JAR" "$@" 62 | -------------------------------------------------------------------------------- /bin/pyspark-hbase: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one or more 5 | # contributor license agreements. See the NOTICE file distributed with 6 | # this work for additional information regarding copyright ownership. 7 | # The ASF licenses this file to You under the Apache License, Version 2.0 8 | # (the "License"); you may not use this file except in compliance with 9 | # the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | # 19 | 20 | if [[ -z "$SPARK_HOME" ]]; then 21 | echo "SPARK_HOME is not set!" 1>&2 22 | exit 1 23 | fi 24 | 25 | export HSPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" 26 | export PYTHONPATH="$HSPARK_HOME/python" 27 | export OLD_PYTHONSTARTUP="$PYTHONSTARTUP" 28 | export PYTHONSTARTUP="$HSPARK_HOME/python/pyspark_hbase/shell.py" 29 | 30 | $SPARK_HOME/bin/pyspark 31 | -------------------------------------------------------------------------------- /conf/hspark.properties: -------------------------------------------------------------------------------- 1 | # hdfs, zookeeper, spark configuration 2 | # master=spark://flaky1:7077 3 | # spark.hadoop.hbase.zookeeper.quorum=flaky1,flaky2,flaky3 4 | # spark.hbase.fs.tmp.dir=hdfs://flaky1:9000/tmp 5 | # spark.hadoop.fs.default.name=hdfs://flaky1:9000/ 6 | 7 | master=local[2] 8 | appName=hspark 9 | 10 | spark.hadoop.hbase.zookeeper.quorum=localhost 11 | spark.hbase.fs.tmp.dir=/tmp 12 | spark.hadoop.fs.default.name=file:/// 13 | -------------------------------------------------------------------------------- /doc/SparkSQLOnHBase_v2.2.docx: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/bomeng/Heracles/3a0096b991ef4534989cab37f881e885727a631d/doc/SparkSQLOnHBase_v2.2.docx -------------------------------------------------------------------------------- /examples/README.md: -------------------------------------------------------------------------------- 1 | ### SparkSQL on HBase examples: 2 | 3 | 1. Example 1: Create and query SparkSQL table map to existing Hbase table 4 | 2. Example 2: Create and query SparkSQL table map to a new HBase table 5 | 3. Example 3: Similar to example 1, but with larger sample file 6 | -------------------------------------------------------------------------------- /examples/sales1m.csv.tar.gz: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/bomeng/Heracles/3a0096b991ef4534989cab37f881e885727a631d/examples/sales1m.csv.tar.gz -------------------------------------------------------------------------------- /examples/spark-hbase-demo1.md: -------------------------------------------------------------------------------- 1 | ## Example 1: Create and query SparkSQL table map to existing Hbase table 2 | In this example, we create SparkSQL table and map it to a existing HBase table. (a single column map to hbase rowkey) 3 | 4 | Steps: 5 | 6 | (1) Create table and populate data in HBase shell 7 | ``` 8 | $HBase_Home/bin/hbase shell 9 | create 'hbase_numbers', 'f' 10 | for i in '1'..'100' do for j in '1'..'2' do put 'hbase_numbers', "row#{i}", "f:c#{j}", "#{i}#{j}" end end 11 | ``` 12 | 13 | (2) Map hbase table with sparksql table in hbase-sql shell 14 | ``` 15 | $SPARK_HBASE_Home/bin/hbase-sql 16 | CREATE TABLE numbers(rowkey STRING, a STRING, b STRING, PRIMARY KEY (rowkey)) MAPPED BY (hbase_numbers, COLS=[a=f.c1, b=f.c2]); 17 | ``` 18 | 19 | (3) Query: 20 | ``` 21 | // test count * 22 | (1) select count(*) from numbers 23 | 24 | // test group by 25 | (2) select a, b from numbers where b > "980" 26 | ``` 27 | -------------------------------------------------------------------------------- /examples/spark-hbase-demo2.md: -------------------------------------------------------------------------------- 1 | ## Example 2: Create and query SparkSQL table map to a new HBase table (multiple columns map to hbase table rowkey) 2 | In this example, we create a new SparkSQL table and map it to a new HBase table with multiple column in rowkey. 3 | 4 | Steps: 5 | 6 | (1) Create table in hbase-sql shell: 7 | ``` 8 | $SPARK_HBASE_Home/bin/hbase-sql 9 | CREATE TABLE teacher(grade int, class int, subject string, teacher_name string, teacher_age int, PRIMARY KEY (grade, class, subject)) MAPPED BY (hbase_teacher, COLS=[teacher_name=teacher.name, teacher_age=teacher.age]); 10 | ``` 11 | 12 | This command will create following tables: 13 | Tables : 14 | 15 | spark : teacher 16 | 17 | hbase : hbase_teacher 18 | 19 | Fields : 20 | [grade,int] 21 | [class,int] 22 | [subject,string] 23 | [teacher_name,string] 24 | [teacher_age,int] 25 | 26 | key columns : grade,class,subject 27 | non-key colums: teacher_name, teacher_age 28 | 29 | (2) Load data from a csv data file: 30 | ``` 31 | LOAD DATA INPATH './examples/teacher1k.csv' INTO TABLE teacher FIELDS TERMINATED BY "," ; 32 | ``` 33 | 34 | (3) Query : 35 | ``` 36 | // test where 37 | (1) select teacher_name,teacher_age from teacher where teacher_age > 25; 38 | 39 | // test like in 40 | (2) select teacher_name,teacher_age,subject from teacher where teacher_name is not null and teacher_name like 'teacher_2_3%' and teacher_age not in (20,21,22,23,24,25) 41 | 42 | // test subquery 43 | (3) select t1.teacher_name,t1.teacher_age from (select * from teacher where teacher_name like 'teacher_2_3%') t1 where t1.teacher_age < 25 44 | 45 | //test group by 46 | (4) select teacher_name, sum(teacher_age) from teacher where grade=1 group by teacher_name 47 | 48 | //test join 49 | (5) select t1.teacher_name, t2.subject, t1.teacher_age from (select teacher_name, teacher_age from teacher where teacher_age >=26 ) t1 join (select teacher_name, subject from teacher where teacher_name like 'teacher_2_3%')t2 on t1.teacher_name=t2.teacher_name 50 | ``` 51 | -------------------------------------------------------------------------------- /examples/spark-hbase-demo3.md: -------------------------------------------------------------------------------- 1 | ## Example 3: Similar to example 2, but with larger sample file 2 | In this example, we create a new SparkSQL table and map it to a new HBase table with multiple column in rowkey. 3 | 4 | (1) Create table in SparkSQL and in HBase 5 | ``` 6 | $SPARK_HBASE_HOME/bin/hbase-sql 7 | CREATE TABLE sales1m(id STRING, product STRING, region STRING, sales INTEGER, quantity INTEGER, PRIMARY KEY (id, product, region)) MAPPED BY (hbase_sales1m, COLS=[sales=f.sales, quantity=f.quantity]); 8 | CREATE TABLE sales1m_onekey(id STRING, product STRING, region STRING, sales INTEGER, quantity INTEGER, PRIMARY KEY (id)) MAPPED BY (hbase_sales1m_onekey, COLS=[product=f.product, region=f.region, sales=f.sales, quantity=f.quantity]); 9 | ``` 10 | 11 | (2) Load data : 12 | ``` 13 | LOAD DATA INPATH './examples/sales1m.csv' INTO TABLE sales1m FIELDS TERMINATED BY "," ; 14 | LOAD DATA INPATH './examples/sales1m.csv' INTO TABLE sales1m_onekey FIELDS TERMINATED BY "," ; 15 | ``` 16 | 17 | (3) Query: 18 | ``` 19 | // test count * 20 | (1) select count(*) from sales1m 21 | 22 | // test group by 23 | (2) select product, region, avg(sales) from sales1m where product="product4" group by product, region; 24 | ``` 25 | -------------------------------------------------------------------------------- /python/pyspark_hbase/__init__.py: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # 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 | # for back compatibility 19 | from pyspark_hbase.sql.context import HBaseSQLContext 20 | 21 | __all__ = [ 22 | ] 23 | -------------------------------------------------------------------------------- /python/pyspark_hbase/shell.py: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # 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 | """ 19 | An interactive shell. 20 | 21 | This file is designed to be launched as a PYTHONSTARTUP script. 22 | """ 23 | 24 | import atexit 25 | import os 26 | import platform 27 | 28 | import py4j 29 | 30 | import pyspark 31 | from pyspark_hbase.sql import HBaseSQLContext, context 32 | 33 | print("You are using HSpark !!!") 34 | try: 35 | context.register(sc) 36 | hsqlContext = HBaseSQLContext(sc) 37 | except py4j.protocol.Py4JError: 38 | print("HBaseSQLContext can not be instantiated, falling back to SQLContext now") 39 | hsqlContext = SQLContext(sc) 40 | except TypeError: 41 | print("HBaseSQLContext can not be instantiated, falling back to SQLContext now") 42 | hsqlContext = SQLContext(sc) 43 | 44 | print("%s available as hsqlContext." % hsqlContext.__class__.__name__) 45 | -------------------------------------------------------------------------------- /python/pyspark_hbase/sql/__init__.py: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # 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 | from pyspark_hbase.sql.context import HBaseSQLContext 19 | 20 | __all__ = [ 21 | 'HBaseSQLContext' 22 | ] 23 | -------------------------------------------------------------------------------- /python/pyspark_hbase/sql/context.py: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # 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 | from pyspark.sql import SQLContext 19 | from py4j.java_gateway import java_import 20 | 21 | def register(sc): 22 | java_import(sc._gateway.jvm, "org.apache.spark.sql.hbase.HBaseSQLContext") 23 | 24 | __all__ = ["HBaseSQLContext"] 25 | 26 | class HBaseSQLContext(SQLContext): 27 | """A variant of Spark SQL that integrates with data stored in HBase. 28 | """ 29 | 30 | def __init__(self, sparkContext): 31 | """Create a new HbaseContext. 32 | 33 | @param sparkContext: The SparkContext to wrap. 34 | """ 35 | SQLContext.__init__(self, sparkContext) 36 | self._scala_HBaseSQLContext = self._get_hbase_ctx() 37 | 38 | @property 39 | def _ssql_ctx(self): 40 | if self._scala_HBaseSQLContext is None: 41 | print ("loading hbase context ..") 42 | self._scala_HBaseSQLContext = self._get_hbase_ctx() 43 | if self._scala_SQLContext is None: 44 | self._scala_SQLContext = self._scala_HBaseSQLContext 45 | return self._scala_HBaseSQLContext 46 | 47 | def _get_hbase_ctx(self): 48 | return self._jvm.HBaseSQLContext(self._jsc.sc()) 49 | 50 | #TODO: add tests if for main 51 | -------------------------------------------------------------------------------- /src/main/protobuf/AdditionalComparators.proto: -------------------------------------------------------------------------------- 1 | option java_package = "org.apache.spark.sql.hbase.util"; 2 | option java_outer_classname = "AdditionalComparatorsProtos"; 3 | option java_generic_services = true; 4 | option java_generate_equals_and_hash = true; 5 | option optimize_for = SPEED; 6 | 7 | 8 | message ByteArrayComparable { 9 | optional bytes value = 1; 10 | } 11 | 12 | message CustomComparator { 13 | required ByteArrayComparable comparable = 1; 14 | } 15 | -------------------------------------------------------------------------------- /src/main/protobuf/CheckDirService.proto: -------------------------------------------------------------------------------- 1 | option java_package = "org.apache.spark.sql.hbase"; 2 | option java_outer_classname = "CheckDirProtos"; 3 | option java_generic_services = true; 4 | option java_generate_equals_and_hash = true; 5 | option optimize_for = SPEED; 6 | 7 | message CheckRequest { 8 | } 9 | 10 | message CheckResponse { 11 | required bool accessible = 1; 12 | } 13 | 14 | service CheckDirService { 15 | rpc getCheckResult(CheckRequest) 16 | returns (CheckResponse); 17 | } 18 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/catalyst/NotPusher.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 org.apache.spark.sql.catalyst 19 | 20 | import org.apache.spark.sql.catalyst.expressions._ 21 | import org.apache.spark.sql.catalyst.rules._ 22 | 23 | /** 24 | * Pushes NOT through And/Or 25 | */ 26 | object NotPusher extends Rule[Expression] { 27 | def apply(pred: Expression): Expression = pred transformDown { 28 | case Not(And(left, right)) => Or(Not(left), Not(right)) 29 | case Not(Or(left, right)) => And(Not(left), Not(right)) 30 | case not @ Not(exp) => 31 | // This pattern has been caught by optimizer but after NOT pushdown 32 | // more opportunities may present 33 | exp match { 34 | case GreaterThan(l, r) => LessThanOrEqual(l, r) 35 | case GreaterThanOrEqual(l, r) => LessThan(l, r) 36 | case LessThan(l, r) => GreaterThanOrEqual(l, r) 37 | case LessThanOrEqual(l, r) => GreaterThan(l, r) 38 | case Not(e) => e 39 | case _ => not 40 | } 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/hbase/CheckDirEndPointImpl.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 org.apache.spark.sql.hbase 19 | 20 | import com.google.protobuf.{RpcCallback, RpcController, Service} 21 | import org.apache.hadoop.hbase.coprocessor._ 22 | import org.apache.hadoop.hbase.{Coprocessor, CoprocessorEnvironment} 23 | import org.apache.log4j.Logger 24 | 25 | /*When we enable coprocessor and codegen at the same time, 26 | * if the current directory of any regionservers is not accessible, 27 | * it will crash the HBase regionserver service!!! 28 | * 29 | * This issue normally happens in CDH service. 30 | * Comparing to standalone HBase service, hbase-regionserver, the start script of CDH, 31 | * doesn't include 'cd $HBASE_HOME', which might lead to the result that 32 | * system doesn't regard '.' as a directory. 33 | * 34 | * Thus, we use this endpoint service to check whether 35 | * the current directory is accessible or not in advance. 36 | * If it is not, we will not use codegen. 37 | * 38 | * For CDH service, the better solution is adding 'cd' 39 | * to hbase-regionserver in /etc/init.d 40 | */ 41 | class CheckDirEndPointImpl 42 | extends CheckDirProtos.CheckDirService with Coprocessor with CoprocessorService { 43 | 44 | private lazy val logger = Logger.getLogger(getClass.getName) 45 | 46 | override def start(env: CoprocessorEnvironment) = { 47 | env match { 48 | case _: RegionCoprocessorEnvironment => 49 | case _ => throw new CoprocessorException("Must be loaded on a table region!") 50 | } 51 | } 52 | 53 | override def stop(env: CoprocessorEnvironment) = {} 54 | 55 | override def getService: Service = this 56 | 57 | override def getCheckResult(controller: RpcController, 58 | request: CheckDirProtos.CheckRequest, 59 | done: RpcCallback[CheckDirProtos.CheckResponse]) = { 60 | val isDir = new java.io.File(".").isDirectory 61 | if (!isDir) { 62 | logger.warn( 63 | """Current directory is not accessible, 64 | |please add 'cd ~' before start regionserver in your regionserver start script.""") 65 | } 66 | val response: CheckDirProtos.CheckResponse = { 67 | CheckDirProtos.CheckResponse.newBuilder().setAccessible(isDir).build() 68 | } 69 | done.run(response) 70 | } 71 | } 72 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/hbase/HBasePartition.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 | package org.apache.spark.sql.hbase 18 | 19 | import org.apache.hadoop.hbase.regionserver.RegionScanner 20 | import org.apache.spark.Partition 21 | import org.apache.spark.internal.Logging 22 | import org.apache.spark.sql.catalyst.expressions.PartialPredicateOperations._ 23 | import org.apache.spark.sql.catalyst.expressions._ 24 | import org.apache.spark.sql.hbase.types.HBaseBytesType 25 | import org.apache.spark.sql.types.Range 26 | 27 | import scala.annotation.meta.param 28 | 29 | private[hbase] class HBasePartition( 30 | val idx: Int, val mappedIndex: Int, 31 | start: Option[HBaseRawType] = None, 32 | end: Option[HBaseRawType] = None, 33 | val server: Option[String] = None, 34 | val filterPredicates: Option[Expression] = None, 35 | @(transient @param) relation: HBaseRelation = null, 36 | @(transient @param) val newScanner:RegionScanner = null) 37 | extends Range[HBaseRawType](start, true, end, false, HBaseBytesType) 38 | with Partition with IndexMappable with Logging { 39 | 40 | override def index: Int = idx 41 | 42 | override def hashCode(): Int = idx 43 | 44 | @transient lazy val startNative: Seq[Any] = relation.nativeKeyConvert(start) 45 | 46 | @transient lazy val endNative: Seq[Any] = relation.nativeKeyConvert(end) 47 | 48 | /** Compute predicate specific for this partition: performed by the Spark slaves 49 | * 50 | * @param relation The HBase relation 51 | * @return the partition-specific predicate 52 | */ 53 | def computePredicate(relation: HBaseRelation): Option[Expression] = { 54 | val predicate = if (filterPredicates.isDefined && 55 | filterPredicates.get.references.exists(_.exprId == relation.partitionKeys.head.exprId)) { 56 | val oriPredicate = filterPredicates.get 57 | val predicateReferences = oriPredicate.references.toSeq 58 | val boundReference = BindReferences.bindReference(oriPredicate, predicateReferences) 59 | val row = new GenericInternalRow(predicateReferences.size) 60 | var rowIndex = 0 61 | var i = 0 62 | var range: Range[_] = null 63 | while (i < relation.keyColumns.size) { 64 | range = relation.generateRange(this, oriPredicate, i) 65 | if (range != null) { 66 | rowIndex = relation.rowIndex(predicateReferences, i) 67 | if (rowIndex >= 0) row.update(rowIndex, range) 68 | // if the non-last dimension range is not point, do not proceed to the next dims 69 | if (i < relation.keyColumns.size - 1 && !range.isPoint) i = relation.keyColumns.size 70 | else i = i + 1 71 | } else i = relation.keyColumns.size 72 | } 73 | val pr = boundReference.partialReduce(row, predicateReferences) 74 | pr match { 75 | case (null, e: Expression) => Some(e) 76 | case (true, _) => None 77 | case (false, _) => Some(Literal(false)) 78 | } 79 | } else filterPredicates 80 | logInfo(predicate.toString) 81 | predicate 82 | } 83 | 84 | override def toString = { 85 | s"HBasePartition: $idx, $mappedIndex, [$start, $end), $filterPredicates" 86 | } 87 | } 88 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/hbase/HBasePartitioner.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 org.apache.spark.sql.hbase 19 | 20 | import org.apache.hadoop.hbase.util.Bytes 21 | import org.apache.spark.Partitioner 22 | import org.apache.spark.util.CollectionsUtils 23 | 24 | object HBasePartitioner { 25 | implicit object HBaseRawOrdering extends Ordering[HBaseRawType] { 26 | def compare(a: HBaseRawType, b: HBaseRawType) = Bytes.compareTo(a, b) 27 | } 28 | } 29 | 30 | class HBasePartitioner (val splitKeys: Array[HBaseRawType]) extends Partitioner { 31 | import HBasePartitioner.HBaseRawOrdering 32 | 33 | type t = HBaseRawType 34 | 35 | lazy private val len = splitKeys.length 36 | 37 | // For pre-split table splitKeys(0) = bytes[0], to remove it, 38 | // otherwise partition 0 always be empty and 39 | // we will miss the last region's date when bulk load 40 | lazy private val realSplitKeys = if (splitKeys.isEmpty) splitKeys else splitKeys.tail 41 | 42 | override def numPartitions = if (len == 0) 1 else len 43 | 44 | @transient private lazy val binarySearch: ((Array[t], t) => Int) = CollectionsUtils.makeBinarySearch[t] 45 | 46 | override def getPartition(key: Any): Int = { 47 | val k = key.asInstanceOf[t] 48 | var partition = 0 49 | if (len <= 128 && len > 0) { 50 | // If we have less than 128 partitions naive search 51 | val ordering = implicitly[Ordering[t]] 52 | while (partition < realSplitKeys.length && ordering.gt(k, realSplitKeys(partition))) { 53 | partition += 1 54 | } 55 | } else { 56 | // Determine which binary search method to use only once. 57 | partition = binarySearch(realSplitKeys, k) 58 | // binarySearch either returns the match location or -[insertion point]-1 59 | if (partition < 0) { 60 | partition = -partition - 1 61 | } 62 | if (partition > realSplitKeys.length) { 63 | partition = realSplitKeys.length 64 | } 65 | } 66 | partition 67 | } 68 | 69 | override def equals(other: Any): Boolean = other match { 70 | case r: HBasePartitioner => 71 | r.splitKeys.sameElements(splitKeys) 72 | case _ => 73 | false 74 | } 75 | 76 | override def hashCode(): Int = { 77 | val prime = 31 78 | var result = 1 79 | var i = 0 80 | while (i < splitKeys.length) { 81 | result = prime * result + splitKeys(i).hashCode 82 | i += 1 83 | } 84 | result = prime * result 85 | result 86 | } 87 | } 88 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/hbase/HBaseSQLConf.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 org.apache.spark.sql.hbase 19 | 20 | import org.apache.spark.sql.internal.SQLConf 21 | 22 | object HBaseSQLConf { 23 | val PARTITION_EXPIRATION = "spark.sql.hbase.partition.expiration" 24 | val SCANNER_FETCH_SIZE = "spark.sql.hbase.scanner.fetchsize" 25 | val USE_COPROCESSOR = "spark.sql.hbase.coprocessor" 26 | val USE_CUSTOMFILTER = "spark.sql.hbase.customfilter" 27 | 28 | val PROVIDER = "provider" 29 | val HBASE = "hbase" 30 | val COLS = "cols" 31 | val KEY_COLS = "keyCols" 32 | val NONKEY_COLS = "nonKeyCols" 33 | val HBASE_TABLENAME = "hbaseTableName" 34 | val ENCODING_FORMAT = "encodingFormat" 35 | } 36 | 37 | /** 38 | * A trait that enables the setting and getting of mutable config parameters/hints. 39 | * 40 | */ 41 | private[hbase] class HBaseSQLConf extends SQLConf { 42 | import HBaseSQLConf._ 43 | 44 | /** The expiration of cached partition (i.e., region) info; defaults to 10 minutes. */ 45 | private[hbase] def partitionExpiration: Long = getConfString(PARTITION_EXPIRATION, "600").toLong 46 | private[hbase] def scannerFetchSize: Int = getConfString(SCANNER_FETCH_SIZE, "1000").toInt 47 | private[hbase] def useCoprocessor: Boolean = getConfString(USE_COPROCESSOR, "false").toBoolean 48 | private[hbase] def useCustomFilter: Boolean = getConfString(USE_CUSTOMFILTER, "true").toBoolean 49 | } 50 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/hbase/HBaseSerializer.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 org.apache.spark.sql.hbase 19 | 20 | import java.io._ 21 | 22 | /** 23 | * the serializer to serialize / de-serialize the objects for HBase embedded execution, 24 | * may be made configurable and use the ones provided by Spark in the future. 25 | */ 26 | private[hbase] object HBaseSerializer { 27 | /** 28 | * serialize the input object to byte array 29 | * @param obj the input object 30 | * @return the serialized byte array 31 | */ 32 | def serialize(obj: Any): Array[Byte] = { 33 | val b = new ByteArrayOutputStream() 34 | val o = new ObjectOutputStream(b) 35 | o.writeObject(obj) 36 | val res = b.toByteArray 37 | o.close() 38 | b.close() 39 | res 40 | } 41 | 42 | /** 43 | * de-serialize the byte array to the original object 44 | * @param bytes the input byte array 45 | * @return the de-serialized object 46 | */ 47 | def deserialize(bytes: Array[Byte]): Any = { 48 | val b = new ByteArrayInputStream(bytes) 49 | val o = new ObjectInputStream(b) 50 | val res = o.readObject() 51 | o.close() 52 | b.close() 53 | res 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/hbase/HBaseShuffledRDD.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 org.apache.spark.sql.hbase 19 | 20 | import org.apache.spark._ 21 | import org.apache.spark.rdd.{RDD, ShuffledRDD, ShuffledRDDPartition} 22 | 23 | import scala.annotation.meta.param 24 | 25 | class HBaseShuffledRDD ( 26 | prevRdd: RDD[(HBaseRawType, Array[HBaseRawType])], 27 | part: Partitioner, 28 | @(transient @param) hbPartitions: Seq[HBasePartition] = Nil) extends ShuffledRDD(prevRdd, part){ 29 | 30 | override def getPartitions: Array[Partition] = { 31 | if (hbPartitions==null || hbPartitions.isEmpty) { 32 | Array.tabulate[Partition](part.numPartitions)(i => new ShuffledRDDPartition(i)) 33 | } else { 34 | // only to be invoked by clients 35 | hbPartitions.toArray 36 | } 37 | } 38 | 39 | override def getPreferredLocations(split: Partition): Seq[String] = { 40 | if (hbPartitions==null || hbPartitions.isEmpty) { 41 | Seq.empty 42 | } else { 43 | split.asInstanceOf[HBasePartition].server.map { 44 | identity[String] 45 | }.toSeq 46 | } 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/hbase/HBaseSparkSession.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 org.apache.spark.sql.hbase 19 | 20 | import org.apache.hadoop.hbase.HBaseConfiguration 21 | import org.apache.spark.SparkContext 22 | import org.apache.spark.api.java.JavaSparkContext 23 | import org.apache.spark.sql._ 24 | import org.apache.spark.sql.catalyst.analysis.Analyzer 25 | import org.apache.spark.sql.catalyst.catalog.ExternalCatalog 26 | import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan 27 | import org.apache.spark.sql.catalyst.rules.Rule 28 | import org.apache.spark.sql.execution.datasources._ 29 | import org.apache.spark.sql.execution.SparkPlanner 30 | import org.apache.spark.sql.hbase.execution.{HBaseSourceAnalysis, HBaseStrategies} 31 | import org.apache.spark.sql.internal.{BaseSessionStateBuilder, SQLConf, SessionState, SharedState} 32 | 33 | class HBaseSparkSession(sc: SparkContext) extends SparkSession(sc) { 34 | self => 35 | 36 | def this(sparkContext: JavaSparkContext) = this(sparkContext.sc) 37 | 38 | @transient 39 | override lazy val sessionState: SessionState = new HBaseSessionStateBuilder(this).build() 40 | 41 | HBaseConfiguration.merge( 42 | sc.hadoopConfiguration, HBaseConfiguration.create(sc.hadoopConfiguration)) 43 | 44 | @transient 45 | override lazy val sharedState: SharedState = 46 | new HBaseSharedState(sc, this.sqlContext) 47 | } 48 | 49 | class HBaseSessionStateBuilder(session: SparkSession, parentState: Option[SessionState] = None) extends BaseSessionStateBuilder(session) { 50 | override lazy val conf: SQLConf = new HBaseSQLConf 51 | 52 | override protected def newBuilder: NewBuilder = new HBaseSessionStateBuilder(_, _) 53 | 54 | override lazy val experimentalMethods: ExperimentalMethods = { 55 | val result = new ExperimentalMethods; 56 | result.extraStrategies = Seq((new SparkPlanner(session.sparkContext, conf, new ExperimentalMethods) 57 | with HBaseStrategies).HBaseDataSource) 58 | result 59 | } 60 | 61 | override lazy val analyzer: Analyzer = { 62 | new Analyzer(catalog, conf) { 63 | override val extendedResolutionRules: Seq[Rule[LogicalPlan]] = 64 | new FindDataSourceTable(session) +: 65 | new ResolveSQLOnFile(session) +: 66 | customResolutionRules 67 | 68 | override val postHocResolutionRules: Seq[Rule[LogicalPlan]] = 69 | PreprocessTableCreation(session) +: 70 | PreprocessTableInsertion(conf) +: 71 | DataSourceAnalysis(conf) +: 72 | HBaseSourceAnalysis(session) +: 73 | customPostHocResolutionRules 74 | 75 | override val extendedCheckRules = 76 | customCheckRules 77 | } 78 | } 79 | } 80 | 81 | class HBaseSharedState(sc: SparkContext, sqlContext: SQLContext) extends SharedState(sc) { 82 | override lazy val externalCatalog: ExternalCatalog = 83 | new HBaseCatalog(sqlContext, sc.hadoopConfiguration) 84 | } 85 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/hbase/HadoopReader.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 org.apache.spark.sql.hbase 19 | 20 | import org.apache.spark.SparkContext 21 | import org.apache.spark.sql.hbase.util.HBaseKVHelper 22 | import org.apache.spark.sql.types._ 23 | 24 | import scala.annotation.meta.param 25 | 26 | /** 27 | * Helper class for scanning files stored in Hadoop - e.g., to read text file when bulk loading. 28 | */ 29 | private[hbase] class HadoopReader( 30 | @(transient @param) sc: SparkContext, 31 | path: String)(baseRelation: HBaseRelation) { 32 | /** make RDD[(SparkImmutableBytesWritable, SparkKeyValue)] from text file. */ 33 | private[hbase] def makeBulkLoadRDDFromTextFile = { 34 | val rdd = sc.textFile(path) 35 | val splitRegex = "," 36 | val relation = baseRelation 37 | 38 | rdd.mapPartitions { iter => 39 | val lineBuffer = HBaseKVHelper.createLineBuffer(relation.output()) 40 | val keyBytes = new Array[(HBaseRawType, DataType)](relation.keyColumns.size) 41 | iter.flatMap { line => 42 | if (line == "") { 43 | None 44 | } else { 45 | // If the last column in the text file is null, the java parser will 46 | // return a String[] containing only the non-null text values. 47 | // In this case we need to append another element (null) to 48 | // the array returned by line.split(splitRegex). 49 | val valueBytes = new Array[HBaseRawType](relation.nonKeyColumns.size) 50 | var textValueArray = line.split(splitRegex) 51 | while (textValueArray.length < relation.output().length) { 52 | textValueArray = textValueArray :+ "" 53 | } 54 | HBaseKVHelper.string2KV(textValueArray, relation, lineBuffer, keyBytes, valueBytes) 55 | val rowKeyData = HBaseKVHelper.encodingRawKeyColumns(keyBytes) 56 | Seq((rowKeyData, valueBytes)) 57 | } 58 | } 59 | } 60 | } 61 | } 62 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/hbase/IndexMappable.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 | package org.apache.spark.sql.hbase 18 | 19 | private[hbase] trait IndexMappable { 20 | def mappedIndex: Int 21 | } 22 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/hbase/ScanPredClassifier.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 org.apache.spark.sql.hbase 19 | 20 | import org.apache.spark.sql.catalyst.expressions._ 21 | import org.apache.spark.sql.hbase.util.DataTypeUtils 22 | 23 | /** 24 | * Classifies a predicate into a pair of (pushdownable, non-pushdownable) predicates 25 | * for a Scan; the logic relationship between the two components of the pair is AND 26 | */ 27 | class ScanPredClassifier(relation: HBaseRelation) { 28 | def apply(pred: Expression): (Option[Expression], Option[Expression]) = { 29 | // post-order bottom-up traversal 30 | pred match { 31 | case And(left, right) => 32 | val (ll, lr) = apply(left) 33 | val (rl, rr) = apply(right) 34 | (ll, lr, rl, rr) match { 35 | // All Nones 36 | case (None, None, None, None) => (None, None) 37 | // Three Nones 38 | case (None, None, None, _) => (None, rr) 39 | case (None, None, _, None) => (rl, None) 40 | case (None, _, None, None) => (None, lr) 41 | case (_, None, None, None) => (ll, None) 42 | // two Nones 43 | case (None, None, _, _) => (rl, rr) 44 | case (None, _, None, _) => (None, Some(And(lr.get, rr.get))) 45 | case (None, _, _, None) => (rl, lr) 46 | case (_, None, None, _) => (ll, rr) 47 | case (_, None, _, None) => (Some(And(ll.get, rl.get)), None) 48 | case (_, _, None, None) => (ll, lr) 49 | // One None 50 | case (None, _, _, _) => (rl, Some(And(lr.get, rr.get))) 51 | case (_, None, _, _) => (Some(And(ll.get, rl.get)), rr) 52 | case (_, _, None, _) => (ll, Some(And(lr.get, rr.get))) 53 | case (_, _, _, None) => (Some(And(ll.get, rl.get)), lr) 54 | // No nones 55 | case _ => (Some(And(ll.get, rl.get)), Some(And(lr.get, rr.get))) 56 | } 57 | case Or(left, right) => 58 | val (ll, lr) = apply(left) 59 | val (rl, rr) = apply(right) 60 | (ll, lr, rl, rr) match { 61 | // All Nones 62 | case (None, None, None, None) => (None, None) 63 | // Three Nones 64 | case (None, None, None, _) => (None, rr) 65 | case (None, None, _, None) => (rl, None) 66 | case (None, _, None, None) => (None, lr) 67 | case (_, None, None, None) => (ll, None) 68 | // two Nones 69 | case (None, None, _, _) => (rl, rr) 70 | case (None, _, None, _) => (None, Some(Or(lr.get, rr.get))) 71 | case (None, _, _, None) => (None, Some(Or(lr.get, rl.get))) 72 | case (_, None, None, _) => (None, Some(Or(ll.get, rr.get))) 73 | case (_, None, _, None) => (Some(Or(ll.get, rl.get)), None) 74 | case (_, _, None, None) => (ll, lr) 75 | // One None 76 | case (None, _, _, _) => (None, Some(pred)) 77 | // Accept increased evaluation complexity for improved pushed down 78 | case (_, None, _, _) => (Some(Or(ll.get, rl.get)), Some(Or(ll.get, rr.get))) 79 | case (_, _, None, _) => (None, Some(pred)) 80 | // Accept increased evaluation complexity for improved pushed down 81 | case (_, _, _, None) => (Some(Or(ll.get, rl.get)), Some(Or(lr.get, rl.get))) 82 | // No nones 83 | // Accept increased evaluation complexity for improved pushed down 84 | case _ => (Some(Or(ll.get, rl.get)), Some(And(Or(ll.get, rr.get), 85 | And(Or(lr.get, rl.get), Or(lr.get, rr.get))))) 86 | } 87 | case EqualTo(left, right) => classifyBinary(left, right, pred) 88 | case LessThan(left, right) => classifyBinary(left, right, pred) 89 | case LessThanOrEqual(left, right) => classifyBinary(left, right, pred) 90 | case GreaterThan(left, right) => classifyBinary(left, right, pred) 91 | case GreaterThanOrEqual(left, right) => classifyBinary(left, right, pred) 92 | case In(value@AttributeReference(_, _, _, _), list) => 93 | if (relation.isNonKey(value) && !list.exists(!_.isInstanceOf[Literal])) { 94 | (Some(pred), None) 95 | } else { 96 | (None, Some(pred)) 97 | } 98 | case InSet(AttributeReference(name, dataType, _, _), hset) 99 | if relation.nonKeyColumns.exists(_.sqlName == name) => 100 | var errorOccurred = false 101 | for (item <- hset if !errorOccurred) { 102 | try { 103 | /** 104 | * Use try-catch to make sure data type conversion is proper, for example, 105 | * Java throws casting exception while doing col2 in (1, 2, 3), if col2 data type 106 | * is ByteType and 1, 2, 3 is Integer. 107 | */ 108 | DataTypeUtils.getBinaryComparator( 109 | relation.bytesUtils.create(dataType), Literal.create(item, dataType)) 110 | } catch { 111 | case _: Exception => errorOccurred = true 112 | } 113 | } 114 | if (errorOccurred) { 115 | (None, Some(pred)) 116 | } else { 117 | (Some(pred), None) 118 | } 119 | // everything else are treated as non pushdownable 120 | case _ => (None, Some(pred)) 121 | } 122 | } 123 | 124 | // returns true if the binary operator of the two args can be pushed down 125 | private def classifyBinary(left: Expression, right: Expression, pred: Expression) 126 | : (Option[Expression], Option[Expression]) = { 127 | (left, right) match { 128 | case (Literal(_, _), AttributeReference(_, _, _, _)) => 129 | if (relation.isNonKey(right.asInstanceOf[AttributeReference])) { 130 | (Some(pred), None) 131 | } else { 132 | (None, Some(pred)) 133 | } 134 | case (AttributeReference(_, _, _, _), Literal(_, _)) => 135 | if (relation.isNonKey(left.asInstanceOf[AttributeReference])) { 136 | (Some(pred), None) 137 | } else { 138 | (None, Some(pred)) 139 | } 140 | case _ => (None, Some(pred)) 141 | } 142 | } 143 | } 144 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/hbase/SparkSqlRegionObserver.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 org.apache.spark.sql.hbase 19 | 20 | import java.util.Properties 21 | 22 | import org.apache.hadoop.hbase._ 23 | import org.apache.hadoop.hbase.client._ 24 | import org.apache.hadoop.hbase.coprocessor._ 25 | import org.apache.hadoop.hbase.regionserver._ 26 | import org.apache.hadoop.hbase.util.Bytes 27 | import org.apache.spark._ 28 | import org.apache.spark.internal.Logging 29 | import org.apache.spark.rdd.RDD 30 | import org.apache.spark.sql.catalyst.InternalRow 31 | import org.apache.spark.sql.catalyst.expressions._ 32 | import org.apache.spark.sql.catalyst.expressions.codegen.{GeneratePredicate, Predicate} 33 | import org.apache.spark.sql.hbase.util.DataTypeUtils 34 | import org.apache.spark.sql.types._ 35 | import org.apache.spark.sql.{Row, SQLContext} 36 | 37 | import scala.annotation.meta.param 38 | 39 | /** 40 | * HBaseCoprocessorSQLReaderRDD: 41 | */ 42 | class HBaseCoprocessorSQLReaderRDD(var relation: HBaseRelation, 43 | val codegenEnabled: Boolean, 44 | var finalOutput: Seq[Attribute], 45 | var otherFilters: Option[Expression], 46 | @(transient @param) sqlContext: SQLContext) 47 | extends RDD[InternalRow](sqlContext.sparkContext, Nil) with Logging { 48 | 49 | @transient var scanner: RegionScanner = _ 50 | 51 | private def createIterator(context: TaskContext): Iterator[InternalRow] = { 52 | val otherFilter: Predicate = { 53 | if (otherFilters.isDefined) { 54 | if (codegenEnabled) { 55 | GeneratePredicate.generate(otherFilters.get, finalOutput) 56 | } else { 57 | InterpretedPredicate.create(otherFilters.get, finalOutput) 58 | } 59 | } else null 60 | } 61 | 62 | val projections = finalOutput.zipWithIndex 63 | var finished: Boolean = false 64 | var gotNext: Boolean = false 65 | val results: java.util.ArrayList[Cell] = new java.util.ArrayList[Cell]() 66 | val row = new GenericInternalRow(finalOutput.size) 67 | 68 | val iterator = new Iterator[InternalRow] { 69 | override def hasNext: Boolean = { 70 | if (!finished) { 71 | if (!gotNext) { 72 | results.clear() 73 | scanner.nextRaw(results) 74 | finished = results.isEmpty 75 | gotNext = true 76 | } 77 | } 78 | if (finished) { 79 | close() 80 | } 81 | !finished 82 | } 83 | 84 | override def next(): InternalRow = { 85 | if (hasNext) { 86 | gotNext = false 87 | relation.buildRowInCoprocessor(projections, results, row) 88 | } else { 89 | null 90 | } 91 | } 92 | 93 | def close() = { 94 | try { 95 | scanner.close() 96 | relation.close() 97 | } catch { 98 | case e: Exception => logWarning("Exception in scanner.close", e) 99 | } 100 | } 101 | } 102 | 103 | if (otherFilter == null) { 104 | new InterruptibleIterator(context, iterator) 105 | } else { 106 | new InterruptibleIterator(context, iterator.filter(otherFilter.eval(_))) 107 | } 108 | } 109 | 110 | override def getPartitions: Array[Partition] = { 111 | Array() 112 | } 113 | 114 | override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { 115 | scanner = split.asInstanceOf[HBasePartition].newScanner 116 | createIterator(context) 117 | } 118 | } 119 | 120 | abstract class BaseRegionScanner extends RegionScanner { 121 | override def getBatch={0}//Achieve this function inherited from RegionScanner 122 | 123 | override def isFilterDone = false 124 | 125 | override def next(result: java.util.List[Cell], scannerContext: ScannerContext)= next(result)// limit: Int=>scannerContext: ScannerContext 126 | 127 | override def reseek(row: Array[Byte]) = throw new DoNotRetryIOException("Unsupported") 128 | 129 | override def getMvccReadPoint = Long.MaxValue 130 | 131 | override def nextRaw(result: java.util.List[Cell]) = next(result) 132 | 133 | override def nextRaw(result: java.util.List[Cell], scannerContext: ScannerContext) = next(result, scannerContext) //limit: Int=>scannerContext: ScannerContext 134 | } 135 | 136 | class SparkSqlRegionObserver extends BaseRegionObserver with Logging { 137 | lazy val EmptyArray = Array[Byte]() 138 | 139 | override def postScannerOpen(e: ObserverContext[RegionCoprocessorEnvironment], 140 | scan: Scan, 141 | s: RegionScanner) = { 142 | val serializedPartitionIndex = scan.getAttribute(CoprocessorConstants.COINDEX) 143 | if (serializedPartitionIndex == null) { 144 | logDebug("Work without coprocessor") 145 | super.postScannerOpen(e, scan, s) 146 | } else { 147 | logDebug("Work with coprocessor") 148 | val partitionIndex: Int = Bytes.toInt(serializedPartitionIndex) 149 | val serializedOutputDataType = scan.getAttribute(CoprocessorConstants.COTYPE) 150 | val outputDataType: Seq[DataType] = 151 | HBaseSerializer.deserialize(serializedOutputDataType).asInstanceOf[Seq[DataType]] 152 | 153 | val serializedRDD = scan.getAttribute(CoprocessorConstants.COKEY) 154 | val subPlanRDD: RDD[Row] = HBaseSerializer.deserialize(serializedRDD).asInstanceOf[RDD[Row]] 155 | 156 | val taskParaInfo = scan.getAttribute(CoprocessorConstants.COTASK) 157 | val (stageId, partitionId, taskAttemptId, attemptNumber) = 158 | HBaseSerializer.deserialize(taskParaInfo).asInstanceOf[(Int, Int, Long, Int)] 159 | val taskContext = new TaskContextImpl( 160 | stageId, partitionId, taskAttemptId, attemptNumber, null, new Properties, null) 161 | 162 | val regionInfo = s.getRegionInfo 163 | val startKey = if (regionInfo.getStartKey.isEmpty) None else Some(regionInfo.getStartKey) 164 | val endKey = if (regionInfo.getEndKey.isEmpty) None else Some(regionInfo.getEndKey) 165 | 166 | val result = subPlanRDD.compute( 167 | new HBasePartition(partitionIndex, partitionIndex, startKey, endKey, newScanner = s), 168 | taskContext) 169 | 170 | new BaseRegionScanner() { 171 | override def getRegionInfo: HRegionInfo = regionInfo 172 | 173 | override def getMaxResultSize: Long = s.getMaxResultSize 174 | 175 | override def close(): Unit = s.close() 176 | 177 | override def next(results: java.util.List[Cell]): Boolean = { 178 | val hasMore: Boolean = result.hasNext 179 | if (hasMore) { 180 | val nextRow = result.next() 181 | val numOfCells = outputDataType.length 182 | for (i <- 0 until numOfCells) { 183 | val data = nextRow(i) 184 | val dataType = outputDataType(i) 185 | val dataOfBytes: HBaseRawType = { 186 | if (data == null) null else DataTypeUtils.dataToBytes(data, dataType) 187 | } 188 | results.add(new KeyValue(EmptyArray, EmptyArray, EmptyArray, dataOfBytes)) 189 | } 190 | } 191 | hasMore 192 | } 193 | } 194 | } 195 | } 196 | } 197 | 198 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/hbase/execution/HBaseSQLTableScan.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 org.apache.spark.sql.hbase.execution 19 | 20 | import org.apache.spark.annotation.DeveloperApi 21 | import org.apache.spark.rdd.RDD 22 | import org.apache.spark.sql.catalyst.InternalRow 23 | import org.apache.spark.sql.catalyst.expressions._ 24 | import org.apache.spark.sql.catalyst.plans.physical.RangePartitioning 25 | import org.apache.spark.sql.execution.SparkPlan 26 | import org.apache.spark.sql.hbase._ 27 | import org.apache.spark.sql.types.StructType 28 | import org.apache.spark.util.Utils 29 | 30 | /** 31 | * :: DeveloperApi :: 32 | * The HBase table scan operator. 33 | */ 34 | @DeveloperApi 35 | case class HBaseSQLTableScan( 36 | relation: HBaseRelation, 37 | output: Seq[Attribute], 38 | result: RDD[InternalRow]) extends SparkPlan { 39 | override def children: Seq[SparkPlan] = Nil 40 | 41 | override def outputPartitioning = { 42 | var ordering = List[SortOrder]() 43 | for (key <- relation.partitionKeys) { 44 | ordering = ordering :+ SortOrder(key, Ascending) 45 | } 46 | RangePartitioning(ordering, relation.partitions.size) 47 | } 48 | 49 | override protected def doExecute(): RDD[InternalRow] = { 50 | val schema = StructType.fromAttributes(output) 51 | result.mapPartitionsInternal { iter => 52 | val proj = UnsafeProjection.create(schema) 53 | iter.map(proj) 54 | } 55 | } 56 | 57 | override def nodeName: String = getClass.getSimpleName 58 | 59 | override def argString: String = 60 | (Utils.truncatedString(output, "[", ", ", "]") :: Nil).mkString(", ") 61 | } 62 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/hbase/execution/hbaseCommands.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 | package org.apache.spark.sql.hbase.execution 18 | 19 | import org.apache.spark.annotation.DeveloperApi 20 | import org.apache.spark.sql._ 21 | import org.apache.spark.sql.catalyst.TableIdentifier 22 | import org.apache.spark.sql.catalyst.expressions.Attribute 23 | import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias 24 | import org.apache.spark.sql.execution.command.RunnableCommand 25 | import org.apache.spark.sql.execution.datasources.LogicalRelation 26 | import org.apache.spark.sql.hbase._ 27 | import org.apache.spark.sql.hbase.util.DataTypeUtils 28 | import org.apache.spark.sql.types._ 29 | 30 | import scala.collection.mutable.ArrayBuffer 31 | 32 | @DeveloperApi 33 | case class AlterDropColCommand(namespace: String, tableName: String, columnName: String) 34 | extends RunnableCommand { 35 | 36 | def run(sparkSession: SparkSession): Seq[Row] = { 37 | sparkSession.sharedState.externalCatalog.asInstanceOf[HBaseCatalog] 38 | .alterTableDropNonKey(namespace, tableName, columnName) 39 | sparkSession.sharedState.externalCatalog.asInstanceOf[HBaseCatalog].stopAdmin() 40 | Seq.empty[Row] 41 | } 42 | } 43 | 44 | @DeveloperApi 45 | case class AlterAddColCommand(namespace: String, 46 | tableName: String, 47 | colName: String, 48 | colType: String, 49 | colFamily: String, 50 | colQualifier: String) extends RunnableCommand { 51 | 52 | def run(sparkSession: SparkSession): Seq[Row] = { 53 | val hbaseCatalog = sparkSession.sharedState.externalCatalog.asInstanceOf[HBaseCatalog] 54 | hbaseCatalog.alterTableAddNonKey(namespace, tableName, 55 | NonKeyColumn(colName, DataTypeUtils.getDataType(colType), colFamily, colQualifier)) 56 | hbaseCatalog.stopAdmin() 57 | Seq.empty[Row] 58 | } 59 | } 60 | 61 | @DeveloperApi 62 | case class InsertValueIntoTableCommand(tid: TableIdentifier, valueSeq: Seq[String]) 63 | extends RunnableCommand { 64 | override def run(sparkSession: SparkSession) = { 65 | val relation: HBaseRelation = sparkSession.sessionState.catalog.externalCatalog 66 | .asInstanceOf[HBaseCatalog] 67 | .getHBaseRelation(tid.database.getOrElse(null), tid.table).getOrElse(null) 68 | 69 | val bytes = valueSeq.zipWithIndex.map(v => 70 | DataTypeUtils.string2TypeData(v._1, relation.schema(v._2).dataType)) 71 | 72 | val rows = sparkSession.sparkContext.makeRDD(Seq(Row.fromSeq(bytes))) 73 | val inputValuesDF = sparkSession.createDataFrame(rows, relation.schema) 74 | relation.insert(inputValuesDF, overwrite = false) 75 | 76 | Seq.empty[Row] 77 | } 78 | 79 | override def output: Seq[Attribute] = Seq.empty 80 | } 81 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/hbase/package.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 | package org.apache.spark.sql 18 | 19 | package object hbase { 20 | type HBaseRawType = Array[Byte] 21 | } 22 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/hbase/types/HBaseBytesType.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 org.apache.spark.sql.hbase.types 19 | 20 | import org.apache.spark.sql.catalyst.ScalaReflectionLock 21 | import org.apache.spark.sql.hbase.HBaseRawType 22 | import org.apache.spark.sql.types._ 23 | 24 | import scala.reflect.runtime.universe.typeTag 25 | 26 | /** 27 | * Almost identical to BinaryType except for a different ordering to be consistent 28 | * with that of HBase's internal ordering 29 | * This is a data type for Low-Level HBase entities. 30 | * It should not be used in High-Level processing 31 | */ 32 | private[hbase] case object HBaseBytesType extends AtomicType /*with PrimitiveType*/ { 33 | override def defaultSize: Int = 4096 34 | private[sql] type InternalType = HBaseRawType 35 | @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized {typeTag[InternalType]} 36 | private[sql] val ordering = new Ordering[InternalType] { 37 | def compare(x: Array[Byte], y: Array[Byte]): Int = { 38 | for (i <- x.indices if i < y.length) { 39 | val a: Int = x(i) & 0xff 40 | val b: Int = y(i) & 0xff 41 | val res = a - b 42 | if (res != 0) return res 43 | } 44 | x.length - y.length 45 | } 46 | } 47 | 48 | private[spark] override def asNullable = this 49 | } 50 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/hbase/util/DataTypeUtils.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 | package org.apache.spark.sql.hbase.util 18 | 19 | import java.nio.ByteBuffer 20 | 21 | import org.apache.hadoop.hbase.filter.{BinaryComparator, ByteArrayComparable} 22 | import org.apache.spark.SparkException 23 | import org.apache.spark.serializer.JavaSerializer 24 | import org.apache.spark.sql.Row 25 | import org.apache.spark.sql.catalyst.InternalRow 26 | import org.apache.spark.sql.catalyst.expressions.Literal 27 | import org.apache.spark.sql.hbase._ 28 | import org.apache.spark.sql.types._ 29 | 30 | /** 31 | * Data Type conversion utilities 32 | */ 33 | object DataTypeUtils { 34 | /** 35 | * convert the byte array to data 36 | * @param src the input byte array 37 | * @param offset the offset in the byte array 38 | * @param length the length of the data, only used by StringType 39 | * @param dt the data type 40 | * @return the actual data converted from byte array 41 | */ 42 | def bytesToData(src: HBaseRawType, offset: Int, length: Int, dt: DataType, 43 | bytesUtils: BytesUtils = BinaryBytesUtils): Any = { 44 | dt match { 45 | case BooleanType => bytesUtils.toBoolean(src, offset, length) 46 | case ByteType => bytesUtils.toByte(src, offset, length) 47 | case DateType => bytesUtils.toDate(src, offset, length) 48 | case DoubleType => bytesUtils.toDouble(src, offset, length) 49 | case FloatType => bytesUtils.toFloat(src, offset, length) 50 | case IntegerType => bytesUtils.toInt(src, offset, length) 51 | case LongType => bytesUtils.toLong(src, offset, length) 52 | case ShortType => bytesUtils.toShort(src, offset, length) 53 | case StringType => bytesUtils.toUTF8String(src, offset, length) 54 | case TimestampType => bytesUtils.toTimestamp(src, offset, length) 55 | case _ => throw new SparkException(s"Unsupported HBase SQL Data Type ${dt.catalogString}") 56 | } 57 | } 58 | 59 | /** 60 | * convert data to byte array 61 | * @param src the input data 62 | * @param dt the data type 63 | * @return the output byte array 64 | */ 65 | def dataToBytes(src: Any, 66 | dt: DataType, 67 | bytesUtils: BytesUtils = BinaryBytesUtils): HBaseRawType = { 68 | // TODO: avoid new instance per invocation 69 | lazy val bu = bytesUtils.create(dt) 70 | dt match { 71 | case BooleanType => bu.toBytes(src.asInstanceOf[Boolean]) 72 | case ByteType => bu.toBytes(src.asInstanceOf[Byte]) 73 | case DateType => bu.toBytes(src.asInstanceOf[Int]) 74 | case DoubleType => bu.toBytes(src.asInstanceOf[Double]) 75 | case FloatType => bu.toBytes(src.asInstanceOf[Float]) 76 | case IntegerType => bu.toBytes(src.asInstanceOf[Int]) 77 | case LongType => bu.toBytes(src.asInstanceOf[Long]) 78 | case ShortType => bu.toBytes(src.asInstanceOf[Short]) 79 | case StringType => bu.toBytes(src) 80 | case TimestampType => bu.toBytes(src.asInstanceOf[Long]) 81 | case _ => new JavaSerializer(null).newInstance().serialize[Any](src).array //TODO 82 | } 83 | } 84 | 85 | /** 86 | * set the row data from byte array 87 | * @param row the row to be set 88 | * @param index the index in the row 89 | * @param src the input byte array 90 | * @param offset the offset in the byte array 91 | * @param length the length of the data, only used by StringType 92 | * @param dt the data type 93 | */ 94 | def setRowColumnFromHBaseRawType(row: InternalRow, 95 | index: Int, 96 | src: HBaseRawType, 97 | offset: Int, 98 | length: Int, 99 | dt: DataType, 100 | bytesUtils: BytesUtils = BinaryBytesUtils): Unit = { 101 | dt match { 102 | case BooleanType => row.setBoolean(index, bytesUtils.toBoolean(src, offset, length)) 103 | case ByteType => row.setByte(index, bytesUtils.toByte(src, offset, length)) 104 | case DateType => row.update(index, bytesUtils.toDate(src, offset, length)) 105 | case DoubleType => row.setDouble(index, bytesUtils.toDouble(src, offset, length)) 106 | case FloatType => row.setFloat(index, bytesUtils.toFloat(src, offset, length)) 107 | case IntegerType => row.setInt(index, bytesUtils.toInt(src, offset, length)) 108 | case LongType => row.setLong(index, bytesUtils.toLong(src, offset, length)) 109 | case ShortType => row.setShort(index, bytesUtils.toShort(src, offset, length)) 110 | case StringType => row.update(index, bytesUtils.toUTF8String(src, offset, length)) 111 | case TimestampType => row.update(index, bytesUtils.toTimestamp(src, offset, length)) 112 | case _ => row.update(index, new JavaSerializer(null).newInstance() 113 | .deserialize[Any](ByteBuffer.wrap(src))) //TODO 114 | } 115 | } 116 | 117 | def string2TypeData(v: String, dt: DataType): Any = { 118 | v match { 119 | case null => null 120 | case _ => 121 | dt match { 122 | // TODO: handle some complex types 123 | case BooleanType => v.toBoolean 124 | case ByteType => v.getBytes()(0) 125 | case DateType => java.sql.Date.valueOf(v) 126 | case DoubleType => v.toDouble 127 | case FloatType => v.toFloat 128 | case IntegerType => v.toInt 129 | case LongType => v.toLong 130 | case ShortType => v.toShort 131 | case StringType => v 132 | case TimestampType => java.sql.Timestamp.valueOf(v) 133 | } 134 | } 135 | } 136 | 137 | /** 138 | * get the data from row based on index 139 | * @param row the input row 140 | * @param index the index of the data 141 | * @param dt the data type 142 | * @return the data from the row based on index 143 | */ 144 | def getRowColumnInHBaseRawType(row: Row, index: Int, dt: DataType, 145 | bytesUtils: BytesUtils = BinaryBytesUtils): HBaseRawType = { 146 | if (row.isNullAt(index)) return new Array[Byte](0) 147 | 148 | val bu = bytesUtils.create(dt) 149 | dt match { 150 | case BooleanType => bu.toBytes(row.getBoolean(index)) 151 | case ByteType => bu.toBytes(row.getByte(index)) 152 | case DateType => bu.toBytes(row.getDate(index)) 153 | case DoubleType => bu.toBytes(row.getDouble(index)) 154 | case FloatType => bu.toBytes(row.getFloat(index)) 155 | case IntegerType => bu.toBytes(row.getInt(index)) 156 | case LongType => bu.toBytes(row.getLong(index)) 157 | case ShortType => bu.toBytes(row.getShort(index)) 158 | case StringType => bu.toBytes(row.getString(index)) 159 | case TimestampType => bu.toBytes(row.getTimestamp(index)) 160 | case _ => throw new SparkException(s"Unsupported HBase SQL Data Type ${dt.catalogString}") 161 | } 162 | } 163 | 164 | /** 165 | * create binary comparator for the input expression 166 | * @param bu the byte utility 167 | * @param expression the input expression 168 | * @return the constructed binary comparator 169 | */ 170 | def getBinaryComparator(bu: ToBytesUtils, expression: Literal): ByteArrayComparable = { 171 | bu match { 172 | case _: BinaryBytesUtils => 173 | expression.dataType match { 174 | case BooleanType => new BinaryComparator(bu.toBytes(expression.value.asInstanceOf[Boolean])) 175 | case ByteType => new BinaryComparator(bu.toBytes(expression.value.asInstanceOf[Byte])) 176 | case DateType => new BinaryComparator(bu.toBytes(expression.value.asInstanceOf[Int])) 177 | case DoubleType => new BinaryComparator(bu.toBytes(expression.value.asInstanceOf[Double])) 178 | case FloatType => new BinaryComparator(bu.toBytes(expression.value.asInstanceOf[Float])) 179 | case IntegerType => new BinaryComparator(bu.toBytes(expression.value.asInstanceOf[Int])) 180 | case LongType => new BinaryComparator(bu.toBytes(expression.value.asInstanceOf[Long])) 181 | case ShortType => new BinaryComparator(bu.toBytes(expression.value.asInstanceOf[Short])) 182 | case StringType => new BinaryComparator(bu.toBytes(expression.value)) 183 | case TimestampType => new BinaryComparator(bu.toBytes(expression.value.asInstanceOf[Long])) 184 | case _ => throw new SparkException("Cannot convert the data type using BinaryComparator") 185 | } 186 | case _: StringBytesUtils => 187 | expression.dataType match { 188 | case BooleanType => new BoolComparator(bu.toBytes(expression.value.asInstanceOf[Boolean])) 189 | case ByteType => new ByteComparator(bu.toBytes(expression.value.asInstanceOf[Byte])) 190 | case DateType => new IntComparator(bu.toBytes(expression.value.asInstanceOf[Int])) 191 | case DoubleType => new DoubleComparator(bu.toBytes(expression.value.asInstanceOf[Double])) 192 | case FloatType => new FloatComparator(bu.toBytes(expression.value.asInstanceOf[Float])) 193 | case IntegerType => new IntComparator(bu.toBytes(expression.value.asInstanceOf[Int])) 194 | case LongType => new LongComparator(bu.toBytes(expression.value.asInstanceOf[Long])) 195 | case ShortType => new ShortComparator(bu.toBytes(expression.value.asInstanceOf[Short])) 196 | case StringType => new BinaryComparator(bu.toBytes(expression.value)) 197 | case TimestampType => new LongComparator(bu.toBytes(expression.value.asInstanceOf[Long])) 198 | case _ => throw new SparkException("Cannot convert the data type using CustomComparator") 199 | } 200 | } 201 | } 202 | 203 | def getDataType(data: String): DataType = { 204 | val dataType = data.toLowerCase 205 | if (dataType == ByteType.catalogString) { 206 | ByteType 207 | } else if (dataType == BooleanType.catalogString) { 208 | BooleanType 209 | } else if (dataType == DateType.catalogString) { 210 | DateType 211 | } else if (dataType == DoubleType.catalogString) { 212 | DoubleType 213 | } else if (dataType == FloatType.catalogString) { 214 | FloatType 215 | } else if (dataType == IntegerType.catalogString) { 216 | IntegerType 217 | } else if (dataType == LongType.catalogString) { 218 | LongType 219 | } else if (dataType == ShortType.catalogString) { 220 | ShortType 221 | } else if (dataType == StringType.catalogString) { 222 | StringType 223 | } else if (dataType == TimestampType.catalogString) { 224 | TimestampType 225 | } else { 226 | throw new SparkException(s"Unrecognized data type: $data") 227 | } 228 | } 229 | } 230 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/hbase/util/HBaseKVHelper.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 org.apache.spark.sql.hbase.util 19 | 20 | import org.apache.spark.sql.Row 21 | import org.apache.spark.sql.catalyst.expressions.Attribute 22 | import org.apache.spark.sql.hbase._ 23 | import org.apache.spark.sql.types._ 24 | 25 | object HBaseKVHelper { 26 | val delimiter: Byte = 0 27 | 28 | /** 29 | * create row key based on key columns information 30 | * for strings, it will add '0x00' as its delimiter 31 | * @param rawKeyColumns sequence of byte array and data type representing the key columns 32 | * @return array of bytes 33 | */ 34 | def encodingRawKeyColumns(rawKeyColumns: Seq[(HBaseRawType, DataType)]): HBaseRawType = { 35 | var length = 0 36 | for (i <- rawKeyColumns.indices) { 37 | length += rawKeyColumns(i)._1.length 38 | if (rawKeyColumns(i)._2 == StringType && i < rawKeyColumns.length - 1) { 39 | length += 1 40 | } 41 | } 42 | val result = new HBaseRawType(length) 43 | var index = 0 44 | var kcIdx = 0 45 | for (rawKeyColumn <- rawKeyColumns) { 46 | Array.copy(rawKeyColumn._1, 0, result, index, rawKeyColumn._1.length) 47 | index += rawKeyColumn._1.length 48 | if (rawKeyColumn._2 == StringType && kcIdx < rawKeyColumns.length - 1) { 49 | result(index) = delimiter 50 | index += 1 51 | } 52 | kcIdx += 1 53 | } 54 | result 55 | } 56 | 57 | /** 58 | * generate the sequence information of key columns from the byte array 59 | * @param rowKey array of bytes 60 | * @param keyColumns the sequence of key columns 61 | * @param keyLength rowkey length: specified if not negative 62 | * @return sequence of information in (offset, length) tuple 63 | */ 64 | def decodingRawKeyColumns(rowKey: HBaseRawType, 65 | keyColumns: Seq[KeyColumn], 66 | keyLength: Int = -1, 67 | startIndex: Int = 0): Seq[(Int, Int)] = { 68 | var index = startIndex 69 | var pos = 0 70 | val limit = if (keyLength < 0) { 71 | rowKey.length 72 | } else { 73 | index + keyLength 74 | } 75 | keyColumns.map { 76 | c => 77 | if (index >= limit) (-1, -1) 78 | else { 79 | val offset = index 80 | if (c.dataType == StringType) { 81 | pos = rowKey.indexOf(delimiter, index) 82 | if (pos == -1 || pos > limit) { 83 | // this is at the last dimension 84 | pos = limit 85 | } 86 | index = pos + 1 87 | (offset, pos - offset) 88 | } else { 89 | val length = c.dataType.asInstanceOf[AtomicType].defaultSize 90 | index += length 91 | (offset, length) 92 | } 93 | } 94 | } 95 | } 96 | 97 | /** 98 | * Takes a record, translate it into HBase row key column and value by matching with metadata 99 | * @param values record that as a sequence of string 100 | * @param relation HBaseRelation 101 | * @param keyBytes output parameter, array of (key column and its type); 102 | * @param valueBytes array of (column family, column qualifier, value) 103 | */ 104 | def string2KV(values: Seq[String], 105 | relation: HBaseRelation, 106 | lineBuffer: Array[ToBytesUtils], 107 | keyBytes: Array[(Array[Byte], DataType)], 108 | valueBytes: Array[HBaseRawType]) = { 109 | assert(values.length == relation.output().length, 110 | s"values length ${values.length} not equals columns length ${relation.output().length}") 111 | 112 | relation.keyColumns.foreach(kc => { 113 | val ordinal = kc.ordinal 114 | keyBytes(kc.order) = (string2Bytes(values(ordinal), lineBuffer(ordinal)), 115 | relation.output()(ordinal).dataType) 116 | }) 117 | for (i <- relation.nonKeyColumns.indices) { 118 | val nkc = relation.nonKeyColumns(i) 119 | val bytes = { 120 | // we should not use the same buffer in bulk-loading otherwise it will lead to corrupted 121 | lineBuffer(nkc.ordinal) = relation.bytesUtils.create(lineBuffer(nkc.ordinal).dataType) 122 | string2Bytes(values(nkc.ordinal), lineBuffer(nkc.ordinal)) 123 | } 124 | valueBytes(i) = bytes 125 | } 126 | } 127 | 128 | private def string2Bytes(v: String, bu: ToBytesUtils): Array[Byte] = { 129 | v match { 130 | case "" => new Array[Byte](0) 131 | case null => new Array[Byte](0) 132 | case _ => 133 | bu.dataType match { 134 | // todo: handle some complex types 135 | case BooleanType => bu.toBytes(v.toBoolean) 136 | case ByteType => bu.toBytes(v) 137 | case DoubleType => bu.toBytes(v.toDouble) 138 | case FloatType => bu.toBytes(v.toFloat) 139 | case IntegerType => bu.toBytes(v.toInt) 140 | case LongType => bu.toBytes(v.toLong) 141 | case ShortType => bu.toBytes(v.toShort) 142 | case StringType => bu.toBytes(v) 143 | } 144 | } 145 | } 146 | 147 | /** 148 | * create a array of buffer that to be used for creating HBase Put object 149 | * @param schema the schema of the line buffer 150 | * @return 151 | */ 152 | private[hbase] def createLineBuffer(schema: Seq[Attribute]): Array[ToBytesUtils] = { 153 | schema.map{x => 154 | BinaryBytesUtils.create(x.dataType) 155 | }.toArray 156 | } 157 | 158 | /** 159 | * create a row key 160 | * @param row the generic row 161 | * @param dataTypeOfKeys sequence of data type 162 | * @return the row key 163 | */ 164 | def makeRowKey(row: Row, dataTypeOfKeys: Seq[DataType]): HBaseRawType = { 165 | val rawKeyCol = dataTypeOfKeys.zipWithIndex.map { 166 | case (dataType, index) => 167 | (DataTypeUtils.getRowColumnInHBaseRawType(row, index, dataType), dataType) 168 | } 169 | 170 | encodingRawKeyColumns(rawKeyCol) 171 | } 172 | } 173 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/hbase/util/Util.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 org.apache.spark.sql.hbase.util 19 | 20 | import java.io._ 21 | import java.util.concurrent.atomic.AtomicInteger 22 | import java.util.zip.{DeflaterOutputStream, InflaterInputStream} 23 | 24 | import org.apache.hadoop.conf.Configuration 25 | import org.apache.hadoop.fs.{FileSystem, Path} 26 | import org.apache.hadoop.hbase.HBaseConfiguration 27 | 28 | object Util { 29 | val iteration = new AtomicInteger(0) 30 | 31 | /** 32 | * Generate a temp folder on hadoop file system. 33 | * @param conf the hadoop configuration 34 | * @param prefix the prefix for that folder name to be generated 35 | * @return the folder name in string format 36 | */ 37 | def getTempFilePath(conf: Configuration, prefix: String): String = { 38 | val fileSystem = FileSystem.get(conf) 39 | val path = new Path(s"$prefix-${System.currentTimeMillis()}-${iteration.getAndIncrement}") 40 | if (fileSystem.exists(path)) { 41 | fileSystem.delete(path, true) 42 | } 43 | path.getName 44 | } 45 | 46 | /** 47 | * Drop the temp folder, return true if it exists and is dropped successfully, 48 | * otherwise return false 49 | * @param conf the hadoop configuration 50 | * @param path the path to be dropped 51 | * @return true if the folder path exists ad is dropped successfully, otherwise return false 52 | */ 53 | def dropTempFilePath(conf: Configuration, path: String): Boolean = { 54 | val fileSystem = FileSystem.get(conf) 55 | val filePath = new Path(path) 56 | if (fileSystem.exists(filePath)) { 57 | fileSystem.delete(filePath, true) 58 | } else { 59 | false 60 | } 61 | } 62 | 63 | def serializeHBaseConfiguration(configuration: Configuration): Array[Byte] = { 64 | val bos = new ByteArrayOutputStream 65 | val deflaterOutputStream = new DeflaterOutputStream(bos) 66 | val dos = new DataOutputStream(deflaterOutputStream) 67 | configuration.write(dos) 68 | dos.close() 69 | bos.toByteArray 70 | } 71 | 72 | def deserializeHBaseConfiguration(arr: Array[Byte]) = { 73 | val conf = HBaseConfiguration.create 74 | conf.readFields(new DataInputStream(new InflaterInputStream(new ByteArrayInputStream(arr)))) 75 | conf 76 | } 77 | } 78 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/hbase/util/comparators.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 org.apache.spark.sql.hbase.util 19 | 20 | import com.google.protobuf.InvalidProtocolBufferException 21 | import org.apache.hadoop.hbase.exceptions.DeserializationException 22 | import org.apache.hadoop.hbase.filter.ByteArrayComparable 23 | import org.apache.hadoop.hbase.util.{ByteStringer, Bytes} 24 | import org.apache.spark.sql.hbase._ 25 | 26 | class CustomComparator(value: Array[Byte]) extends ByteArrayComparable(value) { 27 | def compareTo(value: HBaseRawType, offset: Int, length: Int): Int = { 28 | Bytes.compareTo(this.value, 0, this.value.length, value, offset, length) 29 | } 30 | 31 | def origConvert: AdditionalComparatorsProtos.ByteArrayComparable = { 32 | val builder = AdditionalComparatorsProtos.ByteArrayComparable.newBuilder 33 | if (value != null) builder.setValue(ByteStringer.wrap(value)) 34 | builder.build 35 | } 36 | 37 | /** 38 | * @return The comparator serialized using pb 39 | */ 40 | def toByteArray: Array[Byte] = { 41 | val builder = AdditionalComparatorsProtos.CustomComparator.newBuilder() 42 | builder.setComparable(origConvert) 43 | builder.build.toByteArray 44 | } 45 | } 46 | 47 | object IntComparator { 48 | /** 49 | * @param pbBytes A pb serialized { @link IntComparator} instance 50 | * @return An instance of { @link IntComparator} made from bytes 51 | * @see #toByteArray 52 | */ 53 | def parseFrom(pbBytes: Array[Byte]): IntComparator = { 54 | var proto: AdditionalComparatorsProtos.CustomComparator = null 55 | try { 56 | proto = AdditionalComparatorsProtos.CustomComparator.parseFrom(pbBytes) 57 | } 58 | catch { 59 | case e: InvalidProtocolBufferException => 60 | throw new DeserializationException(e) 61 | } 62 | new IntComparator(proto.getComparable.getValue.toByteArray) 63 | } 64 | } 65 | 66 | class IntComparator(value: Array[Byte]) extends CustomComparator(value) { 67 | override def compareTo(value: HBaseRawType, offset: Int, length: Int): Int = { 68 | StringBytesUtils.toInt(this.value, 0, this.value.length) - 69 | StringBytesUtils.toInt(value, offset, length) 70 | } 71 | } 72 | 73 | object ByteComparator { 74 | /** 75 | * @param pbBytes A pb serialized { @link ByteComparator} instance 76 | * @return An instance of { @link ByteComparator} made from bytes 77 | * @see #toByteArray 78 | */ 79 | def parseFrom(pbBytes: Array[Byte]): ByteComparator = { 80 | var proto: AdditionalComparatorsProtos.CustomComparator = null 81 | try { 82 | proto = AdditionalComparatorsProtos.CustomComparator.parseFrom(pbBytes) 83 | } 84 | catch { 85 | case e: InvalidProtocolBufferException => 86 | throw new DeserializationException(e) 87 | } 88 | new ByteComparator(proto.getComparable.getValue.toByteArray) 89 | } 90 | } 91 | 92 | class ByteComparator(value: Array[Byte]) extends CustomComparator(value) { 93 | override def compareTo(value: HBaseRawType, offset: Int, length: Int): Int = { 94 | StringBytesUtils.toByte(this.value, 0, this.value.length) - 95 | StringBytesUtils.toByte(value, offset, length) 96 | } 97 | } 98 | 99 | object ShortComparator { 100 | /** 101 | * @param pbBytes A pb serialized { @link ShortComparator} instance 102 | * @return An instance of { @link ShortComparator} made from bytes 103 | * @see #toByteArray 104 | */ 105 | def parseFrom(pbBytes: Array[Byte]): ShortComparator = { 106 | var proto: AdditionalComparatorsProtos.CustomComparator = null 107 | try { 108 | proto = AdditionalComparatorsProtos.CustomComparator.parseFrom(pbBytes) 109 | } 110 | catch { 111 | case e: InvalidProtocolBufferException => 112 | throw new DeserializationException(e) 113 | } 114 | new ShortComparator(proto.getComparable.getValue.toByteArray) 115 | } 116 | } 117 | 118 | class ShortComparator(value: Array[Byte]) extends CustomComparator(value) { 119 | override def compareTo(value: HBaseRawType, offset: Int, length: Int): Int = { 120 | StringBytesUtils.toShort(this.value, 0, this.value.length) - 121 | StringBytesUtils.toShort(value, offset, length) 122 | } 123 | } 124 | 125 | object LongComparator { 126 | /** 127 | * @param pbBytes A pb serialized { @link LongComparator} instance 128 | * @return An instance of { @link LongComparator} made from bytes 129 | * @see #toByteArray 130 | */ 131 | def parseFrom(pbBytes: Array[Byte]): LongComparator = { 132 | var proto: AdditionalComparatorsProtos.CustomComparator = null 133 | try { 134 | proto = AdditionalComparatorsProtos.CustomComparator.parseFrom(pbBytes) 135 | } 136 | catch { 137 | case e: InvalidProtocolBufferException => 138 | throw new DeserializationException(e) 139 | } 140 | new LongComparator(proto.getComparable.getValue.toByteArray) 141 | } 142 | } 143 | 144 | class LongComparator(value: Array[Byte]) extends CustomComparator(value) { 145 | override def compareTo(value: HBaseRawType, offset: Int, length: Int): Int = { 146 | val r = StringBytesUtils.toLong(this.value, 0, this.value.length) - 147 | StringBytesUtils.toLong(value, offset, length) 148 | if (r > 0) 1 149 | else if (r == 0) 0 150 | else -1 151 | } 152 | } 153 | 154 | object DoubleComparator { 155 | /** 156 | * @param pbBytes A pb serialized { @link DoubleComparator} instance 157 | * @return An instance of { @link DoubleComparator} made from bytes 158 | * @see #toByteArray 159 | */ 160 | def parseFrom(pbBytes: Array[Byte]): DoubleComparator = { 161 | var proto: AdditionalComparatorsProtos.CustomComparator = null 162 | try { 163 | proto = AdditionalComparatorsProtos.CustomComparator.parseFrom(pbBytes) 164 | } 165 | catch { 166 | case e: InvalidProtocolBufferException => 167 | throw new DeserializationException(e) 168 | } 169 | new DoubleComparator(proto.getComparable.getValue.toByteArray) 170 | } 171 | } 172 | 173 | class DoubleComparator(value: Array[Byte]) extends CustomComparator(value) { 174 | override def compareTo(value: HBaseRawType, offset: Int, length: Int): Int = { 175 | val r = StringBytesUtils.toDouble(this.value, 0, this.value.length) - 176 | StringBytesUtils.toDouble(value, offset, length) 177 | if (r > 0) 1 178 | else if (r == 0) 0 179 | else -1 180 | } 181 | } 182 | 183 | object FloatComparator { 184 | /** 185 | * @param pbBytes A pb serialized { @link FloatComparator} instance 186 | * @return An instance of { @link FloatComparator} made from bytes 187 | * @see #toByteArray 188 | */ 189 | def parseFrom(pbBytes: Array[Byte]): FloatComparator = { 190 | var proto: AdditionalComparatorsProtos.CustomComparator = null 191 | try { 192 | proto = AdditionalComparatorsProtos.CustomComparator.parseFrom(pbBytes) 193 | } 194 | catch { 195 | case e: InvalidProtocolBufferException => 196 | throw new DeserializationException(e) 197 | } 198 | new FloatComparator(proto.getComparable.getValue.toByteArray) 199 | } 200 | } 201 | 202 | class FloatComparator(value: Array[Byte]) extends CustomComparator(value) { 203 | override def compareTo(value: HBaseRawType, offset: Int, length: Int): Int = { 204 | val r = StringBytesUtils.toFloat(this.value, 0, this.value.length) - 205 | StringBytesUtils.toFloat(value, offset, length) 206 | if (r > 0) 1 207 | else if (r == 0) 0 208 | else -1 209 | } 210 | } 211 | 212 | object BoolComparator { 213 | /** 214 | * @param pbBytes A pb serialized { @link BoolComparator} instance 215 | * @return An instance of { @link BoolComparator} made from bytes 216 | * @see #toByteArray 217 | */ 218 | def parseFrom(pbBytes: Array[Byte]): BoolComparator = { 219 | var proto: AdditionalComparatorsProtos.CustomComparator = null 220 | try { 221 | proto = AdditionalComparatorsProtos.CustomComparator.parseFrom(pbBytes) 222 | } 223 | catch { 224 | case e: InvalidProtocolBufferException => 225 | throw new DeserializationException(e) 226 | } 227 | new BoolComparator(proto.getComparable.getValue.toByteArray) 228 | } 229 | } 230 | 231 | class BoolComparator(value: Array[Byte]) extends CustomComparator(value) { 232 | override def compareTo(value: HBaseRawType, offset: Int, length: Int): Int = { 233 | if (StringBytesUtils.toBoolean(this.value, 0, this.value.length)) 1 234 | else -1 235 | } 236 | } 237 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/types/PartialOrderingDataType.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 | package org.apache.spark.sql.types 18 | 19 | import org.apache.spark.sql.types._ 20 | 21 | import scala.annotation.meta.getter 22 | import scala.reflect.runtime.universe.TypeTag 23 | 24 | abstract class PartialOrderingDataType extends DataType { 25 | private[sql] type InternalType 26 | def toPartiallyOrderingDataType(s: Any, dt: AtomicType): InternalType 27 | @(transient @getter) private[sql] val tag: TypeTag[InternalType] 28 | private[sql] val partialOrdering: PartialOrdering[InternalType] 29 | } 30 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/types/RangeType.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 | package org.apache.spark.sql.types 18 | 19 | import java.util.concurrent.ConcurrentHashMap 20 | 21 | import org.apache.spark.sql.catalyst.ScalaReflectionLock 22 | import org.apache.spark.sql.types._ 23 | 24 | import scala.language.implicitConversions 25 | import scala.math.PartialOrdering 26 | import scala.reflect.runtime.universe.typeTag 27 | 28 | class Range[+T](val start: Option[T], // None for open ends 29 | val startInclusive: Boolean, 30 | val end: Option[T], // None for open ends 31 | val endInclusive: Boolean, 32 | val dt: AtomicType) extends Serializable { 33 | require(dt != null && !(start.isDefined && end.isDefined && 34 | ((dt.ordering.eq(start.get, end.get) && 35 | (!startInclusive || !endInclusive)) || 36 | dt.ordering.gt(start.get.asInstanceOf[dt.InternalType], end.get.asInstanceOf[dt.InternalType]))), 37 | "Inappropriate range parameters") 38 | @transient lazy val isPoint: Boolean = start.isDefined && end.isDefined && 39 | startInclusive && endInclusive && start.get.equals(end.get) 40 | } 41 | 42 | private[sql] class RangeType[T] extends PartialOrderingDataType { 43 | override def defaultSize: Int = 4096 44 | private[sql] type InternalType = Range[T] 45 | @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } 46 | 47 | private[spark] override def asNullable: RangeType[T] = this 48 | 49 | /** 50 | * Convert a value to a point range 51 | * @param s value to be converted from 52 | * @param dt runtime type 53 | * @return 54 | */ 55 | override def toPartiallyOrderingDataType(s: Any, dt: AtomicType): InternalType = s match { 56 | case r: InternalType => r 57 | case _ => 58 | new Range(Some(s.asInstanceOf[T] ), true, Some(s.asInstanceOf[T] ), true, dt) 59 | } 60 | 61 | val partialOrdering = new PartialOrdering[InternalType] { 62 | // Right now we just support comparisons between a range and a point 63 | // In the future when more generic range comparisons, these two methods 64 | // must be functional as expected 65 | // return -2 if a < b; -1 if a <= b; 0 if a = b; 1 if a >= b; 2 if a > b 66 | def tryCompare(a: InternalType, b: InternalType): Option[Int] = { 67 | val aRange = a.asInstanceOf[Range[T]] 68 | val aStartInclusive = aRange.startInclusive 69 | val aStart = aRange.start.getOrElse(null).asInstanceOf[aRange.dt.InternalType] 70 | val aEnd = aRange.end.getOrElse(null).asInstanceOf[aRange.dt.InternalType] 71 | val aEndInclusive = aRange.endInclusive 72 | val bRange = b.asInstanceOf[Range[T]] 73 | val bStart = bRange.start.getOrElse(null).asInstanceOf[aRange.dt.InternalType] 74 | val bEnd = bRange.end.getOrElse(null).asInstanceOf[aRange.dt.InternalType] 75 | val bStartInclusive = bRange.startInclusive 76 | val bEndInclusive = bRange.endInclusive 77 | 78 | // return 1 iff aStart > bEnd 79 | // return 1 iff aStart = bEnd, aStartInclusive & bEndInclusive are not true at same position 80 | if ((aStart != null && bEnd != null) 81 | && (aRange.dt.ordering.gt(aStart, bEnd) 82 | || (aRange.dt.ordering.equiv(aStart, bEnd) && !(aStartInclusive && bEndInclusive)))) { 83 | Some(2) 84 | } // Vice versa 85 | else if ((bStart != null && aEnd != null) 86 | && (aRange.dt.ordering.gt(bStart, aEnd) 87 | || (aRange.dt.ordering.equiv(bStart, aEnd) && !(bStartInclusive && aEndInclusive)))) { 88 | Some(-2) 89 | } else if (aStart != null && aEnd != null && bStart != null && bEnd != null && 90 | aRange.dt.ordering.equiv(bStart, aEnd) 91 | && aRange.dt.ordering.equiv(aStart, aEnd) 92 | && aRange.dt.ordering.equiv(bStart, bEnd) 93 | && (aStartInclusive && aEndInclusive && bStartInclusive && bEndInclusive)) { 94 | Some(0) 95 | } else if (aEnd != null && bStart != null && aRange.dt.ordering.equiv(aEnd, bStart) 96 | && aEndInclusive && bStartInclusive) { 97 | Some(-1) 98 | } else if (aStart != null && bEnd != null && aRange.dt.ordering.equiv(aStart, bEnd) 99 | && aStartInclusive && bEndInclusive) { 100 | Some(1) 101 | } else { 102 | None 103 | } 104 | } 105 | 106 | def lteq(a: InternalType, b: InternalType): Boolean = { 107 | // [(aStart, aEnd)] and [(bStart, bEnd)] 108 | // [( and )] mean the possibilities of the inclusive and exclusive condition 109 | val aRange = a.asInstanceOf[Range[T]] 110 | val aStartInclusive = aRange.startInclusive 111 | val aEnd = if (aRange.end.isEmpty) null else aRange.end.get 112 | val aEndInclusive = aRange.endInclusive 113 | val bRange = b.asInstanceOf[Range[T]] 114 | val bStart = if (bRange.start.isEmpty) null else bRange.start.get 115 | val bStartInclusive = bRange.startInclusive 116 | val bEndInclusive = bRange.endInclusive 117 | 118 | // Compare two ranges, return true iff the upper bound of the lower range is lteq to 119 | // the lower bound of the upper range. Because the exclusive boundary could be null, which 120 | // means the boundary could be infinity, we need to further check this conditions. 121 | val result = 122 | (aStartInclusive, aEndInclusive, bStartInclusive, bEndInclusive) match { 123 | // [(aStart, aEnd] compare to [bStart, bEnd)] 124 | case (_, true, true, _) => 125 | if (aRange.dt.ordering.lteq(aEnd.asInstanceOf[aRange.dt.InternalType], 126 | bStart.asInstanceOf[aRange.dt.InternalType])) { 127 | true 128 | } else { 129 | false 130 | } 131 | // [(aStart, aEnd] compare to (bStart, bEnd)] 132 | case (_, true, false, _) => 133 | if (bStart != null && aRange.dt.ordering.lteq(aEnd.asInstanceOf[aRange.dt.InternalType], 134 | bStart.asInstanceOf[aRange.dt.InternalType])) { 135 | true 136 | } else { 137 | false 138 | } 139 | // [(aStart, aEnd) compare to [bStart, bEnd)] 140 | case (_, false, true, _) => 141 | if (aEnd != null && aRange.dt.ordering.lteq(aEnd.asInstanceOf[aRange.dt.InternalType], 142 | bStart.asInstanceOf[aRange.dt.InternalType])) { 143 | true 144 | } else { 145 | false 146 | } 147 | // [(aStart, aEnd) compare to (bStart, bEnd)] 148 | case (_, false, false, _) => 149 | if (aEnd != null && bStart != null && 150 | aRange.dt.ordering.lteq(aEnd.asInstanceOf[aRange.dt.InternalType], 151 | bStart.asInstanceOf[aRange.dt.InternalType])) { 152 | true 153 | } else { 154 | false 155 | } 156 | } 157 | 158 | result 159 | } 160 | } 161 | } 162 | 163 | object RangeType { 164 | import scala.collection.JavaConverters._ 165 | import scala.reflect.runtime.universe.TypeTag 166 | private val typeMap = new ConcurrentHashMap[TypeTag[_], RangeType[_]].asScala 167 | 168 | implicit class partialOrdering(dt: AtomicType) { 169 | private[sql] def toRangeType[T]: RangeType[T] = 170 | typeMap.getOrElseUpdate(dt.tag, new RangeType[T]).asInstanceOf[RangeType[T]] 171 | } 172 | } 173 | -------------------------------------------------------------------------------- /src/test/java/org/apache/spark/sql/hbase/api/java/JavaAPISuite.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 org.apache.spark.sql.hbase.api.java; 19 | 20 | import java.util.List; 21 | 22 | import org.apache.hadoop.hbase.HBaseTestingUtility; 23 | import org.apache.spark.SparkConf; 24 | import org.apache.spark.api.java.JavaSparkContext; 25 | import org.apache.spark.sql.Row; 26 | import org.apache.spark.sql.SparkSession; 27 | import org.apache.spark.sql.hbase.HBaseSparkSession; 28 | import org.apache.spark.sql.hbase.TestBase; 29 | import org.junit.After; 30 | import org.junit.Before; 31 | import org.junit.Test; 32 | 33 | import java.io.Serializable; 34 | 35 | public class JavaAPISuite extends TestBase implements Serializable { 36 | private transient SparkSession ss; 37 | private transient HBaseTestingUtility testUtil = null; 38 | private transient JavaSparkContext sc = null; 39 | 40 | @Before 41 | public void setUp() { 42 | System.setProperty("spark.hadoop.hbase.zookeeper.quorum", "localhost"); 43 | SparkConf scf = new SparkConf(true); 44 | sc = new JavaSparkContext("local", "JavaAPISuite", scf); 45 | ss = new HBaseSparkSession(sc); 46 | testUtil = new HBaseTestingUtility(ss.sparkContext().hadoopConfiguration()); 47 | try { 48 | testUtil.cleanupTestDir(); 49 | testUtil.startMiniZKCluster(); 50 | testUtil.startMiniHBaseCluster(1, 1); 51 | } catch (Exception e) { 52 | e.printStackTrace(); 53 | } 54 | } 55 | 56 | @After 57 | public void tearDown() { 58 | try { 59 | testUtil.shutdownMiniHBaseCluster(); 60 | testUtil.shutdownMiniZKCluster(); 61 | testUtil.cleanupTestDir(); 62 | } catch (Exception e) { 63 | throw new RuntimeException(e); 64 | } 65 | ss = null; 66 | sc.stop(); 67 | sc = null; 68 | } 69 | 70 | @Test 71 | public void testCreateInsertRetrieveTable() { 72 | final String hb_staging_table = "HbStagingTable"; 73 | final String staging_table = "StagingTable"; 74 | final String insert_sql = "INSERT INTO TABLE " + staging_table + " VALUES (\"strcol\" , \"bytecol\" , \"shortcol\" , \"intcol\" ," + 75 | " \"longcol\" , \"floatcol\" , \"doublecol\")"; 76 | final String retrieve_sql = "SELECT * FROM " + staging_table; 77 | String create_sql = "CREATE TABLE " + staging_table + " (strcol STRING, bytecol STRING, shortcol STRING, intcol STRING, longcol STRING, floatcol STRING, doublecol STRING) TBLPROPERTIES(" + 78 | "'hbaseTableName'='" + hb_staging_table +"'," + 79 | "'keyCols'='doublecol;strcol;intcol'," + 80 | "'nonKeyCols'='bytecol,cf1,hbytecol;shortcol,cf1,hshortcol;longcol,cf2,hlongcol;floatcol,cf2,hfloatcol')"; 81 | ss.sql(create_sql).collect(); 82 | ss.sql(insert_sql).collect(); 83 | List rows = ss.sql(retrieve_sql).collectAsList(); 84 | 85 | assert (rows.get(0).toString().equals("[strcol,bytecol,shortcol,intcol,longcol,floatcol,doublecol]")); 86 | } 87 | } 88 | -------------------------------------------------------------------------------- /src/test/resources/131_regions.txt: -------------------------------------------------------------------------------- 1 | 1,0 2 | 2,0 3 | 3,0 4 | 4,0 5 | 5,0 6 | 6,0 7 | 7,0 8 | 8,0 9 | 9,0 10 | 10,0 11 | 11,0 12 | 12,0 13 | 13,0 14 | 14,0 15 | 15,0 16 | 16,0 17 | 17,0 18 | 18,0 19 | 19,0 20 | 20,0 21 | 21,0 22 | 22,0 23 | 23,0 24 | 24,0 25 | 25,0 26 | 26,0 27 | 27,0 28 | 28,0 29 | 29,0 30 | 30,0 31 | 31,0 32 | 32,0 33 | 33,0 34 | 34,0 35 | 35,0 36 | 36,0 37 | 37,0 38 | 38,0 39 | 39,0 40 | 40,0 41 | 41,0 42 | 42,0 43 | 43,0 44 | 44,0 45 | 45,0 46 | 46,0 47 | 47,0 48 | 48,0 49 | 49,0 50 | 50,0 51 | 51,0 52 | 52,0 53 | 53,0 54 | 54,0 55 | 55,0 56 | 56,0 57 | 57,0 58 | 58,0 59 | 59,0 60 | 60,0 61 | 61,0 62 | 62,0 63 | 63,0 64 | 64,0 65 | 65,0 66 | 66,0 67 | 67,0 68 | 68,0 69 | 69,0 70 | 70,0 71 | 71,0 72 | 72,0 73 | 73,0 74 | 74,0 75 | 75,0 76 | 76,0 77 | 77,0 78 | 78,0 79 | 79,0 80 | 80,0 81 | 81,0 82 | 82,0 83 | 83,0 84 | 84,0 85 | 85,0 86 | 86,0 87 | 87,0 88 | 88,0 89 | 89,0 90 | 90,0 91 | 91,0 92 | 92,0 93 | 93,0 94 | 94,0 95 | 95,0 96 | 96,0 97 | 97,0 98 | 98,0 99 | 99,0 100 | 100,0 101 | 101,0 102 | 102,0 103 | 103,0 104 | 104,0 105 | 105,0 106 | 106,0 107 | 107,0 108 | 108,0 109 | 109,0 110 | 110,0 111 | 111,0 112 | 112,0 113 | 113,0 114 | 114,0 115 | 115,0 116 | 116,0 117 | 117,0 118 | 118,0 119 | 119,0 120 | 120,0 121 | 121,0 122 | 122,0 123 | 123,0 124 | 124,0 125 | 125,0 126 | 126,0 127 | 127,0 128 | 128,0 129 | 129,0 130 | 130,0 131 | 131,0 132 | 132,0 133 | 133,0 134 | 134,0 135 | 135,0 136 | 136,0 137 | 137,0 138 | 138,0 139 | 139,0 140 | 140,0 141 | 141,0 142 | 142,0 143 | 143,0 144 | 144,0 145 | 145,0 146 | 146,0 147 | 147,0 148 | 148,0 149 | 149,0 150 | 150,0 151 | 151,0 152 | 152,0 153 | 153,0 154 | 154,0 155 | 155,0 156 | 156,0 157 | 157,0 158 | 158,0 159 | 159,0 160 | 160,0 161 | 161,0 162 | 162,0 163 | 163,0 164 | 164,0 165 | 165,0 166 | 166,0 167 | 167,0 168 | 168,0 169 | 169,0 170 | 170,0 171 | 171,0 172 | 172,0 173 | 173,0 174 | 174,0 175 | 175,0 176 | 176,0 177 | 177,0 178 | 178,0 179 | 179,0 180 | 180,0 181 | 181,0 182 | 182,0 183 | 183,0 184 | 184,0 185 | 185,0 186 | 186,0 187 | 187,0 188 | 188,0 189 | 189,0 190 | 190,0 191 | 191,0 192 | 192,0 193 | 193,0 194 | 194,0 195 | 195,0 196 | 196,0 197 | 197,0 198 | 198,0 199 | 199,0 200 | 200,0 201 | 201,0 202 | 202,0 203 | 203,0 204 | 204,0 205 | 205,0 206 | 206,0 207 | 207,0 208 | 208,0 209 | 209,0 210 | 210,0 211 | 211,0 212 | 212,0 213 | 213,0 214 | 214,0 215 | 215,0 216 | 216,0 217 | 217,0 218 | 218,0 219 | 219,0 220 | 220,0 221 | 221,0 222 | 222,0 223 | 223,0 224 | 224,0 225 | 225,0 226 | 226,0 227 | 227,0 228 | 228,0 229 | 229,0 230 | 230,0 231 | 231,0 232 | 232,0 233 | 233,0 234 | 234,0 235 | 235,0 236 | 236,0 237 | 237,0 238 | 238,0 239 | 239,0 240 | 240,0 241 | 241,0 242 | 242,0 243 | 243,0 244 | 244,0 245 | 245,0 246 | 246,0 247 | 247,0 248 | 248,0 249 | 249,0 250 | 250,0 251 | 251,0 252 | 252,0 253 | 253,0 254 | 254,0 255 | 255,0 256 | 256,0 257 | 257,0 258 | 258,0 259 | 259,0 260 | 260,0 261 | -------------------------------------------------------------------------------- /src/test/resources/cf.txt: -------------------------------------------------------------------------------- 1 | 1,101,1001,-1,-101 2,102,1002,-2,-102 3,103,1003,-3,-103 4,104,1004,-4,-104 5,105,1005,-5,-105 6,106,1006,-6,-106 7,107,1007,-7,-107 8,108,1008,-8,-108 9,109,1009,-9,-109 10,110,1010,-10,-110 11,111,1011,-11,-111 12,112,1012,-12,-112 13,113,1013,-13,-113 14,114,1014,-14,-114 15,115,1015,-15,-115 16,116,1016,-16,-116 17,117,1017,-17,-117 18,118,1018,-18,-118 19,119,1019,-19,-119 20,120,1020,-20,-120 21,121,1021,-21,-121 22,122,1022,-22,-122 23,123,1023,-23,-123 24,124,1024,-24,-124 25,125,1025,-25,-125 26,126,1026,-26,-126 27,127,1027,-27,-127 -------------------------------------------------------------------------------- /src/test/resources/joinTable1.txt: -------------------------------------------------------------------------------- 1 | RowA1,a,12345,23456789,3456789012345,45657.89, 5678912.345678 2 | RowA2,a,12346,23456790,3456789012346,45657.90, 5678912.345679 3 | Row2,b,12342,23456782,3456789012342,45657.82, 5678912.345682 4 | Row3,c,12343,23456783,3456789012343,45657.83, 5678912.345683 5 | Row4,d,12344,23456784,3456789012344,45657.84, 5678912.345684 6 | Row5,e,12345,23456785,3456789012345,45657.85, 5678912.345685 7 | Row6,f,12346,23456786,3456789012346,45657.86, 5678912.345686 8 | Row7,g,12347,23456787,3456789012347,45657.87, 5678912.345687 9 | Row8,h,12348,23456788,3456789012348,45657.88, 5678912.345688 10 | Row9,i,12349,23456789,3456789012349,45657.89, 5678912.345689 11 | RowA10a,j,12340,23456780,3456789012340,45657.80, 5678912.345690 12 | RowA10b,j,12341,23456781,3456789012341,45657.81, 5678912.345691 13 | RowA10c,j,12342,23456782,3456789012342,45657.82, 5678912.345692 14 | -------------------------------------------------------------------------------- /src/test/resources/joinTable2.txt: -------------------------------------------------------------------------------- 1 | RowB1,a,12345,23456789,3456789012345,45657.89, 5678912.345678 2 | Row2,b1,12342,23456782,3456789012342,45657.82, 5678912.345682 3 | Row2,b2,12342,23456782,3456789012342,45657.82, 5678912.345683 4 | Row2,b3,12342,23456782,3456789012342,45657.82, 5678912.345684 5 | Row2,b4,12342,23456782,3456789012342,45657.82, 5678912.345685 6 | Row3,c,12343,23456783,3456789012343,45657.83, 5678912.345683 7 | Row4,d,12344,23456784,3456789012344,45657.84, 5678912.345684 8 | Row5,e,12345,23456785,3456789012345,45657.85, 5678912.345685 9 | Row6,f,12346,23456786,3456789012346,45657.86, 5678912.345686 10 | Row7,g,12347,23456787,3456789012347,45657.87, 5678912.345687 11 | Row8,h,12348,23456788,3456789012348,45657.88, 5678912.345688 12 | Row9,i,12349,23456789,3456789012349,45657.89, 5678912.345689 13 | RowB10a,j,12340,23456780,3456789012340,45657.80, 5678912.345690 14 | RowB10b,k,12341,23456781,3456789012341,45657.81, 5678912.345691 -------------------------------------------------------------------------------- /src/test/resources/joinTable3.txt: -------------------------------------------------------------------------------- 1 | RowC1,a,12345,23456789,3456789012345,45657.89, 5678912.345678 2 | RowC2,a,12346,23456790,3456789012346,45657.90, 5678912.345679 3 | Row2,b,12342,23456782,3456789012342,45657.82, 5678912.345682 4 | Row3,c,12343,23456783,3456789012343,45657.83, 5678912.345683 5 | Row4,d,12344,23456784,3456789012344,45657.84, 5678912.345684 6 | Row5,e,12345,23456785,3456789012345,45657.85, 5678912.345685 7 | Row6,f,12346,23456786,3456789012346,45657.86, 5678912.345686 8 | Row7,g,12347,23456787,3456789012347,45657.87, 5678912.345687 9 | Row8,h,12348,23456788,3456789012348,45657.88, 5678912.345688 10 | Row9,i,12349,23456789,3456789012349,45657.89, 5678912.345689 11 | RowC10a,j,12340,23456780,3456789012340,45657.80, 5678912.345690 12 | RowC10b,j,12341,23456781,3456789012341,45657.81, 5678912.345691 13 | RowC10c,j,12342,23456782,3456789012342,45657.82, 5678912.345692 14 | -------------------------------------------------------------------------------- /src/test/resources/joinTable4.txt: -------------------------------------------------------------------------------- 1 | RowD1,a,12345,23456789,3456789012345,45657.89, 5678912.345678 2 | RowD2,a,12346,23456790,3456789012346,45657.90, 5678912.345679 3 | Row2,b,12342,23456782,3456789012342,45657.82, 5678912.345682 4 | Row3,c,12343,23456783,3456789012343,45657.83, 5678912.345683 5 | Row4,d,12344,23456784,3456789012344,45657.84, 5678912.345684 6 | Row5,e,12345,23456785,3456789012345,45657.85, 5678912.345685 7 | Row6,f,12346,23456786,3456789012346,45657.86, 5678912.345686 8 | Row7,g,12347,23456787,3456789012347,45657.87, 5678912.345687 9 | Row8,h,12348,23456788,3456789012348,45657.88, 5678912.345688 10 | Row9,i,12349,23456789,3456789012349,45657.89, 5678912.345689 11 | RowD10a,j,12340,23456780,3456789012340,45657.80, 5678912.345690 12 | RowD10b,j,12341,23456781,3456789012341,45657.81, 5678912.345691 13 | RowD10c,j,12342,23456782,3456789012342,45657.82, 5678912.345692 14 | -------------------------------------------------------------------------------- /src/test/resources/loadData.txt: -------------------------------------------------------------------------------- 1 | row5,5,10 2 | row4,4,8 3 | row5,5,10 4 | row6,6,12 -------------------------------------------------------------------------------- /src/test/resources/loadNullableData.txt: -------------------------------------------------------------------------------- 1 | row1,,8,101 2 | row2,2,,102 3 | row3,3,10, 4 | row4,,, -------------------------------------------------------------------------------- /src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # 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 | # Set everything to be logged to the file core/target/unit-tests.log 19 | log4j.rootLogger=WARN,CA,FA 20 | 21 | #Console Appender 22 | log4j.appender.CA=org.apache.log4j.ConsoleAppender 23 | log4j.appender.CA.layout=org.apache.log4j.PatternLayout 24 | log4j.appender.CA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %p %c: %m%n 25 | log4j.appender.CA.Threshold = INFO 26 | 27 | 28 | #File Appender 29 | log4j.appender.FA=org.apache.log4j.FileAppender 30 | log4j.appender.FA.append=false 31 | log4j.appender.FA.file=target/unit-tests.log 32 | log4j.appender.FA.layout=org.apache.log4j.PatternLayout 33 | log4j.appender.FA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %p %c{1}: %m%n 34 | log4j.appender.FA.Threshold = INFO 35 | 36 | log4j.logger.org.mortbay=WARN 37 | 38 | log4j.logger.BlockStateChange=WARN 39 | log4j.logger.org.eclipse.jetty=WARN 40 | log4j.logger.org.apache.hadoop.hbase.ZNodeClearer=ERROR 41 | log4j.logger.org.apache.hadoop.hbase=WARN 42 | log4j.logger.org.apache.hadoop=WARN 43 | 44 | log4j.logger.org.apache.zookeeper=ERROR 45 | log4j.logger.org.apache.zookeeper.server=ERROR 46 | 47 | log4j.logger.org.apache.spark.sql.hbase=DEBUG 48 | log4j.logger.org.apache.spark=WARN 49 | log4j.logger.org.scalatest=WARN 50 | -------------------------------------------------------------------------------- /src/test/resources/onecoljoin1.txt: -------------------------------------------------------------------------------- 1 | 1 2 | 2 -------------------------------------------------------------------------------- /src/test/resources/onecoljoin2.txt: -------------------------------------------------------------------------------- 1 | 1 2 | 2 -------------------------------------------------------------------------------- /src/test/resources/people.txt: -------------------------------------------------------------------------------- 1 | 1,xiaoming,16,id_1,teacherW 2 | 2,xiaoming,16,id_2,teacherW 3 | 3,xiaoming,16,id_3,teacherW 4 | 4,xiaoming,16,id_4,teacherW 5 | 5,xiaoming,16,id_5,teacherW 6 | 6,xiaoming,16,id_6,teacherW 7 | 7,xiaoming,16,id_7,teacherW 8 | 8,xiaoming,16,id_8,teacherW 9 | 9,xiaoming,16,id_9,teacherW 10 | 10,xiaoming,16,id_10,teacherW 11 | 11,xiaoming,16,id_11,teacherW 12 | 12,xiaoming,16,id_12,teacherW 13 | 13,xiaoming,16,id_13,teacherW 14 | 14,xiaoming,16,id_14,teacherW 15 | 15,xiaoming,16,id_15,teacherW 16 | 16,xiaoming,16,id_16,teacherW 17 | 17,xiaoming,16,id_17,teacherW 18 | 18,xiaoming,16,id_18,teacherW 19 | 19,xiaoming,16,id_19,teacherW 20 | 1001,lihua,20,A1000, 21 | 1002,lihua,20,A1000, -------------------------------------------------------------------------------- /src/test/resources/splitLoadData.txt: -------------------------------------------------------------------------------- 1 | 1,6,val6 2 | 2,12,val12 3 | 3,18,val18 4 | 4,24,val24 5 | 5,30,val30 6 | 6,36,val36 7 | 7,42,val42 8 | 8,48,val48 9 | 9,54,val54 10 | 10,60,val60 11 | 11,66,val66 12 | 12,72,val72 13 | 13,78,val78 14 | 14,84,val84 15 | 15,90,val90 16 | 16,96,val96 -------------------------------------------------------------------------------- /src/test/resources/splitLoadData1.txt: -------------------------------------------------------------------------------- 1 | 1,0a,1024,v1 2 | 1024,0b,0,v2 3 | 2048,cc,1024,v3 4 | 4096,0a,0,v4 5 | 4096,0b,1024,v5 6 | 4096,cc,0,v6 7 | 4096,cc,1024,v7 -------------------------------------------------------------------------------- /src/test/resources/store_sales_stringformat.txt: -------------------------------------------------------------------------------- 1 | 01857000000007,2452260,46712,1857,147954,890396,791,97633,19,75,7,69,34.19,58.46,50.86,0.00,3509.34,2359.11,4033.74,210.56,0.00,3509.34,3719.90,1150.23 2 | 03163000000007,2452260,46712,3163,147954,890396,791,97633,19,22,7,82,69.53,87.60,34.16,0.00,2801.12,5701.46,7183.20,0.00,0.00,2801.12,2801.12,-2900.34 3 | 00707000000010,2451966,60226,707,180451,71288,6925,92515,13,145,10,83,10.26,17.33,9.18,0.00,761.94,851.58,1438.39,45.71,0.00,761.94,807.65,-89.64 4 | 16335000000010,2451966,60226,16335,180451,71288,6925,92515,13,175,10,66,82.35,137.52,82.51,0.00,5445.66,5435.10,9076.32,0.00,0.00,5445.66,5445.66,10.56 5 | 18669000000011,2452420,68961,18669,36900,781292,2484,71923,25,258,11,68,7.16,12.88,5.66,107.76,384.88,486.88,875.84,8.31,107.76,277.12,285.43,-209.76 6 | 00007000000029,2451121,45001,7,134631,1267519,54,67989,14,324,29,33,52.33,93.14,88.48,0.00,2919.84,1726.89,3073.62,29.19,0.00,2919.84,2949.03,1192.95 7 | 00574000000029,2451121,45001,574,134631,1267519,54,67989,14,342,29,33,68.24,116.69,57.17,1056.50,1886.61,2251.92,3850.77,66.40,1056.50,830.11,896.51,-1421.81 8 | 18814000000029,2451121,,18814,,,,,,,29,,95.63,184.56,,0.00,0.00,4398.98,8489.76,,0.00,,,-4398.98 9 | 01579000000030,2451390,52042,1579,75937,499127,7164,47554,28,231,30,21,64.00,81.28,29.26,0.00,614.46,1344.00,1706.88,36.86,0.00,614.46,651.32,-729.54 10 | 12919000000030,2451390,52042,12919,75937,499127,7164,47554,28,200,30,33,61.96,67.53,14.85,210.72,490.05,2044.68,2228.49,19.55,210.72,279.33,298.88,-1765.35 11 | -------------------------------------------------------------------------------- /src/test/resources/teacher.txt: -------------------------------------------------------------------------------- 1 | 1,1,subject_1,teacher_1_1_1,25 2 | 1,1,subject_2,teacher_1_2_1,25 3 | 1,1,subject_3,teacher_1_3_1,25 4 | 1,1,subject_4,teacher_1_4_1,25 5 | 1,1,subject_5,teacher_1_5_1,25 6 | 1,2,subject_1,teacher_1_1_1,25 7 | 1,2,subject_2,teacher_1_2_1,25 8 | 1,2,subject_3,teacher_1_3_1,25 9 | 1,2,subject_4,teacher_1_4_1,25 10 | 1,2,subject_5,teacher_1_5_1,25 11 | 1,3,subject_1,teacher_1_1_1,25 12 | 1,3,subject_2,teacher_1_2_1,25 -------------------------------------------------------------------------------- /src/test/resources/testTable.txt: -------------------------------------------------------------------------------- 1 | Row2,b,12342,23456782,3456789012342,45657.82, 5678912.345682 2 | Row4,d,12344,23456784,3456789012344,45657.84, 5678912.345684 3 | Row5,e,12345,23456785,3456789012345,45657.85, 5678912.345685 4 | Row7,g,12347,23456787,3456789012347,45657.87, 5678912.345687 5 | Row9,i,12349,23456789,3456789012349,45657.89, 5678912.345689 6 | Row0,j,12340,23456780,3456789012340,45657.80, 5678912.345690 7 | Row6,f,12346,23456786,3456789012346,45657.86, 5678912.345686 8 | Row3,c,12343,23456783,3456789012343,45657.83, 5678912.345683 9 | Row1,a,12345,23456789,3456789012345,45657.89, 5678912.345678 10 | Row8,h,12348,23456788,3456789012348,45657.88, 5678912.345688 11 | Row9,i,12349,23456789,3456789012349,45657.89, 5678912.345689 12 | 13 | -------------------------------------------------------------------------------- /src/test/resources/users.parquet/.part-r-00001.gz.parquet.crc: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/bomeng/Heracles/3a0096b991ef4534989cab37f881e885727a631d/src/test/resources/users.parquet/.part-r-00001.gz.parquet.crc -------------------------------------------------------------------------------- /src/test/resources/users.parquet/.part-r-00002.gz.parquet.crc: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/bomeng/Heracles/3a0096b991ef4534989cab37f881e885727a631d/src/test/resources/users.parquet/.part-r-00002.gz.parquet.crc -------------------------------------------------------------------------------- /src/test/resources/users.parquet/_SUCCESS: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/bomeng/Heracles/3a0096b991ef4534989cab37f881e885727a631d/src/test/resources/users.parquet/_SUCCESS -------------------------------------------------------------------------------- /src/test/resources/users.parquet/_common_metadata: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/bomeng/Heracles/3a0096b991ef4534989cab37f881e885727a631d/src/test/resources/users.parquet/_common_metadata -------------------------------------------------------------------------------- /src/test/resources/users.parquet/_metadata: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/bomeng/Heracles/3a0096b991ef4534989cab37f881e885727a631d/src/test/resources/users.parquet/_metadata -------------------------------------------------------------------------------- /src/test/resources/users.parquet/part-r-00001.gz.parquet: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/bomeng/Heracles/3a0096b991ef4534989cab37f881e885727a631d/src/test/resources/users.parquet/part-r-00001.gz.parquet -------------------------------------------------------------------------------- /src/test/resources/users.parquet/part-r-00002.gz.parquet: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/bomeng/Heracles/3a0096b991ef4534989cab37f881e885727a631d/src/test/resources/users.parquet/part-r-00002.gz.parquet -------------------------------------------------------------------------------- /src/test/scala/org/apache/spark/sql/hbase/BytesUtilsSuite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 org.apache.spark.sql.hbase 19 | 20 | import java.util.GregorianCalendar 21 | 22 | import org.apache.hadoop.hbase.util.Bytes 23 | import org.apache.spark.internal.Logging 24 | import org.apache.spark.sql.catalyst.util.DateTimeUtils 25 | import org.apache.spark.sql.hbase.types.HBaseBytesType 26 | import org.apache.spark.sql.hbase.util.BinaryBytesUtils 27 | import org.apache.spark.sql.types._ 28 | import org.apache.spark.unsafe.types.UTF8String 29 | import org.scalatest.{BeforeAndAfterAll, FunSuite} 30 | 31 | class BytesUtilsSuite extends FunSuite with BeforeAndAfterAll with Logging { 32 | test("Bytes Ordering Test") { 33 | val s = Seq(-257, -256, -255, -129, -128, -127, -64, -16, -4, -1, 34 | 0, 1, 4, 16, 64, 127, 128, 129, 255, 256, 257) 35 | val result = s.map(i => (i, BinaryBytesUtils.create(IntegerType).toBytes(i))) 36 | .sortWith((f, s) => 37 | HBaseBytesType.ordering.gt( 38 | f._2.asInstanceOf[HBaseBytesType.InternalType], 39 | s._2.asInstanceOf[HBaseBytesType.InternalType])) 40 | assert(result.map(a => a._1) == s.sorted.reverse) 41 | } 42 | 43 | def compare(a: Array[Byte], b: Array[Byte]): Int = { 44 | val length = Math.min(a.length, b.length) 45 | var result: Int = 0 46 | for (i <- 0 until length) { 47 | val diff: Int = (a(i) & 0xff).asInstanceOf[Byte] - (b(i) & 0xff).asInstanceOf[Byte] 48 | if (diff != 0) { 49 | result = diff 50 | } 51 | } 52 | result 53 | } 54 | 55 | test("Bytes Utility Test") { 56 | assert(BinaryBytesUtils.toBoolean(BinaryBytesUtils.create(BooleanType) 57 | .toBytes(input = true), 0) === true) 58 | assert(BinaryBytesUtils.toBoolean(BinaryBytesUtils.create(BooleanType) 59 | .toBytes(input = false), 0) === false) 60 | 61 | assert(BinaryBytesUtils.toDouble(BinaryBytesUtils.create(DoubleType).toBytes(12.34d), 0) 62 | === 12.34d) 63 | assert(BinaryBytesUtils.toDouble(BinaryBytesUtils.create(DoubleType).toBytes(-12.34d), 0) 64 | === -12.34d) 65 | 66 | assert(BinaryBytesUtils.toFloat(BinaryBytesUtils.create(FloatType).toBytes(12.34f), 0) 67 | === 12.34f) 68 | assert(BinaryBytesUtils.toFloat(BinaryBytesUtils.create(FloatType).toBytes(-12.34f), 0) 69 | === -12.34f) 70 | 71 | assert(BinaryBytesUtils.toInt(BinaryBytesUtils.create(IntegerType).toBytes(12), 0) 72 | === 12) 73 | assert(BinaryBytesUtils.toInt(BinaryBytesUtils.create(IntegerType).toBytes(-12), 0) 74 | === -12) 75 | 76 | assert(BinaryBytesUtils.toLong(BinaryBytesUtils.create(LongType).toBytes(1234l), 0) 77 | === 1234l) 78 | assert(BinaryBytesUtils.toLong(BinaryBytesUtils.create(LongType).toBytes(-1234l), 0) 79 | === -1234l) 80 | 81 | assert(BinaryBytesUtils.toShort(BinaryBytesUtils.create(ShortType) 82 | .toBytes(12.asInstanceOf[Short]), 0) === 12) 83 | assert(BinaryBytesUtils.toShort(BinaryBytesUtils.create(ShortType) 84 | .toBytes(-12.asInstanceOf[Short]), 0) === -12) 85 | 86 | assert(BinaryBytesUtils.toUTF8String(BinaryBytesUtils.create(StringType).toBytes("abc"), 0, 3) 87 | === UTF8String.fromString("abc")) 88 | assert(BinaryBytesUtils.toUTF8String(BinaryBytesUtils.create(StringType).toBytes(""), 0, 0) 89 | === UTF8String.fromString("")) 90 | 91 | assert(BinaryBytesUtils.toByte(BinaryBytesUtils.create(ByteType) 92 | .toBytes(5.asInstanceOf[Byte]), 0) === 5) 93 | assert(BinaryBytesUtils.toByte(BinaryBytesUtils.create(ByteType) 94 | .toBytes(-5.asInstanceOf[Byte]), 0) === -5) 95 | 96 | assert(compare(BinaryBytesUtils.create(IntegerType).toBytes(128), 97 | BinaryBytesUtils.create(IntegerType).toBytes(-128)) > 0) 98 | 99 | val date = new java.sql.Date(new java.util.Date().getTime) 100 | assert(BinaryBytesUtils.toDate(BinaryBytesUtils.create(DateType).toBytes(date), 0, 4) === 101 | DateTimeUtils.millisToDays(date.getTime)) 102 | } 103 | 104 | test("byte array plus one") { 105 | var byteArray = Array[Byte](0x01.toByte, 127.toByte) 106 | assert(Bytes.compareTo(BinaryBytesUtils.addOne(byteArray), Array[Byte](0x01.toByte, 0x80.toByte)) == 0) 107 | 108 | byteArray = Array[Byte](0xff.toByte, 0xff.toByte) 109 | assert(BinaryBytesUtils.addOne(byteArray) == null) 110 | 111 | byteArray = Array[Byte](0x02.toByte, 0xff.toByte) 112 | assert(Bytes.compareTo(BinaryBytesUtils.addOne(byteArray), Array[Byte](0x03.toByte, 0x00.toByte)) == 0) 113 | } 114 | 115 | test("value comparison") { 116 | val b1 = BinaryBytesUtils.create(ByteType).toBytes(-2.asInstanceOf[Byte]) 117 | val b2 = BinaryBytesUtils.create(ByteType).toBytes(4.asInstanceOf[Byte]) 118 | assert(Bytes.compareTo(b1, b2) < 0) 119 | 120 | val s1 = BinaryBytesUtils.create(ShortType).toBytes(-2.asInstanceOf[Short]) 121 | val s2 = BinaryBytesUtils.create(ShortType).toBytes(4.asInstanceOf[Short]) 122 | assert(Bytes.compareTo(s1, s2) < 0) 123 | 124 | val i1 = BinaryBytesUtils.create(IntegerType).toBytes(-2) 125 | val i2 = BinaryBytesUtils.create(IntegerType).toBytes(4) 126 | assert(Bytes.compareTo(i1, i2) < 0) 127 | 128 | val f1 = BinaryBytesUtils.create(FloatType).toBytes(-1.23f) 129 | val f2 = BinaryBytesUtils.create(FloatType).toBytes(100f) 130 | assert(Bytes.compareTo(f1, f2) < 0) 131 | 132 | val d1 = BinaryBytesUtils.create(DoubleType).toBytes(-1.23) 133 | val d2 = BinaryBytesUtils.create(DoubleType).toBytes(456.0) 134 | assert(Bytes.compareTo(d1, d2) < 0) 135 | 136 | val l1 = BinaryBytesUtils.create(DoubleType).toBytes(-123L) 137 | val l2 = BinaryBytesUtils.create(DoubleType).toBytes(456L) 138 | assert(Bytes.compareTo(l1, l2) < 0) 139 | 140 | val calendar1 = new GregorianCalendar(2013, 1, 28, 13, 24, 56) 141 | val calendar2 = new GregorianCalendar(2016, 12, 8, 15, 24, 56) 142 | 143 | val date1 = BinaryBytesUtils.create(DateType).toBytes(new java.sql.Date(calendar1.getTimeInMillis)) 144 | val date2 = BinaryBytesUtils.create(DateType).toBytes(new java.sql.Date(calendar2.getTimeInMillis)) 145 | assert(Bytes.compareTo(date1, date2) < 0) 146 | 147 | val timestamp1 = BinaryBytesUtils.create(TimestampType).toBytes(new java.sql.Timestamp(calendar1.getTimeInMillis)) 148 | val timestamp2 = BinaryBytesUtils.create(TimestampType).toBytes(new java.sql.Timestamp(calendar2.getTimeInMillis)) 149 | assert(Bytes.compareTo(timestamp1, timestamp2) < 0) 150 | } 151 | } 152 | -------------------------------------------------------------------------------- /src/test/scala/org/apache/spark/sql/hbase/HBaseAdvancedSQLQuerySuite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 org.apache.spark.sql.hbase 19 | 20 | import org.apache.spark.sql.internal.SQLConf 21 | import org.apache.spark.sql.types.{MetadataBuilder, StructType} 22 | import org.apache.spark.sql.{DataFrame, Row} 23 | 24 | class HBaseAdvancedSQLQuerySuite extends TestBaseWithSplitData { 25 | import org.apache.spark.sql.hbase.TestHbase._ 26 | import org.apache.spark.sql.hbase.TestHbase.implicits._ 27 | 28 | test("aggregation with codegen") { 29 | val originalValue = TestHbase.sessionState.conf.wholeStageEnabled 30 | TestHbase.sessionState.conf.setConfString(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") 31 | val result = sql("SELECT col1 FROM ta GROUP BY col1").collect() 32 | assert(result.length == 14, s"aggregation with codegen test failed on size") 33 | TestHbase.sessionState.conf.setConfString(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, originalValue.toString) 34 | } 35 | 36 | test("dsl simple select 0") { 37 | val tableA = sql("SELECT * FROM ta") 38 | checkAnswer( 39 | tableA.where('col7 === 1).orderBy('col2.asc).select('col4), 40 | Row(1) :: Nil) 41 | checkAnswer( 42 | tableA.where('col2 === 6).orderBy('col2.asc).select('col7), 43 | Row(-31) :: Nil) 44 | } 45 | 46 | test("metadata is propagated correctly") { 47 | val tableA = sql("SELECT col7, col1, col3 FROM ta") 48 | val schema = tableA.schema 49 | val docKey = "doc" 50 | val docValue = "first name" 51 | val metadata = new MetadataBuilder() 52 | .putString(docKey, docValue) 53 | .build() 54 | val schemaWithMeta = new StructType(Array( 55 | schema("col7"), schema("col1").copy(metadata = metadata), schema("col3"))) 56 | val personWithMeta = createDataFrame(tableA.rdd, schemaWithMeta) 57 | def validateMetadata(rdd: DataFrame): Unit = { 58 | assert(rdd.schema("col1").metadata.getString(docKey) == docValue) 59 | } 60 | personWithMeta.createOrReplaceTempView("personWithMeta") 61 | validateMetadata(personWithMeta.select($"col1")) 62 | validateMetadata(personWithMeta.select($"col1")) 63 | validateMetadata(personWithMeta.select($"col7", $"col1")) 64 | validateMetadata(sql("SELECT * FROM personWithMeta")) 65 | validateMetadata(sql("SELECT col7, col1 FROM personWithMeta")) 66 | validateMetadata(sql("SELECT * FROM personWithMeta JOIN salary ON col7 = personId")) 67 | validateMetadata(sql("SELECT col1, salary FROM personWithMeta JOIN salary ON col7 = personId")) 68 | } 69 | } 70 | -------------------------------------------------------------------------------- /src/test/scala/org/apache/spark/sql/hbase/HBaseAggregateQueriesSuite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 org.apache.spark.sql.hbase 19 | 20 | class HBaseAggregateQueriesSuite extends 21 | TestBaseWithNonSplitData { 22 | var testnm = "Group by with cols in select list and with order by" 23 | test("Group by with cols in select list and with order by") { 24 | val query = 25 | s"""SELECT count(1) as cnt, intcol, floatcol, strcol, max(bytecol) bytecol, max(shortcol) shortcol, 26 | max(floatcol) floatcolmax, max(doublecol) doublecol, max(longcol) from $TestTableName 27 | WHERE strcol LIKE '%Row%' AND shortcol < 12345 AND doublecol > 5678912.345681 28 | AND doublecol < 5678912.345684 29 | GROUP BY intcol, floatcol, strcol ORDER BY strcol DESC""" 30 | 31 | testGroupBy(testnm, query) 32 | } 33 | 34 | testnm = "Group by with cols in select list and with having and order by" 35 | test("Group by with cols in select list and with having and order by") { 36 | val query = s"""SELECT count(1) as cnt, intcol, floatcol, strcol, max(bytecol) bytecolmax, 37 | max(shortcol) shortcolmax, max(floatcol) floatcolmax, max(doublecol) doublecolmax, 38 | max(longcol) longcolmax 39 | FROM $TestTableName 40 | WHERE strcol like '%Row%' AND shortcol < 12345 AND doublecol > 5678912.345681 41 | AND doublecol < 5678912.345685 42 | GROUP BY intcol, floatcol, strcol 43 | HAVING max(doublecol) < 5678912.345684 44 | ORDER BY strcol DESC""".stripMargin 45 | testGroupBy(testnm, query) 46 | } 47 | 48 | def testGroupBy(testName: String, query: String) = { 49 | val result1 = runSql(query) 50 | assert(result1.length == 2, s"$testName failed on size") 51 | val exparr = Array( 52 | Array(1, 23456783, 45657.83F, "Row3", 'c', 12343, 45657.83F, 5678912.345683, 3456789012343L), 53 | Array(1, 23456782, 45657.82F, "Row2", 'b', 12342, 45657.82F, 5678912.345682, 3456789012342L)) 54 | 55 | val res = { 56 | for (rx <- exparr.indices) 57 | yield compareWithTol(result1(rx).toSeq, exparr(rx), s"Row$rx failed") 58 | }.foldLeft(true) { case (res1, newres) => res1 && newres} 59 | assert(res, "One or more rows did not match expected") 60 | 61 | logInfo(s"$query came back with $result1.length results") 62 | logInfo(result1.mkString) 63 | 64 | logInfo(s"Test $testName completed successfully") 65 | } 66 | 67 | testnm = "Another Group by with cols in select list and with having and order by" 68 | test("Another Group by with cols in select list and with having and order by") { 69 | val query1 = 70 | s"""SELECT count(1) as cnt, intcol, floatcol, strcol, max(bytecol) bytecolmax, max(shortcol) shortcolmax, 71 | max(floatcol) floatcolmax, max(doublecol) doublecolmax, max(longcol) longcolmax FROM $TestTableName 72 | WHERE strcol LIKE '%Row%' AND shortcol < 12345 AND doublecol > 5678912.345681 73 | AND doublecol < 5678912.345685 74 | GROUP BY intcol, floatcol, strcol HAVING max(doublecol) < 5678912.345684 ORDER BY strcol DESC""" 75 | .stripMargin 76 | 77 | val result1 = runSql(query1) 78 | assert(result1.length == 2, s"$testnm failed on size") 79 | val exparr = Array( 80 | Array(1, 23456783, 45657.83F, "Row3", 'c', 12343, 45657.83F, 5678912.345683, 3456789012343L), 81 | Array(1, 23456782, 45657.82F, "Row2", 'b', 12342, 45657.82F, 5678912.345682, 3456789012342L)) 82 | 83 | val res = { 84 | for (rx <- exparr.indices) 85 | yield compareWithTol(result1(rx).toSeq, exparr(rx), s"Row$rx failed") 86 | }.foldLeft(true) { case (res1, newres) => res1 && newres} 87 | assert(res, "One or more rows did not match expected") 88 | 89 | logInfo(s"$query1 came back with $result1.length results") 90 | logInfo(result1.mkString) 91 | 92 | logInfo(s"Test $testnm completed successfully") 93 | } 94 | } 95 | 96 | -------------------------------------------------------------------------------- /src/test/scala/org/apache/spark/sql/hbase/HBaseBasicOperationSuite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 org.apache.spark.sql.hbase 19 | 20 | import org.apache.hadoop.hbase.TableName 21 | import org.apache.spark.sql.catalyst.util.DateTimeUtils 22 | 23 | /** 24 | * Test insert / query against the table 25 | */ 26 | class HBaseBasicOperationSuite extends TestBaseWithSplitData { 27 | import org.apache.spark.sql.hbase.TestHbase._ 28 | 29 | override def afterAll() = { 30 | if (TestHbase.hbaseAdmin.tableExists(TableName.valueOf("ht0"))) { 31 | TestHbase.hbaseAdmin.disableTable(TableName.valueOf("ht0")) 32 | TestHbase.hbaseAdmin.deleteTable(TableName.valueOf("ht0")) 33 | } 34 | if (TestHbase.hbaseAdmin.tableExists(TableName.valueOf("ht1"))) { 35 | TestHbase.hbaseAdmin.disableTable(TableName.valueOf("ht1")) 36 | TestHbase.hbaseAdmin.deleteTable(TableName.valueOf("ht1")) 37 | } 38 | super.afterAll() 39 | } 40 | 41 | test("DateType test") { 42 | sql( """CREATE TABLE date_table (c1 DATE, c2 DATE) TBLPROPERTIES( 43 | 'hbaseTableName'='date_table', 44 | 'keyCols'='c1', 45 | 'nonKeyCols'='c2,f,c')""") 46 | sql("insert into table date_table values ('2010-12-31', '2010-01-01')") 47 | sql("insert into table date_table values ('2011-12-31', '2011-01-01')") 48 | sql("insert into table date_table values ('2012-12-31', '2012-01-01')") 49 | 50 | val result1 = sql("select * from date_table where c1 < cast('2012-12-31' as date) order by c2 desc").collect 51 | assert(result1.length == 2) 52 | 53 | val result2 = sql("select * from date_table where c2 < cast('2012-01-01' as date) order by c2 desc").collect 54 | assert(result2.length == 2) 55 | assert(result2(0).get(0).toString == "2011-12-31") 56 | assert(result2(0).get(1).toString == "2011-01-01") 57 | assert(result2(1).get(0).toString == "2010-12-31") 58 | assert(result2(1).get(1).toString == "2010-01-01") 59 | sql("drop table date_table") 60 | } 61 | 62 | test("TimestampType test") { 63 | sql( """CREATE TABLE ts_table (c1 TIMESTAMP, c2 TIMESTAMP) TBLPROPERTIES( 64 | 'hbaseTableName'='ts_table', 65 | 'keyCols'='c1', 66 | 'nonKeyCols'='c2,f,c')""") 67 | sql("insert into table ts_table values ('2009-08-07 03:14:15', '2009-08-07 13:14:15')") 68 | sql("insert into table ts_table values ('2010-08-07 03:14:15', '2010-08-07 13:14:15')") 69 | sql("insert into table ts_table values ('2011-08-07 03:14:15', '2011-08-07 13:14:15')") 70 | 71 | val result1 = sql("select * from ts_table where c1 < cast('2011-08-07 03:14:15' as timestamp) order by c2 desc").collect 72 | assert(result1.length == 2) 73 | 74 | val result2 = sql("select * from ts_table where c2 < cast('2011-08-07 13:14:15' as timestamp) order by c2 desc").collect 75 | assert(result2.length == 2) 76 | assert(result2(0).get(0).toString == "2010-08-07 03:14:15.0") 77 | assert(result2(0).get(1).toString == "2010-08-07 13:14:15.0") 78 | assert(result2(1).get(0).toString == "2009-08-07 03:14:15.0") 79 | assert(result2(1).get(1).toString == "2009-08-07 13:14:15.0") 80 | sql("drop table ts_table") 81 | } 82 | 83 | test("Insert Into table in StringFormat") { 84 | sql( """CREATE TABLE tb0 (column2 INTEGER, column1 INTEGER, column4 FLOAT, column3 SHORT) TBLPROPERTIES( 85 | 'hbaseTableName'='default.ht0', 86 | 'keyCols'='column1', 87 | 'nonKeyCols'='column2,family0,qualifier0;column3,family1,qualifier1;column4,family2,qualifier2')""") 88 | 89 | assert(sql( """SELECT * FROM tb0""").collect().length == 0) 90 | sql( """INSERT INTO TABLE tb0 SELECT col4,col4,col6,col3 FROM ta""") 91 | assert(sql( """SELECT * FROM tb0""").collect().length == 14) 92 | 93 | sql( """SELECT * FROM tb0""").show 94 | sql( """SELECT * FROM tb0 where column2 > 200""").show 95 | 96 | sql( """DROP TABLE tb0""") 97 | dropNativeHbaseTable("default.ht0") 98 | } 99 | 100 | test("Insert and Query Single Row") { 101 | sql( """CREATE TABLE tb1 (column1 INTEGER, column2 STRING) TBLPROPERTIES( 102 | 'hbaseTableName'='ht1', 103 | 'keyCols'='column1', 104 | 'nonKeyCols'='column2,cf,cq')""" 105 | ) 106 | 107 | assert(sql( """SELECT * FROM tb1""").collect().length == 0) 108 | sql( """INSERT INTO TABLE tb1 VALUES (1024, "abc")""") 109 | sql( """INSERT INTO TABLE tb1 VALUES (1028, "abd")""") 110 | assert(sql( """SELECT * FROM tb1""").collect().length == 2) 111 | assert( 112 | sql( """SELECT * FROM tb1 WHERE (column1 = 1023 AND column2 ="abc")""").collect().length == 0) 113 | assert(sql( 114 | """SELECT * FROM tb1 WHERE (column1 = 1024) 115 | |OR (column1 = 1028 AND column2 ="abd")""".stripMargin).collect().length == 2) 116 | 117 | sql( """DROP TABLE tb1""") 118 | dropNativeHbaseTable("ht1") 119 | } 120 | 121 | test("Insert and Query Single Row in StringFormat") { 122 | sql( """CREATE TABLE tb1 (col1 STRING,col2 BOOLEAN,col3 SHORT,col4 INTEGER,col5 LONG,col6 FLOAT,col7 DOUBLE) TBLPROPERTIES( 123 | 'hbaseTableName'='ht2', 124 | 'keyCols'='col1', 125 | 'nonKeyCols'='col2,cf1,cq11;col3,cf1,cq12;col4,cf1,cq13;col5,cf2,cq21;col6,cf2,cq22;col7,cf2,cq23')""".stripMargin 126 | ) 127 | 128 | assert(sql( """SELECT * FROM tb1""").collect().length == 0) 129 | sql( """INSERT INTO TABLE tb1 VALUES ("row1", false, 1000, 5050 , 50000 , 99.99 , 999.999)""") 130 | sql( """INSERT INTO TABLE tb1 VALUES ("row2", false, 99 , 10000, 9999 , 1000.1, 5000.5)""") 131 | sql( """INSERT INTO TABLE tb1 VALUES ("row3", true , 555 , 999 , 100000, 500.05, 10000.01)""") 132 | sql( """SELECT col1 FROM tb1 where col2 assert(r.getString(0) == s)} 134 | sql( """SELECT * FROM tb1 where col3>500 order by col3""") 135 | .collect().zip(Seq("row3", "row1")).foreach{case (r,s) => assert(r.getString(0) == s)} 136 | sql( """SELECT * FROM tb1 where col4>5000 order by col4""") 137 | .collect().zip(Seq("row1", "row2")).foreach{case (r,s) => assert(r.getString(0) == s)} 138 | sql( """SELECT * FROM tb1 where col5>50000 order by col5""") 139 | .collect().zip(Seq("row3")).foreach{case (r,s) => assert(r.getString(0) == s)} 140 | sql( """SELECT * FROM tb1 where col6>500 order by col6""") 141 | .collect().zip(Seq("row3", "row2")).foreach{case (r,s) => assert(r.getString(0) == s)} 142 | sql( """SELECT * FROM tb1 where col7>5000 order by col7""") 143 | .collect().zip(Seq("row2", "row3")).foreach{case (r,s) => assert(r.getString(0) == s)} 144 | 145 | sql( """DROP TABLE tb1""") 146 | dropNativeHbaseTable("ht2") 147 | } 148 | 149 | test("Select test 0") { 150 | assert(sql( """SELECT * FROM ta""").count() == 14) 151 | } 152 | 153 | test("Count(*/1) and Non-Key Column Query") { 154 | assert(sql( """SELECT count(*) FROM ta""").collect()(0).get(0) == 14) 155 | assert(sql( """SELECT count(*) FROM ta where col2 < 8""").collect()(0).get(0) == 7) 156 | assert(sql( """SELECT count(*) FROM ta where col4 < 0""").collect()(0).get(0) == 7) 157 | assert(sql( """SELECT count(1) FROM ta where col2 < 8""").collect()(0).get(0) == 7) 158 | assert(sql( """SELECT count(1) FROM ta where col4 < 0""").collect()(0).get(0) == 7) 159 | } 160 | 161 | test("InSet Query") { 162 | assert(sql( """SELECT count(*) FROM ta where col2 IN (1, 2, 3)""").collect()(0).get(0) == 3) 163 | assert(sql( """SELECT count(*) FROM ta where col4 IN (1, 2, 3)""").collect()(0).get(0) == 1) 164 | } 165 | 166 | test("Point Aggregate Query") { 167 | sql( """CREATE TABLE tb2 (column2 INTEGER,column1 INTEGER,column4 FLOAT,column3 SHORT) TBLPROPERTIES( 168 | 'hbaseTableName'='default.ht0', 169 | 'keyCols'='column1;column2', 170 | 'nonKeyCols'='column3,family1,qualifier1;column4,family2,qualifier2')""" 171 | ) 172 | sql( """INSERT INTO TABLE tb2 SELECT col4,col4,col6,col3 FROM ta""") 173 | val result = sql( """SELECT count(*) FROM tb2 where column1=1 AND column2=1""").collect() 174 | assert(result.length == 1) 175 | assert(result(0).get(0) == 1) 176 | } 177 | 178 | test("Select test 1 (AND, OR)") { 179 | assert(sql( """SELECT * FROM ta WHERE col7 = 255 OR col7 = 127""").collect().length == 2) 180 | assert(sql( """SELECT * FROM ta WHERE col7 < 0 AND col4 < -255""").collect().length == 4) 181 | } 182 | 183 | test("Select test 2 (WHERE)") { 184 | assert(sql( """SELECT * FROM ta WHERE col7 > 128""").count() == 3) 185 | assert(sql( """SELECT * FROM ta WHERE (col7 - 10 > 128) AND col1 = ' p255 '""").collect().length == 1) 186 | assert(sql( """SELECT * FROM ta WHERE (col7 > 1) AND (col7 < 1)""").collect().length == 0) 187 | assert(sql( """SELECT * FROM ta WHERE (col7 > 1) OR (col7 < 1)""").collect().length == 13) 188 | assert(sql( 189 | """SELECT * FROM ta WHERE 190 | |((col7 = 1) AND (col1 < ' p255 ') AND (col1 > ' p255 ')) OR 191 | |((col7 = 2) AND (col1 < ' p255 ') AND (col1 > ' p255 ')) 192 | """.stripMargin).collect().length == 0) 193 | assert(sql( 194 | """SELECT * FROM ta WHERE 195 | |((col7 = 1) AND (col3 < 128) AND (col3 > 128)) OR 196 | |((col7 = 2) AND (col3 < 127) AND (col3 > 127)) 197 | """.stripMargin).collect().length == 0) 198 | } 199 | 200 | test("Select test 3 (ORDER BY)") { 201 | val result = sql( """SELECT col1, col7 FROM ta ORDER BY col7 DESC""").collect() 202 | val sortedResult = result.sortWith( 203 | (r1, r2) => r1(1).asInstanceOf[Int] > r2(1).asInstanceOf[Int]) 204 | for ((r1, r2) <- result zip sortedResult) { 205 | assert(r1.equals(r2)) 206 | } 207 | } 208 | 209 | test("Select test 4 (join)") { 210 | val enabled = sqlContext.getConf("spark.sql.crossJoin.enabled") 211 | sqlContext.setConf("spark.sql.crossJoin.enabled", "true") 212 | assert(sql( """SELECT ta.col2 FROM ta join tb on ta.col4=tb.col7""").collect().length == 2) 213 | assert(sql( """SELECT * FROM ta FULL OUTER JOIN tb WHERE tb.col7 = 1""").collect().length == 14) 214 | assert(sql( """SELECT * FROM ta LEFT JOIN tb WHERE tb.col7 = 1""").collect().length == 14) 215 | assert(sql( """SELECT * FROM ta RIGHT JOIN tb WHERE tb.col7 = 1""").collect().length == 14) 216 | sqlContext.setConf("spark.sql.crossJoin.enabled", enabled) 217 | } 218 | 219 | // alter table is not supported for now 220 | ignore("Alter Add column and Alter Drop column") { 221 | assert(sql( """SELECT * FROM ta""").collect()(0).length == 7) 222 | sql( """ALTER TABLE ta ADD col8 STRING MAPPED BY (col8 = cf1.cf13)""") 223 | assert(sql( """SELECT * FROM ta""").collect()(0).length == 8) 224 | sql( """ALTER TABLE ta DROP col8""") 225 | assert(sql( """SELECT * FROM ta""").collect()(0).length == 7) 226 | } 227 | } 228 | -------------------------------------------------------------------------------- /src/test/scala/org/apache/spark/sql/hbase/HBaseBasicQueriesSuite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 org.apache.spark.sql.hbase 19 | 20 | class HBaseBasicQueriesSuite extends TestBaseWithNonSplitData { 21 | var testnm = "StarOperator * with limit" 22 | test("StarOperator * with limit") { 23 | val query1 = 24 | s"""SELECT * FROM $TestTableName LIMIT 3""" 25 | .stripMargin 26 | 27 | val result1 = runSql(query1) 28 | assert(result1.length == 3, s"$testnm failed on size") 29 | val exparr = Array(Array("Row1", 'a', 12345, 23456789, 3456789012345L, 45657.89F, 5678912.345678), 30 | Array("Row2", 'b', 12342, 23456782, 3456789012342L, 45657.82F, 5678912.345682), 31 | Array("Row3", 'c', 12343, 23456783, 3456789012343L, 45657.83F, 5678912.345683)) 32 | 33 | var res = { 34 | for (rx <- 0 until 3) 35 | yield compareWithTol(result1(rx).toSeq, exparr(rx), s"Row$rx failed") 36 | }.foldLeft(true) { case (res1, newres) => res1 && newres} 37 | assert(res, "One or more rows did not match expected") 38 | 39 | logInfo(s"$query1 came back with ${result1.length} results") 40 | logInfo(result1.mkString) 41 | 42 | val sql2 = 43 | s"""SELECT * FROM $TestTableName LIMIT 2""" 44 | .stripMargin 45 | 46 | val results = runSql(sql2) 47 | logInfo(s"$sql2 came back with ${results.length} results") 48 | assert(results.length == 2, s"$testnm failed assertion on size") 49 | res = { 50 | for (rx <- 0 until 2) 51 | yield compareWithTol(result1(rx).toSeq, exparr(rx), s"Row$rx failed") 52 | }.foldLeft(true) { case (res1, newres) => res1 && newres} 53 | logInfo(results.mkString) 54 | assert(res, "One or more rows did not match expected") 55 | 56 | logInfo(s"Test $testnm completed successfully") 57 | } 58 | 59 | testnm = "Select all cols with filter" 60 | test("Select all cols with filter") { 61 | val query1 = 62 | s"""SELECT * FROM $TestTableName WHERE shortcol < 12345 LIMIT 2""" 63 | .stripMargin 64 | 65 | val result1 = runSql(query1) 66 | logInfo(s"$query1 came back with ${result1.length} results") 67 | assert(result1.length == 2, s"$testnm failed on size") 68 | val exparr = Array( 69 | Array("Row2", 'b', 12342, 23456782, 3456789012342L, 45657.82F, 5678912.345682), 70 | Array("Row3", 'c', 12343, 23456783, 3456789012343L, 45657.83F, 5678912.345683)) 71 | 72 | val res = { 73 | for (rx <- 0 until 2) 74 | yield compareWithTol(result1(rx).toSeq, exparr(rx), s"Row$rx failed") 75 | }.foldLeft(true) { case (res1, newres) => res1 && newres} 76 | logInfo(result1.mkString) 77 | assert(res, "One or more rows did not match expected") 78 | 79 | logInfo(s"Test $testnm completed successfully") 80 | } 81 | 82 | testnm = "Select all cols with order by" 83 | test("Select all cols with order by") { 84 | val query1 = 85 | s"""SELECT * FROM $TestTableName WHERE shortcol < 12344 ORDER BY strcol DESC LIMIT 2""" 86 | .stripMargin 87 | 88 | val result1 = runSql(query1) 89 | assert(result1.length == 2, s"$testnm failed on size") 90 | val exparr = Array( 91 | Array("Row3", 'c', 12343, 23456783, 3456789012343L, 45657.83F, 5678912.345683), 92 | Array("Row2", 'b', 12342, 23456782, 3456789012342L, 45657.82F, 5678912.345682)) 93 | 94 | val res = { 95 | for (rx <- 0 until 2) 96 | yield compareWithTol(result1(rx).toSeq, exparr(rx), s"Row$rx failed") 97 | }.foldLeft(true) { case (res1, newres) => res1 && newres} 98 | assert(res, "One or more rows did not match expected") 99 | 100 | logInfo(s"Test $testnm completed successfully") 101 | } 102 | 103 | testnm = "Select same column twice" 104 | test("Select same column twice") { 105 | val query1 = 106 | s"""SELECT doublecol AS double1, doublecol AS doublecol 107 | | FROM $TestTableName 108 | | WHERE doublecol > 5678912.345681 AND doublecol < 5678912.345683""" 109 | .stripMargin 110 | 111 | val result1 = runSql(query1) 112 | logInfo(s"$query1 came back with ${result1.length} results") 113 | assert(result1.length == 1, s"$testnm failed on size") 114 | val exparr = Array( 115 | Array(5678912.345682, 5678912.345682)) 116 | 117 | assert(result1.length == 1, s"$testnm failed assertion on size") 118 | val res = { 119 | for (rx <- 0 until 1) 120 | yield compareWithTol(result1(rx).toSeq, exparr(rx), s"Row$rx failed") 121 | }.foldLeft(true) { case (res1, newres) => res1 && newres} 122 | logInfo(result1.mkString) 123 | assert(res, "One or more rows did not match expected") 124 | 125 | logInfo(s"Test $testnm completed successfully") 126 | } 127 | 128 | testnm = "Select specific cols with filter" 129 | test("Select specific cols with filter") { 130 | val query1 = 131 | s"""SELECT doublecol AS double1, -1 * doublecol AS minusdouble, 132 | | substr(strcol, 2) as substrcol, doublecol, strcol, 133 | | bytecol, shortcol, intcol, longcol, floatcol FROM $TestTableName WHERE strcol LIKE 134 | | '%Row%' AND shortcol < 12345 135 | | AND doublecol > 5678912.345681 AND doublecol < 5678912.345683 LIMIT 2""" 136 | .stripMargin 137 | 138 | val result1 = runSql(query1) 139 | logInfo(s"$query1 came back with ${result1.length} results") 140 | assert(result1.length == 1, s"$testnm failed on size") 141 | val exparr = Array( 142 | Array(5678912.345682, -5678912.345682, "ow2", 5678912.345682, 143 | "Row2", 'b', 12342, 23456782, 3456789012342L, 45657.82F)) 144 | 145 | assert(result1.length == 1, s"$testnm failed assertion on size") 146 | val res = { 147 | for (rx <- 0 until 1) 148 | yield compareWithTol(result1(rx).toSeq, exparr(rx), s"Row$rx failed") 149 | }.foldLeft(true) { case (res1, newres) => res1 && newres} 150 | logInfo(result1.mkString) 151 | assert(res, "One or more rows did not match expected") 152 | 153 | logInfo(s"Test $testnm completed successfully") 154 | } 155 | 156 | testnm = "Mixed And/or predicates" 157 | test("Mixed And/or predicates") { 158 | val query1 = s"""SELECT doublecol AS double1, -1 * doublecol AS minusdouble, 159 | substr(strcol, 2) AS substrcol, doublecol, strcol, 160 | bytecol, shortcol, intcol, longcol, floatcol FROM $TestTableName 161 | WHERE strcol LIKE '%Row%' 162 | AND shortcol < 12345 163 | AND doublecol > 5678912.345681 AND doublecol < 5678912.345683 164 | OR (doublecol = 5678912.345683 AND strcol IS NOT NULL) 165 | OR (doublecol = 5678912.345683 AND strcol IS NOT NULL or intcol > 12345 AND intcol < 0) 166 | OR (doublecol <> 5678912.345683 AND (strcol IS NULL or intcol > 12345 AND intcol < 0)) 167 | AND floatcol IS NOT NULL 168 | AND (intcol IS NOT NULL and intcol > 0) 169 | AND (intcol < 0 OR intcol IS NOT NULL)""".stripMargin 170 | 171 | val result1 = runSql(query1) 172 | logInfo(s"$query1 came back with ${result1.length} results") 173 | assert(result1.length == 2, s"$testnm failed on size") 174 | val exparr = Array( 175 | Array(5678912.345682, -5678912.345682, "ow2", 5678912.345682, 176 | "Row2", 'b', 12342, 23456782, 3456789012342L, 45657.82F), 177 | Array(5678912.345683, -5678912.345683, "ow3", 5678912.345683, 178 | "Row3", -29, 12343, 23456783, 3456789012343L, 45657.83)) 179 | 180 | val res = { 181 | for (rx <- 0 until 1) 182 | yield compareWithTol(result1(rx).toSeq, exparr(rx), s"Row$rx failed") 183 | }.foldLeft(true) { case (res1, newres) => res1 && newres} 184 | logInfo(result1.mkString) 185 | assert(res, "One or more rows did not match expected") 186 | 187 | logInfo(s"Test $testnm completed successfully") 188 | } 189 | 190 | testnm = "In predicates" 191 | test("In predicates") { 192 | val query1 = s"""SELECT doublecol AS double1, -1 * doublecol AS minusdouble, 193 | substr(strcol, 2) AS substrcol, doublecol, strcol, 194 | bytecol, shortcol, intcol, longcol, floatcol FROM $TestTableName 195 | WHERE doublecol IN (doublecol + 5678912.345682 - doublecol, doublecol + 5678912.345683 - doublecol)""".stripMargin 196 | 197 | val result1 = runSql(query1) 198 | logInfo(s"$query1 came back with ${result1.length} results") 199 | assert(result1.length == 2, s"$testnm failed on size") 200 | val exparr = Array( 201 | Array(5678912.345682, -5678912.345682, "ow2", 5678912.345682, 202 | "Row2", 'b', 12342, 23456782, 3456789012342L, 45657.82F), 203 | Array(5678912.345683, -5678912.345683, "ow3", 5678912.345683, 204 | "Row3", -29, 12343, 23456783, 3456789012343L, 45657.83)) 205 | 206 | val res = { 207 | for (rx <- 0 until 1) 208 | yield compareWithTol(result1(rx).toSeq, exparr(rx), s"Row$rx failed") 209 | }.foldLeft(true) { case (res1, newres) => res1 && newres} 210 | logInfo(result1.mkString) 211 | assert(res, "One or more rows did not match expected") 212 | 213 | logInfo(s"Test $testnm completed successfully") 214 | } 215 | 216 | testnm = "InSet predicates" 217 | test("InSet predicates") { 218 | val query1 = s"""SELECT doublecol AS double1, -1 * doublecol AS minusdouble, 219 | substr(strcol, 2) AS substrcol, doublecol, strcol, 220 | bytecol, shortcol, intcol, longcol, floatcol FROM $TestTableName 221 | WHERE doublecol IN (5678912.345682, 5678912.345683)""".stripMargin 222 | 223 | val result1 = runSql(query1) 224 | logInfo(s"$query1 came back with ${result1.length} results") 225 | assert(result1.length == 2, s"$testnm failed on size") 226 | val exparr = Array( 227 | Array(5678912.345682, -5678912.345682, "ow2", 5678912.345682, 228 | "Row2", 'b', 12342, 23456782, 3456789012342L, 45657.82F), 229 | Array(5678912.345683, -5678912.345683, "ow3", 5678912.345683, 230 | "Row3", -29, 12343, 23456783, 3456789012343L, 45657.83)) 231 | 232 | val res = { 233 | for (rx <- 0 until 1) 234 | yield compareWithTol(result1(rx).toSeq, exparr(rx), s"Row$rx failed") 235 | }.foldLeft(true) { case (res1, newres) => res1 && newres} 236 | logInfo(result1.mkString) 237 | assert(res, "One or more rows did not match expected") 238 | 239 | logInfo(s"Test $testnm completed successfully") 240 | } 241 | } 242 | -------------------------------------------------------------------------------- /src/test/scala/org/apache/spark/sql/hbase/HBaseCFTestSuite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 org.apache.spark.sql.hbase 19 | 20 | import org.apache.hadoop.hbase._ 21 | 22 | /** 23 | * This is a custom filter test suite running against mini-cluster 24 | */ 25 | class HBaseCFTestSuite extends TestBase { 26 | private val namespace = "default" 27 | private val tableName = "cf" 28 | private val hbaseTableName = "cf_htable" 29 | private val hbaseFamilies = Seq("f") 30 | 31 | private val csvPaths = Array("src/test/resources", "sql/hbase/src/test/resources") 32 | private val csvFile = "cf.txt" 33 | private val tpath = for (csvPath <- csvPaths if new java.io.File(csvPath).exists()) yield { 34 | logInfo(s"Following path exists $csvPath") 35 | csvPath 36 | } 37 | private[hbase] val csvPath = tpath(0) 38 | 39 | override protected def beforeAll() = { 40 | /** 41 | * create hbase table if it does not exists 42 | */ 43 | super.beforeAll() 44 | if (!TestHbase.sharedState.externalCatalog.asInstanceOf[HBaseCatalog].tableExists(namespace, hbaseTableName)) { 45 | val descriptor = new HTableDescriptor(TableName.valueOf(tableName)) 46 | hbaseFamilies.foreach { f => descriptor.addFamily(new HColumnDescriptor(f))} 47 | try { 48 | TestHbase.hbaseAdmin.createTable(descriptor) 49 | } catch { 50 | case e: TableExistsException => 51 | logError(s"Table already exists $tableName", e) 52 | } finally { 53 | TestHbase.sharedState.externalCatalog.asInstanceOf[HBaseCatalog].stopAdmin() 54 | } 55 | } 56 | 57 | /** 58 | * drop the existing logical table if it exists 59 | */ 60 | if (TestHbase.sharedState.externalCatalog.asInstanceOf[HBaseCatalog].tableExists(namespace, tableName)) { 61 | val dropSql = "DROP TABLE " + tableName 62 | try { 63 | runSql(dropSql) 64 | } catch { 65 | case e: IllegalStateException => 66 | logError(s"Error occurs while dropping the table $tableName", e) 67 | } 68 | } 69 | 70 | /** 71 | * create table 72 | */ 73 | val createSql = 74 | s"""CREATE TABLE cf (k1 INTEGER, k2 INTEGER, k3 INTEGER, nk1 INTEGER, nk2 INTEGER) TBLPROPERTIES( 75 | 'hbaseTableName'='cf_htable', 76 | 'keyCols'='k1;k2;k3', 77 | 'nonKeyCols'='nk1,f,nk1;nk2,f,nk2')""" 78 | .stripMargin 79 | 80 | try { 81 | runSql(createSql) 82 | } catch { 83 | case e: IllegalStateException => 84 | logError(s"Error occurs while creating the table $tableName", e) 85 | } 86 | 87 | /** 88 | * load the data 89 | */ 90 | val loadSql = "LOAD DATA LOCAL INPATH '" + s"$csvPath/$csvFile" + 91 | s"' INTO TABLE $tableName" 92 | try { 93 | runSql(loadSql) 94 | } catch { 95 | case e: IllegalStateException => 96 | logError(s"Error occurs while loading the data $tableName", e) 97 | } 98 | } 99 | 100 | override protected def afterAll() = { 101 | runSql(s"DROP TABLE $tableName") 102 | super.afterAll() 103 | } 104 | 105 | test("Query 0") { 106 | val sql = "SELECT * FROM cf" 107 | val rows = runSql(sql) 108 | assert(rows.length == 27) 109 | } 110 | 111 | test("Query 1") { 112 | val sql = "SELECT * FROM cf WHERE k1 = 1 OR k1 = 10 OR k1 = 20" 113 | val rows = runSql(sql) 114 | assert(rows.length == 3) 115 | } 116 | 117 | test("Query 2") { 118 | val sql = "SELECT * FROM cf WHERE k1 < 2 OR k1 = 10 OR k1 > 20" 119 | val rows = runSql(sql) 120 | assert(rows.length == 9) 121 | } 122 | 123 | test("Query 3") { 124 | val sql = "SELECT * FROM cf WHERE (k1 = 1 OR k1 = 10 OR k1 = 20) AND (k2 = 101 OR k2 = 110 OR k2 = 120) AND (k3 = 1001 OR k3 = 1010 OR k3 = 1020)" 125 | val rows = runSql(sql) 126 | assert(rows.length == 3) 127 | } 128 | 129 | test("Query 4") { 130 | val sql = "SELECT * FROM cf WHERE (k2 = 101 OR k2 = 110 OR k2 = 120) AND (k3 = 1001 OR k3 = 1010 OR k3 = 1020)" 131 | val rows = runSql(sql) 132 | assert(rows.length == 3) 133 | } 134 | 135 | test("Query 5") { 136 | val sql = "SELECT * FROM cf WHERE (k3 = 1001 OR k3 = 1010 OR k3 = 1020)" 137 | val rows = runSql(sql) 138 | assert(rows.length == 3) 139 | } 140 | 141 | test("Query 6") { 142 | val sql = "SELECT * FROM cf WHERE (nk1 = -1 OR nk1 = -10)" 143 | val rows = runSql(sql) 144 | assert(rows.length == 2) 145 | } 146 | 147 | test("Query 7") { 148 | val sql = "SELECT * FROM cf WHERE (nk2 = -101 OR nk2 = -110)" 149 | val rows = runSql(sql) 150 | assert(rows.length == 2) 151 | } 152 | 153 | test("Query 8") { 154 | val sql = "SELECT * FROM cf WHERE k1 = 10 AND k2 = 110 AND (k3 = 1001 OR k3 = 1010 OR k3 = 1020)" 155 | val rows = runSql(sql) 156 | assert(rows.length == 1) 157 | } 158 | 159 | test("Query 9") { 160 | val sql = "SELECT * FROM cf WHERE k1 = 10 AND k2 = 110 AND k3 = 1010" 161 | val rows = runSql(sql) 162 | assert(rows.length == 1) 163 | } 164 | 165 | test("Query 10") { 166 | val sql = "SELECT * FROM cf WHERE k1 = 10 AND k2 = 110 AND k3 = 1010 AND (nk2 = -101 OR nk2 = -110)" 167 | val rows = runSql(sql) 168 | assert(rows.length == 1) 169 | } 170 | } 171 | -------------------------------------------------------------------------------- /src/test/scala/org/apache/spark/sql/hbase/HBaseCatalogTestSuite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 | package org.apache.spark.sql.hbase 18 | 19 | import org.apache.hadoop.hbase._ 20 | import org.apache.spark.sql.catalyst.expressions.GenericRow 21 | import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias 22 | import org.apache.spark.sql.execution.datasources.LogicalRelation 23 | import org.apache.spark.sql.hbase.util.HBaseKVHelper 24 | import org.apache.spark.sql.types._ 25 | 26 | import scala.collection.Seq 27 | 28 | class HBaseCatalogTestSuite extends TestBase { 29 | val (catalog, configuration) = (TestHbase.sharedState.externalCatalog.asInstanceOf[HBaseCatalog], 30 | TestHbase.sparkContext.hadoopConfiguration) 31 | 32 | test("Create Table") { 33 | // prepare the test data 34 | val namespace = "default" 35 | val tableName = "testTable" 36 | val hbaseTableName = "hbaseTable" 37 | val family1 = "family1" 38 | val family2 = "family2" 39 | 40 | if (!catalog.checkHBaseTableExists(TableName.valueOf(namespace, hbaseTableName))) { 41 | val admin = catalog.admin 42 | val desc = new HTableDescriptor(TableName.valueOf(hbaseTableName)) 43 | desc.addFamily(new HColumnDescriptor(family1)) 44 | desc.addFamily(new HColumnDescriptor(family2)) 45 | admin.createTable(desc) 46 | admin.close() 47 | } 48 | 49 | var allColumns = List[AbstractColumn]() 50 | allColumns = allColumns :+ KeyColumn("column2", IntegerType, 1) 51 | allColumns = allColumns :+ KeyColumn("column1", StringType, 0) 52 | allColumns = allColumns :+ NonKeyColumn("column4", FloatType, family2, "qualifier2") 53 | allColumns = allColumns :+ NonKeyColumn("column3", BooleanType, family1, "qualifier1") 54 | 55 | val splitKeys: Array[Array[Byte]] = Array( 56 | new GenericRow(Array(1024.0, "Upen", 128: Short)), 57 | new GenericRow(Array(1024.0, "Upen", 256: Short)), 58 | new GenericRow(Array(4096.0, "SF", 512: Short)) 59 | ).map(HBaseKVHelper.makeRowKey(_, Seq(DoubleType, StringType, ShortType))) 60 | 61 | catalog.createTable(tableName, namespace, hbaseTableName, allColumns, splitKeys) 62 | 63 | assert(catalog.tableExists(namespace, tableName) === true) 64 | catalog.stopAdmin() 65 | } 66 | 67 | test("Get Table") { 68 | // prepare the test data 69 | val hbaseNamespace = "default" 70 | val tableName = "testTable" 71 | val hbaseTableName = "hbaseTable" 72 | 73 | val oresult = catalog.getHBaseRelation(hbaseNamespace, tableName) 74 | assert(oresult.isDefined) 75 | val result = oresult.get 76 | assert(result.tableName === tableName) 77 | assert(result.hbaseNamespace === hbaseNamespace) 78 | assert(result.hbaseTableName === hbaseTableName) 79 | assert(result.keyColumns.size === 2) 80 | assert(result.nonKeyColumns.size === 2) 81 | assert(result.allColumns.size === 4) 82 | 83 | // check the data type 84 | assert(result.keyColumns.head.dataType === StringType) 85 | assert(result.keyColumns(1).dataType === IntegerType) 86 | assert(result.nonKeyColumns(1).dataType === FloatType) 87 | assert(result.nonKeyColumns.head.dataType === BooleanType) 88 | 89 | assert(result.nonKeyColumns.map(_.family) == List("family1", "family2")) 90 | val keyColumns = Seq(KeyColumn("column1", StringType, 0), KeyColumn("column2", IntegerType, 1)) 91 | assert(result.keyColumns.equals(keyColumns)) 92 | catalog.stopAdmin() 93 | } 94 | 95 | test("Alter Table") { 96 | val namespace = "default" 97 | val tableName = "testTable" 98 | 99 | val family1 = "family1" 100 | val column = NonKeyColumn("column5", BooleanType, family1, "qualifier3") 101 | 102 | catalog.alterTableAddNonKey(namespace, tableName, column) 103 | 104 | var result = catalog.getHBaseRelation(namespace, tableName) 105 | var table = result.get 106 | assert(table.allColumns.size === 5) 107 | 108 | catalog.alterTableDropNonKey(namespace, tableName, column.sqlName) 109 | result = catalog.getHBaseRelation(namespace, tableName) 110 | table = result.get 111 | assert(table.allColumns.size === 4) 112 | catalog.stopAdmin() 113 | } 114 | 115 | test("Delete Table") { 116 | // prepare the test data 117 | val namespace = "default" 118 | val tableName = "testTable" 119 | 120 | catalog.dropTable(namespace, tableName, true, true) 121 | assert(catalog.tableExists(namespace, tableName) === false) 122 | catalog.stopAdmin() 123 | } 124 | 125 | test("Check Logical Table Exist") { 126 | val namespace = "default" 127 | val tableName = "non-exist" 128 | 129 | assert(catalog.tableExists(namespace, tableName) === false) 130 | catalog.stopAdmin() 131 | } 132 | 133 | test("Namespce operations") { 134 | runSql("CREATE DATABASE db1") 135 | runSql(s"""CREATE TABLE db1.t1 (c1 INT, c2 INT) TBLPROPERTIES('hbaseTableName'='ht', 136 | 'keyCols'='c1','nonKeyCols'='c2,cf,q')""") 137 | assert(runSql("SHOW DATABASES").length == 3) 138 | assert(runSql("SHOW tables IN db1").length == 1) 139 | runSql("DROP DATABASE db1 CASCADE") 140 | assert(runSql("SHOW DATABASES").length == 2) 141 | } 142 | } 143 | -------------------------------------------------------------------------------- /src/test/scala/org/apache/spark/sql/hbase/HBaseInsertTableSuite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 org.apache.spark.sql.hbase 19 | 20 | import org.apache.spark.sql.Row 21 | 22 | class HBaseInsertTableSuite extends TestBaseWithNonSplitData { 23 | 24 | var testnm = "Insert all rows to the table from other table" 25 | test("Insert all rows to the table from other table") { 26 | val createQuery = s"""CREATE TABLE insertTestTable (strcol STRING, bytecol BYTE, shortcol SHORT, intcol INT, longcol LONG, floatcol FLOAT, doublecol DOUBLE) TBLPROPERTIES( 27 | 'hbaseTableName'='hinsertTestTable', 28 | 'keyCols'='doublecol;strcol;intcol', 29 | 'nonKeyCols'='bytecol,cf1,hbytecol;shortcol,cf1,hshortcol;longcol,cf2,hlongcol;floatcol,cf2,hfloatco')""" 30 | .stripMargin 31 | runSql(createQuery) 32 | 33 | val insertQuery = 34 | s"""INSERT INTO TABLE insertTestTable SELECT * FROM $TestTableName""" 35 | .stripMargin 36 | runSql(insertQuery) 37 | 38 | val testQuery = "SELECT * FROM insertTestTable" 39 | val testResult = runSql(testQuery) 40 | val targetResult = runSql(s"SELECT * FROM $TestTableName") 41 | assert(testResult.length == targetResult.length, s"$testnm failed on size") 42 | 43 | compareResults(testResult, targetResult) 44 | 45 | runSql("DROP TABLE insertTestTable") 46 | } 47 | 48 | testnm = "Insert few rows to the table from other table after applying filter" 49 | test("Insert few rows to the table from other table after applying filter") { 50 | val createQuery = s"""CREATE TABLE insertTestTableFilter (strcol STRING, bytecol BYTE, shortcol SHORT, intcol INT, longcol LONG, floatcol FLOAT, doublecol DOUBLE) TBLPROPERTIES( 51 | 'hbaseTableName'='hinsertTestTableFilter', 52 | 'keyCols'='doublecol;strcol;intcol', 53 | 'nonKeyCols'='bytecol,cf1,hbytecol;shortcol,cf1,hshortcol;longcol,cf2,hlongcol;floatcol,cf2,hfloatco')""".stripMargin 54 | .stripMargin 55 | runSql(createQuery) 56 | 57 | val insertQuery = 58 | s"""insert into table insertTestTableFilter select * from $TestTableName 59 | where doublecol > 5678912.345681""" 60 | .stripMargin 61 | runSql(insertQuery) 62 | 63 | val testQuery = "select * from insertTestTableFilter" 64 | val testResult = runSql(testQuery) 65 | val targetResult = runSql(s"select * from $TestTableName where doublecol > 5678912.345681") 66 | assert(testResult.length == targetResult.length, s"$testnm failed on size") 67 | 68 | compareResults(testResult, targetResult) 69 | 70 | runSql("Drop Table insertTestTableFilter") 71 | } 72 | 73 | def compareResults(fetchResult: Array[Row], targetResult: Array[Row]) = { 74 | val res = { 75 | for (rx <- targetResult.indices) 76 | yield compareWithTol(fetchResult(rx).toSeq, targetResult(rx).toSeq, s"Row$rx failed") 77 | }.foldLeft(true) { case (res1, newres) => res1 && newres} 78 | assert(res, "One or more rows did not match expected") 79 | } 80 | 81 | testnm = "Insert few columns to the table from other table" 82 | test("Insert few columns to the table from other table") { 83 | val createQuery = s"""CREATE TABLE insertTestTableFewCols (strcol STRING, bytecol BYTE, shortcol SHORT, intcol INTEGER) TBLPROPERTIES( 84 | 'hbaseTableName'='insertTestTableFewCols', 85 | 'keyCols'='strcol;intcol', 86 | 'nonKeyCols'='bytecol,cf1,hbytecol;shortcol,cf1,hshortcol')""" 87 | .stripMargin 88 | runSql(createQuery) 89 | 90 | val insertQuery = 91 | s"""INSERT INTO TABLE insertTestTableFewCols SELECT strcol, bytecol, 92 | shortcol, intcol FROM $TestTableName ORDER BY strcol""" 93 | .stripMargin 94 | runSql(insertQuery) 95 | 96 | val testQuery = 97 | "SELECT strcol, bytecol, shortcol, intcol FROM insertTestTableFewCols ORDER BY strcol" 98 | val testResult = runSql(testQuery) 99 | val targetResult = 100 | runSql(s"SELECT strcol, bytecol, shortcol, intcol FROM $TestTableName ORDER BY strcol") 101 | assert(testResult.length == targetResult.length, s"$testnm failed on size") 102 | 103 | compareResults(testResult, targetResult) 104 | 105 | runSql("DROP TABLE insertTestTableFewCols") 106 | } 107 | 108 | testnm = "Insert into values test" 109 | test("Insert into values test") { 110 | val createQuery = s"""CREATE TABLE insertValuesTest (strcol STRING, bytecol BYTE, shortcol SHORT, intcol INTEGER) TBLPROPERTIES( 111 | 'hbaseTableName'='hinsertValuesTest', 112 | 'keyCols'='strcol;intcol', 113 | 'nonKeyCols'='bytecol,cf1,hbytecol;shortcol,cf1,hshortcol')""" 114 | .stripMargin 115 | runSql(createQuery) 116 | 117 | val insertQuery1 = s"INSERT INTO TABLE insertValuesTest VALUES('Row0',97,12340,23456780)" 118 | val insertQuery2 = s"INSERT INTO TABLE insertValuesTest VALUES('Row1',98,12345,23456789)" 119 | val insertQuery3 = s"INSERT INTO TABLE insertValuesTest VALUES('Row2',99,12342,23456782)" 120 | runSql(insertQuery1) 121 | runSql(insertQuery2) 122 | runSql(insertQuery3) 123 | 124 | val testQuery = "SELECT * FROM insertValuesTest ORDER BY strcol" 125 | val testResult = runSql(testQuery) 126 | assert(testResult.length == 3, s"$testnm failed on size") 127 | 128 | val exparr = Array(Array("Row0", 'a', 12340, 23456780), 129 | Array("Row1", 'b', 12345, 23456789), 130 | Array("Row2", 'c', 12342, 23456782)) 131 | 132 | val res = { 133 | for (rx <- 0 until 3) 134 | yield compareWithTol(testResult(rx).toSeq, exparr(rx), s"Row$rx failed") 135 | }.foldLeft(true) { case (res1, newres) => res1 && newres} 136 | assert(res, "One or more rows did not match expected") 137 | 138 | runSql("DROP TABLE insertValuesTest") 139 | } 140 | 141 | testnm = "Insert nullable values test" 142 | test("Insert nullable values test") { 143 | val createQuery = s"""CREATE TABLE insertNullValuesTest (strcol STRING, bytecol BYTE, shortcol SHORT, intcol INTEGER) TBLPROPERTIES( 144 | 'hbaseTableName'='hinsertNullValuesTest', 145 | 'keyCols'='strcol', 146 | 'nonKeyCols'='bytecol,cf1,hbytecol;shortcol,cf1,hshortcol;intcol,cf1,hintcol')""" 147 | .stripMargin 148 | runSql(createQuery) 149 | 150 | val insertQuery1 = s"INSERT INTO TABLE insertNullValuesTest VALUES('Row0', null, 12340, 23456780)" 151 | val insertQuery2 = s"INSERT INTO TABLE insertNullValuesTest VALUES('Row1', 'b', null, 23456789)" 152 | val insertQuery3 = s"INSERT INTO TABLE insertNullValuesTest VALUES('Row2', 'c', 12342, null)" 153 | runSql(insertQuery1) 154 | runSql(insertQuery2) 155 | runSql(insertQuery3) 156 | 157 | val selectAllQuery = "SELECT * FROM insertNullValuesTest ORDER BY strcol" 158 | val selectAllResult = runSql(selectAllQuery) 159 | 160 | assert(selectAllResult.length == 3, s"$testnm failed on size") 161 | 162 | var currentResultRow: Int = 0 163 | 164 | // check 1st result row 165 | assert(selectAllResult(currentResultRow).length == 4, s"$testnm failed on row size (# of cols)") 166 | assert(selectAllResult(currentResultRow)(0) === s"Row0", s"$testnm failed on returned Row0, key value") 167 | assert(selectAllResult(currentResultRow)(1) == null, s"$testnm failed on returned Row0, null col1 value") 168 | assert(selectAllResult(currentResultRow)(2) == 12340, s"$testnm failed on returned Row0, col2 value") 169 | assert(selectAllResult(currentResultRow)(3) == 23456780, s"$testnm failed on returned Row0, col3 value") 170 | 171 | currentResultRow += 1 172 | 173 | // check 2nd result row 174 | assert(selectAllResult(currentResultRow)(0) === s"Row1", s"$testnm failed on returned Row1, key value") 175 | // skip comparison of actual and expected bytecol value 176 | assert(selectAllResult(currentResultRow)(2) == null, s"$testnm failed on returned Row1, null col2 value") 177 | assert(selectAllResult(currentResultRow)(3) == 23456789, s"$testnm failed on returned Row1, col3 value") 178 | 179 | currentResultRow += 1 180 | 181 | // check 3rd result row 182 | assert(selectAllResult(currentResultRow)(0) === s"Row2", s"$testnm failed on returned Row2, key value") 183 | // skip comparison of actual and expected bytecol value 184 | assert(selectAllResult(currentResultRow)(2) == 12342, s"$testnm failed on returned Row2, col2 value") 185 | assert(selectAllResult(currentResultRow)(3) == null, s"$testnm failed on returned Row2, null col3 value") 186 | 187 | // test 'where col is not null' 188 | 189 | val selectWhereIsNotNullQuery = "SELECT * FROM insertNullValuesTest WHERE intcol IS NOT NULL ORDER BY strcol" 190 | val selectWhereIsNotNullResult = runSql(selectWhereIsNotNullQuery) 191 | assert(selectWhereIsNotNullResult.length == 2, s"$testnm failed on size") 192 | 193 | currentResultRow = 0 194 | // check 1st result row 195 | assert(selectWhereIsNotNullResult(currentResultRow)(0) === s"Row0", s"$testnm failed on returned Row0, key value") 196 | assert(selectWhereIsNotNullResult(currentResultRow)(1) == null, s"$testnm failed on returned Row0, null col1 value") 197 | assert(selectWhereIsNotNullResult(currentResultRow)(2) == 12340, s"$testnm failed on returned Row0, col2 value") 198 | assert(selectWhereIsNotNullResult(currentResultRow)(3) == 23456780, s"$testnm failed on returned Row0, col3 value") 199 | 200 | currentResultRow += 1 201 | // check 2nd result row 202 | assert(selectWhereIsNotNullResult(currentResultRow)(0) === s"Row1", s"$testnm failed on returned Row1, key value") 203 | // skip comparison of actual and expected bytecol value 204 | assert(selectWhereIsNotNullResult(currentResultRow)(2) == null, s"$testnm failed on returned Row1, null col2 value") 205 | assert(selectWhereIsNotNullResult(currentResultRow)(3) == 23456789, s"$testnm failed on returned Row1, col3 value") 206 | 207 | 208 | runSql(" Drop Table insertNullValuesTest") 209 | } 210 | 211 | 212 | } 213 | -------------------------------------------------------------------------------- /src/test/scala/org/apache/spark/sql/hbase/TestBase.scala: -------------------------------------------------------------------------------- 1 | 2 | /* 3 | * Licensed to the Apache Software Foundation (ASF) under one or more 4 | * contributor license agreements. See the NOTICE file distributed with 5 | * this work for additional information regarding copyright ownership. 6 | * The ASF licenses this file to You under the Apache License, Version 2.0 7 | * (the "License"); you may not use this file except in compliance with 8 | * the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.spark.sql.hbase 20 | 21 | import java.io.File 22 | import java.util.Date 23 | 24 | import org.apache.hadoop.hbase.{HColumnDescriptor, HTableDescriptor, TableExistsException, TableName} 25 | import org.apache.spark.internal.Logging 26 | import org.apache.spark.sql.catalyst.plans.logical 27 | import org.apache.spark.sql.catalyst.util._ 28 | import org.apache.spark.sql.{DataFrame, Row} 29 | import org.scalatest.{BeforeAndAfterAll, FunSuite, Suite} 30 | 31 | abstract class TestBase 32 | extends FunSuite with BeforeAndAfterAll with Logging { 33 | self: Suite => 34 | 35 | val startTime = (new Date).getTime 36 | val hbaseHome = { 37 | val loader = this.getClass.getClassLoader 38 | val url = loader.getResource("loadData.txt") 39 | val file = new File(url.getPath) 40 | val parent = file.getParentFile 41 | parent.getAbsolutePath 42 | } 43 | if (hbaseHome == null || hbaseHome.isEmpty) 44 | logError("Spark Home is not defined; may lead to unexpected error!") 45 | 46 | /** 47 | * Runs the plan and makes sure the answer matches the expected result. 48 | * @param rdd the [[DataFrame]] to be executed 49 | * @param expectedAnswer the expected result, can either be an Any, Seq[Product], or Seq[ Seq[Any] ]. 50 | */ 51 | protected def checkAnswer(rdd: DataFrame, expectedAnswer: Seq[Row]): Unit = { 52 | val isSorted = rdd.logicalPlan.collect { case s: logical.Sort => s}.nonEmpty 53 | def prepareAnswer(answer: Seq[Row]): Seq[Row] = { 54 | // Converts data to types that we can do equality comparison using Scala collections. 55 | // For BigDecimal type, the Scala type has a better definition of equality test (similar to 56 | // Java's java.math.BigDecimal.compareTo). 57 | val converted: Seq[Row] = answer.map { s => 58 | Row.fromSeq(s.toSeq.map { 59 | case d: java.math.BigDecimal => BigDecimal(d) 60 | case o => o 61 | }) 62 | } 63 | if (!isSorted) converted.sortBy(_.toString()) else converted 64 | } 65 | val sparkAnswer = try rdd.collect().toSeq catch { 66 | case e: Exception => 67 | fail( 68 | s""" 69 | |Exception thrown while executing query: 70 | |${rdd.queryExecution} 71 | |== Exception == 72 | |$e 73 | |${org.apache.spark.sql.catalyst.util.stackTraceToString(e)} 74 | """.stripMargin) 75 | } 76 | 77 | if (prepareAnswer(expectedAnswer) != prepareAnswer(sparkAnswer)) { 78 | fail( s""" 79 | |Results do not match for query: 80 | |${rdd.logicalPlan} 81 | |== Analyzed Plan == 82 | |${rdd.queryExecution.analyzed} 83 | |== Physical Plan == 84 | |${rdd.queryExecution.executedPlan} 85 | |== Results == 86 | |${ 87 | sideBySide( 88 | s"== Correct Answer - ${expectedAnswer.size} ==" +: 89 | prepareAnswer(expectedAnswer).map(_.toString()), 90 | s"== Spark Answer - ${sparkAnswer.size} ==" +: 91 | prepareAnswer(sparkAnswer).map(_.toString())).mkString("\n") 92 | } 93 | """.stripMargin) 94 | } 95 | } 96 | 97 | protected def checkAnswer(rdd: DataFrame, expectedAnswer: Row): Unit = { 98 | checkAnswer(rdd, Seq(expectedAnswer)) 99 | } 100 | 101 | def runSql(sql: String): Array[Row] = { 102 | logInfo(sql) 103 | TestHbase.sql(sql).collect() 104 | } 105 | 106 | override protected def afterAll(): Unit = { 107 | val msg = s"Test ${getClass.getName} completed at ${(new java.util.Date).toString} duration=${((new java.util.Date).getTime - startTime) / 1000}" 108 | logInfo(msg) 109 | super.afterAll() 110 | } 111 | 112 | val CompareTol = 1e-6 113 | 114 | def compareWithTol(actarr: Seq[Any], exparr: Seq[Any], emsg: String): Boolean = { 115 | actarr.zip(exparr).forall { case (aa, ee) => 116 | val eq = (aa, ee) match { 117 | case (a: Double, e: Double) => 118 | Math.abs(a - e) <= CompareTol 119 | case (a: Float, e: Float) => 120 | Math.abs(a - e) <= CompareTol 121 | case (a: Byte, e) => true //For now, we assume it is ok 122 | case (a, e) => 123 | if (a == null && e == null) { 124 | logDebug(s"a=null e=null") 125 | } else { 126 | logDebug(s"atype=${a.getClass.getName} etype=${e.getClass.getName}") 127 | } 128 | a == e 129 | case _ => throw new IllegalArgumentException("Expected tuple") 130 | } 131 | if (!eq) { 132 | logError(s"$emsg: Mismatch- act=$aa exp=$ee") 133 | } 134 | eq 135 | } 136 | } 137 | 138 | def verify(testName: String, sql: String, result1: Seq[Seq[Any]], exparr: Seq[Seq[Any]]) = { 139 | val res = { 140 | for (rx <- exparr.indices) 141 | yield compareWithTol(result1(rx), exparr(rx), s"Row$rx failed") 142 | }.foldLeft(true) { case (res1, newres) => res1 && newres} 143 | 144 | logInfo(s"$sql came back with ${result1.size} results") 145 | logInfo(result1.mkString) 146 | assert(res, "One or more rows did not match expected") 147 | } 148 | 149 | def createNativeHbaseTable(tableName: String, families: Seq[String]) = { 150 | val hbaseAdmin = TestHbase.hbaseAdmin 151 | val hdesc = new HTableDescriptor(TableName.valueOf(tableName)) 152 | families.foreach { f => hdesc.addFamily(new HColumnDescriptor(f))} 153 | try { 154 | hbaseAdmin.createTable(hdesc) 155 | } catch { 156 | case e: TableExistsException => 157 | logError(s"Table already exists $tableName", e) 158 | } 159 | } 160 | 161 | def dropNativeHbaseTable(tableName: String) = { 162 | try { 163 | val hbaseAdmin = TestHbase.hbaseAdmin 164 | if (hbaseAdmin.tableExists(TableName.valueOf(tableName))) { 165 | hbaseAdmin.disableTable(TableName.valueOf(tableName)) 166 | hbaseAdmin.deleteTable(TableName.valueOf(tableName)) 167 | } 168 | } catch { 169 | case e: TableExistsException => 170 | logError(s"Table already exists $tableName", e) 171 | } 172 | } 173 | 174 | def loadData(tableName: String, loadFile: String) = { 175 | // then load data into table 176 | val loadSql = s"LOAD DATA LOCAL INPATH '$loadFile' INTO TABLE $tableName" 177 | runSql(loadSql) 178 | } 179 | } 180 | -------------------------------------------------------------------------------- /src/test/scala/org/apache/spark/sql/hbase/TestBaseWithNonSplitData.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase 2 | 3 | import org.apache.hadoop.hbase._ 4 | 5 | /* 6 | * Licensed to the Apache Software Foundation (ASF) under one or more 7 | * contributor license agreements. See the NOTICE file distributed with 8 | * this work for additional information regarding copyright ownership. 9 | * The ASF licenses this file to You under the Apache License, Version 2.0 10 | * (the "License"); you may not use this file except in compliance with 11 | * the License. You may obtain a copy of the License at 12 | * 13 | * http://www.apache.org/licenses/LICENSE-2.0 14 | * 15 | * Unless required by applicable law or agreed to in writing, software 16 | * distributed under the License is distributed on an "AS IS" BASIS, 17 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 18 | * See the License for the specific language governing permissions and 19 | * limitations under the License. 20 | */ 21 | 22 | /** 23 | * CreateTableAndLoadData 24 | * 25 | */ 26 | class TestBaseWithNonSplitData extends TestBase { 27 | private val namespace = "default" 28 | val TestTableName = "TestTable" 29 | val TestHBaseTableName: String = s"Hb$TestTableName" 30 | val TestHbaseColFamilies = Seq("cf1", "cf2") 31 | 32 | val CsvPaths = Array("src/test/resources", "sql/hbase/src/test/resources") 33 | val DefaultLoadFile = "testTable.txt" 34 | 35 | private val tpath = for (csvPath <- CsvPaths 36 | if new java.io.File(csvPath).exists() 37 | ) yield { 38 | logInfo(s"Following path exists $csvPath") 39 | csvPath 40 | } 41 | private[hbase] val CsvPath = tpath(0) 42 | 43 | override protected def beforeAll() = { 44 | super.beforeAll() 45 | val testTableCreationSQL = s"""CREATE TABLE $TestTableName (strcol STRING, bytecol BYTE, shortcol SHORT, intcol INT, longcol LONG, floatcol FLOAT, doublecol DOUBLE) TBLPROPERTIES( 46 | 'hbaseTableName'='$TestHBaseTableName', 47 | 'keyCols'='doublecol;strcol;intcol', 48 | 'nonKeyCols'='bytecol,cf1,hbytecol;shortcol,cf1,hshortcol;longcol,cf2,hlongcol;floatcol,cf2,hfloatcol')""" 49 | createTable(TestTableName, TestHBaseTableName, testTableCreationSQL) 50 | loadData(TestTableName, s"$CsvPath/$DefaultLoadFile") 51 | } 52 | 53 | override protected def afterAll() = { 54 | runSql("DROP TABLE " + TestTableName) 55 | dropNativeHbaseTable(TestHBaseTableName) 56 | super.afterAll() 57 | } 58 | 59 | def createTable(tableName: String, hbaseTable: String, creationSQL: String) = { 60 | val hbaseAdmin = TestHbase.hbaseAdmin 61 | if (!hbaseAdmin.tableExists(TableName.valueOf(hbaseTable))) { 62 | createNativeHbaseTable(hbaseTable, TestHbaseColFamilies) 63 | } 64 | 65 | if (TestHbase.sharedState.externalCatalog.tableExists(namespace, tableName)) { 66 | val dropSql = s"DROP TABLE $tableName" 67 | runSql(dropSql) 68 | } 69 | 70 | try { 71 | logInfo(s"invoking $creationSQL ..") 72 | runSql(creationSQL) 73 | } catch { 74 | case e: TableExistsException => 75 | logInfo("IF NOT EXISTS still not implemented so we get the following exception", e) 76 | } 77 | } 78 | } 79 | -------------------------------------------------------------------------------- /src/test/scala/org/apache/spark/sql/hbase/TestBaseWithSplitData.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 org.apache.spark.sql.hbase 19 | 20 | import java.io.{ByteArrayOutputStream, DataOutputStream} 21 | import java.util.concurrent.Executors 22 | 23 | import org.apache.hadoop.hbase._ 24 | import org.apache.hadoop.hbase.client._ 25 | import org.apache.hadoop.hbase.util.Bytes 26 | import org.apache.spark.sql.Row 27 | import org.apache.spark.sql.catalyst.expressions.GenericRow 28 | import org.apache.spark.sql.hbase.util.{BinaryBytesUtils, DataTypeUtils, HBaseKVHelper} 29 | import org.apache.spark.sql.types._ 30 | 31 | /** 32 | * HBaseMainTest 33 | * create HbTestTable and metadata table, and insert some data 34 | */ 35 | class TestBaseWithSplitData extends TestBase { 36 | val namespace = "default" 37 | val TableName_a: String = "ta" 38 | val TableName_b: String = "tb" 39 | val HbaseTableName = TableName.valueOf("ht") 40 | val Metadata_Table = TableName.valueOf("metadata") 41 | var alreadyInserted = false 42 | 43 | override protected def beforeAll() = { 44 | super.beforeAll() 45 | setupData(useMultiplePartitions = true, needInsertData = true) 46 | TestData 47 | } 48 | 49 | override protected def afterAll() = { 50 | runSql("DROP TABLE " + TableName_a) 51 | runSql("DROP TABLE " + TableName_b) 52 | dropNativeHbaseTable("ht") 53 | super.afterAll() 54 | } 55 | 56 | def createTable(useMultiplePartitions: Boolean) = { 57 | // delete the existing hbase table 58 | if (TestHbase.hbaseAdmin.tableExists(HbaseTableName)) { 59 | TestHbase.hbaseAdmin.disableTable(HbaseTableName) 60 | TestHbase.hbaseAdmin.deleteTable(HbaseTableName) 61 | } 62 | if (TestHbase.hbaseAdmin.tableExists(Metadata_Table)) { 63 | TestHbase.hbaseAdmin.disableTable(Metadata_Table) 64 | TestHbase.hbaseAdmin.deleteTable(Metadata_Table) 65 | } 66 | 67 | var allColumns = List[AbstractColumn]() 68 | allColumns = allColumns :+ KeyColumn("col1", StringType, 1) 69 | allColumns = allColumns :+ NonKeyColumn("col2", ByteType, "cf1", "cq11") 70 | allColumns = allColumns :+ KeyColumn("col3", ShortType, 2) 71 | allColumns = allColumns :+ NonKeyColumn("col4", IntegerType, "cf1", "cq12") 72 | allColumns = allColumns :+ NonKeyColumn("col5", LongType, "cf2", "cq21") 73 | allColumns = allColumns :+ NonKeyColumn("col6", FloatType, "cf2", "cq22") 74 | allColumns = allColumns :+ KeyColumn("col7", IntegerType, 0) 75 | 76 | val splitKeys: Array[Array[Byte]] = if (useMultiplePartitions) { 77 | Array( 78 | new GenericRow(Array(256, " p256 ", 128: Short)), 79 | new GenericRow(Array(32, " p32 ", 256: Short)), 80 | new GenericRow(Array(-32, " n32 ", 128: Short)), 81 | new GenericRow(Array(-256, " n256 ", 256: Short)), 82 | new GenericRow(Array(-128, " n128 ", 128: Short)), 83 | new GenericRow(Array(0, " zero ", 256: Short)), 84 | new GenericRow(Array(128, " p128 ", 512: Short)) 85 | ).map(HBaseKVHelper.makeRowKey(_, Seq(IntegerType, StringType, ShortType))) 86 | } else { 87 | null 88 | } 89 | 90 | TestHbase.sharedState.externalCatalog.asInstanceOf[HBaseCatalog].createTable( 91 | TableName_a, namespace, HbaseTableName.getNameAsString, allColumns, splitKeys) 92 | 93 | runSql(s"""CREATE TABLE $TableName_b (col1 STRING, col2 BYTE, col3 SHORT, col4 INT, col5 LONG, col6 FLOAT, col7 INT) TBLPROPERTIES( 94 | 'hbaseTableName'='$HbaseTableName', 95 | 'keyCols'='col7;col1;col3', 96 | 'nonKeyCols'='col2,cf1,cq11;col4,cf1,cq12;col5,cf2,cq21;col6,cf2,cq21')""".stripMargin) 97 | 98 | if (!TestHbase.hbaseAdmin.tableExists(HbaseTableName)) { 99 | throw new IllegalArgumentException("where is our table?") 100 | } 101 | } 102 | 103 | def checkHBaseTableExists(hbaseTable: TableName): Boolean = { 104 | TestHbase.hbaseAdmin.tableExists(hbaseTable) 105 | } 106 | 107 | def insertTestData() = { 108 | if (!checkHBaseTableExists(HbaseTableName)) { 109 | throw new IllegalStateException(s"Unable to find table $HbaseTableName") 110 | } 111 | 112 | def putNewTableIntoHBase(keys: Seq[Any], keysType: Seq[DataType], 113 | vals: Seq[Any], valsType: Seq[DataType]): Unit = { 114 | val row = new GenericRow(keys.toArray) 115 | val key = makeRowKey(row, keysType) 116 | val put = new Put(key) 117 | Seq((vals.head, valsType.head, "cf1", "cq11"), 118 | (vals(1), valsType(1), "cf1", "cq12"), 119 | (vals(2), valsType(2), "cf2", "cq21"), 120 | (vals(3), valsType(3), "cf2", "cq22")).foreach { 121 | case (rowValue, rowType, colFamily, colQualifier) => 122 | addRowVals(put, rowValue, rowType, colFamily, colQualifier) 123 | } 124 | val executor = Executors.newFixedThreadPool(10) 125 | val connection = ConnectionFactory.createConnection( 126 | TestHbase.sparkContext.hadoopConfiguration, executor) 127 | val table = connection.getTable(HbaseTableName) 128 | try { 129 | table.put(put) 130 | } finally { 131 | table.close() 132 | connection.close() 133 | } 134 | } 135 | 136 | putNewTableIntoHBase(Seq(-257, " n257 ", 128: Short), 137 | Seq(IntegerType, StringType, ShortType), 138 | Seq[Any](1.toByte, -2048, 12345678901234L, 1234.5678F), 139 | Seq(ByteType, IntegerType, LongType, FloatType)) 140 | 141 | putNewTableIntoHBase(Seq(-255, " n255 ", 128: Short), 142 | Seq(IntegerType, StringType, ShortType), 143 | Seq[Any](2.toByte, -1024, 12345678901234L, 1234.5678F), 144 | Seq(ByteType, IntegerType, LongType, FloatType)) 145 | 146 | putNewTableIntoHBase(Seq(-129, " n129 ", 128: Short), 147 | Seq(IntegerType, StringType, ShortType), 148 | Seq[Any](3.toByte, -512, 12345678901234L, 1234.5678F), 149 | Seq(ByteType, IntegerType, LongType, FloatType)) 150 | 151 | putNewTableIntoHBase(Seq(-127, " n127 ", 128: Short), 152 | Seq(IntegerType, StringType, ShortType), 153 | Seq[Any](4.toByte, -256, 12345678901234L, 1234.5678F), 154 | Seq(ByteType, IntegerType, LongType, FloatType)) 155 | 156 | putNewTableIntoHBase(Seq(-33, " n33 ", 128: Short), 157 | Seq(IntegerType, StringType, ShortType), 158 | Seq[Any](5.toByte, -128, 12345678901234L, 1234.5678F), 159 | Seq(ByteType, IntegerType, LongType, FloatType)) 160 | 161 | putNewTableIntoHBase(Seq(-31, " n31 ", 128: Short), 162 | Seq(IntegerType, StringType, ShortType), 163 | Seq[Any](6.toByte, -64, 12345678901234L, 1234.5678F), 164 | Seq(ByteType, IntegerType, LongType, FloatType)) 165 | 166 | putNewTableIntoHBase(Seq(-1, " n1 ", 128: Short), 167 | Seq(IntegerType, StringType, ShortType), 168 | Seq[Any](7.toByte, -1, 12345678901234L, 1234.5678F), 169 | Seq(ByteType, IntegerType, LongType, FloatType)) 170 | 171 | putNewTableIntoHBase(Seq(1, " p1 ", 128: Short), 172 | Seq(IntegerType, StringType, ShortType), 173 | Seq[Any](8.toByte, 1, 12345678901234L, 1234.5678F), 174 | Seq(ByteType, IntegerType, LongType, FloatType)) 175 | 176 | putNewTableIntoHBase(Seq(31, " p31 ", 128: Short), 177 | Seq(IntegerType, StringType, ShortType), 178 | Seq[Any](9.toByte, 4, 12345678901234L, 1234.5678F), 179 | Seq(ByteType, IntegerType, LongType, FloatType)) 180 | 181 | putNewTableIntoHBase(Seq(33, " p33 ", 128: Short), 182 | Seq(IntegerType, StringType, ShortType), 183 | Seq[Any](10.toByte, 64, 12345678901234L, 1234.5678F), 184 | Seq(ByteType, IntegerType, LongType, FloatType)) 185 | 186 | putNewTableIntoHBase(Seq(127, " p127 ", 128: Short), 187 | Seq(IntegerType, StringType, ShortType), 188 | Seq[Any](11.toByte, 128, 12345678901234L, 1234.5678F), 189 | Seq(ByteType, IntegerType, LongType, FloatType)) 190 | 191 | putNewTableIntoHBase(Seq(129, " p129 ", 128: Short), 192 | Seq(IntegerType, StringType, ShortType), 193 | Seq[Any](12.toByte, 256, 12345678901234L, 1234.5678F), 194 | Seq(ByteType, IntegerType, LongType, FloatType)) 195 | 196 | putNewTableIntoHBase(Seq(255, " p255 ", 128: Short), 197 | Seq(IntegerType, StringType, ShortType), 198 | Seq[Any](13.toByte, 512, 12345678901234L, 1234.5678F), 199 | Seq(ByteType, IntegerType, LongType, FloatType)) 200 | 201 | putNewTableIntoHBase(Seq(257, " p257 ", 128: Short), 202 | Seq(IntegerType, StringType, ShortType), 203 | Seq[Any](14.toByte, 1024, 12345678901234L, 1234.5678F), 204 | Seq(ByteType, IntegerType, LongType, FloatType)) 205 | } 206 | 207 | def makeRowKey(row: Row, dataTypeOfKeys: Seq[DataType]) = { 208 | val rawKeyCol = dataTypeOfKeys.zipWithIndex.map { 209 | case (dataType, index) => 210 | (DataTypeUtils.getRowColumnInHBaseRawType(row, index, dataType), 211 | dataType) 212 | } 213 | 214 | HBaseKVHelper.encodingRawKeyColumns(rawKeyCol) 215 | } 216 | 217 | def addRowVals(put: Put, rowValue: Any, rowType: DataType, 218 | colFamily: String, colQualifier: String) = { 219 | val bos = new ByteArrayOutputStream() 220 | val dos = new DataOutputStream(bos) 221 | val bu = BinaryBytesUtils.create(rowType) 222 | rowType match { 223 | case StringType => dos.write(bu.toBytes(rowValue.asInstanceOf[String])) 224 | case IntegerType => dos.write(bu.toBytes(rowValue.asInstanceOf[Int])) 225 | case BooleanType => dos.write(bu.toBytes(rowValue.asInstanceOf[Boolean])) 226 | case ByteType => dos.write(bu.toBytes(rowValue.asInstanceOf[Byte])) 227 | case DoubleType => dos.write(bu.toBytes(rowValue.asInstanceOf[Double])) 228 | case FloatType => dos.write(bu.toBytes(rowValue.asInstanceOf[Float])) 229 | case LongType => dos.write(bu.toBytes(rowValue.asInstanceOf[Long])) 230 | case ShortType => dos.write(bu.toBytes(rowValue.asInstanceOf[Short])) 231 | case _ => throw new Exception("Unsupported HBase SQL Data Type") 232 | } 233 | put.addImmutable(Bytes.toBytes(colFamily), Bytes.toBytes(colQualifier), bos.toByteArray) 234 | } 235 | 236 | def setupData(useMultiplePartitions: Boolean, needInsertData: Boolean = false) { 237 | if (needInsertData && !alreadyInserted) { 238 | createTable(useMultiplePartitions) 239 | insertTestData() 240 | alreadyInserted = true 241 | } 242 | } 243 | } 244 | -------------------------------------------------------------------------------- /src/test/scala/org/apache/spark/sql/hbase/TestData.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 org.apache.spark.sql.hbase 19 | 20 | import java.sql.Timestamp 21 | 22 | import org.apache.spark.sql.catalyst.plans.logical 23 | 24 | case class TestData(k: Int, v: String) 25 | 26 | object TestData { 27 | import TestHbase.implicits._ 28 | 29 | val testData = TestHbase.sparkContext.parallelize( 30 | (1 to 100).map(i => TestData(i, i.toString))).toDF() 31 | testData.createOrReplaceTempView("testData") 32 | 33 | val negativeData = TestHbase.sparkContext.parallelize( 34 | (1 to 100).map(i => TestData(-i, (-i).toString))).toDF() 35 | negativeData.createOrReplaceTempView("negativeData") 36 | 37 | case class LargeAndSmallInts(a: Int, b: Int) 38 | 39 | val largeAndSmallInts = 40 | TestHbase.sparkContext.parallelize( 41 | LargeAndSmallInts(2147483644, 1) :: 42 | LargeAndSmallInts(1, 2) :: 43 | LargeAndSmallInts(2147483645, 1) :: 44 | LargeAndSmallInts(2, 2) :: 45 | LargeAndSmallInts(2147483646, 1) :: 46 | LargeAndSmallInts(3, 2) :: Nil).toDF() 47 | largeAndSmallInts.createOrReplaceTempView("largeAndSmallInts") 48 | 49 | case class TestData2(a: Int, b: Int) 50 | 51 | val testData2 = 52 | TestHbase.sparkContext.parallelize( 53 | TestData2(1, 1) :: 54 | TestData2(1, 2) :: 55 | TestData2(2, 1) :: 56 | TestData2(2, 2) :: 57 | TestData2(3, 1) :: 58 | TestData2(3, 2) :: Nil).toDF() 59 | testData2.createOrReplaceTempView("testData2") 60 | 61 | case class DecimalData(a: BigDecimal, b: BigDecimal) 62 | 63 | val decimalData = 64 | TestHbase.sparkContext.parallelize( 65 | DecimalData(1, 1) :: 66 | DecimalData(1, 2) :: 67 | DecimalData(2, 1) :: 68 | DecimalData(2, 2) :: 69 | DecimalData(3, 1) :: 70 | DecimalData(3, 2) :: Nil).toDF() 71 | decimalData.createOrReplaceTempView("decimalData") 72 | 73 | case class BinaryData(a: Array[Byte], b: Int) 74 | 75 | val binaryData = 76 | TestHbase.sparkContext.parallelize( 77 | BinaryData("12".getBytes, 1) :: 78 | BinaryData("22".getBytes, 5) :: 79 | BinaryData("122".getBytes, 3) :: 80 | BinaryData("121".getBytes, 2) :: 81 | BinaryData("123".getBytes, 4) :: Nil).toDF() 82 | binaryData.createOrReplaceTempView("binaryData") 83 | 84 | case class TestData3(a: Int, b: Option[Int]) 85 | 86 | val testData3 = 87 | TestHbase.sparkContext.parallelize( 88 | TestData3(1, None) :: 89 | TestData3(2, Some(2)) :: Nil).toDF() 90 | testData3.createOrReplaceTempView("testData3") 91 | 92 | case class UpperCaseData(N: Int, L: String) 93 | 94 | val upperCaseData = 95 | TestHbase.sparkContext.parallelize( 96 | UpperCaseData(1, "A") :: 97 | UpperCaseData(2, "B") :: 98 | UpperCaseData(3, "C") :: 99 | UpperCaseData(4, "D") :: 100 | UpperCaseData(5, "E") :: 101 | UpperCaseData(6, "F") :: Nil).toDF() 102 | upperCaseData.createOrReplaceTempView("upperCaseData") 103 | 104 | case class LowerCaseData(n: Int, l: String) 105 | 106 | val lowerCaseData = 107 | TestHbase.sparkContext.parallelize( 108 | LowerCaseData(1, "a") :: 109 | LowerCaseData(2, "b") :: 110 | LowerCaseData(3, "c") :: 111 | LowerCaseData(4, "d") :: Nil).toDF() 112 | lowerCaseData.createOrReplaceTempView("lowerCaseData") 113 | 114 | case class ArrayData(dt: Seq[Int], nestedData: Seq[Seq[Int]]) 115 | 116 | val arrayData = 117 | TestHbase.sparkContext.parallelize( 118 | ArrayData(Seq(1, 2, 3), Seq(Seq(1, 2, 3))) :: 119 | ArrayData(Seq(2, 3, 4), Seq(Seq(2, 3, 4))) :: Nil) 120 | arrayData.toDF().createOrReplaceTempView("arrayData") 121 | 122 | case class MapData(data: scala.collection.Map[Int, String]) 123 | 124 | val mapData = 125 | TestHbase.sparkContext.parallelize( 126 | MapData(Map(1 -> "a1", 2 -> "b1", 3 -> "c1", 4 -> "d1", 5 -> "e1")) :: 127 | MapData(Map(1 -> "a2", 2 -> "b2", 3 -> "c2", 4 -> "d2")) :: 128 | MapData(Map(1 -> "a3", 2 -> "b3", 3 -> "c3")) :: 129 | MapData(Map(1 -> "a4", 2 -> "b4")) :: 130 | MapData(Map(1 -> "a5")) :: Nil) 131 | mapData.toDF().createOrReplaceTempView("mapData") 132 | 133 | case class StringData(s: String) 134 | 135 | val repeatedData = 136 | TestHbase.sparkContext.parallelize(List.fill(2)(StringData("test"))).toDF() 137 | repeatedData.createOrReplaceTempView("repeatedData") 138 | 139 | val nullableRepeatedData = 140 | TestHbase.sparkContext.parallelize( 141 | List.fill(2)(StringData(null)) ++ 142 | List.fill(2)(StringData("test"))).toDF() 143 | nullableRepeatedData.createOrReplaceTempView("nullableRepeatedData") 144 | 145 | case class NullInts(a: Integer) 146 | 147 | val nullInts = 148 | TestHbase.sparkContext.parallelize( 149 | NullInts(1) :: 150 | NullInts(2) :: 151 | NullInts(3) :: 152 | NullInts(null) :: Nil 153 | ).toDF() 154 | nullInts.createOrReplaceTempView("nullInts") 155 | 156 | val allNulls = 157 | TestHbase.sparkContext.parallelize( 158 | NullInts(null) :: 159 | NullInts(null) :: 160 | NullInts(null) :: 161 | NullInts(null) :: Nil).toDF() 162 | allNulls.createOrReplaceTempView("allNulls") 163 | 164 | case class NullStrings(n: Int, s: String) 165 | 166 | val nullStrings = 167 | TestHbase.sparkContext.parallelize( 168 | NullStrings(1, "abc") :: 169 | NullStrings(2, "ABC") :: 170 | NullStrings(3, null) :: Nil).toDF() 171 | nullStrings.createOrReplaceTempView("nullStrings") 172 | 173 | case class TableName(tableName: String) 174 | 175 | TestHbase.sparkContext.parallelize(TableName("test") :: Nil).toDF() 176 | .createOrReplaceTempView("tableName") 177 | 178 | val unparsedStrings = 179 | TestHbase.sparkContext.parallelize( 180 | "1, A1, true, null" :: 181 | "2, B2, false, null" :: 182 | "3, C3, true, null" :: 183 | "4, D4, true, 2147483644" :: Nil) 184 | 185 | case class TimestampField(time: Timestamp) 186 | 187 | val timestamps = TestHbase.sparkContext.parallelize((1 to 3).map { i => 188 | TimestampField(new Timestamp(i)) 189 | }).toDF() 190 | timestamps.createOrReplaceTempView("timestamps") 191 | 192 | case class IntField(i: Int) 193 | 194 | // An RDD with 4 elements and 8 partitions 195 | val withEmptyParts = TestHbase.sparkContext.parallelize((1 to 4).map(IntField), 8).toDF() 196 | withEmptyParts.createOrReplaceTempView("withEmptyParts") 197 | 198 | case class Person(id: Int, name: String, age: Int) 199 | 200 | case class Salary(personId: Int, salary: Double) 201 | 202 | val person = TestHbase.sparkContext.parallelize( 203 | Person(0, "mike", 30) :: 204 | Person(1, "jim", 20) :: Nil) 205 | person.toDF().createOrReplaceTempView("person") 206 | val salary = TestHbase.sparkContext.parallelize( 207 | Salary(0, 2000.0) :: 208 | Salary(1, 1000.0) :: Nil).toDF() 209 | salary.createOrReplaceTempView("salary") 210 | 211 | case class ComplexData(m: Map[Int, String], s: TestData, a: Seq[Int], b: Boolean) 212 | 213 | val complexData = 214 | TestHbase.sparkContext.parallelize( 215 | ComplexData(Map(1 -> "1"), TestData(1, "1"), Seq(1), b = true) 216 | :: ComplexData(Map(2 -> "2"), TestData(2, "2"), Seq(2), b = false) 217 | :: Nil).toDF() 218 | complexData.createOrReplaceTempView("complexData") 219 | } 220 | -------------------------------------------------------------------------------- /src/test/scala/org/apache/spark/sql/hbase/TestHbase.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * 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 org.apache.spark.sql.hbase 19 | 20 | import org.apache.hadoop.hbase.{HBaseTestingUtility, MiniHBaseCluster} 21 | import org.apache.spark.{SparkConf, SparkContext} 22 | 23 | 24 | object TestHbase extends HBaseSparkSession( 25 | new SparkContext("local[2]", "TestSQLContext", new SparkConf(true) 26 | .set("spark.hadoop.hbase.zookeeper.quorum", "localhost"))) { 27 | 28 | @transient val testUtil: HBaseTestingUtility = 29 | new HBaseTestingUtility(sparkContext.hadoopConfiguration) 30 | 31 | testUtil.getConfiguration.set("hbase.fs.tmp.dir", System.getProperty("java.io.tmpdir")) 32 | 33 | testUtil.startMiniZKCluster 34 | logDebug(s"Spin up hbase minicluster with 1 master, 1 RS, 1 dataNode") 35 | 36 | @transient val cluster: MiniHBaseCluster = testUtil.startMiniHBaseCluster(1, 1) 37 | logInfo(s"Started HBaseMiniCluster with regions = ${cluster.countServedRegions}") 38 | 39 | logInfo(s"Configuration zkPort=" 40 | + s"${sparkContext.hadoopConfiguration.get("hbase.zookeeper.property.clientPort")}") 41 | 42 | // The following operation will initialize the HBaseCatalog. 43 | // And it should be done after starting MiniHBaseCluster 44 | sharedState.externalCatalog.asInstanceOf[HBaseCatalog].deploySuccessfully_internal = Some(true) 45 | sharedState.externalCatalog.asInstanceOf[HBaseCatalog].pwdIsAccessible = true 46 | 47 | def hbaseAdmin = sharedState.externalCatalog.asInstanceOf[HBaseCatalog].admin 48 | } 49 | --------------------------------------------------------------------------------