From 2bfc5f1c078e7530ca6506a397e74fa38668b1a5 Mon Sep 17 00:00:00 2001 From: "Tak Lon (Stephen) Wu" Date: Thu, 13 Jan 2022 14:11:25 -0800 Subject: [PATCH 01/32] HBASE-26664 hbase-connector upgrades extra-enforcer-rules to 1.5.1 (#90) - hbase-connector fails to build with -Prelease after maven-enforcer-plugin upgraded to 3.0.0 from 3.0.0-M3 Signed-off-by: Peter Somogyi --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 034bff2a..93bf7578 100644 --- a/pom.xml +++ b/pom.xml @@ -143,7 +143,7 @@ 3.1.2 3.0.0-M5 3.0.0 - 1.2 + 1.5.1 0.14.0 2.5.0 From 036729dea0d1041e957f0349254f4ea1cf7cfba9 Mon Sep 17 00:00:00 2001 From: "Tak Lon (Stephen) Wu" Date: Mon, 7 Mar 2022 12:08:13 -0800 Subject: [PATCH 02/32] HBASE-22338 LICENSE file only contains Apache 2.0 (#93) Signed-off-by: Sean Busbey --- hbase-connectors-assembly/pom.xml | 90 +++++++++++++++++++ .../main/assembly/hbase-connectors-bin.xml | 18 +++- .../main/resources/supplemental-models.xml | 39 ++++++++ pom.xml | 11 ++- 4 files changed, 153 insertions(+), 5 deletions(-) create mode 100644 hbase-connectors-assembly/src/main/resources/supplemental-models.xml diff --git a/hbase-connectors-assembly/pom.xml b/hbase-connectors-assembly/pom.xml index af7356fb..6e857d8b 100755 --- a/hbase-connectors-assembly/pom.xml +++ b/hbase-connectors-assembly/pom.xml @@ -50,6 +50,38 @@ + + org.codehaus.mojo + build-helper-maven-plugin + + + negate-license-bundles-property + + bsh-property + + + skip.license.check = !${license.bundles.dependencies}; + + skip.license.check + + + + + + create-license-file-path-property + + regex-property + + + license.aggregate.path + ${project.build.directory}/maven-shared-archive-resources/META-INF/LICENSE + \\ + / + false + + + + maven-assembly-plugin @@ -82,9 +114,11 @@ test build-classpath + copy-dependencies ${project.parent.basedir}/target/cached_classpath.txt + ${project.build.directory}/dependency @@ -119,6 +153,8 @@ supplemental-models.xml + + src/main/resources/supplemental-models.xml @@ -151,6 +187,60 @@ + + org.codehaus.mojo + buildnumber-maven-plugin + ${buildnumber.maven.version} + + + org.apache.maven.plugins + maven-enforcer-plugin + ${enforcer.version} + + + org.codehaus.mojo + extra-enforcer-rules + ${extra.enforcer.version} + + + + + check-aggregate-license + + process-resources + + enforce + + + + + + File license = new File("${license.aggregate.path}"); + + // Beanshell does not support try-with-resources, + // so we must close this scanner manually + Scanner scanner = new Scanner(license); + + while (scanner.hasNextLine()) { + if (scanner.nextLine().startsWith("ERROR:")) { + scanner.close(); + return false; + } + } + scanner.close(); + return true; + + + License errors detected, for more detail find ERROR in + ${license.aggregate.path} + + + + ${skip.license.check} + + + + diff --git a/hbase-connectors-assembly/src/main/assembly/hbase-connectors-bin.xml b/hbase-connectors-assembly/src/main/assembly/hbase-connectors-bin.xml index c38c74b3..1a3caf41 100755 --- a/hbase-connectors-assembly/src/main/assembly/hbase-connectors-bin.xml +++ b/hbase-connectors-assembly/src/main/assembly/hbase-connectors-bin.xml @@ -52,12 +52,22 @@ - ../LICENSE.txt - ${file.separator} + ${project.build.directory}/maven-shared-archive-resources/META-INF/LICENSE + . + LICENSE.txt + unix - ../NOTICE.txt - ${file.separator} + ${project.build.directory}/maven-shared-archive-resources/META-INF/NOTICE + . + NOTICE.txt + unix + + + ${basedir}/src/main/resources/META-INF/LEGAL + . + LEGAL + unix ../README.md diff --git a/hbase-connectors-assembly/src/main/resources/supplemental-models.xml b/hbase-connectors-assembly/src/main/resources/supplemental-models.xml new file mode 100644 index 00000000..4e2a7aa1 --- /dev/null +++ b/hbase-connectors-assembly/src/main/resources/supplemental-models.xml @@ -0,0 +1,39 @@ + + + + + + + + + javax.xml.stream + stax-api + + + CDDL 1.1 + https://github.com/javaee/activation/blob/master/LICENSE.txt + repo + + + + + diff --git a/pom.xml b/pom.xml index 93bf7578..4d9bc566 100644 --- a/pom.xml +++ b/pom.xml @@ -128,6 +128,15 @@ ${maven.build.timestamp} 1.8 ${compileSource} + + false + false + false + false + false + 1.4 + 3.5.0 2.4.9 1.6.0 @@ -391,7 +400,7 @@ org.codehaus.mojo buildnumber-maven-plugin - 1.4 + ${buildnumber.maven.version} validate From 25d60ab2d57b700908cfc12d0d54338121d02883 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Horv=C3=A1th=20D=C3=B3ra?= Date: Fri, 5 Aug 2022 12:44:14 +0200 Subject: [PATCH 03/32] HBASE-27272: Enable code coverage reporting to SonarQube in hbase-connectors (#99) Signed-off-by: Balazs Meszaros --- dev-support/code-coverage/README.md | 49 ++++++++++ dev-support/code-coverage/run-coverage.sh | 69 ++++++++++++++ pom.xml | 76 +++++++++++++++ spark/hbase-spark/pom.xml | 92 ++++++++++++++++++ test-reporting/pom.xml | 109 ++++++++++++++++++++++ 5 files changed, 395 insertions(+) create mode 100644 dev-support/code-coverage/README.md create mode 100755 dev-support/code-coverage/run-coverage.sh create mode 100644 test-reporting/pom.xml diff --git a/dev-support/code-coverage/README.md b/dev-support/code-coverage/README.md new file mode 100644 index 00000000..b6af086f --- /dev/null +++ b/dev-support/code-coverage/README.md @@ -0,0 +1,49 @@ + + +# Code analysis + +The `run-coverage.sh` script runs maven with the coverage profile which generates the test coverage data for both java +and scala classes. +If the required parameters are given it also runs the sonar analysis and uploads the results to the given SonarQube +Server. + +## Running code analysis + +After running the script the code coverage results are generated under the `test-reporting/target/code-coverage/` +folder. +The JaCoCo code coverage library generated reports can be found under the `jacoco-reports` folder and the SCoverage +generated results can be found under the `scoverage-reports` folder. + +Here is how you can generate the code coverage reports: + +```./dev-support/code-coverage/run-coverage.sh``` + +## Publishing coverage results to SonarQube + +The required parameters for publishing the results to SonarQube are: + +- host URL, +- login credentials, +- project key + +The project name is an optional parameter. + +Here is an example command for running and publishing the coverage data: + +```./dev-support/code-coverage/run-coverage.sh -l ProjectCredentials -u https://exampleserver.com -k Project_Key -n Project_Name``` \ No newline at end of file diff --git a/dev-support/code-coverage/run-coverage.sh b/dev-support/code-coverage/run-coverage.sh new file mode 100755 index 00000000..c56d8461 --- /dev/null +++ b/dev-support/code-coverage/run-coverage.sh @@ -0,0 +1,69 @@ +#!/usr/bin/env bash +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +usage() { + echo + echo "options:" + echo "-h Display help" + echo "-u SonarQube Host URL" + echo "-l SonarQube Login Credentials" + echo "-k SonarQube Project Key" + echo "-n SonarQube Project Name" + echo + echo "Important:" + echo " The required parameters for publishing the coverage results to SonarQube:" + echo " - Host URL" + echo " - Login Credentials" + echo " - Project Key" + echo +} + +execute() { +SCRIPT_DIR="$(cd -- "$(dirname -- "${BASH_SOURCE[0]}")" &>/dev/null && pwd)" +MAIN_POM="${SCRIPT_DIR}/../../pom.xml" + + mvn -B -e -f "$MAIN_POM" clean install -DskipTests -DskipShade -Pcoverage + + mvn -B -e -f "$MAIN_POM" package -fn -Pcoverage + + # If the required parameters are given, the code coverage results are uploaded to the SonarQube Server + if [ -n "$SONAR_LOGIN" ] && [ -n "$SONAR_PROJECT_KEY" ] && [ -n "$SONAR_URL" ]; then + mvn -B -e -Pcoverage sonar:sonar -Dsonar.clover.reportPath=./target/clover/clover.xml \ + -Dsonar.host.url="$SONAR_URL" -Dsonar.login="$SONAR_LOGIN" -Dsonar.projectKey="$SONAR_PROJECT_KEY" -Dsonar.projectName="$SONAR_PROJECT_NAME" + fi +} + +while getopts ":u:l:k:n:h" option; do + case $option in + u) SONAR_URL=${OPTARG:-} ;; + l) SONAR_LOGIN=${OPTARG:-} ;; + k) SONAR_PROJECT_KEY=${OPTARG:-} ;; + n) SONAR_PROJECT_NAME=${OPTARG:-} ;; + h) # Display usage + usage + exit + ;; + \?) # Invalid option + echo "Error: Invalid option" + exit + ;; + esac +done + +# Start code analysis +execute diff --git a/pom.xml b/pom.xml index 4d9bc566..ba889f64 100644 --- a/pom.xml +++ b/pom.xml @@ -167,6 +167,10 @@ 3.0.1-b08 hbase-hadoop2-compat false + 3.9.1.2184 + 1.4.11 + 1.0.0 + 0.8.8 @@ -713,5 +717,77 @@ + + + coverage + + false + + + test-reporting + + + . + test-reporting/**/* + **/example/**/*,**/DumpToStringListener*,**/KafkaProxy* + **/target/classes + ${project.build.directory}/surefire-reports + + ${sonar.projectBaseDir}/test-reporting/target/site/jacoco-aggregate/jacoco.xml + + + ${sonar.projectBaseDir}/spark/hbase-spark/target/scoverage.xml + + + ${project.basedir} + ${main.basedir}/test-reporting/target/code-coverage + ${codeCoverageReportRootDir}/jacoco.exec + ${codeCoverageReportRootDir}/jacoco-reports + ${codeCoverageReportRootDir}/scoverage-reports + + + + + org.jacoco + jacoco-maven-plugin + + + org.sonarsource.scanner.maven + sonar-maven-plugin + ${sonar-maven-plugin.version} + + + + + + + org.scoverage + scoverage-maven-plugin + ${scoverage.version} + + src/test/java + IntegrationTest* + + + + org.jacoco + jacoco-maven-plugin + ${jacoco.version} + + **/example/**/* + + + + prepare-agent + + prepare-agent + + + + + + + + diff --git a/spark/hbase-spark/pom.xml b/spark/hbase-spark/pom.xml index 26c5c70f..7ac48283 100644 --- a/spark/hbase-spark/pom.xml +++ b/spark/hbase-spark/pom.xml @@ -509,6 +509,98 @@ + + + coverage + + false + + + src/main/ + ${project.parent.parent.basedir} + + + + + net.alchim31.maven + scala-maven-plugin + + src/test/scala + src/main/scala + + + + default-sbt-compile + + compile + testCompile + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + true + true + + + + org.scoverage + scoverage-maven-plugin + + 1.4.11 + true + true + false + ${project.build.sourceEncoding} + ${scoverageReportDir} + ${scoverageReportDir} + + + + instrument + + pre-compile + post-compile + + + + package + + package + report + + + + scoverage-report + + + report-only + + prepare-package + + + + + + + + + org.scoverage + scoverage-maven-plugin + + + + report-only + + + + + + + diff --git a/test-reporting/pom.xml b/test-reporting/pom.xml new file mode 100644 index 00000000..0a5f5df5 --- /dev/null +++ b/test-reporting/pom.xml @@ -0,0 +1,109 @@ + + + + 4.0.0 + + + hbase-connectors + org.apache.hbase.connectors + ${revision} + + + test-reporting + pom + ${revision} + Test Reporting + Test Reporting for Apache HBase Connectors + + + + ${project.parent.basedir} + + + + + + org.apache.hbase.connectors.kafka + hbase-kafka-proxy + ${revision} + + + org.apache.hbase.connectors.spark + hbase-spark + ${revision} + + + + org.scala-lang + scala-library + + + + org.scala-lang + scalap + + + com.google.code.findbugs + jsr305 + + + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + ${surefire.version} + + ${argLine} -Xms256m -Xmx2048m + 1 + random + + + + + + + + org.jacoco + jacoco-maven-plugin + + + report + + report-aggregate + + package + + ${jacocoReportDir} + ${project.build.sourceEncoding} + ${project.reporting.outputEncoding} + + + + + + + + \ No newline at end of file From 496cd582198f71544259ecfa9e207aca9ba18389 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Horv=C3=A1th=20D=C3=B3ra?= Date: Mon, 15 Aug 2022 10:53:40 +0200 Subject: [PATCH 04/32] HBASE-27285: Fix sonar report paths (#103) Signed-off-by: Peter Somogyi Reviewed-by: Mate Szalay-Beko --- dev-support/code-coverage/run-coverage.sh | 4 ++-- pom.xml | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dev-support/code-coverage/run-coverage.sh b/dev-support/code-coverage/run-coverage.sh index c56d8461..5379e01c 100755 --- a/dev-support/code-coverage/run-coverage.sh +++ b/dev-support/code-coverage/run-coverage.sh @@ -43,8 +43,8 @@ MAIN_POM="${SCRIPT_DIR}/../../pom.xml" # If the required parameters are given, the code coverage results are uploaded to the SonarQube Server if [ -n "$SONAR_LOGIN" ] && [ -n "$SONAR_PROJECT_KEY" ] && [ -n "$SONAR_URL" ]; then - mvn -B -e -Pcoverage sonar:sonar -Dsonar.clover.reportPath=./target/clover/clover.xml \ - -Dsonar.host.url="$SONAR_URL" -Dsonar.login="$SONAR_LOGIN" -Dsonar.projectKey="$SONAR_PROJECT_KEY" -Dsonar.projectName="$SONAR_PROJECT_NAME" + mvn -B -e -Pcoverage sonar:sonar -Dsonar.host.url="$SONAR_URL" -Dsonar.login="$SONAR_LOGIN" \ + -Dsonar.projectKey="$SONAR_PROJECT_KEY" -Dsonar.projectName="$SONAR_PROJECT_NAME" fi } diff --git a/pom.xml b/pom.xml index ba889f64..186d21f5 100644 --- a/pom.xml +++ b/pom.xml @@ -733,10 +733,10 @@ **/target/classes ${project.build.directory}/surefire-reports - ${sonar.projectBaseDir}/test-reporting/target/site/jacoco-aggregate/jacoco.xml + ${sonar.projectBaseDir}/test-reporting/target/code-coverage/jacoco-reports/jacoco.xml - ${sonar.projectBaseDir}/spark/hbase-spark/target/scoverage.xml + ${sonar.projectBaseDir}/test-reporting/target/code-coverage/scoverage-reports/scoverage.xml ${project.basedir} From 707e1c584a5e1653fb9bfb24d7429c761982adf3 Mon Sep 17 00:00:00 2001 From: richardantal Date: Tue, 29 Nov 2022 13:29:52 +0100 Subject: [PATCH 05/32] HBASE-27397 Spark-hbase support for 'startWith' predicate (#105) Signed-off-by: Balazs Meszaros --- .../hadoop/hbase/spark/DefaultSource.scala | 12 +++ .../hbase/spark/DynamicLogicExpression.scala | 21 ++++ .../spark/datasources/HBaseTableScanRDD.scala | 4 + .../hbase/spark/datasources/Utils.scala | 32 ++++++ .../hbase/spark/DefaultSourceSuite.scala | 50 +++++++++ .../hadoop/hbase/spark/StartsWithSuite.scala | 102 ++++++++++++++++++ 6 files changed, 221 insertions(+) create mode 100644 spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/StartsWithSuite.scala diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala index 84e9123f..0da7373f 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala @@ -538,6 +538,18 @@ case class HBaseRelation ( valueArray += byteValue } new GreaterThanOrEqualLogicExpression(attr, valueArray.length - 1) + case StringStartsWith(attr, value) => + val field = catalog.getField(attr) + if (field != null) { + if (field.isRowKey) { + val p = Utils.toBytes(value, field) + val endRange = Utils.incrementByteArray(p) + parentRowKeyFilter.mergeIntersect(new RowKeyFilter(null, new ScanRange(endRange, false, p, true))) + } + val byteValue = Utils.toBytes(value, field) + valueArray += byteValue + } + new StartsWithLogicExpression(attr, valueArray.length - 1) case Or(left, right) => val leftExpression = transverseFilterTree(parentRowKeyFilter, valueArray, left) val rightSideRowKeyFilter = new RowKeyFilter diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DynamicLogicExpression.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DynamicLogicExpression.scala index 4c35a7b4..dab311e2 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DynamicLogicExpression.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DynamicLogicExpression.scala @@ -130,6 +130,24 @@ class EqualLogicExpression (val columnName:String, } } +@InterfaceAudience.Private +class StartsWithLogicExpression (val columnName:String, + val valueFromQueryIndex:Int) extends DynamicLogicExpression{ + override def execute(columnToCurrentRowValueMap: + util.HashMap[String, ByteArrayComparable], + valueFromQueryValueArray:Array[Array[Byte]]): Boolean = { + val currentRowValue = columnToCurrentRowValueMap.get(columnName) + val valueFromQuery = valueFromQueryValueArray(valueFromQueryIndex) + + currentRowValue != null && valueFromQuery != null && currentRowValue.length >= valueFromQuery.length && + Bytes.equals(valueFromQuery,0, valueFromQuery.length, currentRowValue.bytes, + currentRowValue.offset, valueFromQuery.length) + } + override def appendToExpression(strBuilder: StringBuilder): Unit = { + strBuilder.append(columnName + " startsWith " + valueFromQueryIndex) + } +} + @InterfaceAudience.Private class IsNullLogicExpression (val columnName:String, val isNot:Boolean) extends DynamicLogicExpression{ @@ -242,6 +260,9 @@ object DynamicLogicExpressionBuilder { } else if (command.equals("!=")) { (new EqualLogicExpression(expressionArray(offSet), expressionArray(offSet + 2).toInt, true), offSet + 3) + } else if (command.equals("startsWith")) { + (new StartsWithLogicExpression(expressionArray(offSet), + expressionArray(offSet + 2).toInt), offSet + 3) } else if (command.equals("isNull")) { (new IsNullLogicExpression(expressionArray(offSet), false), offSet + 2) } else if (command.equals("isNotNull")) { diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala index 346983c1..fe325f7d 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala @@ -87,6 +87,10 @@ class HBaseTableScanRDD(relation: HBaseRelation, None } }.toArray + if (log.isDebugEnabled) { + logDebug(s"Partitions: ${ps.size}"); + ps.foreach(x => logDebug(x.toString)) + } regions.release() ShutdownHookManager.affixShutdownHook( new Thread() { override def run() { diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/Utils.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/Utils.scala index 6b96bcc5..05d80d41 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/Utils.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/Utils.scala @@ -89,4 +89,36 @@ object Utils { } } } + + // increment Byte array's value by 1 + def incrementByteArray(array: Array[Byte]): Array[Byte] = { + if (array.length == 0) { + return null + } + var index = -1 // index of the byte we have to increment + var a = array.length - 1 + + while (a >= 0) { + if (array(a) != (-1).toByte) { + index = a + a = -1 // break from the loop because we found a non -1 element + } + a = a - 1 + } + + if (index < 0) { + return null + } + val returnArray = new Array[Byte](array.length) + + for (a <- 0 until index) { + returnArray(a) = array(a) + } + returnArray(index) = (array(index) + 1).toByte + for (a <- index + 1 until array.length) { + returnArray(a) = 0.toByte + } + + returnArray + } } diff --git a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DefaultSourceSuite.scala b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DefaultSourceSuite.scala index 72a84cf1..366c9baa 100644 --- a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DefaultSourceSuite.scala +++ b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DefaultSourceSuite.scala @@ -954,6 +954,56 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { assert(s.count() == 6) } + test("filtered query01") { + val sql = sqlContext + import sql.implicits._ + val df = withCatalog(writeCatalog) + val s = df.filter(col("col0").startsWith("row00")) + .select("col0", "col1") + s.show() + assert(s.count() == 10) + } + + test("startsWith filtered query 1") { + val sql = sqlContext + import sql.implicits._ + val df = withCatalog(writeCatalog) + val s = df.filter(col("col0").startsWith("row005")) + .select("col0", "col1") + s.show() + assert(s.count() == 1) + } + + test("startsWith filtered query 2") { + val sql = sqlContext + import sql.implicits._ + val df = withCatalog(writeCatalog) + val s = df.filter(col("col0").startsWith("row")) + .select("col0", "col1") + s.show() + assert(s.count() == 256) + } + + test("startsWith filtered query 3") { + val sql = sqlContext + import sql.implicits._ + val df = withCatalog(writeCatalog) + val s = df.filter(col("col0").startsWith("row19")) + .select("col0", "col1") + s.show() + assert(s.count() == 10) + } + + test("startsWith filtered query 4") { + val sql = sqlContext + import sql.implicits._ + val df = withCatalog(writeCatalog) + val s = df.filter(col("col0").startsWith("")) + .select("col0", "col1") + s.show() + assert(s.count() == 256) + } + test("Timestamp semantics") { val sql = sqlContext import sql.implicits._ diff --git a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/StartsWithSuite.scala b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/StartsWithSuite.scala new file mode 100644 index 00000000..2dde0aef --- /dev/null +++ b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/StartsWithSuite.scala @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.spark + + +import org.apache.hadoop.hbase.spark.datasources.Utils +import org.apache.hadoop.hbase.util.Bytes +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} + +class StartsWithSuite extends FunSuite with +BeforeAndAfterEach with BeforeAndAfterAll with Logging { + + test("simple1") { + val t = new Array[Byte](2) + t(0) = 1.toByte + t(1) = 2.toByte + + val expected = new Array[Byte](2) + expected(0) = 1.toByte + expected(1) = 3.toByte + + val res = Utils.incrementByteArray(t) + assert(res.sameElements(expected)) + } + + test("simple2") { + val t = new Array[Byte](1) + t(0) = 87.toByte + + val expected = new Array[Byte](1) + expected(0) = 88.toByte + + val res = Utils.incrementByteArray(t) + assert(res.sameElements(expected)) + } + + test("overflow1") { + val t = new Array[Byte](2) + t(0) = 1.toByte + t(1) = (-1).toByte + + val expected = new Array[Byte](2) + expected(0) = 2.toByte + expected(1) = 0.toByte + + val res = Utils.incrementByteArray(t) + + assert(res.sameElements(expected)) + } + + test("overflow2") { + val t = new Array[Byte](2) + t(0) = (-1).toByte + t(1) = (-1).toByte + + val expected = null + + val res = Utils.incrementByteArray(t) + + assert(res == expected) + } + + test("max-min-value") { + val t = new Array[Byte](2) + t(0) = 1.toByte + t(1) = (127).toByte + + val expected = new Array[Byte](2) + expected(0) = 1.toByte + expected(1) = (-128).toByte + + val res = Utils.incrementByteArray(t) + assert(res.sameElements(expected)) + } + + test("complicated") { + val imput = "row005" + val expectedOutput = "row006" + + val t = Bytes.toBytes(imput) + val expected = Bytes.toBytes(expectedOutput) + + val res = Utils.incrementByteArray(t) + assert(res.sameElements(expected)) + } + +} From c221896a6acb5fdd9ae98f9f2eb1ce2c76699ab2 Mon Sep 17 00:00:00 2001 From: Istvan Toth Date: Thu, 9 Feb 2023 14:36:40 +0100 Subject: [PATCH 06/32] HBASE-27624 Cannot Specify Namespace via the hbase.table Option in Spark Connector (#107) Signed-off-by: Peter Somogyi --- .../spark/datasources/HBaseTableCatalog.scala | 16 +++++++++++++--- .../hadoop/hbase/spark/HBaseCatalogSuite.scala | 6 +++++- 2 files changed, 18 insertions(+), 4 deletions(-) diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableCatalog.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableCatalog.scala index 61e77882..6ac0454c 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableCatalog.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableCatalog.scala @@ -289,9 +289,19 @@ object HBaseTableCatalog { @deprecated("Please use new json format to define HBaseCatalog") // TODO: There is no need to deprecate since this is the first release. def convert(parameters: Map[String, String]): Map[String, String] = { - val tableName = parameters.get(TABLE_KEY).getOrElse(null) + val nsTableName = parameters.get(TABLE_KEY).getOrElse(null) // if the hbase.table is not defined, we assume it is json format already. - if (tableName == null) return parameters + if (nsTableName == null) return parameters + val tableParts = nsTableName.trim.split(':') + val tableNamespace = if (tableParts.length == 1) { + "default" + } else if (tableParts.length == 2) { + tableParts(0) + } else { + throw new IllegalArgumentException("Invalid table name '" + nsTableName + + "' should be ':' or '' ") + } + val tableName = tableParts(tableParts.length - 1) val schemaMappingString = parameters.getOrElse(SCHEMA_COLUMNS_MAPPING_KEY, "") import scala.collection.JavaConverters._ val schemaMap = generateSchemaMappingMap(schemaMappingString).asScala.map(_._2.asInstanceOf[SchemaQualifierDefinition]) @@ -304,7 +314,7 @@ object HBaseTableCatalog { } val jsonCatalog = s"""{ - |"table":{"namespace":"default", "name":"${tableName}"}, + |"table":{"namespace":"${tableNamespace}", "name":"${tableName}"}, |"rowkey":"${rowkey.mkString(":")}", |"columns":{ |${cols.mkString(",")} diff --git a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseCatalogSuite.scala b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseCatalogSuite.scala index d6245a64..326c2395 100644 --- a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseCatalogSuite.scala +++ b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseCatalogSuite.scala @@ -87,11 +87,13 @@ class HBaseCatalogSuite extends FunSuite with BeforeAndAfterEach with BeforeAndA test("convert") { val m = Map("hbase.columns.mapping" -> "KEY_FIELD STRING :key, A_FIELD STRING c:a, B_FIELD DOUBLE c:b, C_FIELD BINARY c:c,", - "hbase.table" -> "t1") + "hbase.table" -> "NAMESPACE:TABLE") val map = HBaseTableCatalog.convert(m) val json = map.get(HBaseTableCatalog.tableCatalog).get val parameters = Map(HBaseTableCatalog.tableCatalog->json) val t = HBaseTableCatalog(parameters) + assert(t.namespace === "NAMESPACE") + assert(t.name == "TABLE") assert(t.getField("KEY_FIELD").isRowKey) assert(DataTypeParserWrapper.parse("STRING") === t.getField("A_FIELD").dt) assert(!t.getField("A_FIELD").isRowKey) @@ -104,6 +106,8 @@ class HBaseCatalogSuite extends FunSuite with BeforeAndAfterEach with BeforeAndA "KEY_FIELD STRING :key, A_FIELD STRING c:a, B_FIELD DOUBLE c:b, C_FIELD BINARY c:c,", "hbase.table" -> "t1") val t = HBaseTableCatalog(m) + assert(t.namespace === "default") + assert(t.name == "t1") assert(t.getField("KEY_FIELD").isRowKey) assert(DataTypeParserWrapper.parse("STRING") === t.getField("A_FIELD").dt) assert(!t.getField("A_FIELD").isRowKey) From 672f3deab0101025b74583c0e4510745b347f485 Mon Sep 17 00:00:00 2001 From: Sergey Soldatov Date: Tue, 14 Feb 2023 04:31:38 -0800 Subject: [PATCH 07/32] HBASE-27630: hbase-spark bulkload stage directory limited to hdfs only (#108) Signed-off-by: Peter Somogyi Reviewed-by: Istvan Toth --- .../hadoop/hbase/spark/HBaseContext.scala | 6 +- .../hadoop/hbase/spark/BulkLoadSuite.scala | 63 ++++++++++++++++++- 2 files changed, 64 insertions(+), 5 deletions(-) diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala index ee860c42..82a77b99 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala @@ -656,7 +656,7 @@ class HBaseContext(@transient val sc: SparkContext, hbaseForeachPartition(this, (it, conn) => { val conf = broadcastedConf.value.value - val fs = FileSystem.get(conf) + val fs = new Path(stagingDir).getFileSystem(conf) val writerMap = new mutable.HashMap[ByteArrayWrapper, WriterLength] var previousRow:Array[Byte] = HConstants.EMPTY_BYTE_ARRAY var rollOverRequested = false @@ -791,7 +791,7 @@ class HBaseContext(@transient val sc: SparkContext, hbaseForeachPartition(this, (it, conn) => { val conf = broadcastedConf.value.value - val fs = FileSystem.get(conf) + val fs = new Path(stagingDir).getFileSystem(conf) val writerMap = new mutable.HashMap[ByteArrayWrapper, WriterLength] var previousRow:Array[Byte] = HConstants.EMPTY_BYTE_ARRAY var rollOverRequested = false @@ -973,7 +973,7 @@ class HBaseContext(@transient val sc: SparkContext, val wl = writerMap.getOrElseUpdate(new ByteArrayWrapper(family), { val familyDir = new Path(stagingDir, Bytes.toString(family)) - fs.mkdirs(familyDir) + familyDir.getFileSystem(conf).mkdirs(familyDir); val loc:HRegionLocation = { try { diff --git a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/BulkLoadSuite.scala b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/BulkLoadSuite.scala index 59435bb9..1c148a90 100644 --- a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/BulkLoadSuite.scala +++ b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/BulkLoadSuite.scala @@ -18,16 +18,20 @@ package org.apache.hadoop.hbase.spark import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.hadoop.hbase.client.{Get, ConnectionFactory} +import org.apache.hadoop.hbase.client.{ConnectionFactory, Get} import org.apache.hadoop.hbase.io.hfile.{CacheConfig, HFile} import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles -import org.apache.hadoop.hbase.{HConstants, CellUtil, HBaseTestingUtility, TableName} +import org.apache.hadoop.hbase.{CellUtil, HBaseTestingUtility, HConstants, TableName} import org.apache.hadoop.hbase.util.Bytes import org.apache.hadoop.hbase.spark.HBaseRDDFunctions._ import org.apache.spark.SparkContext import org.junit.rules.TemporaryFolder import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} +import java.io.File +import java.net.URI +import java.nio.file.Files + class BulkLoadSuite extends FunSuite with BeforeAndAfterEach with BeforeAndAfterAll with Logging { @transient var sc: SparkContext = null @@ -65,6 +69,61 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { sc.stop() } + test ("Staging dir: Test usage of staging dir on a separate filesystem") { + val config = TEST_UTIL.getConfiguration + + logInfo(" - creating table " + tableName) + TEST_UTIL.createTable(TableName.valueOf(tableName), + Array(Bytes.toBytes(columnFamily1), Bytes.toBytes(columnFamily2))) + + // Test creates rdd with 2 column families and + // write those to hfiles on local filesystem + // using bulkLoad functionality. We don't check the load functionality + // due the limitations of the HBase Minicluster + + val rdd = sc.parallelize(Array( + (Bytes.toBytes("1"), + (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo1"))), + (Bytes.toBytes("2"), + (Bytes.toBytes(columnFamily2), Bytes.toBytes("b"), Bytes.toBytes("bar.2"))))) + + val hbaseContext = new HBaseContext(sc, config) + val uri = Files.createTempDirectory("tmpDirPrefix").toUri + val stagingUri = new URI(uri + "staging_dir") + val stagingFolder = new File(stagingUri) + val fs = new Path(stagingUri.toString).getFileSystem(config) + try { + hbaseContext.bulkLoad[(Array[Byte], (Array[Byte], Array[Byte], Array[Byte]))](rdd, + TableName.valueOf(tableName), + t => { + val rowKey = t._1 + val family: Array[Byte] = t._2._1 + val qualifier = t._2._2 + val value: Array[Byte] = t._2._3 + + val keyFamilyQualifier = new KeyFamilyQualifier(rowKey, family, qualifier) + + Seq((keyFamilyQualifier, value)).iterator + }, + stagingUri.toString) + + assert(fs.listStatus(new Path(stagingFolder.getPath)).length == 2) + + } finally { + val admin = ConnectionFactory.createConnection(config).getAdmin + try { + admin.disableTable(TableName.valueOf(tableName)) + admin.deleteTable(TableName.valueOf(tableName)) + } finally { + admin.close() + } + fs.delete(new Path(stagingFolder.getPath), true) + + testFolder.delete() + + } + } + test("Wide Row Bulk Load: Test multi family and multi column tests " + "with all default HFile Configs.") { val config = TEST_UTIL.getConfiguration From ca7733ab566d8a274f65e4aced88fbdba8e6c209 Mon Sep 17 00:00:00 2001 From: Peter Somogyi Date: Tue, 28 Feb 2023 18:36:42 +0100 Subject: [PATCH 08/32] HBASE-27678 Update checkstyle to 8.45.1 (#111) Signed-off-by: Balazs Meszaros --- pom.xml | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 186d21f5..ba1c3fe4 100644 --- a/pom.xml +++ b/pom.xml @@ -148,8 +148,9 @@ ${hadoop-three.version} 1.7.25 1.2.17 - 8.28 - 3.1.2 + 8.45.1 + 3.2.1 + 2.5.0 3.0.0-M5 3.0.0 1.5.1 @@ -429,7 +430,7 @@ org.apache.hbase hbase-checkstyle - ${hbase.version} + ${hbase.checkstyle.version} com.puppycrawl.tools From bb2902fc98d7c586a5424eb1e4e8c3b811863a52 Mon Sep 17 00:00:00 2001 From: Nihal Jain Date: Wed, 1 Mar 2023 15:02:35 +0530 Subject: [PATCH 09/32] HBASE-27639 Support hbase-connectors compilation with HBase 2.5.3, Hadoop 3.2.4 and Spark 3.2.3 (#110) * Added mockito-all as otherwise we get java.lang.NoClassDefFoundError: org/mockito/stubbing/Answer at org.apache.hadoop.hdfs.MiniDFSCluster.isNameNodeUp() for hadoop 3.2.4 * Exlclude hadoop-client-api and hadoop-client-runtime coming from Spark 3.2, else minicluster.start() fails * Exclude lower versioned paranamer coming from avro otherwise tests fail with java.lang.ArrayIndexOutOfBoundsException * Added spark.hadoopRDD.ignoreEmptySplits for test due to behaviour change in spark 3.2, where the below conf is true by default. We will get empty table as result (for small sized tables) for HBase version not having HBASE-26340 Signed-off-by: Rajeshbabu Chintaguntla Signed-off-by: Peter Somogyi Reviewed-by: Istvan Toth --- pom.xml | 13 ++++ spark/hbase-spark-it/pom.xml | 13 ++++ spark/hbase-spark/pom.xml | 13 ++++ .../hbase/spark/TestJavaHBaseContext.java | 23 ++++-- .../TestJavaHBaseContextForLargeRows.java | 74 +++++++++++++++++++ 5 files changed, 130 insertions(+), 6 deletions(-) create mode 100644 spark/hbase-spark/src/test/java/org/apache/hadoop/hbase/spark/TestJavaHBaseContextForLargeRows.java diff --git a/pom.xml b/pom.xml index ba1c3fe4..1bfe0cf7 100644 --- a/pom.xml +++ b/pom.xml @@ -142,6 +142,7 @@ 1.6.0 0.5.0 4.12 + 1.8.5 4.0.1 2.8.5 3.2.0 @@ -195,6 +196,12 @@ org.apache.avro avro ${avro.version} + + + com.thoughtworks.paranamer + paranamer + + org.slf4j @@ -233,6 +240,12 @@ junit ${junit.version} + + org.mockito + mockito-all + ${mockito-all.version} + test + org.apache.hbase.thirdparty hbase-shaded-miscellaneous diff --git a/spark/hbase-spark-it/pom.xml b/spark/hbase-spark-it/pom.xml index 976edba3..a7f42366 100644 --- a/spark/hbase-spark-it/pom.xml +++ b/spark/hbase-spark-it/pom.xml @@ -247,6 +247,14 @@ com.google.code.findbugs jsr305 + + org.apache.hadoop + hadoop-client-api + + + org.apache.hadoop + hadoop-client-runtime + @@ -287,6 +295,11 @@ junit test + + org.mockito + mockito-all + test + diff --git a/spark/hbase-spark/pom.xml b/spark/hbase-spark/pom.xml index 7ac48283..d8e290e5 100644 --- a/spark/hbase-spark/pom.xml +++ b/spark/hbase-spark/pom.xml @@ -88,6 +88,14 @@ xerces xercesImpl + + org.apache.hadoop + hadoop-client-api + + + org.apache.hadoop + hadoop-client-runtime + @@ -115,6 +123,11 @@ junit test + + org.mockito + mockito-all + test + org.scalatest scalatest_${scala.binary.version} diff --git a/spark/hbase-spark/src/test/java/org/apache/hadoop/hbase/spark/TestJavaHBaseContext.java b/spark/hbase-spark/src/test/java/org/apache/hadoop/hbase/spark/TestJavaHBaseContext.java index 793ed8ef..6601eb70 100644 --- a/spark/hbase-spark/src/test/java/org/apache/hadoop/hbase/spark/TestJavaHBaseContext.java +++ b/spark/hbase-spark/src/test/java/org/apache/hadoop/hbase/spark/TestJavaHBaseContext.java @@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; @@ -69,21 +70,29 @@ public class TestJavaHBaseContext implements Serializable { public static final HBaseClassTestRule TIMEOUT = HBaseClassTestRule.forClass(TestJavaHBaseContext.class); - private static transient JavaSparkContext JSC; + protected static transient JavaSparkContext JSC; private static HBaseTestingUtility TEST_UTIL; private static JavaHBaseContext HBASE_CONTEXT; private static final Logger LOG = LoggerFactory.getLogger(TestJavaHBaseContext.class); - byte[] tableName = Bytes.toBytes("t1"); - byte[] columnFamily = Bytes.toBytes("c"); + protected byte[] tableName = Bytes.toBytes("t1"); + protected byte[] columnFamily = Bytes.toBytes("c"); byte[] columnFamily1 = Bytes.toBytes("d"); String columnFamilyStr = Bytes.toString(columnFamily); String columnFamilyStr1 = Bytes.toString(columnFamily1); @BeforeClass public static void setUpBeforeClass() throws Exception { + // NOTE: We need to do this due to behaviour change in spark 3.2, where the below conf is true + // by default. We will get empty table as result (for small sized tables) for HBase version not + // having HBASE-26340 + SparkConf sparkConf = new SparkConf().set("spark.hadoopRDD.ignoreEmptySplits", "false"); + JSC = new JavaSparkContext("local", "JavaHBaseContextSuite", sparkConf); - JSC = new JavaSparkContext("local", "JavaHBaseContextSuite"); + init(); + } + + protected static void init() throws Exception { TEST_UTIL = new HBaseTestingUtility(); Configuration conf = TEST_UTIL.getConfiguration(); @@ -503,11 +512,12 @@ public String call(Result result) throws Exception { } } - private void populateTableWithMockData(Configuration conf, TableName tableName) + protected void populateTableWithMockData(Configuration conf, TableName tableName) throws IOException { try ( Connection conn = ConnectionFactory.createConnection(conf); - Table table = conn.getTable(tableName)) { + Table table = conn.getTable(tableName); + Admin admin = conn.getAdmin()) { List puts = new ArrayList<>(5); @@ -517,6 +527,7 @@ private void populateTableWithMockData(Configuration conf, TableName tableName) puts.add(put); } table.put(puts); + admin.flush(tableName); } } } diff --git a/spark/hbase-spark/src/test/java/org/apache/hadoop/hbase/spark/TestJavaHBaseContextForLargeRows.java b/spark/hbase-spark/src/test/java/org/apache/hadoop/hbase/spark/TestJavaHBaseContextForLargeRows.java new file mode 100644 index 00000000..81908ad5 --- /dev/null +++ b/spark/hbase-spark/src/test/java/org/apache/hadoop/hbase/spark/TestJavaHBaseContextForLargeRows.java @@ -0,0 +1,74 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.spark; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.MiscTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.spark.api.java.JavaSparkContext; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.experimental.categories.Category; + +@Category({ MiscTests.class, MediumTests.class }) +public class TestJavaHBaseContextForLargeRows extends TestJavaHBaseContext { + + @ClassRule public static final HBaseClassTestRule TIMEOUT = + HBaseClassTestRule.forClass(TestJavaHBaseContextForLargeRows.class); + + @BeforeClass public static void setUpBeforeClass() throws Exception { + JSC = new JavaSparkContext("local", "JavaHBaseContextSuite"); + + init(); + } + + protected void populateTableWithMockData(Configuration conf, TableName tableName) + throws IOException { + try (Connection conn = ConnectionFactory.createConnection(conf); + Table table = conn.getTable(tableName); + Admin admin = conn.getAdmin()) { + + List puts = new ArrayList<>(5); + + for (int i = 1; i < 6; i++) { + Put put = new Put(Bytes.toBytes(Integer.toString(i))); + // We are trying to generate a large row value here + char[] chars = new char[1024 * 1024 * 2]; + // adding '0' to convert int to char + Arrays.fill(chars, (char) (i + '0')); + put.addColumn(columnFamily, columnFamily, Bytes.toBytes(String.valueOf(chars))); + puts.add(put); + } + table.put(puts); + admin.flush(tableName); + } + } +} From 9fd25206dc69e8f04332b5c3f1a79f31de98f8b4 Mon Sep 17 00:00:00 2001 From: Nihal Jain Date: Fri, 3 Mar 2023 14:29:03 +0530 Subject: [PATCH 10/32] HBASE-27680 Bump hbase, hbase-thirdparty, hadoop and spark for hbase-connectors (#113) Signed-off-by: Balazs Meszaros --- pom.xml | 8 ++++---- spark/pom.xml | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/pom.xml b/pom.xml index 1bfe0cf7..e8c76103 100644 --- a/pom.xml +++ b/pom.xml @@ -138,14 +138,14 @@ 1.4 3.5.0 - 2.4.9 + 2.4.16 1.6.0 0.5.0 4.12 1.8.5 - 4.0.1 - 2.8.5 - 3.2.0 + 4.1.4 + 2.10.0 + 3.2.4 ${hadoop-three.version} 1.7.25 1.2.17 diff --git a/spark/pom.xml b/spark/pom.xml index 050d35ad..3127a1fb 100644 --- a/spark/pom.xml +++ b/spark/pom.xml @@ -45,10 +45,10 @@ 0.6.1 2.12.5 - 3.1.2 + 3.3.2 - 2.12.10 + 2.12.15 2.12 From 99d25ff45890fa5d5b6b0113f6eedd369733c1b1 Mon Sep 17 00:00:00 2001 From: Peter Somogyi Date: Mon, 6 Mar 2023 15:14:10 +0100 Subject: [PATCH 11/32] HBASE-27679 Bump junit to 4.13.2 (#112) Signed-off-by: Balazs Meszaros Signed-off-by: Duo Zhang --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index e8c76103..4c029398 100644 --- a/pom.xml +++ b/pom.xml @@ -141,7 +141,7 @@ 2.4.16 1.6.0 0.5.0 - 4.12 + 4.13.2 1.8.5 4.1.4 2.10.0 From 2d9972497e87c1265c9f3abda98461863ec0d458 Mon Sep 17 00:00:00 2001 From: Istvan Toth Date: Mon, 17 Apr 2023 10:54:11 +0200 Subject: [PATCH 12/32] HBASE-27801 Remove redundant avro.version property from Kafka connector (#115) Signed-off-by: Peter Somogyi --- kafka/pom.xml | 3 --- 1 file changed, 3 deletions(-) diff --git a/kafka/pom.xml b/kafka/pom.xml index a829e06c..bf6aff8e 100644 --- a/kafka/pom.xml +++ b/kafka/pom.xml @@ -35,9 +35,6 @@ hbase-kafka-model hbase-kafka-proxy - - 1.7.7 - From 416e589ea5dc5ac1a1b0ad17fe5d7658380e5d62 Mon Sep 17 00:00:00 2001 From: Zhen Wang <643348094@qq.com> Date: Wed, 2 Aug 2023 15:45:07 +0800 Subject: [PATCH 13/32] HBASE-27705: Respect SparkContext hadoop configuration (#114) Signed-off-by: Reid Chan --- .../scala/org/apache/hadoop/hbase/spark/DefaultSource.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala index 0da7373f..11702a07 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala @@ -130,7 +130,8 @@ case class HBaseRelation ( val hbaseContext:HBaseContext = if (useHBaseContext) { LatestHBaseContextCache.latest } else { - val config = HBaseConfiguration.create() + val hadoopConfig = sqlContext.sparkContext.hadoopConfiguration + val config = HBaseConfiguration.create(hadoopConfig) configResources.map(resource => resource.split(",").foreach(r => config.addResource(r))) new HBaseContext(sqlContext.sparkContext, config) } From 57abfcc10f782b4089387df3cdc5d3a544e67e33 Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Wed, 2 Aug 2023 15:58:01 +0800 Subject: [PATCH 14/32] HBASE-27176 [hbase-connectors] Fail to build hbase-connectors because of checkstyle error (#98) Signed-off-by: Reid Chan --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 4c029398..e12e59dd 100644 --- a/pom.xml +++ b/pom.xml @@ -464,7 +464,7 @@ hbase/checkstyle.xml hbase/checkstyle-suppressions.xml true - true + false From 5d56e16466669ba561905e3fc6bae76bc658776b Mon Sep 17 00:00:00 2001 From: Zhen Wang <643348094@qq.com> Date: Wed, 2 Aug 2023 16:03:03 +0800 Subject: [PATCH 15/32] [HBASE-27656] Make sure the close method of the SmartSonnection is called (#109) Signed-off-by: Reid Chan --- .../apache/hadoop/hbase/spark/HBaseContext.scala | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala index 82a77b99..091e78eb 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala @@ -478,8 +478,11 @@ class HBaseContext(@transient val sc: SparkContext, applyCreds // specify that this is a proxy user val smartConn = HBaseConnectionCache.getConnection(config) - f(it, smartConn.connection) - smartConn.close() + try { + f(it, smartConn.connection) + } finally { + if (smartConn != null) smartConn.close() + } } private def getConf(configBroadcast: Broadcast[SerializableWritable[Configuration]]): @@ -518,9 +521,11 @@ class HBaseContext(@transient val sc: SparkContext, applyCreds val smartConn = HBaseConnectionCache.getConnection(config) - val res = mp(it, smartConn.connection) - smartConn.close() - res + try { + mp(it, smartConn.connection) + } finally { + if (smartConn != null) smartConn.close() + } } /** From f8ee8f6b78432d8fa0884105ec04fdd58c3e37b3 Mon Sep 17 00:00:00 2001 From: ILuffZhe <37180946+ILuffZhe@users.noreply.github.com> Date: Wed, 2 Aug 2023 19:54:32 +0800 Subject: [PATCH 16/32] HBASE-27488 [hbase-connectors] Duplicate result when searching HBase by Spark (#106) Signed-off-by: Reid Chan --- .../spark/datasources/HBaseTableScanRDD.scala | 18 +++++++++++------- .../hbase/spark/DefaultSourceSuite.scala | 16 ++++++++++++++++ 2 files changed, 27 insertions(+), 7 deletions(-) diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala index fe325f7d..c334076f 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala @@ -114,16 +114,20 @@ class HBaseTableScanRDD(relation: HBaseRelation, hbaseContext: HBaseContext): Iterator[Result] = { g.grouped(relation.bulkGetSize).flatMap{ x => val gets = new ArrayList[Get](x.size) + val rowkeySet = new mutable.HashSet[String]() x.foreach{ y => - val g = new Get(y) - handleTimeSemantics(g) - columns.foreach { d => - if (!d.isRowKey) { - g.addColumn(d.cfBytes, d.colBytes) + if (!rowkeySet.contains(y.mkString("Array(", ", ", ")"))) { + val g = new Get(y) + handleTimeSemantics(g) + columns.foreach { d => + if (!d.isRowKey) { + g.addColumn(d.cfBytes, d.colBytes) + } } + filter.foreach(g.setFilter(_)) + gets.add(g) + rowkeySet.add(y.mkString("Array(", ", ", ")")) } - filter.foreach(g.setFilter(_)) - gets.add(g) } hbaseContext.applyCreds() val tmp = tbr.get(gets) diff --git a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DefaultSourceSuite.scala b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DefaultSourceSuite.scala index 366c9baa..47145d35 100644 --- a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DefaultSourceSuite.scala +++ b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DefaultSourceSuite.scala @@ -297,6 +297,22 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { assert(executionRules.rowKeyFilter.ranges.size == 0) } + /** + * A example of query three fields and also only using rowkey points for the filter, + * some rowkey points are duplicate. + */ + test("Test rowKey point only rowKey query, which contains duplicate rowkey") { + val results = sqlContext.sql("SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseTable1 " + + "WHERE " + + "(KEY_FIELD = 'get1' or KEY_FIELD = 'get2' or KEY_FIELD = 'get1')").take(10) + val executionRules = DefaultSourceStaticUtils.lastFiveExecutionRules.poll() + assert(results.length == 2) + assert(executionRules.dynamicLogicExpression.toExpressionString. + equals("( KEY_FIELD == 0 OR KEY_FIELD == 1 )")) + assert(executionRules.rowKeyFilter.points.size == 2) + assert(executionRules.rowKeyFilter.ranges.size == 0) + } + /** * A example of query three fields and also only using cell points for the filter */ From 4b81148869b59db64ebea16238e1b2d6894b67c2 Mon Sep 17 00:00:00 2001 From: Yohei Kishimoto Date: Fri, 4 Aug 2023 12:16:24 +0900 Subject: [PATCH 17/32] HBASE-26863 fix incorrect rowkey pushdown (#95) Signed-off-by: Reid Chan --- .../hadoop/hbase/spark/DefaultSource.scala | 16 ++++++--------- .../hbase/spark/DefaultSourceSuite.scala | 20 +++++++++++++++++++ 2 files changed, 26 insertions(+), 10 deletions(-) diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala index 11702a07..1a6cde21 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala @@ -606,11 +606,13 @@ class ScanRange(var upperBound:Array[Byte], var isUpperBoundEqualTo:Boolean, isLowerBoundEqualTo = if (lowerBoundCompare == 0) isLowerBoundEqualTo && other.isLowerBoundEqualTo + else if (lowerBoundCompare < 0) other.isLowerBoundEqualTo else isLowerBoundEqualTo isUpperBoundEqualTo = if (upperBoundCompare == 0) isUpperBoundEqualTo && other.isUpperBoundEqualTo - else isUpperBoundEqualTo + else if (upperBoundCompare < 0) isUpperBoundEqualTo + else other.isUpperBoundEqualTo } /** @@ -656,7 +658,6 @@ class ScanRange(var upperBound:Array[Byte], var isUpperBoundEqualTo:Boolean, * @return True is overlap false is not overlap */ def getOverLapScanRange(other:ScanRange): ScanRange = { - var leftRange:ScanRange = null var rightRange:ScanRange = null @@ -672,14 +673,9 @@ class ScanRange(var upperBound:Array[Byte], var isUpperBoundEqualTo:Boolean, } if (hasOverlap(leftRange, rightRange)) { - // Find the upper bound and lower bound - if (compareRange(leftRange.upperBound, rightRange.upperBound) >= 0) { - new ScanRange(rightRange.upperBound, rightRange.isUpperBoundEqualTo, - rightRange.lowerBound, rightRange.isLowerBoundEqualTo) - } else { - new ScanRange(leftRange.upperBound, leftRange.isUpperBoundEqualTo, - rightRange.lowerBound, rightRange.isLowerBoundEqualTo) - } + val result = new ScanRange(upperBound, isUpperBoundEqualTo, lowerBound, isLowerBoundEqualTo) + result.mergeIntersect(other) + result } else { null } diff --git a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DefaultSourceSuite.scala b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DefaultSourceSuite.scala index 47145d35..0f402db8 100644 --- a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DefaultSourceSuite.scala +++ b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DefaultSourceSuite.scala @@ -627,6 +627,26 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { assert(scanRange1.isUpperBoundEqualTo) } + test("Test Rowkey And with complex logic (HBASE-26863)") { + val results = sqlContext.sql("SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseTable1 " + + "WHERE " + + "( KEY_FIELD >= 'get1' AND KEY_FIELD <= 'get3' ) AND (A_FIELD = 'foo1' OR B_FIELD = '8')").take(10) + val executionRules = DefaultSourceStaticUtils.lastFiveExecutionRules.poll() + assert(results.length == 2) + + assert(executionRules.dynamicLogicExpression.toExpressionString + == "( ( ( KEY_FIELD isNotNull AND KEY_FIELD >= 0 ) AND KEY_FIELD <= 1 ) AND ( A_FIELD == 2 OR B_FIELD == 3 ) )") + + assert(executionRules.rowKeyFilter.points.size == 0) + assert(executionRules.rowKeyFilter.ranges.size == 1) + + val scanRange1 = executionRules.rowKeyFilter.ranges.get(0).get + assert(Bytes.equals(scanRange1.lowerBound,Bytes.toBytes("get1"))) + assert(Bytes.equals(scanRange1.upperBound, Bytes.toBytes("get3"))) + assert(scanRange1.isLowerBoundEqualTo) + assert(scanRange1.isUpperBoundEqualTo) + } + test("Test table that doesn't exist") { val catalog = s"""{ |"table":{"namespace":"default", "name":"t1NotThere"}, From bfcdf68e0bd0ea5f1ac00908a719a343d1bcdf75 Mon Sep 17 00:00:00 2001 From: Subrat Mishra Date: Fri, 4 Aug 2023 10:43:54 +0530 Subject: [PATCH 18/32] HBASE-27625 Bump commons-lang3 to 3.12.0, surefire to 3.0.0 and dependency manage reload4j to 1.2.25 to support jdk11 (#116) Signed-off-by: Istvan Toth Signed-off-by: Reid Chan --- pom.xml | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index e12e59dd..f06c147e 100644 --- a/pom.xml +++ b/pom.xml @@ -149,10 +149,11 @@ ${hadoop-three.version} 1.7.25 1.2.17 + 1.2.25 8.45.1 3.2.1 2.5.0 - 3.0.0-M5 + 3.0.0 3.0.0 1.5.1 0.14.0 @@ -161,7 +162,7 @@ 0.5.0 2.11.0 1.7.7 - 3.6 + 3.12.0 3.10.6.Final @@ -218,6 +219,11 @@ log4j ${log4j.version} + + ch.qos.reload4j + reload4j + ${reload4j.version} + org.glassfish javax.el From 35f962c0d224ca3ccb8075164cc009c351f8ca57 Mon Sep 17 00:00:00 2001 From: Subrat Mishra Date: Thu, 10 Aug 2023 08:21:47 +0530 Subject: [PATCH 19/32] HBASE-27883: Use log4j2 instead of log4j for logging and also removed slf4j-log4j12. (#117) Signed-off-by: Peter Somogyi Signed-off-by: Reid Chan --- kafka/hbase-kafka-proxy/pom.xml | 8 ++++++++ pom.xml | 30 ++++++++++++++++++++---------- spark/hbase-spark-it/pom.xml | 16 ++++++++++++++++ spark/hbase-spark/pom.xml | 12 ++++++++---- 4 files changed, 52 insertions(+), 14 deletions(-) diff --git a/kafka/hbase-kafka-proxy/pom.xml b/kafka/hbase-kafka-proxy/pom.xml index 356838d7..51f4267b 100755 --- a/kafka/hbase-kafka-proxy/pom.xml +++ b/kafka/hbase-kafka-proxy/pom.xml @@ -118,6 +118,14 @@ junit junit + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + diff --git a/pom.xml b/pom.xml index f06c147e..d0a6039d 100644 --- a/pom.xml +++ b/pom.xml @@ -138,7 +138,7 @@ 1.4 3.5.0 - 2.4.16 + 2.5.4 1.6.0 0.5.0 4.13.2 @@ -147,8 +147,8 @@ 2.10.0 3.2.4 ${hadoop-three.version} - 1.7.25 - 1.2.17 + 1.7.36 + 2.17.2 1.2.25 8.45.1 3.2.1 @@ -206,18 +206,28 @@ org.slf4j - slf4j-log4j12 + slf4j-api ${slf4j.version} - org.slf4j - slf4j-api - ${slf4j.version} + org.apache.logging.log4j + log4j-api + ${log4j2.version} + + + org.apache.logging.log4j + log4j-core + ${log4j2.version} + + + org.apache.logging.log4j + log4j-slf4j-impl + ${log4j2.version} - log4j - log4j - ${log4j.version} + org.apache.logging.log4j + log4j-1.2-api + ${log4j2.version} ch.qos.reload4j diff --git a/spark/hbase-spark-it/pom.xml b/spark/hbase-spark-it/pom.xml index a7f42366..524bc8fc 100644 --- a/spark/hbase-spark-it/pom.xml +++ b/spark/hbase-spark-it/pom.xml @@ -205,6 +205,14 @@ com.google.code.findbugs jsr305 + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + @@ -218,6 +226,14 @@ com.google.code.findbugs jsr305 + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + diff --git a/spark/hbase-spark/pom.xml b/spark/hbase-spark/pom.xml index d8e290e5..6c192ed4 100644 --- a/spark/hbase-spark/pom.xml +++ b/spark/hbase-spark/pom.xml @@ -37,10 +37,6 @@ org.slf4j slf4j-api - - org.slf4j - slf4j-log4j12 - org.apache.hbase.thirdparty hbase-shaded-miscellaneous @@ -399,6 +395,14 @@ com.google.code.findbugs jsr305 + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + From c1eb13ac1080f433f0c9f9a889f58e9b6d3a712e Mon Sep 17 00:00:00 2001 From: Nihal Jain Date: Fri, 11 Aug 2023 08:12:34 +0530 Subject: [PATCH 20/32] HBASE-27178 Use spotless to format code (including scala code) (#119) Signed-off-by: Reid Chan --- dev-support/.scalafmt.conf | 33 ++ dev-support/eclipse.importorder | 19 ++ dev-support/hbase_eclipse_formatter.xml | 418 ++++++++++++++++++++++++ dev-support/license-header | 17 + pom.xml | 125 +++++++ 5 files changed, 612 insertions(+) create mode 100644 dev-support/.scalafmt.conf create mode 100644 dev-support/eclipse.importorder create mode 100644 dev-support/hbase_eclipse_formatter.xml create mode 100644 dev-support/license-header diff --git a/dev-support/.scalafmt.conf b/dev-support/.scalafmt.conf new file mode 100644 index 00000000..0b8ff0e7 --- /dev/null +++ b/dev-support/.scalafmt.conf @@ -0,0 +1,33 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Template based off apache spark: https://github.com/apache/spark/blob/master/dev/.scalafmt.conf +align = none +align.openParenDefnSite = false +align.openParenCallSite = false +align.tokens = [] +importSelectors = "singleLine" +optIn = { + configStyleArguments = false +} +danglingParentheses.preset = false +docstrings.style = Asterisk +# See https://github.com/scalameta/scalafmt/issues/1387 +docstrings.wrap = no +maxColumn = 100 +runner.dialect = scala212 +version = 3.7.12 diff --git a/dev-support/eclipse.importorder b/dev-support/eclipse.importorder new file mode 100644 index 00000000..68f4e8b2 --- /dev/null +++ b/dev-support/eclipse.importorder @@ -0,0 +1,19 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +#Organize Import Order +3=org.apache.hadoop.hbase.shaded +2=org.apache.hbase.thirdparty +1= +0=\# diff --git a/dev-support/hbase_eclipse_formatter.xml b/dev-support/hbase_eclipse_formatter.xml new file mode 100644 index 00000000..99000a62 --- /dev/null +++ b/dev-support/hbase_eclipse_formatter.xml @@ -0,0 +1,418 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/dev-support/license-header b/dev-support/license-header new file mode 100644 index 00000000..2379ddac --- /dev/null +++ b/dev-support/license-header @@ -0,0 +1,17 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ diff --git a/pom.xml b/pom.xml index d0a6039d..1d5ef6c8 100644 --- a/pom.xml +++ b/pom.xml @@ -151,6 +151,7 @@ 2.17.2 1.2.25 8.45.1 + 2.27.2 3.2.1 2.5.0 3.0.0 @@ -684,6 +685,130 @@ + + com.diffplug.spotless + spotless-maven-plugin + ${spotless.version} + + + + + **/generated/* + **/package-info.java + + + + Remove unhelpful javadoc stubs + (?m)^ *\* *@(?:param|throws|return) *\w* *\n + + + + + Purge single returns tag multi line + (?m)^ */\*\*\n *\* *@return *(.*) *\n *\*/$ + /** Returns $1 */ + + + Purge single returns tag single line + ^ */\*\* *@return *(.*) *\*/$ + /** Returns $1 */ + + + + ${session.executionRootDirectory}/dev-support/hbase_eclipse_formatter.xml + + + ${session.executionRootDirectory}/dev-support/eclipse.importorder + + + + + + + + + src/main/scala/**/*.scala + src/test/scala/**/*.scala + src/main/scala/**/*.sc + src/test/scala/**/*.sc + + + + 3.7.12 + ${session.executionRootDirectory}/dev-support/.scalafmt.conf + + + + + false + + + + + + + + **/*.xml + **/*.sh + **/*.py + **/Jenkinsfile* + **/*.md + *.md + **/*.txt + *.txt + + + **/target/** + **/dependency-reduced-pom.xml + + + + + + + + + src/main/java/**/*.java + src/test/java/**/*.java + src/main/scala/**/*.scala + src/test/scala/**/*.scala + src/main/scala/**/*.sc + src/test/scala/**/*.sc + + + **/generated/* + **/package-info.java + + + ${session.executionRootDirectory}/dev-support/license-header + package + + + + + From ee2b51c7b8e6b05ccc725ccdd72ce3560621bf96 Mon Sep 17 00:00:00 2001 From: Nihal Jain Date: Wed, 16 Aug 2023 08:28:55 +0530 Subject: [PATCH 21/32] HBASE-28006 Run spotless:apply on code base (#120) Signed-off-by: Reid Chan --- CHANGELOG.md | 2 - README.md | 2 +- RELEASENOTES.md | 1 - dev-support/.scalafmt.conf | 27 +- dev-support/code-coverage/README.md | 2 +- dev-support/jenkins/Jenkinsfile | 2 +- .../jenkins/jenkins_precommit_github_yetus.sh | 2 +- hbase-connectors-assembly/pom.xml | 44 +- kafka/README.md | 2 +- kafka/hbase-kafka-model/pom.xml | 77 +- kafka/hbase-kafka-proxy/pom.xml | 51 +- .../apache/hadoop/hbase/kafka/DropRule.java | 8 +- .../hbase/kafka/DumpToStringListener.java | 28 +- .../hbase/kafka/KafkaBridgeConnection.java | 66 +- .../apache/hadoop/hbase/kafka/KafkaProxy.java | 245 ++-- .../hbase/kafka/KafkaTableForBridge.java | 116 +- .../org/apache/hadoop/hbase/kafka/Rule.java | 41 +- .../hadoop/hbase/kafka/TopicRoutingRules.java | 97 +- .../apache/hadoop/hbase/kafka/TopicRule.java | 9 +- .../hbase/kafka/ProducerForTesting.java | 30 +- .../hadoop/hbase/kafka/TestDropRule.java | 122 +- .../hbase/kafka/TestProcessMutations.java | 30 +- .../hbase/kafka/TestQualifierMatching.java | 30 +- .../hadoop/hbase/kafka/TestRouteRules.java | 126 +- kafka/pom.xml | 5 +- pom.xml | 133 +-- spark/README.md | 1 - spark/hbase-spark-it/pom.xml | 292 ++--- .../spark/IntegrationTestSparkBulkLoad.java | 144 +-- spark/hbase-spark-protocol-shaded/pom.xml | 4 +- spark/hbase-spark-protocol/pom.xml | 4 +- spark/hbase-spark/pom.xml | 36 +- .../hbase/spark/SparkSQLPushDownFilter.java | 162 ++- .../JavaHBaseBulkDeleteExample.java | 25 +- .../hbasecontext/JavaHBaseBulkGetExample.java | 37 +- .../JavaHBaseBulkLoadExample.java | 47 +- .../hbasecontext/JavaHBaseBulkPutExample.java | 31 +- .../JavaHBaseDistributedScan.java | 27 +- .../JavaHBaseMapGetPutExample.java | 54 +- .../JavaHBaseStreamingBulkPutExample.java | 39 +- .../hbase/spark/BulkLoadPartitioner.scala | 24 +- .../hbase/spark/ByteArrayComparable.scala | 10 +- .../hadoop/hbase/spark/ByteArrayWrapper.scala | 27 +- .../ColumnFamilyQualifierMapKeyWrapper.scala | 57 +- .../hadoop/hbase/spark/DefaultSource.scala | 1041 +++++++++-------- .../hbase/spark/DynamicLogicExpression.scala | 214 ++-- .../spark/FamiliesQualifiersValues.scala | 39 +- .../hbase/spark/FamilyHFileWriteOptions.scala | 27 +- .../hbase/spark/HBaseConnectionCache.scala | 83 +- .../hadoop/hbase/spark/HBaseContext.scala | 898 +++++++------- .../hbase/spark/HBaseDStreamFunctions.scala | 66 +- .../hbase/spark/HBaseRDDFunctions.scala | 139 ++- .../hadoop/hbase/spark/JavaHBaseContext.scala | 298 ++--- .../hbase/spark/KeyFamilyQualifier.scala | 27 +- .../apache/hadoop/hbase/spark/Logging.scala | 18 +- .../hadoop/hbase/spark/NewHBaseRDD.scala | 34 +- .../hbase/spark/datasources/Bound.scala | 150 ++- .../datasources/DataTypeParserWrapper.scala | 18 +- .../spark/datasources/HBaseResources.scala | 38 +- .../spark/datasources/HBaseSparkConf.scala | 40 +- .../spark/datasources/HBaseTableCatalog.scala | 274 +++-- .../spark/datasources/HBaseTableScanRDD.scala | 198 ++-- .../spark/datasources/JavaBytesEncoder.scala | 127 +- .../spark/datasources/NaiveEncoder.scala | 242 ++-- .../spark/datasources/SchemaConverters.scala | 324 ++--- .../hbase/spark/datasources/SerDes.scala | 16 +- .../SerializableConfiguration.scala | 20 +- .../hbase/spark/datasources/Utils.scala | 34 +- .../hbase/spark/datasources/package.scala | 19 +- .../example/datasources/AvroSource.scala | 59 +- .../spark/example/datasources/DataType.scala | 82 +- .../example/datasources/HBaseSource.scala | 65 +- .../hbasecontext/HBaseBulkDeleteExample.scala | 39 +- .../hbasecontext/HBaseBulkGetExample.scala | 45 +- .../hbasecontext/HBaseBulkPutExample.scala | 62 +- .../HBaseBulkPutExampleFromFile.scala | 49 +- .../HBaseBulkPutTimestampExample.scala | 65 +- .../HBaseDistributedScanExample.scala | 28 +- .../HBaseStreamingBulkPutExample.scala | 33 +- .../example/rdd/HBaseBulkDeleteExample.scala | 39 +- .../example/rdd/HBaseBulkGetExample.scala | 49 +- .../example/rdd/HBaseBulkPutExample.scala | 103 +- .../rdd/HBaseForeachPartitionExample.scala | 68 +- .../rdd/HBaseMapPartitionExample.scala | 84 +- .../hbase/spark/TestJavaHBaseContext.java | 98 +- .../TestJavaHBaseContextForLargeRows.java | 16 +- .../hadoop/hbase/spark/BulkLoadSuite.scala | 559 +++++---- .../hbase/spark/DefaultSourceSuite.scala | 607 ++++++---- .../spark/DynamicLogicExpressionSuite.scala | 69 +- .../hbase/spark/HBaseCatalogSuite.scala | 54 +- .../spark/HBaseConnectionCacheSuite.scala | 66 +- .../hbase/spark/HBaseContextSuite.scala | 186 +-- .../spark/HBaseDStreamFunctionsSuite.scala | 111 +- .../hbase/spark/HBaseRDDFunctionsSuite.scala | 350 +++--- .../hadoop/hbase/spark/HBaseTestSource.scala | 49 +- .../hbase/spark/PartitionFilterSuite.scala | 544 ++++----- .../hadoop/hbase/spark/StartsWithSuite.scala | 20 +- .../hbase/spark/TableOutputFormatSuite.scala | 44 +- spark/pom.xml | 7 +- test-reporting/pom.xml | 2 +- 100 files changed, 5437 insertions(+), 4918 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 8879d49e..2bd209c3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -121,5 +121,3 @@ limitations under the License. | [HBASE-22210](https://issues.apache.org/jira/browse/HBASE-22210) | Fix hbase-connectors-assembly to include every jar | Major | hbase-connectors | | [HBASE-22266](https://issues.apache.org/jira/browse/HBASE-22266) | Add yetus personality to connectors to avoid scaladoc issues | Major | hbase-connectors | | [HBASE-22257](https://issues.apache.org/jira/browse/HBASE-22257) | Remove json4s and jackson dependency from hbase spark connector | Major | hbase-connectors | - - diff --git a/README.md b/README.md index 3534f511..d47b5a29 100644 --- a/README.md +++ b/README.md @@ -18,7 +18,7 @@ limitations under the License. # hbase-connectors -Connectors for [Apache HBase™](https://hbase.apache.org) +Connectors for [Apache HBase™](https://hbase.apache.org) * [Kafka Proxy](https://github.com/apache/hbase-connectors/tree/master/kafka) * [Spark](https://github.com/apache/hbase-connectors/tree/master/spark) diff --git a/RELEASENOTES.md b/RELEASENOTES.md index b3bbacf8..dc8844d3 100644 --- a/RELEASENOTES.md +++ b/RELEASENOTES.md @@ -150,4 +150,3 @@ New features in hbase-spark: \* support for Dataframe writes, \* avro support, \* catalog can be defined in json. - diff --git a/dev-support/.scalafmt.conf b/dev-support/.scalafmt.conf index 0b8ff0e7..70dccfcc 100644 --- a/dev-support/.scalafmt.conf +++ b/dev-support/.scalafmt.conf @@ -16,18 +16,39 @@ # # Template based off apache spark: https://github.com/apache/spark/blob/master/dev/.scalafmt.conf +# Align settings align = none align.openParenDefnSite = false align.openParenCallSite = false align.tokens = [] + +# Rewrites +rewrite.rules = [Imports] + +# Imports +rewrite.imports.sort = scalastyle +rewrite.imports.groups = [ + [".*"], + ["org.apache.hbase.thirdparty\\..*"], + ["org.apache.hadoop.hbase.shaded\\..*"] +] +rewrite.imports.contiguousGroups = no importSelectors = "singleLine" -optIn = { - configStyleArguments = false -} + +# Newlines +newlines.beforeCurlyLambdaParams = multiline +newlines.afterCurlyLambdaParams = squash danglingParentheses.preset = false +optIn.configStyleArguments = false + +# Scaladoc docstrings.style = Asterisk # See https://github.com/scalameta/scalafmt/issues/1387 docstrings.wrap = no + +# Max column maxColumn = 100 + +# Version runner.dialect = scala212 version = 3.7.12 diff --git a/dev-support/code-coverage/README.md b/dev-support/code-coverage/README.md index b6af086f..1917686e 100644 --- a/dev-support/code-coverage/README.md +++ b/dev-support/code-coverage/README.md @@ -46,4 +46,4 @@ The project name is an optional parameter. Here is an example command for running and publishing the coverage data: -```./dev-support/code-coverage/run-coverage.sh -l ProjectCredentials -u https://exampleserver.com -k Project_Key -n Project_Name``` \ No newline at end of file +```./dev-support/code-coverage/run-coverage.sh -l ProjectCredentials -u https://exampleserver.com -k Project_Key -n Project_Name``` diff --git a/dev-support/jenkins/Jenkinsfile b/dev-support/jenkins/Jenkinsfile index e9c2c12e..eabf47b3 100644 --- a/dev-support/jenkins/Jenkinsfile +++ b/dev-support/jenkins/Jenkinsfile @@ -139,4 +139,4 @@ pipeline { } } } -} \ No newline at end of file +} diff --git a/dev-support/jenkins/jenkins_precommit_github_yetus.sh b/dev-support/jenkins/jenkins_precommit_github_yetus.sh index 39264fcb..5a03adac 100755 --- a/dev-support/jenkins/jenkins_precommit_github_yetus.sh +++ b/dev-support/jenkins/jenkins_precommit_github_yetus.sh @@ -119,4 +119,4 @@ YETUS_ARGS+=("--github-repo=apache/hbase-connectors") echo "Launching yetus with command line:" echo "${TESTPATCHBIN} ${YETUS_ARGS[*]}" -/usr/bin/env bash "${TESTPATCHBIN}" "${YETUS_ARGS[@]}" \ No newline at end of file +/usr/bin/env bash "${TESTPATCHBIN}" "${YETUS_ARGS[@]}" diff --git a/hbase-connectors-assembly/pom.xml b/hbase-connectors-assembly/pom.xml index 6e857d8b..4bed23ae 100755 --- a/hbase-connectors-assembly/pom.xml +++ b/hbase-connectors-assembly/pom.xml @@ -1,6 +1,5 @@ - - + + 4.0.0 - hbase-connectors org.apache.hbase.connectors + hbase-connectors ${revision} ../ hbase-connectors-assembly - Apache HBase Connectors - Assembly - - Module that does project assembly and that is all that it does. - pom + Apache HBase Connectors - Assembly + Module that does project assembly and that is all that it does. true @@ -87,10 +84,10 @@ distro-assembly - package single + package hbase-connectors-${revision} false @@ -111,11 +108,11 @@ create-hbase-connectors-generated-classpath - test build-classpath copy-dependencies + test ${project.parent.basedir}/target/cached_classpath.txt ${project.build.directory}/dependency @@ -137,8 +134,7 @@ ${build.year} - ${license.debug.print.included} - + ${license.debug.print.included} ${license.bundles.dependencies} ${license.bundles.jquery} ${license.bundles.logo} @@ -148,8 +144,7 @@ org.apache.hbase:hbase-resource-bundle:${hbase.version} - org.apache.hbase:hbase-resource-bundle:${hbase.version} - + org.apache.hbase:hbase-resource-bundle:${hbase.version} supplemental-models.xml @@ -168,18 +163,17 @@ concat-NOTICE-files - package exec + package env bash -c cat maven-shared-archive-resources/META-INF/NOTICE \ - `find ${project.build.directory}/dependency -iname NOTICE -or -iname NOTICE.txt` - + `find ${project.build.directory}/dependency -iname NOTICE -or -iname NOTICE.txt` ${project.build.directory}/NOTICE.aggregate ${project.build.directory} @@ -206,16 +200,15 @@ check-aggregate-license - - process-resources enforce + + process-resources - - File license = new File("${license.aggregate.path}"); + File license = new File("${license.aggregate.path}"); // Beanshell does not support try-with-resources, // so we must close this scanner manually @@ -228,12 +221,9 @@ } } scanner.close(); - return true; - - - License errors detected, for more detail find ERROR in - ${license.aggregate.path} - + return true; + License errors detected, for more detail find ERROR in + ${license.aggregate.path} ${skip.license.check} diff --git a/kafka/README.md b/kafka/README.md index 02770529..6ac6f0f2 100755 --- a/kafka/README.md +++ b/kafka/README.md @@ -20,7 +20,7 @@ limitations under the License. Welcome to the HBase kafka proxy. The purpose of this proxy is to act as a _fake peer_. It receives replication events from a peer cluster and applies a set of rules (stored in -a _kafka-route-rules.xml_ file) to determine if the event should be forwarded to a +a _kafka-route-rules.xml_ file) to determine if the event should be forwarded to a kafka topic. If the mutation matches a rule, the mutation is converted to an avro object and the item is placed into the topic. diff --git a/kafka/hbase-kafka-model/pom.xml b/kafka/hbase-kafka-model/pom.xml index a9dc5fd6..ec5081ae 100644 --- a/kafka/hbase-kafka-model/pom.xml +++ b/kafka/hbase-kafka-model/pom.xml @@ -1,6 +1,5 @@ - + + + org.eclipse.m2e + lifecycle-mapping + 1.0.0 + + + + + + org.apache.avro + avro-maven-plugin + [1.7.7,) + + schema + + + + + + + + + + + + org.apache.avro @@ -45,10 +74,10 @@ ${avro.version} - generate-sources schema + generate-sources ${project.basedir}/src/main/avro/ ${project.build.directory}/generated-sources/java @@ -65,14 +94,14 @@ add-source - generate-sources - add-source + add-source + generate-sources - - ${project.build.directory}/generated-sources/java/ - + + ${project.build.directory}/generated-sources/java/ + @@ -97,36 +126,6 @@ maven-source-plugin - - - - - org.eclipse.m2e - lifecycle-mapping - 1.0.0 - - - - - - org.apache.avro - avro-maven-plugin - [1.7.7,) - - schema - - - - - - - - - - - - @@ -140,10 +139,10 @@ license-javadocs - prepare-package copy-resources + prepare-package ${project.build.directory}/apidocs diff --git a/kafka/hbase-kafka-proxy/pom.xml b/kafka/hbase-kafka-proxy/pom.xml index 51f4267b..8df70527 100755 --- a/kafka/hbase-kafka-proxy/pom.xml +++ b/kafka/hbase-kafka-proxy/pom.xml @@ -1,6 +1,5 @@ - + - maven-assembly-plugin - - true - - - - org.apache.maven.plugins - maven-checkstyle-plugin - - - org.slf4j @@ -172,9 +149,9 @@ org.apache.kafka kafka-clients + ${kafka-clients.version} test test - ${kafka-clients.version} @@ -183,6 +160,28 @@ ${commons-lang3.version} + + + + org.apache.maven.plugins + maven-site-plugin + + true + + + + + maven-assembly-plugin + + true + + + + org.apache.maven.plugins + maven-checkstyle-plugin + + + @@ -195,10 +194,10 @@ license-javadocs - prepare-package copy-resources + prepare-package ${project.build.directory}/apidocs diff --git a/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/DropRule.java b/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/DropRule.java index 8bc1effe..01f7a349 100644 --- a/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/DropRule.java +++ b/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/DropRule.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -6,9 +6,9 @@ * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

+ * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. diff --git a/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/DumpToStringListener.java b/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/DumpToStringListener.java index 477cddd8..c968acc8 100755 --- a/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/DumpToStringListener.java +++ b/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/DumpToStringListener.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -6,9 +6,9 @@ * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

+ * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. @@ -22,13 +22,11 @@ import java.util.Iterator; import java.util.Properties; import java.util.stream.Collectors; - import org.apache.avro.io.BinaryDecoder; import org.apache.avro.io.DecoderFactory; import org.apache.avro.specific.SpecificDatumReader; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.VersionInfo; - import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -44,14 +42,14 @@ import org.apache.hbase.thirdparty.org.apache.commons.cli.ParseException; /** - * connects to kafka and reads from the passed in topics. Parses each message into an avro object + * connects to kafka and reads from the passed in topics. Parses each message into an avro object * and dumps it to the console. */ @InterfaceAudience.Private public final class DumpToStringListener { private static final Logger LOG = LoggerFactory.getLogger(DumpToStringListener.class); - private DumpToStringListener(){ + private DumpToStringListener() { } public static void main(String[] args) { @@ -59,10 +57,9 @@ public static void main(String[] args) { VersionInfo.logVersion(); Options options = new Options(); - options.addRequiredOption("k", "kafkabrokers", true, "Kafka Brokers " + - "(comma delimited)"); - options.addRequiredOption("t", "kafkatopics", true,"Kafka Topics " - + "to subscribe to (comma delimited)"); + options.addRequiredOption("k", "kafkabrokers", true, "Kafka Brokers " + "(comma delimited)"); + options.addRequiredOption("t", "kafkatopics", true, + "Kafka Topics " + "to subscribe to (comma delimited)"); CommandLine commandLine = null; try { @@ -73,7 +70,7 @@ public static void main(String[] args) { } SpecificDatumReader dreader = - new SpecificDatumReader<>(HBaseKafkaEvent.SCHEMA$); + new SpecificDatumReader<>(HBaseKafkaEvent.SCHEMA$); String topic = commandLine.getOptionValue('t'); Properties props = new Properties(); @@ -105,8 +102,9 @@ public static void main(String[] args) { private static void printUsageAndExit(Options options, int exitCode) { HelpFormatter formatter = new HelpFormatter(); formatter.printHelp("hbase " + DumpToStringListener.class.getName(), "", options, - "\n[--kafkabrokers ] " + - "[-k ] \n", true); + "\n[--kafkabrokers ] " + + "[-k ] \n", + true); System.exit(exitCode); } } diff --git a/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/KafkaBridgeConnection.java b/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/KafkaBridgeConnection.java index 68a88aff..bcb6704a 100644 --- a/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/KafkaBridgeConnection.java +++ b/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/KafkaBridgeConnection.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -6,9 +6,9 @@ * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

+ * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. @@ -22,7 +22,6 @@ import java.io.IOException; import java.util.Properties; import java.util.concurrent.ExecutorService; - import org.apache.avro.io.DatumWriter; import org.apache.avro.specific.SpecificDatumWriter; import org.apache.commons.lang3.StringUtils; @@ -44,27 +43,25 @@ /** * a alternative implementation of a connection object that forwards the mutations to a kafka queue * depending on the routing rules (see kafka-route-rules.xml). - * */ + */ @InterfaceAudience.Private public class KafkaBridgeConnection implements Connection { private final Configuration conf; private volatile boolean closed = false; private TopicRoutingRules routingRules; - private Producer producer; + private Producer producer; private DatumWriter avroWriter = - new SpecificDatumWriter<>(HBaseKafkaEvent.getClassSchema()); - - - /** - * Public constructor - * @param conf hbase configuration - * @param pool executor pool - * @param user user who requested connection - * @throws IOException on error - */ - public KafkaBridgeConnection(Configuration conf, - ExecutorService pool, - User user) throws IOException { + new SpecificDatumWriter<>(HBaseKafkaEvent.getClassSchema()); + + /** + * Public constructor + * @param conf hbase configuration + * @param pool executor pool + * @param user user who requested connection + * @throws IOException on error + */ + public KafkaBridgeConnection(Configuration conf, ExecutorService pool, User user) + throws IOException { this.conf = conf; setupRules(); startKafkaConnection(); @@ -72,12 +69,12 @@ public KafkaBridgeConnection(Configuration conf, /** * for testing. - * @param conf hbase configuration + * @param conf hbase configuration * @param routingRules a set of routing rules - * @param producer a kafka producer + * @param producer a kafka producer */ public KafkaBridgeConnection(Configuration conf, TopicRoutingRules routingRules, - Producer producer) { + Producer producer) { this.conf = conf; this.producer = producer; this.routingRules = routingRules; @@ -86,7 +83,7 @@ public KafkaBridgeConnection(Configuration conf, TopicRoutingRules routingRules, private void setupRules() throws IOException { String file = this.conf.get(KafkaProxy.KAFKA_PROXY_RULES_FILE); routingRules = new TopicRoutingRules(); - try (FileInputStream fin = new FileInputStream(file);){ + try (FileInputStream fin = new FileInputStream(file);) { routingRules.parseRules(fin); } } @@ -94,28 +91,26 @@ private void setupRules() throws IOException { private void startKafkaConnection() throws IOException { Properties configProperties = new Properties(); - String kafkaPropsFile = conf.get(KafkaProxy.KAFKA_PROXY_KAFKA_PROPERTIES,""); - if (!StringUtils.isEmpty(kafkaPropsFile)){ - try (FileInputStream fs = new java.io.FileInputStream( - new File(kafkaPropsFile))){ + String kafkaPropsFile = conf.get(KafkaProxy.KAFKA_PROXY_KAFKA_PROPERTIES, ""); + if (!StringUtils.isEmpty(kafkaPropsFile)) { + try (FileInputStream fs = new java.io.FileInputStream(new File(kafkaPropsFile))) { configProperties.load(fs); } } else { - String kafkaServers =conf.get(KafkaProxy.KAFKA_PROXY_KAFKA_BROKERS); + String kafkaServers = conf.get(KafkaProxy.KAFKA_PROXY_KAFKA_BROKERS); configProperties.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaServers); } configProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, - "org.apache.kafka.common.serialization.ByteArraySerializer"); + "org.apache.kafka.common.serialization.ByteArraySerializer"); configProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, - "org.apache.kafka.common.serialization.ByteArraySerializer"); + "org.apache.kafka.common.serialization.ByteArraySerializer"); this.producer = new KafkaProducer(configProperties); } - - @Override - public void abort(String why, Throwable e) {} + public void abort(String why, Throwable e) { + } @Override public boolean isAborted() { @@ -194,7 +189,8 @@ public TableBuilder setWriteRpcTimeout(int timeout) { @Override public Table build() { - return new KafkaTableForBridge(tn,passedInConfiguration,routingRules,producer,avroWriter) ; + return new KafkaTableForBridge(tn, passedInConfiguration, routingRules, producer, + avroWriter); } }; } diff --git a/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/KafkaProxy.java b/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/KafkaProxy.java index 2c1aca12..5dda9576 100755 --- a/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/KafkaProxy.java +++ b/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/KafkaProxy.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -6,9 +6,9 @@ * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

+ * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. @@ -24,7 +24,6 @@ import java.util.Map; import java.util.UUID; import java.util.stream.Collectors; - import org.apache.commons.lang3.StringUtils; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; @@ -45,7 +44,6 @@ import org.apache.hadoop.hbase.util.VersionInfo; import org.apache.hadoop.util.GenericOptionsParser; import org.apache.yetus.audience.InterfaceAudience; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -55,14 +53,10 @@ import org.apache.hbase.thirdparty.org.apache.commons.cli.Options; import org.apache.hbase.thirdparty.org.apache.commons.cli.ParseException; - - /** - * hbase to kafka bridge. - * - * Starts up a region server and receives replication events, just like a peer - * cluster member. It takes the events and cell by cell determines how to - * route them (see kafka-route-rules.xml) + * hbase to kafka bridge. Starts up a region server and receives replication events, just like a + * peer cluster member. It takes the events and cell by cell determines how to route them (see + * kafka-route-rules.xml) */ @InterfaceAudience.Private public final class KafkaProxy { @@ -72,47 +66,45 @@ public final class KafkaProxy { public static final String KAFKA_PROXY_KAFKA_PROPERTIES = "kafkaproxy.kafka.properties"; public static final String KAFKA_PROXY_KAFKA_BROKERS = "kafkaproxy.kafka.brokers"; - private static Map DEFAULT_PROPERTIES = new HashMap<>(); - private static Map CAN_OVERRIDE_DEFAULT_PROPERTIES = new HashMap<>(); - + private static Map DEFAULT_PROPERTIES = new HashMap<>(); + private static Map CAN_OVERRIDE_DEFAULT_PROPERTIES = new HashMap<>(); static { - DEFAULT_PROPERTIES.put("hbase.cluster.distributed","true"); - DEFAULT_PROPERTIES.put("zookeeper.znode.parent","/kafkaproxy"); - DEFAULT_PROPERTIES.put("hbase.regionserver.info.port","17010"); + DEFAULT_PROPERTIES.put("hbase.cluster.distributed", "true"); + DEFAULT_PROPERTIES.put("zookeeper.znode.parent", "/kafkaproxy"); + DEFAULT_PROPERTIES.put("hbase.regionserver.info.port", "17010"); DEFAULT_PROPERTIES.put("hbase.client.connection.impl", - "org.apache.hadoop.hbase.kafka.KafkaBridgeConnection"); - DEFAULT_PROPERTIES.put("hbase.regionserver.admin.service","false"); - DEFAULT_PROPERTIES.put("hbase.regionserver.client.service","false"); - DEFAULT_PROPERTIES.put("hbase.wal.provider", - "org.apache.hadoop.hbase.wal.DisabledWALProvider"); - DEFAULT_PROPERTIES.put("hbase.regionserver.workers","false"); - DEFAULT_PROPERTIES.put("hfile.block.cache.size","0.0001"); - DEFAULT_PROPERTIES.put("hbase.mob.file.cache.size","0"); - DEFAULT_PROPERTIES.put("hbase.masterless","true"); - DEFAULT_PROPERTIES.put("hbase.regionserver.metahandler.count","1"); - DEFAULT_PROPERTIES.put("hbase.regionserver.replication.handler.count","1"); - DEFAULT_PROPERTIES.put("hbase.regionserver.handler.count","1"); - DEFAULT_PROPERTIES.put("hbase.ipc.server.read.threadpool.size","3"); - - CAN_OVERRIDE_DEFAULT_PROPERTIES.put("hbase.regionserver.port","17020"); + "org.apache.hadoop.hbase.kafka.KafkaBridgeConnection"); + DEFAULT_PROPERTIES.put("hbase.regionserver.admin.service", "false"); + DEFAULT_PROPERTIES.put("hbase.regionserver.client.service", "false"); + DEFAULT_PROPERTIES.put("hbase.wal.provider", "org.apache.hadoop.hbase.wal.DisabledWALProvider"); + DEFAULT_PROPERTIES.put("hbase.regionserver.workers", "false"); + DEFAULT_PROPERTIES.put("hfile.block.cache.size", "0.0001"); + DEFAULT_PROPERTIES.put("hbase.mob.file.cache.size", "0"); + DEFAULT_PROPERTIES.put("hbase.masterless", "true"); + DEFAULT_PROPERTIES.put("hbase.regionserver.metahandler.count", "1"); + DEFAULT_PROPERTIES.put("hbase.regionserver.replication.handler.count", "1"); + DEFAULT_PROPERTIES.put("hbase.regionserver.handler.count", "1"); + DEFAULT_PROPERTIES.put("hbase.ipc.server.read.threadpool.size", "3"); + + CAN_OVERRIDE_DEFAULT_PROPERTIES.put("hbase.regionserver.port", "17020"); } private static void printUsageAndExit(Options options, int exitCode) { HelpFormatter formatter = new HelpFormatter(); formatter.printHelp("hbase kafkaproxy start", "", options, - "\nTo run the kafka proxy as a daemon, execute " + - "hbase-connectors-daemon.sh start|stop kafkaproxy \n" + - "[--kafkabrokers (or -b) ] \n" + - "[--routerulesfile (or -r) ] \n" + - "[--kafkaproperties (or -f) ] \n" + - "[--peername (or -p) name of hbase peer to use (defaults to hbasekafka)]\n " + - "[--znode (or -z) root znode (defaults to /kafkaproxy)] \n" + - "[--enablepeer (or -e) enable peer on startup (defaults to false)]\n " + - "[--auto (or -a) auto create peer] " + - "\n", true); + "\nTo run the kafka proxy as a daemon, execute " + + "hbase-connectors-daemon.sh start|stop kafkaproxy \n" + + "[--kafkabrokers (or -b) ] \n" + + "[--routerulesfile (or -r) ] \n" + + "[--kafkaproperties (or -f) ] \n" + + "[--peername (or -p) name of hbase peer to use (defaults to hbasekafka)]\n " + + "[--znode (or -z) root znode (defaults to /kafkaproxy)] \n" + + "[--enablepeer (or -e) enable peer on startup (defaults to false)]\n " + + "[--auto (or -a) auto create peer] " + "\n", + true); System.exit(exitCode); } @@ -130,24 +122,20 @@ private KafkaProxy() { */ public static void main(String[] args) throws Exception { - Map otherProps = new HashMap<>(); + Map otherProps = new HashMap<>(); Options options = new Options(); - options.addRequiredOption("b", "kafkabrokers", true, - "Kafka Brokers (comma delimited)"); + options.addRequiredOption("b", "kafkabrokers", true, "Kafka Brokers (comma delimited)"); options.addOption("r", "routerulesfile", true, "file that has routing rules (defaults to conf/kafka-route-rules.xml"); options.addOption("f", "kafkaproperties", true, "Path to properties file that has the kafka connection properties"); - options.addRequiredOption("p", "peername", true, - "Name of hbase peer"); + options.addRequiredOption("p", "peername", true, "Name of hbase peer"); options.addOption("z", "znode", true, - "root zode to use in zookeeper (defaults to /kafkaproxy)"); - options.addOption("a", "autopeer", false, - "Create a peer automatically to the hbase cluster"); - options.addOption("e", "enablepeer", false, - "enable peer on startup (defaults to false)"); + "root zode to use in zookeeper (defaults to /kafkaproxy)"); + options.addOption("a", "autopeer", false, "Create a peer automatically to the hbase cluster"); + options.addOption("e", "enablepeer", false, "enable peer on startup (defaults to false)"); LOG.info("STARTING executorService " + HRegionServer.class.getSimpleName()); VersionInfo.logVersion(); @@ -168,9 +156,8 @@ public static void main(String[] args) throws Exception { printUsageAndExit(options, -1); } - - String peer=""; - if (!commandLine.hasOption('p')){ + String peer = ""; + if (!commandLine.hasOption('p')) { System.err.println("hbase peer id is required"); System.exit(-1); } else { @@ -180,154 +167,141 @@ public static void main(String[] args) throws Exception { boolean createPeer = false; boolean enablePeer = false; - if (commandLine.hasOption('a')){ - createPeer=true; + if (commandLine.hasOption('a')) { + createPeer = true; } - if (commandLine.hasOption("a")){ - enablePeer=true; + if (commandLine.hasOption("a")) { + enablePeer = true; } - String rulesFile = StringUtils.defaultIfBlank( - commandLine.getOptionValue("r"),"kafka-route-rules.xml"); + String rulesFile = + StringUtils.defaultIfBlank(commandLine.getOptionValue("r"), "kafka-route-rules.xml"); - if (!new File(rulesFile).exists()){ - if (KafkaProxy.class.getClassLoader().getResource(rulesFile)!=null){ + if (!new File(rulesFile).exists()) { + if (KafkaProxy.class.getClassLoader().getResource(rulesFile) != null) { rulesFile = KafkaProxy.class.getClassLoader().getResource(rulesFile).getFile(); } else { - System.err.println("Rules file " + rulesFile + - " is invalid"); + System.err.println("Rules file " + rulesFile + " is invalid"); System.exit(-1); } } - otherProps.put(KafkaProxy.KAFKA_PROXY_RULES_FILE,rulesFile); + otherProps.put(KafkaProxy.KAFKA_PROXY_RULES_FILE, rulesFile); - if (commandLine.hasOption('f')){ - otherProps.put(KafkaProxy.KAFKA_PROXY_KAFKA_PROPERTIES,commandLine.getOptionValue('f')); - } else if (commandLine.hasOption('b')){ - otherProps.put(KafkaProxy.KAFKA_PROXY_KAFKA_BROKERS,commandLine.getOptionValue('b')); + if (commandLine.hasOption('f')) { + otherProps.put(KafkaProxy.KAFKA_PROXY_KAFKA_PROPERTIES, commandLine.getOptionValue('f')); + } else if (commandLine.hasOption('b')) { + otherProps.put(KafkaProxy.KAFKA_PROXY_KAFKA_BROKERS, commandLine.getOptionValue('b')); } else { System.err.println("Kafka connection properites or brokers must be specified"); System.exit(-1); } - String zookeeperQ = conf.get("hbase.zookeeper.quorum") + ":" + - conf.get("hbase.zookeeper.property.clientPort"); + String zookeeperQ = + conf.get("hbase.zookeeper.quorum") + ":" + conf.get("hbase.zookeeper.property.clientPort"); ExponentialBackoffRetry retryPolicy = new ExponentialBackoffRetry(20000, 20); - try (CuratorFramework zk = CuratorFrameworkFactory.newClient(zookeeperQ, retryPolicy); - ) { + try (CuratorFramework zk = CuratorFrameworkFactory.newClient(zookeeperQ, retryPolicy);) { zk.start(); String rootZnode = "/kafkaproxy"; - setupZookeeperZnodes(zk,rootZnode,peer); - checkForOrCreateReplicationPeer(conf,zk,rootZnode,peer,createPeer,enablePeer); + setupZookeeperZnodes(zk, rootZnode, peer); + checkForOrCreateReplicationPeer(conf, zk, rootZnode, peer, createPeer, enablePeer); } @SuppressWarnings("unchecked") Class regionServerClass = (Class) conf - .getClass(HConstants.REGION_SERVER_IMPL, HRegionServer.class); + .getClass(HConstants.REGION_SERVER_IMPL, HRegionServer.class); List allArgs = DEFAULT_PROPERTIES.keySet().stream() - .map((argKey)->("-D"+argKey+"="+ DEFAULT_PROPERTIES.get(argKey))) - .collect(Collectors.toList()); + .map((argKey) -> ("-D" + argKey + "=" + DEFAULT_PROPERTIES.get(argKey))) + .collect(Collectors.toList()); allArgs.addAll(CAN_OVERRIDE_DEFAULT_PROPERTIES.keySet().stream() - .filter((argKey)->commandLineConf.get(argKey,"").equalsIgnoreCase("")) - .map((argKey)->("-D"+argKey+"="+ CAN_OVERRIDE_DEFAULT_PROPERTIES.get(argKey))) - .collect(Collectors.toList())); + .filter((argKey) -> commandLineConf.get(argKey, "").equalsIgnoreCase("")) + .map((argKey) -> ("-D" + argKey + "=" + CAN_OVERRIDE_DEFAULT_PROPERTIES.get(argKey))) + .collect(Collectors.toList())); - for (Map.Entry k : commandLineConf){ - allArgs.add("-D"+k.getKey()+"="+k.getValue()); + for (Map.Entry k : commandLineConf) { + allArgs.add("-D" + k.getKey() + "=" + k.getValue()); } - otherProps.keySet().stream() - .map((argKey)->("-D"+argKey+"="+ otherProps.get(argKey))) - .forEach((item)->allArgs.add(item)); + otherProps.keySet().stream().map((argKey) -> ("-D" + argKey + "=" + otherProps.get(argKey))) + .forEach((item) -> allArgs.add(item)); - Arrays.stream(restArgs) - .filter((arg)->(arg.startsWith("-D")||arg.equals("start"))) - .forEach((arg)->allArgs.add(arg)); + Arrays.stream(restArgs).filter((arg) -> (arg.startsWith("-D") || arg.equals("start"))) + .forEach((arg) -> allArgs.add(arg)); // is start there? - if (allArgs.stream() - .filter((arg)->arg.equalsIgnoreCase("start")).count() < 1){ + if (allArgs.stream().filter((arg) -> arg.equalsIgnoreCase("start")).count() < 1) { allArgs.add("start"); } - String[] newArgs=new String[allArgs.size()]; + String[] newArgs = new String[allArgs.size()]; allArgs.toArray(newArgs); new HRegionServerCommandLine(regionServerClass).doMain(newArgs); } - /** * Set up the needed znodes under the rootZnode - * @param zk CuratorFramework framework instance + * @param zk CuratorFramework framework instance * @param rootZnode Root znode * @throws Exception If an error occurs */ - public static void setupZookeeperZnodes(CuratorFramework zk, String rootZnode,String peer) - throws Exception { + public static void setupZookeeperZnodes(CuratorFramework zk, String rootZnode, String peer) + throws Exception { // always gives the same uuid for the same name UUID uuid = UUID.nameUUIDFromBytes(Bytes.toBytes(peer)); String newValue = uuid.toString(); - byte []uuidBytes = Bytes.toBytes(newValue); - String idPath=rootZnode+"/hbaseid"; + byte[] uuidBytes = Bytes.toBytes(newValue); + String idPath = rootZnode + "/hbaseid"; if (zk.checkExists().forPath(idPath) == null) { zk.create().forPath(rootZnode); - zk.create().forPath(rootZnode +"/hbaseid",uuidBytes); + zk.create().forPath(rootZnode + "/hbaseid", uuidBytes); } else { // If the znode is there already make sure it has the // expected value for the peer name. byte[] znodeBytes = zk.getData().forPath(idPath).clone(); - if (!Bytes.equals(znodeBytes,uuidBytes)){ + if (!Bytes.equals(znodeBytes, uuidBytes)) { String oldValue = Bytes.toString(znodeBytes); - LOG.warn("znode "+idPath+" has unexpected value "+ oldValue - +" expecting " + newValue + " " - + " (did the peer name for the proxy change?) " - + "Updating value"); + LOG.warn("znode " + idPath + " has unexpected value " + oldValue + " expecting " + newValue + + " " + " (did the peer name for the proxy change?) " + "Updating value"); zk.setData().forPath(idPath, uuidBytes); } } } /** - * Poll for the configured peer or create it if it does not exist - * (controlled by createIfMissing) - * @param hbaseConf the hbase configuratoin - * @param zk CuratorFramework object - * @param basePath base znode. - * @param peerName id if the peer to check for/create - * @param enablePeer if the peer is detected or created, enable it. + * Poll for the configured peer or create it if it does not exist (controlled by createIfMissing) + * @param hbaseConf the hbase configuratoin + * @param zk CuratorFramework object + * @param basePath base znode. + * @param peerName id if the peer to check for/create + * @param enablePeer if the peer is detected or created, enable it. * @param createIfMissing if the peer doesn't exist, create it and peer to it. */ - public static void checkForOrCreateReplicationPeer(Configuration hbaseConf, - CuratorFramework zk, - String basePath, - String peerName, boolean createIfMissing, - boolean enablePeer) { + public static void checkForOrCreateReplicationPeer(Configuration hbaseConf, CuratorFramework zk, + String basePath, String peerName, boolean createIfMissing, boolean enablePeer) { try (Connection conn = ConnectionFactory.createConnection(hbaseConf); - Admin admin = conn.getAdmin()) { + Admin admin = conn.getAdmin()) { boolean peerThere = false; while (!peerThere) { try { ReplicationPeerConfig peerConfig = admin.getReplicationPeerConfig(peerName); - if (peerConfig !=null) { - peerThere=true; + if (peerConfig != null) { + peerThere = true; } } catch (ReplicationPeerNotFoundException e) { if (createIfMissing) { ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder(); // get the current cluster's ZK config - String zookeeperQ = hbaseConf.get("hbase.zookeeper.quorum") + - ":" + - hbaseConf.get("hbase.zookeeper.property.clientPort"); - String znodePath = zookeeperQ + ":"+basePath; + String zookeeperQ = hbaseConf.get("hbase.zookeeper.quorum") + ":" + + hbaseConf.get("hbase.zookeeper.property.clientPort"); + String znodePath = zookeeperQ + ":" + basePath; ReplicationPeerConfig rconf = builder.setClusterKey(znodePath).build(); admin.addReplicationPeer(peerName, rconf); peerThere = true; @@ -335,20 +309,19 @@ public static void checkForOrCreateReplicationPeer(Configuration hbaseConf, } if (peerThere) { - if (enablePeer){ + if (enablePeer) { LOG.info("enable peer,{}", peerName); List peers = admin.listReplicationPeers().stream() - .filter(peer -> peer.getPeerId().equals(peerName)) - .filter(peer -> !peer.isEnabled()) - .collect(Collectors.toList()); - if (!peers.isEmpty()){ + .filter(peer -> peer.getPeerId().equals(peerName)).filter(peer -> !peer.isEnabled()) + .collect(Collectors.toList()); + if (!peers.isEmpty()) { admin.enableReplicationPeer(peerName); } } break; } else { - LOG.info("peer "+ - peerName+" not found, service will not completely start until the peer exists"); + LOG.info("peer " + peerName + + " not found, service will not completely start until the peer exists"); } Thread.sleep(5000); } @@ -356,7 +329,7 @@ public static void checkForOrCreateReplicationPeer(Configuration hbaseConf, LOG.info("found replication peer " + peerName); } catch (Exception e) { - LOG.error("Exception running proxy ",e); + LOG.error("Exception running proxy ", e); } } } diff --git a/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/KafkaTableForBridge.java b/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/KafkaTableForBridge.java index 31baa44e..6103aaee 100755 --- a/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/KafkaTableForBridge.java +++ b/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/KafkaTableForBridge.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -6,9 +6,9 @@ * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

+ * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. @@ -24,7 +24,6 @@ import java.util.List; import java.util.concurrent.Future; import java.util.stream.Collectors; - import org.apache.avro.io.BinaryEncoder; import org.apache.avro.io.DatumWriter; import org.apache.avro.io.EncoderFactory; @@ -49,7 +48,6 @@ import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils; - @InterfaceAudience.Private public class KafkaTableForBridge implements Table { private Logger LOG = LoggerFactory.getLogger(KafkaTableForBridge.class); @@ -58,14 +56,14 @@ public class KafkaTableForBridge implements Table { private final TableName tableName; private byte[] tableAsBytes; - private Producer producer; + private Producer producer; private TopicRoutingRules routingRules; private DatumWriter avroWriter; private static final class CheckMutation { - byte[]qualifier; - byte[]family; + byte[] qualifier; + byte[] family; Cell cell; List topics = new ArrayList<>(); } @@ -75,106 +73,94 @@ public RegionLocator getRegionLocator() throws IOException { throw new UnsupportedOperationException(); } - public KafkaTableForBridge(TableName tableName, - Configuration conf, - TopicRoutingRules routingRules, - Producer producer, - DatumWriter avroWriter){ - this.conf=conf; - this.tableName=tableName; - this.tableAsBytes=this.tableName.toBytes(); - this.routingRules=routingRules; - this.producer=producer; - this.avroWriter=avroWriter; + public KafkaTableForBridge(TableName tableName, Configuration conf, + TopicRoutingRules routingRules, Producer producer, + DatumWriter avroWriter) { + this.conf = conf; + this.tableName = tableName; + this.tableAsBytes = this.tableName.toBytes(); + this.routingRules = routingRules; + this.producer = producer; + this.avroWriter = avroWriter; } - private List> processMutation(CheckMutation check, boolean isDelete){ + private List> processMutation(CheckMutation check, + boolean isDelete) { HBaseKafkaEvent event = new HBaseKafkaEvent(); - event.setKey(ByteBuffer.wrap(check.cell.getRowArray(), - check.cell.getRowOffset(), - check.cell.getRowLength())); + event.setKey(ByteBuffer.wrap(check.cell.getRowArray(), check.cell.getRowOffset(), + check.cell.getRowLength())); event.setTable(ByteBuffer.wrap(tableAsBytes)); event.setDelete(isDelete); event.setTimestamp(check.cell.getTimestamp()); event.setFamily(ByteBuffer.wrap(check.family)); event.setQualifier(ByteBuffer.wrap(check.qualifier)); - event.setValue(ByteBuffer.wrap(check.cell.getValueArray(), - check.cell.getValueOffset(), - check.cell.getValueLength())); + event.setValue(ByteBuffer.wrap(check.cell.getValueArray(), check.cell.getValueOffset(), + check.cell.getValueLength())); - return check.topics.stream() - .map((topic)->new Pair(topic,event)) - .collect(Collectors.toList()); + return check.topics.stream().map((topic) -> new Pair(topic, event)) + .collect(Collectors.toList()); } - private boolean keep(CheckMutation ret){ - if (!routingRules.isExclude(this.tableName,ret.family, ret.qualifier)){ + private boolean keep(CheckMutation ret) { + if (!routingRules.isExclude(this.tableName, ret.family, ret.qualifier)) { return true; } return false; } - private CheckMutation addTopics(CheckMutation ret){ - ret.topics= routingRules.getTopics(this.tableName,ret.family,ret.qualifier); + private CheckMutation addTopics(CheckMutation ret) { + ret.topics = routingRules.getTopics(this.tableName, ret.family, ret.qualifier); return ret; } - private ProducerRecord toByteArray(ByteArrayOutputStream bout, - Pair event, - BinaryEncoder encoder) { + private ProducerRecord toByteArray(ByteArrayOutputStream bout, + Pair event, BinaryEncoder encoder) { try { bout.reset(); BinaryEncoder encoderUse = EncoderFactory.get().binaryEncoder(bout, encoder); avroWriter.write(event.getSecond(), encoderUse); encoder.flush(); - return new ProducerRecord(event.getFirst(), - event.getSecond().getKey().array(), - bout.toByteArray()); - } catch (Exception e){ + return new ProducerRecord(event.getFirst(), + event.getSecond().getKey().array(), bout.toByteArray()); + } catch (Exception e) { throw new RuntimeException(e); } } @Override public void batch(final List actions, Object[] results) - throws IOException, InterruptedException { + throws IOException, InterruptedException { ByteArrayOutputStream bout = new ByteArrayOutputStream(); BinaryEncoder encoderUse = EncoderFactory.get().binaryEncoder(bout, null); - LOG.debug("got {} inputs ",actions.size()); + LOG.debug("got {} inputs ", actions.size()); List> sends = new ArrayList<>(); - actions.stream() - .filter((row)->row instanceof Mutation) - .map((row)->(Mutation)row) - .flatMap((row)->{ + actions.stream().filter((row) -> row instanceof Mutation).map((row) -> (Mutation) row) + .flatMap((row) -> { Mutation mut = (Mutation) row; boolean isDelete = mut instanceof Delete; return mut.getFamilyCellMap().keySet().stream() - .flatMap((family)->mut.getFamilyCellMap().get(family).stream()) - .map((cell)->{ - CheckMutation ret = new CheckMutation(); - ret.family=CellUtil.cloneFamily(cell); - ret.qualifier=CellUtil.cloneQualifier(cell); - ret.cell=cell; - return ret; - }) - .filter((check)->keep(check)) - .map((check)->addTopics(check)) - .filter((check)->!CollectionUtils.isEmpty(check.topics)) - .flatMap((check)->processMutation(check,isDelete).stream()); - }) - .map((event)->toByteArray(bout,event,encoderUse)) - .forEach((item)->sends.add(producer.send(item))); + .flatMap((family) -> mut.getFamilyCellMap().get(family).stream()).map((cell) -> { + CheckMutation ret = new CheckMutation(); + ret.family = CellUtil.cloneFamily(cell); + ret.qualifier = CellUtil.cloneQualifier(cell); + ret.cell = cell; + return ret; + }).filter((check) -> keep(check)).map((check) -> addTopics(check)) + .filter((check) -> !CollectionUtils.isEmpty(check.topics)) + .flatMap((check) -> processMutation(check, isDelete).stream()); + }).map((event) -> toByteArray(bout, event, encoderUse)) + .forEach((item) -> sends.add(producer.send(item))); // make sure the sends are done before returning - sends.stream().forEach((sendResult)->{ + sends.stream().forEach((sendResult) -> { try { sendResult.get(); - } catch (Exception e){ - LOG.error("Exception caught when getting result",e); + } catch (Exception e) { + LOG.error("Exception caught when getting result", e); throw new RuntimeException(e); } }); diff --git a/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/Rule.java b/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/Rule.java index 7d02025b..08453ff2 100644 --- a/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/Rule.java +++ b/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/Rule.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -6,9 +6,9 @@ * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

+ * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. @@ -22,29 +22,28 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.yetus.audience.InterfaceAudience; - /** * Implements the matching logic for a rule */ @InterfaceAudience.Private public abstract class Rule { TableName tableName; - private byte [] columnFamily; - private byte [] qualifier; + private byte[] columnFamily; + private byte[] qualifier; boolean qualifierStartsWith = false; boolean qualifierEndsWith = false; - byte []ast = Bytes.toBytes("*"); + byte[] ast = Bytes.toBytes("*"); /** * Indicates if the table,column family, and qualifier match the rule - * @param tryTable table name to test + * @param tryTable table name to test * @param tryColumFamily column family to test - * @param tryQualifier qualifier to test + * @param tryQualifier qualifier to test * @return true if values match the rule */ - public boolean match(TableName tryTable, byte [] tryColumFamily, byte [] tryQualifier) { + public boolean match(TableName tryTable, byte[] tryColumFamily, byte[] tryQualifier) { boolean tableMatch = tableMatch(tryTable); boolean columnFamilyMatch = columnFamilyMatch(tryColumFamily); boolean qualfierMatch = qualifierMatch(tryQualifier); @@ -57,7 +56,7 @@ public boolean match(TableName tryTable, byte [] tryColumFamily, byte [] tryQual * @param tryQualifier qualifier to test * @return true if the qualifier matches */ - public boolean qualifierMatch(byte [] tryQualifier) { + public boolean qualifierMatch(byte[] tryQualifier) { if (qualifier != null) { if (qualifierStartsWith && qualifierEndsWith) { @@ -78,7 +77,7 @@ public boolean qualifierMatch(byte [] tryQualifier) { * @param tryColumFamily column family to test * @return true if the column family matches the rule */ - public boolean columnFamilyMatch(byte [] tryColumFamily) { + public boolean columnFamilyMatch(byte[] tryColumFamily) { if (columnFamily != null) { return Bytes.equals(this.columnFamily, tryColumFamily); } @@ -101,7 +100,7 @@ public boolean tableMatch(TableName tryTable) { * set the column family for the rule * @param columnFamily column family to set */ - public void setColumnFamily(byte [] columnFamily) { + public void setColumnFamily(byte[] columnFamily) { this.columnFamily = columnFamily; } @@ -109,7 +108,7 @@ public void setColumnFamily(byte [] columnFamily) { * set the qualifier value for the rule * @param qualifier qualifier to set */ - public void setQualifier(byte []qualifier) { + public void setQualifier(byte[] qualifier) { this.qualifier = qualifier; if (startsWith(qualifier, ast)) { qualifierEndsWith = true; @@ -129,11 +128,11 @@ public void setQualifier(byte []qualifier) { /** * Tests if data starts with startsWith - * @param data byte array to test + * @param data byte array to test * @param startsWith array that we want to see if data starts with * @return true if data starts with startsWith */ - public static boolean startsWith(byte [] data, byte [] startsWith) { + public static boolean startsWith(byte[] data, byte[] startsWith) { if (startsWith.length > data.length) { return false; } @@ -152,11 +151,11 @@ public static boolean startsWith(byte [] data, byte [] startsWith) { /** * Tests if data ends with endsWith - * @param data byte array to test + * @param data byte array to test * @param endsWith array that we want to see if data ends with * @return true if data ends with endsWith */ - public static boolean endsWith(byte [] data, byte [] endsWith) { + public static boolean endsWith(byte[] data, byte[] endsWith) { if (endsWith.length > data.length) { return false; } @@ -168,7 +167,7 @@ public static boolean endsWith(byte [] data, byte [] endsWith) { int endStart = data.length - endsWith.length; for (int i = 0; i < endsWith.length; i++) { - //if (endsWith[i]!=data[(data.length-1)-(endsWith.length+i)]){ + // if (endsWith[i]!=data[(data.length-1)-(endsWith.length+i)]){ if (endsWith[i] != data[endStart + i]) { return false; } @@ -202,13 +201,11 @@ public byte[] getColumnFamily() { /** * get the qualifier for the rule - * @return qualfier */ public byte[] getQualifier() { return qualifier; } - /** * indicates if the qualfier is a wildcard like *foo * @return true if rule is like *foo diff --git a/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/TopicRoutingRules.java b/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/TopicRoutingRules.java index c8b818c6..e4e9312f 100644 --- a/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/TopicRoutingRules.java +++ b/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/TopicRoutingRules.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -6,9 +6,9 @@ * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

+ * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hbase.kafka; - import java.io.File; import java.io.FileInputStream; import java.io.InputStream; @@ -25,7 +24,6 @@ import java.util.List; import javax.xml.parsers.DocumentBuilder; import javax.xml.parsers.DocumentBuilderFactory; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.util.Bytes; import org.apache.yetus.audience.InterfaceAudience; @@ -34,41 +32,20 @@ import org.w3c.dom.NodeList; /** - * The topic routing/drop rules. - * - * <rules> - * <rule .... /> - * - * </rules> - * - * - * - * A wildcard can only be at the beginning or at the end (can be at both sides). - * - * drop rules are always evaluated first. - * - * drop examples: - * <rule action="drop" table="default:MyTable" /> - * Do not send replication events for table MyTable - * - * <rule action="drop" table="default:MyTable" columnFamily="data"/> - * Do not send replication events for table MyTable's column family data - * - * <rule action="drop" table="default:MyTable" columnFamily="data" qualfier="dhold:*"/> - * Do not send replication events for any qualiifier on table MyTable with column family data - * - * routeRules examples: - * - * <rule action="routeRules" table="default:MyTable" topic="mytopic"/> - * routeRules all replication events for table default:Mytable to topic mytopic - * - * <rule action="routeRules" table="default:MyTable" columnFamily="data" topic="mytopic"/> - * routeRules all replication events for table default:Mytable column family data to topic mytopic - * - * <rule action="routeRules" table="default:MyTable" columnFamily="data" topic="mytopic" - * qualifier="hold:*"/> - * routeRules all replication events for qualifiers that start with hold: for table - * default:Mytable column family data to topic mytopic + * The topic routing/drop rules. <rules> <rule .... /> </rules> A wildcard can + * only be at the beginning or at the end (can be at both sides). drop rules are always evaluated + * first. drop examples: <rule action="drop" table="default:MyTable" /> Do not send + * replication events for table MyTable <rule action="drop" table="default:MyTable" + * columnFamily="data"/> Do not send replication events for table MyTable's column family data + * <rule action="drop" table="default:MyTable" columnFamily="data" qualfier="dhold:*"/> Do not + * send replication events for any qualiifier on table MyTable with column family data routeRules + * examples: <rule action="routeRules" table="default:MyTable" topic="mytopic"/> routeRules + * all replication events for table default:Mytable to topic mytopic <rule action="routeRules" + * table="default:MyTable" columnFamily="data" topic="mytopic"/> routeRules all replication + * events for table default:Mytable column family data to topic mytopic <rule action="routeRules" + * table="default:MyTable" columnFamily="data" topic="mytopic" qualifier="hold:*"/> routeRules + * all replication events for qualifiers that start with hold: for table default:Mytable column + * family data to topic mytopic */ @InterfaceAudience.Private public class TopicRoutingRules { @@ -100,7 +77,7 @@ public TopicRoutingRules(File source) throws Exception { * @throws Exception error loading rule set */ public void reloadIfFile() throws Exception { - if (this.sourceFile!=null){ + if (this.sourceFile != null) { List dropRulesSave = this.dropRules; List routeRulesSave = this.routeRules; @@ -108,15 +85,15 @@ public void reloadIfFile() throws Exception { List dropRulesNew = new ArrayList<>(); List routeRulesNew = new ArrayList<>(); - parseRules(fin,dropRulesNew,routeRulesNew); + parseRules(fin, dropRulesNew, routeRulesNew); this.dropRules = dropRulesNew; this.routeRules = routeRulesNew; - } catch (Exception e){ + } catch (Exception e) { // roll back - this.dropRules=dropRulesSave; - this.routeRules=routeRulesSave; + this.dropRules = dropRulesSave; + this.routeRules = routeRulesSave; // re-throw throw e; } @@ -130,18 +107,18 @@ public void reloadIfFile() throws Exception { public void parseRules(InputStream input) { List dropRulesNew = new ArrayList<>(); List routeRulesNew = new ArrayList<>(); - parseRules(input,dropRulesNew,routeRulesNew); + parseRules(input, dropRulesNew, routeRulesNew); this.dropRules = dropRulesNew; this.routeRules = routeRulesNew; } /** * Parse the XML in the InputStream into route/drop rules and store them in the passed in Lists - * @param input inputstream the contains the ruleset - * @param dropRules list to accumulate drop rules + * @param input inputstream the contains the ruleset + * @param dropRules list to accumulate drop rules * @param routeRules list to accumulate route rules */ - public void parseRules(InputStream input,List dropRules, List routeRules) { + public void parseRules(InputStream input, List dropRules, List routeRules) { try { DocumentBuilderFactory dbFactory = DocumentBuilderFactory.newInstance(); DocumentBuilder dBuilder = dbFactory.newDocumentBuilder(); @@ -149,7 +126,7 @@ public void parseRules(InputStream input,List dropRules, List dropRules, List dropRules, List routeRules) { @@ -186,13 +163,13 @@ public void parseRule(Element n, List dropRules, List route /** * Indicates if a cell mutation should be dropped instead of routed to kafka. - * @param table table name to check + * @param table table name to check * @param columnFamily column family to check - * @param qualifer qualifier name to check + * @param qualifer qualifier name to check * @return if the mutation should be dropped instead of routed to Kafka */ - public boolean isExclude(final TableName table, final byte []columnFamily, - final byte[] qualifer) { + public boolean isExclude(final TableName table, final byte[] columnFamily, + final byte[] qualifer) { for (DropRule r : getDropRules()) { if (r.match(table, columnFamily, qualifer)) { return true; @@ -203,12 +180,12 @@ public boolean isExclude(final TableName table, final byte []columnFamily, /** * Get topics for the table/column family/qualifier combination - * @param table table name to check + * @param table table name to check * @param columnFamily column family to check - * @param qualifer qualifier name to check + * @param qualifer qualifier name to check * @return list of topics that match the passed in values (or empty for none). */ - public List getTopics(TableName table, byte []columnFamily, byte []qualifer) { + public List getTopics(TableName table, byte[] columnFamily, byte[] qualifer) { List ret = new ArrayList<>(); for (TopicRule r : getRouteRules()) { if (r.match(table, columnFamily, qualifer)) { diff --git a/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/TopicRule.java b/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/TopicRule.java index 5e5b6bfd..fbb6e74c 100644 --- a/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/TopicRule.java +++ b/kafka/hbase-kafka-proxy/src/main/java/org/apache/hadoop/hbase/kafka/TopicRule.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -6,9 +6,9 @@ * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

+ * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. @@ -21,7 +21,6 @@ import java.util.HashSet; import java.util.Set; import java.util.stream.Collectors; - import org.apache.yetus.audience.InterfaceAudience; /** diff --git a/kafka/hbase-kafka-proxy/src/test/java/org/apache/hadoop/hbase/kafka/ProducerForTesting.java b/kafka/hbase-kafka-proxy/src/test/java/org/apache/hadoop/hbase/kafka/ProducerForTesting.java index e800501b..93c0cd44 100644 --- a/kafka/hbase-kafka-proxy/src/test/java/org/apache/hadoop/hbase/kafka/ProducerForTesting.java +++ b/kafka/hbase-kafka-proxy/src/test/java/org/apache/hadoop/hbase/kafka/ProducerForTesting.java @@ -1,16 +1,19 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more contributor license - * agreements. See the NOTICE file distributed with this work for additional information regarding - * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. You may obtain a - * copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software distributed under the License - * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express - * or implied. See the License for the specific language governing permissions and limitations under - * the License. +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package org.apache.hadoop.hbase.kafka; @@ -19,7 +22,6 @@ import java.util.List; import java.util.Map; import java.util.concurrent.Future; - import org.apache.avro.io.BinaryDecoder; import org.apache.avro.io.DecoderFactory; import org.apache.avro.specific.SpecificDatumReader; diff --git a/kafka/hbase-kafka-proxy/src/test/java/org/apache/hadoop/hbase/kafka/TestDropRule.java b/kafka/hbase-kafka-proxy/src/test/java/org/apache/hadoop/hbase/kafka/TestDropRule.java index e10bb043..1c583826 100644 --- a/kafka/hbase-kafka-proxy/src/test/java/org/apache/hadoop/hbase/kafka/TestDropRule.java +++ b/kafka/hbase-kafka-proxy/src/test/java/org/apache/hadoop/hbase/kafka/TestDropRule.java @@ -1,22 +1,24 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more contributor license - * agreements. See the NOTICE file distributed with this work for additional information regarding - * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. You may obtain a - * copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software distributed under the License - * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express - * or implied. See the License for the specific language governing permissions and limitations under - * the License. +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package org.apache.hadoop.hbase.kafka; import java.io.ByteArrayInputStream; import java.nio.charset.StandardCharsets; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; @@ -31,24 +33,19 @@ @Category(SmallTests.class) public class TestDropRule { private static final String DROP_RULE1 = - ""; + ""; private static final String DROP_RULE2 = - ""; - private static final String DROP_RULE3 = - ""; + ""; + private static final String DROP_RULE3 = ""; - private static final String DROP_RULE4 = - ""; - private static final String DROP_RULE5 = - ""; + private static final String DROP_RULE4 = ""; + private static final String DROP_RULE5 = ""; - private static final String DROP_RULE6 = - ""; + private static final String DROP_RULE6 = ""; @Test public void testDropies1() { @@ -75,7 +72,7 @@ public void testDropies2() { Assert.assertEquals(TableName.valueOf("default:MyTable"), rules.getDropRules().get(0).getTableName()); Assert.assertTrue(Bytes.equals("data".getBytes(StandardCharsets.UTF_8), - rules.getDropRules().get(0).getColumnFamily())); + rules.getDropRules().get(0).getColumnFamily())); Assert.assertNull(rules.getDropRules().get(0).getQualifier()); Assert.assertEquals(0, rules.getRouteRules().size()); } catch (Exception e) { @@ -92,9 +89,9 @@ public void testDropies3() { Assert.assertEquals(TableName.valueOf("default:MyTable"), rules.getDropRules().get(0).getTableName()); Assert.assertTrue(Bytes.equals("data".getBytes(StandardCharsets.UTF_8), - rules.getDropRules().get(0).getColumnFamily())); + rules.getDropRules().get(0).getColumnFamily())); Assert.assertTrue(Bytes.equals("dhold".getBytes(StandardCharsets.UTF_8), - rules.getDropRules().get(0).getQualifier())); + rules.getDropRules().get(0).getQualifier())); Assert.assertEquals(0, rules.getRouteRules().size()); } catch (Exception e) { Assert.fail(e.getMessage()); @@ -110,19 +107,18 @@ public void testDropies4() { Assert.assertEquals(TableName.valueOf("default:MyTable"), rules.getDropRules().get(0).getTableName()); Assert.assertTrue(Bytes.equals("data".getBytes(StandardCharsets.UTF_8), - rules.getDropRules().get(0).getColumnFamily())); + rules.getDropRules().get(0).getColumnFamily())); Assert.assertTrue(Bytes.equals("dhold:".getBytes(StandardCharsets.UTF_8), - rules.getDropRules().get(0).getQualifier())); + rules.getDropRules().get(0).getQualifier())); Assert.assertEquals(0, rules.getRouteRules().size()); DropRule drop = rules.getDropRules().get(0); Assert.assertFalse(drop.match(TableName.valueOf("default:MyTable"), - "data".getBytes(StandardCharsets.UTF_8), "blah".getBytes(StandardCharsets.UTF_8))); + "data".getBytes(StandardCharsets.UTF_8), "blah".getBytes(StandardCharsets.UTF_8))); Assert.assertFalse(drop.match(TableName.valueOf("default:MyTable"), - "data".getBytes(StandardCharsets.UTF_8), "dholdme".getBytes(StandardCharsets.UTF_8))); + "data".getBytes(StandardCharsets.UTF_8), "dholdme".getBytes(StandardCharsets.UTF_8))); Assert.assertTrue(drop.match(TableName.valueOf("default:MyTable"), - "data".getBytes(StandardCharsets.UTF_8), - "dhold:me".getBytes(StandardCharsets.UTF_8))); + "data".getBytes(StandardCharsets.UTF_8), "dhold:me".getBytes(StandardCharsets.UTF_8))); } catch (Exception e) { Assert.fail(e.getMessage()); } @@ -137,20 +133,19 @@ public void testDropies5() { Assert.assertEquals(TableName.valueOf("default:MyTable"), rules.getDropRules().get(0).getTableName()); Assert.assertTrue(Bytes.equals("data".getBytes(StandardCharsets.UTF_8), - rules.getDropRules().get(0).getColumnFamily())); + rules.getDropRules().get(0).getColumnFamily())); Assert.assertTrue(Bytes.equals("pickme".getBytes(StandardCharsets.UTF_8), - rules.getDropRules().get(0).getQualifier())); + rules.getDropRules().get(0).getQualifier())); Assert.assertEquals(0, rules.getRouteRules().size()); DropRule drop = rules.getDropRules().get(0); Assert.assertFalse(drop.match(TableName.valueOf("default:MyTable"), - "data".getBytes(StandardCharsets.UTF_8), "blah".getBytes(StandardCharsets.UTF_8))); + "data".getBytes(StandardCharsets.UTF_8), "blah".getBytes(StandardCharsets.UTF_8))); Assert.assertFalse(drop.match(TableName.valueOf("default:MyTable"), - "data".getBytes(StandardCharsets.UTF_8), - "blacickme".getBytes(StandardCharsets.UTF_8))); - Assert.assertTrue(drop.match(TableName.valueOf("default:MyTable"), - "data".getBytes(StandardCharsets.UTF_8), - "hithere.pickme".getBytes(StandardCharsets.UTF_8))); + "data".getBytes(StandardCharsets.UTF_8), "blacickme".getBytes(StandardCharsets.UTF_8))); + Assert.assertTrue( + drop.match(TableName.valueOf("default:MyTable"), "data".getBytes(StandardCharsets.UTF_8), + "hithere.pickme".getBytes(StandardCharsets.UTF_8))); } catch (Exception e) { Assert.fail(e.getMessage()); } @@ -163,31 +158,30 @@ public void testDropies6() { rules.parseRules(new ByteArrayInputStream(DROP_RULE6.getBytes(StandardCharsets.UTF_8))); Assert.assertEquals(1, rules.getDropRules().size()); Assert.assertEquals(TableName.valueOf("default:MyTable"), - rules.getDropRules().get(0).getTableName()); + rules.getDropRules().get(0).getTableName()); Assert.assertTrue(Bytes.equals("data".getBytes(StandardCharsets.UTF_8), - rules.getDropRules().get(0).getColumnFamily())); + rules.getDropRules().get(0).getColumnFamily())); Assert.assertTrue(Bytes.equals("pickme".getBytes(StandardCharsets.UTF_8), - rules.getDropRules().get(0).getQualifier())); + rules.getDropRules().get(0).getQualifier())); Assert.assertEquals(0, rules.getRouteRules().size()); DropRule drop = rules.getDropRules().get(0); Assert.assertFalse(drop.match(TableName.valueOf("default:MyTable"), - "data".getBytes(StandardCharsets.UTF_8), "blah".getBytes(StandardCharsets.UTF_8))); + "data".getBytes(StandardCharsets.UTF_8), "blah".getBytes(StandardCharsets.UTF_8))); Assert.assertFalse(drop.match(TableName.valueOf("default:MyTable"), - "data".getBytes(StandardCharsets.UTF_8), - "blacickme".getBytes(StandardCharsets.UTF_8))); - Assert.assertTrue(drop.match(TableName.valueOf("default:MyTable"), - "data".getBytes(StandardCharsets.UTF_8), - "hithere.pickme".getBytes(StandardCharsets.UTF_8))); - Assert.assertTrue(drop.match(TableName.valueOf("default:MyTable"), - "data".getBytes(StandardCharsets.UTF_8), - "pickme.pleaze.do.it".getBytes(StandardCharsets.UTF_8))); - Assert.assertFalse(drop.match(TableName.valueOf("default:MyTable"), - "data".getBytes(StandardCharsets.UTF_8), - "please.pickme.pleaze".getBytes(StandardCharsets.UTF_8))); - Assert.assertTrue(drop.match(TableName.valueOf("default:MyTable"), - "data".getBytes(StandardCharsets.UTF_8), - "pickme.pleaze.pickme".getBytes(StandardCharsets.UTF_8))); + "data".getBytes(StandardCharsets.UTF_8), "blacickme".getBytes(StandardCharsets.UTF_8))); + Assert.assertTrue( + drop.match(TableName.valueOf("default:MyTable"), "data".getBytes(StandardCharsets.UTF_8), + "hithere.pickme".getBytes(StandardCharsets.UTF_8))); + Assert.assertTrue( + drop.match(TableName.valueOf("default:MyTable"), "data".getBytes(StandardCharsets.UTF_8), + "pickme.pleaze.do.it".getBytes(StandardCharsets.UTF_8))); + Assert.assertFalse( + drop.match(TableName.valueOf("default:MyTable"), "data".getBytes(StandardCharsets.UTF_8), + "please.pickme.pleaze".getBytes(StandardCharsets.UTF_8))); + Assert.assertTrue( + drop.match(TableName.valueOf("default:MyTable"), "data".getBytes(StandardCharsets.UTF_8), + "pickme.pleaze.pickme".getBytes(StandardCharsets.UTF_8))); } catch (Exception e) { Assert.fail(e.getMessage()); } diff --git a/kafka/hbase-kafka-proxy/src/test/java/org/apache/hadoop/hbase/kafka/TestProcessMutations.java b/kafka/hbase-kafka-proxy/src/test/java/org/apache/hadoop/hbase/kafka/TestProcessMutations.java index 534d4a63..177027e2 100644 --- a/kafka/hbase-kafka-proxy/src/test/java/org/apache/hadoop/hbase/kafka/TestProcessMutations.java +++ b/kafka/hbase-kafka-proxy/src/test/java/org/apache/hadoop/hbase/kafka/TestProcessMutations.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -6,9 +6,9 @@ * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

+ * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. @@ -21,7 +21,6 @@ import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Put; @@ -39,14 +38,13 @@ @Category(SmallTests.class) public class TestProcessMutations { private static final String ROUTE_RULE1 = - ""; + ""; ProducerForTesting myTestingProducer; @Before public void setup() { - this.myTestingProducer=new ProducerForTesting(); + this.myTestingProducer = new ProducerForTesting(); } @Test @@ -55,22 +53,22 @@ public void testSendMessage() { try { rules.parseRules(new ByteArrayInputStream(ROUTE_RULE1.getBytes(StandardCharsets.UTF_8))); Configuration conf = new Configuration(); - KafkaBridgeConnection connection = new KafkaBridgeConnection(conf,rules,myTestingProducer); + KafkaBridgeConnection connection = new KafkaBridgeConnection(conf, rules, myTestingProducer); long zeTimestamp = System.currentTimeMillis(); - Put put = new Put("key1".getBytes(StandardCharsets.UTF_8),zeTimestamp); + Put put = new Put("key1".getBytes(StandardCharsets.UTF_8), zeTimestamp); put.addColumn("FAMILY".getBytes(StandardCharsets.UTF_8), - "not foo".getBytes(StandardCharsets.UTF_8), - "VALUE should NOT pass".getBytes(StandardCharsets.UTF_8)); + "not foo".getBytes(StandardCharsets.UTF_8), + "VALUE should NOT pass".getBytes(StandardCharsets.UTF_8)); put.addColumn("FAMILY".getBytes(StandardCharsets.UTF_8), - "foo".getBytes(StandardCharsets.UTF_8), - "VALUE should pass".getBytes(StandardCharsets.UTF_8)); + "foo".getBytes(StandardCharsets.UTF_8), + "VALUE should pass".getBytes(StandardCharsets.UTF_8)); Table myTable = connection.getTable(TableName.valueOf("MyNamespace:MyTable")); List rows = new ArrayList<>(); rows.add(put); - myTable.batch(rows,new Object[0]); + myTable.batch(rows, new Object[0]); Assert.assertFalse(myTestingProducer.getMessages().isEmpty()); - } catch (Exception e){ + } catch (Exception e) { Assert.fail(e.getMessage()); } } diff --git a/kafka/hbase-kafka-proxy/src/test/java/org/apache/hadoop/hbase/kafka/TestQualifierMatching.java b/kafka/hbase-kafka-proxy/src/test/java/org/apache/hadoop/hbase/kafka/TestQualifierMatching.java index c965f123..229a43fc 100644 --- a/kafka/hbase-kafka-proxy/src/test/java/org/apache/hadoop/hbase/kafka/TestQualifierMatching.java +++ b/kafka/hbase-kafka-proxy/src/test/java/org/apache/hadoop/hbase/kafka/TestQualifierMatching.java @@ -1,21 +1,23 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more contributor license - * agreements. See the NOTICE file distributed with this work for additional information regarding - * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. You may obtain a - * copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software distributed under the License - * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express - * or implied. See the License for the specific language governing permissions and limitations under - * the License. +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package org.apache.hadoop.hbase.kafka; import java.nio.charset.StandardCharsets; - import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Assert; import org.junit.Test; diff --git a/kafka/hbase-kafka-proxy/src/test/java/org/apache/hadoop/hbase/kafka/TestRouteRules.java b/kafka/hbase-kafka-proxy/src/test/java/org/apache/hadoop/hbase/kafka/TestRouteRules.java index 0c2f184d..6705f3ec 100644 --- a/kafka/hbase-kafka-proxy/src/test/java/org/apache/hadoop/hbase/kafka/TestRouteRules.java +++ b/kafka/hbase-kafka-proxy/src/test/java/org/apache/hadoop/hbase/kafka/TestRouteRules.java @@ -1,26 +1,27 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more contributor license - * agreements. See the NOTICE file distributed with this work for additional information regarding - * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. You may obtain a - * copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software distributed under the License - * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express - * or implied. See the License for the specific language governing permissions and limitations under - * the License. +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package org.apache.hadoop.hbase.kafka; import java.io.ByteArrayInputStream; import java.nio.charset.StandardCharsets; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; - import org.junit.Assert; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -33,24 +34,23 @@ public class TestRouteRules { private static final String TEST_TABLE = "default:MyTable"; private static final String ROUTE_RULE1 = - ""; + ""; private static final String ROUTE_RULE2 = - ""; private static final String ROUTE_RULE3 = - ""; private static final String ROUTE_RULE4 = - ""; private static final String ROUTE_RULE5 = - ""; private static final String ROUTE_RULE6 = - ""; @Test @@ -77,9 +77,8 @@ public void testTopic2() { Assert.assertEquals(1, rules.getRouteRules().size()); Assert.assertEquals(TableName.valueOf(TEST_TABLE), rules.getRouteRules().get(0).getTableName()); - Assert.assertTrue( - Bytes.equals("data".getBytes(StandardCharsets.UTF_8), - rules.getRouteRules().get(0).getColumnFamily())); + Assert.assertTrue(Bytes.equals("data".getBytes(StandardCharsets.UTF_8), + rules.getRouteRules().get(0).getColumnFamily())); Assert.assertNull(rules.getRouteRules().get(0).getQualifier()); Assert.assertEquals(0, rules.getDropRules().size()); } catch (Exception e) { @@ -96,9 +95,9 @@ public void testTopic3() { Assert.assertEquals(TableName.valueOf(TEST_TABLE), rules.getRouteRules().get(0).getTableName()); Assert.assertTrue(Bytes.equals("data".getBytes(StandardCharsets.UTF_8), - rules.getRouteRules().get(0).getColumnFamily())); + rules.getRouteRules().get(0).getColumnFamily())); Assert.assertTrue(Bytes.equals("dhold".getBytes(StandardCharsets.UTF_8), - rules.getRouteRules().get(0).getQualifier())); + rules.getRouteRules().get(0).getQualifier())); Assert.assertTrue(rules.getRouteRules().get(0).getTopics().contains("foo")); Assert.assertEquals(1, rules.getRouteRules().get(0).getTopics().size()); @@ -117,23 +116,18 @@ public void testTopic4() { Assert.assertEquals(TableName.valueOf(TEST_TABLE), rules.getRouteRules().get(0).getTableName()); Assert.assertTrue(Bytes.equals("data".getBytes(StandardCharsets.UTF_8), - rules.getRouteRules().get(0).getColumnFamily())); + rules.getRouteRules().get(0).getColumnFamily())); Assert.assertTrue(Bytes.equals("dhold:".getBytes(StandardCharsets.UTF_8), - rules.getRouteRules().get(0).getQualifier())); + rules.getRouteRules().get(0).getQualifier())); Assert.assertEquals(0, rules.getDropRules().size()); TopicRule route = rules.getRouteRules().get(0); - Assert.assertFalse( - route.match(TableName.valueOf(TEST_TABLE), - "data".getBytes(StandardCharsets.UTF_8), - "blah".getBytes(StandardCharsets.UTF_8))); - Assert.assertFalse( - route.match(TableName.valueOf(TEST_TABLE), - "data".getBytes(StandardCharsets.UTF_8), - "dholdme".getBytes(StandardCharsets.UTF_8))); + Assert.assertFalse(route.match(TableName.valueOf(TEST_TABLE), + "data".getBytes(StandardCharsets.UTF_8), "blah".getBytes(StandardCharsets.UTF_8))); + Assert.assertFalse(route.match(TableName.valueOf(TEST_TABLE), + "data".getBytes(StandardCharsets.UTF_8), "dholdme".getBytes(StandardCharsets.UTF_8))); Assert.assertTrue(route.match(TableName.valueOf(TEST_TABLE), - "data".getBytes(StandardCharsets.UTF_8), - "dhold:me".getBytes(StandardCharsets.UTF_8))); + "data".getBytes(StandardCharsets.UTF_8), "dhold:me".getBytes(StandardCharsets.UTF_8))); } catch (Exception e) { Assert.fail(e.getMessage()); @@ -149,22 +143,19 @@ public void testTopic5() { Assert.assertEquals(TableName.valueOf(TEST_TABLE), rules.getRouteRules().get(0).getTableName()); Assert.assertTrue(Bytes.equals("data".getBytes(StandardCharsets.UTF_8), - rules.getRouteRules().get(0).getColumnFamily())); + rules.getRouteRules().get(0).getColumnFamily())); Assert.assertTrue(Bytes.equals("pickme".getBytes(StandardCharsets.UTF_8), - rules.getRouteRules().get(0).getQualifier())); + rules.getRouteRules().get(0).getQualifier())); Assert.assertEquals(0, rules.getDropRules().size()); TopicRule route = rules.getRouteRules().get(0); - Assert.assertFalse( - route.match(TableName.valueOf(TEST_TABLE), - "data".getBytes(StandardCharsets.UTF_8), - "blah".getBytes(StandardCharsets.UTF_8))); Assert.assertFalse(route.match(TableName.valueOf(TEST_TABLE), - "data".getBytes(StandardCharsets.UTF_8), - "blacickme".getBytes(StandardCharsets.UTF_8))); - Assert.assertTrue(route.match(TableName.valueOf(TEST_TABLE), - "data".getBytes(StandardCharsets.UTF_8), - "hithere.pickme".getBytes(StandardCharsets.UTF_8))); + "data".getBytes(StandardCharsets.UTF_8), "blah".getBytes(StandardCharsets.UTF_8))); + Assert.assertFalse(route.match(TableName.valueOf(TEST_TABLE), + "data".getBytes(StandardCharsets.UTF_8), "blacickme".getBytes(StandardCharsets.UTF_8))); + Assert.assertTrue( + route.match(TableName.valueOf(TEST_TABLE), "data".getBytes(StandardCharsets.UTF_8), + "hithere.pickme".getBytes(StandardCharsets.UTF_8))); } catch (Exception e) { Assert.fail(e.getMessage()); @@ -180,31 +171,28 @@ public void testTopic6() { Assert.assertEquals(TableName.valueOf(TEST_TABLE), rules.getRouteRules().get(0).getTableName()); Assert.assertTrue(Bytes.equals("data".getBytes(StandardCharsets.UTF_8), - rules.getRouteRules().get(0).getColumnFamily())); + rules.getRouteRules().get(0).getColumnFamily())); Assert.assertTrue(Bytes.equals("pickme".getBytes(StandardCharsets.UTF_8), - rules.getRouteRules().get(0).getQualifier())); + rules.getRouteRules().get(0).getQualifier())); Assert.assertEquals(0, rules.getDropRules().size()); TopicRule route = rules.getRouteRules().get(0); - Assert.assertFalse( - route.match(TableName.valueOf(TEST_TABLE), - "data".getBytes(StandardCharsets.UTF_8), - "blah".getBytes(StandardCharsets.UTF_8))); Assert.assertFalse(route.match(TableName.valueOf(TEST_TABLE), - "data".getBytes(StandardCharsets.UTF_8), - "blacickme".getBytes(StandardCharsets.UTF_8))); - Assert.assertTrue(route.match(TableName.valueOf(TEST_TABLE), - "data".getBytes(StandardCharsets.UTF_8), - "hithere.pickme".getBytes(StandardCharsets.UTF_8))); - Assert.assertTrue(route.match(TableName.valueOf(TEST_TABLE), - "data".getBytes(StandardCharsets.UTF_8), - "pickme.pleaze.do.it".getBytes(StandardCharsets.UTF_8))); + "data".getBytes(StandardCharsets.UTF_8), "blah".getBytes(StandardCharsets.UTF_8))); Assert.assertFalse(route.match(TableName.valueOf(TEST_TABLE), - "data".getBytes(StandardCharsets.UTF_8), - "please.pickme.pleaze".getBytes(StandardCharsets.UTF_8))); - Assert.assertTrue(route.match(TableName.valueOf(TEST_TABLE), - "data".getBytes(StandardCharsets.UTF_8), - "pickme.pleaze.pickme".getBytes(StandardCharsets.UTF_8))); + "data".getBytes(StandardCharsets.UTF_8), "blacickme".getBytes(StandardCharsets.UTF_8))); + Assert.assertTrue( + route.match(TableName.valueOf(TEST_TABLE), "data".getBytes(StandardCharsets.UTF_8), + "hithere.pickme".getBytes(StandardCharsets.UTF_8))); + Assert.assertTrue( + route.match(TableName.valueOf(TEST_TABLE), "data".getBytes(StandardCharsets.UTF_8), + "pickme.pleaze.do.it".getBytes(StandardCharsets.UTF_8))); + Assert.assertFalse( + route.match(TableName.valueOf(TEST_TABLE), "data".getBytes(StandardCharsets.UTF_8), + "please.pickme.pleaze".getBytes(StandardCharsets.UTF_8))); + Assert.assertTrue( + route.match(TableName.valueOf(TEST_TABLE), "data".getBytes(StandardCharsets.UTF_8), + "pickme.pleaze.pickme".getBytes(StandardCharsets.UTF_8))); } catch (Exception e) { Assert.fail(e.getMessage()); diff --git a/kafka/pom.xml b/kafka/pom.xml index bf6aff8e..50b9e051 100644 --- a/kafka/pom.xml +++ b/kafka/pom.xml @@ -1,6 +1,5 @@ - - + + ${revision} - Apache HBase Connectors pom - - Connectors to Apache HBase. - + Apache HBase Connectors + Connectors to Apache HBase. http://hbase.apache.org 2018 @@ -58,21 +56,7 @@ repo - - kafka - spark - hbase-connectors-assembly - - - scm:git:git://gitbox.apache.org/repos/asf/hbase-connectors.git - scm:git:https://gitbox.apache.org/repos/asf/hbase-connectors.git - https://gitbox.apache.org/repos/asf?p=hbase-connectors.git;a=summary - - - - JIRA - http://issues.apache.org/jira/browse/HBASE - + + JIRA + http://issues.apache.org/jira/browse/HBASE + 1.0.1-SNAPSHOT @@ -344,8 +342,8 @@ ${hbase.version} - hbase-it org.apache.hbase + hbase-it ${hbase.version} test-jar test @@ -359,13 +357,6 @@ - - - kr.motd.maven - os-maven-plugin - ${os.maven.version} - - @@ -436,18 +427,18 @@ org.codehaus.mojo buildnumber-maven-plugin ${buildnumber.maven.version} + + yyyy + build.year + - validate create-timestamp + validate - - yyyy - build.year - [${java.min.version},) Java is out of date. HBase requires at least version ${java.min.version} of the JDK to properly build from source. - See the reference guide on building for more information: http://hbase.apache.org/book.html#build - + See the reference guide on building for more information: http://hbase.apache.org/book.html#build @@ -573,10 +562,10 @@ banned-illegal-imports - process-sources enforce + process-sources @@ -644,12 +633,10 @@ true 512 - - You should never use this style of annotations(i.e, 'this is for test only') + You should never use this style of annotations(i.e, 'this is for test only') in IA.Public or IA.LimitedPrivate classes. Use IA.Private to tell users this is not for public use. - For IA.Private classes, use RestrictedApi annotation in error prone instead. - + For IA.Private classes, use RestrictedApi annotation in error prone instead. org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting @@ -671,17 +658,17 @@ flatten - process-resources flatten + process-resources flatten.clean - clean clean + clean @@ -810,6 +797,13 @@ + + + kr.motd.maven + os-maven-plugin + ${os.maven.version} + + @@ -822,10 +816,10 @@ apache-rat-plugin - package check + package @@ -841,8 +835,7 @@ of the JDK to properly build from source. You appear to be using a newer dependency. You can use either "mvn -version" or "mvn enforcer:display-info" to verify what version is active. Non-release builds can temporarily build with a newer JDK version by setting the - 'compileSource' property (eg. mvn -DcompileSource=1.8 clean package). - + 'compileSource' property (eg. mvn -DcompileSource=1.8 clean package). @@ -887,13 +880,9 @@ **/example/**/*,**/DumpToStringListener*,**/KafkaProxy* **/target/classes ${project.build.directory}/surefire-reports - - ${sonar.projectBaseDir}/test-reporting/target/code-coverage/jacoco-reports/jacoco.xml - - - ${sonar.projectBaseDir}/test-reporting/target/code-coverage/scoverage-reports/scoverage.xml - - + ${sonar.projectBaseDir}/test-reporting/target/code-coverage/jacoco-reports/jacoco.xml + ${sonar.projectBaseDir}/test-reporting/target/code-coverage/scoverage-reports/scoverage.xml + ${project.basedir} ${main.basedir}/test-reporting/target/code-coverage ${codeCoverageReportRootDir}/jacoco.exec @@ -901,17 +890,6 @@ ${codeCoverageReportRootDir}/scoverage-reports - - - org.jacoco - jacoco-maven-plugin - - - org.sonarsource.scanner.maven - sonar-maven-plugin - ${sonar-maven-plugin.version} - - @@ -942,6 +920,17 @@ + + + org.jacoco + jacoco-maven-plugin + + + org.sonarsource.scanner.maven + sonar-maven-plugin + ${sonar-maven-plugin.version} + + diff --git a/spark/README.md b/spark/README.md index 2dd90eb7..7515fb8d 100755 --- a/spark/README.md +++ b/spark/README.md @@ -39,4 +39,3 @@ Note: to build the connector with Spark 2.x, compile it with `-Dscala.binary.ver - The server-side configuration is needed for column filter pushdown - if you cannot perform the server-side configuration, consider using `.option("hbase.spark.pushdown.columnfilter", false)` - The Scala library version must match the Scala version (2.11 or 2.12) used for compiling the connector. - diff --git a/spark/hbase-spark-it/pom.xml b/spark/hbase-spark-it/pom.xml index 524bc8fc..6fed9299 100644 --- a/spark/hbase-spark-it/pom.xml +++ b/spark/hbase-spark-it/pom.xml @@ -1,6 +1,6 @@ - + - - - - - - - org.apache.maven.plugins - maven-source-plugin - - - - maven-assembly-plugin - - true - - - - org.apache.maven.plugins - maven-failsafe-plugin - ${surefire.version} - - - org.apache.maven.surefire - surefire-junit4 - ${surefire.version} - - - - - ${integrationtest.include} - - - ${unittest.include} - **/*$* - - ${test.output.tofile} - false - false - - - - integration-test - integration-test - - integration-test - - - - verify - verify - - verify - - - - - - - - - - - org.apache.maven.plugins - maven-failsafe-plugin - - false - 1 - true - - 1800 - -enableassertions -Xmx${failsafe.Xmx} - -Djava.security.egd=file:/dev/./urandom -XX:+CMSClassUnloadingEnabled - -verbose:gc -XX:+PrintCommandLineFlags -XX:+PrintFlagsFinal - - - - org.apache.maven.plugins - maven-enforcer-plugin - - - - banned-hbase-spark - - enforce - - - true - - - - banned-scala - - enforce - - - true - - - - - - maven-dependency-plugin - - - create-mrapp-generated-classpath - generate-test-resources - - build-classpath - - - - ${project.build.directory}/test-classes/spark-generated-classpath - - - - - - org.apache.maven.plugins - maven-checkstyle-plugin - - - net.revelc.code - warbucks-maven-plugin - - - @@ -301,8 +172,8 @@ org.apache.spark spark-streaming_${scala.binary.version} ${spark.version} - test-jar tests + test-jar test @@ -317,21 +188,135 @@ test + + + + + + org.apache.maven.plugins + maven-source-plugin + + + + maven-assembly-plugin + + true + + + + org.apache.maven.plugins + maven-failsafe-plugin + ${surefire.version} + + + ${integrationtest.include} + + + ${unittest.include} + **/*$* + + ${test.output.tofile} + false + false + + + + org.apache.maven.surefire + surefire-junit4 + ${surefire.version} + + + + + integration-test + + integration-test + + integration-test + + + verify + + verify + + verify + + + + + - - - - skipIntegrationTests - - - skipIntegrationTests - - - - true - - - + + + + org.apache.maven.plugins + maven-failsafe-plugin + + false + 1 + true + + 1800 + -enableassertions -Xmx${failsafe.Xmx} + -Djava.security.egd=file:/dev/./urandom -XX:+CMSClassUnloadingEnabled + -verbose:gc -XX:+PrintCommandLineFlags -XX:+PrintFlagsFinal + + + + org.apache.maven.plugins + maven-enforcer-plugin + + + + banned-hbase-spark + + enforce + + + true + + + + banned-scala + + enforce + + + true + + + + + + maven-dependency-plugin + + + create-mrapp-generated-classpath + + build-classpath + + generate-test-resources + + + ${project.build.directory}/test-classes/spark-generated-classpath + + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + + + net.revelc.code + warbucks-maven-plugin + + + @@ -357,4 +342,19 @@ + + + + skipIntegrationTests + + + skipIntegrationTests + + + + true + + + + diff --git a/spark/hbase-spark-it/src/test/java/org/apache/hadoop/hbase/spark/IntegrationTestSparkBulkLoad.java b/spark/hbase-spark-it/src/test/java/org/apache/hadoop/hbase/spark/IntegrationTestSparkBulkLoad.java index 802938ef..07669157 100644 --- a/spark/hbase-spark-it/src/test/java/org/apache/hadoop/hbase/spark/IntegrationTestSparkBulkLoad.java +++ b/spark/hbase-spark-it/src/test/java/org/apache/hadoop/hbase/spark/IntegrationTestSparkBulkLoad.java @@ -1,5 +1,4 @@ -/** - * +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -28,7 +27,6 @@ import java.util.Map; import java.util.Random; import java.util.Set; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; @@ -75,16 +73,12 @@ import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; /** - * Test Bulk Load and Spark on a distributed cluster. - * It starts an Spark job that creates linked chains. - * This test mimic {@link IntegrationTestBulkLoad} in mapreduce. - * - * Usage on cluster: - * First add hbase related jars and hbase-spark.jar into spark classpath. - * - * spark-submit --class org.apache.hadoop.hbase.spark.IntegrationTestSparkBulkLoad - * HBASE_HOME/lib/hbase-spark-it-XXX-tests.jar -m slowDeterministic - * -Dhbase.spark.bulkload.chainlength=300 + * Test Bulk Load and Spark on a distributed cluster. It starts an Spark job that creates linked + * chains. This test mimic {@link IntegrationTestBulkLoad} in mapreduce. Usage on cluster: First add + * hbase related jars and hbase-spark.jar into spark classpath. spark-submit --class + * org.apache.hadoop.hbase.spark.IntegrationTestSparkBulkLoad + * HBASE_HOME/lib/hbase-spark-it-XXX-tests.jar -m slowDeterministic + * -Dhbase.spark.bulkload.chainlength=300 */ public class IntegrationTestSparkBulkLoad extends IntegrationTestBase { @@ -98,7 +92,7 @@ public class IntegrationTestSparkBulkLoad extends IntegrationTestBase { private static int DEFAULT_BULKLOAD_CHAIN_LENGTH = 200000; private static String BULKLOAD_IMPORT_ROUNDS = "hbase.spark.bulkload.importround"; - private static int DEFAULT_BULKLOAD_IMPORT_ROUNDS = 1; + private static int DEFAULT_BULKLOAD_IMPORT_ROUNDS = 1; private static String CURRENT_ROUND_NUM = "hbase.spark.bulkload.current.roundnum"; @@ -116,7 +110,7 @@ public class IntegrationTestSparkBulkLoad extends IntegrationTestBase { private boolean load = false; private boolean check = false; - private static final byte[] CHAIN_FAM = Bytes.toBytes("L"); + private static final byte[] CHAIN_FAM = Bytes.toBytes("L"); private static final byte[] SORT_FAM = Bytes.toBytes("S"); private static final byte[] DATA_FAM = Bytes.toBytes("D"); @@ -138,8 +132,8 @@ public void runLoad() throws Exception { * @throws Exception if an HBase operation or getting the test directory fails */ public void runLinkedListSparkJob(int iteration) throws Exception { - String jobName = IntegrationTestSparkBulkLoad.class.getSimpleName() + " _load " + - EnvironmentEdgeManager.currentTime(); + String jobName = IntegrationTestSparkBulkLoad.class.getSimpleName() + " _load " + + EnvironmentEdgeManager.currentTime(); LOG.info("Running iteration " + iteration + "in Spark Job"); @@ -155,24 +149,20 @@ public void runLinkedListSparkJob(int iteration) throws Exception { hbaseConf.setInt(CURRENT_ROUND_NUM, iteration); int partitionNum = hbaseConf.getInt(BULKLOAD_PARTITIONS_NUM, DEFAULT_BULKLOAD_PARTITIONS_NUM); - JavaSparkContext jsc = new JavaSparkContext(sparkConf); JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, hbaseConf); - LOG.info("Partition RDD into " + partitionNum + " parts"); List temp = new ArrayList<>(); - JavaRDD> rdd = jsc.parallelize(temp, partitionNum). - mapPartitionsWithIndex(new LinkedListCreationMapper(new SerializableWritable<>(hbaseConf)), - false); + JavaRDD> rdd = jsc.parallelize(temp, partitionNum).mapPartitionsWithIndex( + new LinkedListCreationMapper(new SerializableWritable<>(hbaseConf)), false); hbaseContext.bulkLoad(rdd, getTablename(), new ListToKeyValueFunc(), output.toUri().getPath(), - new HashMap<>(), false, HConstants.DEFAULT_MAX_FILE_SIZE); + new HashMap<>(), false, HConstants.DEFAULT_MAX_FILE_SIZE); - try (Connection conn = ConnectionFactory.createConnection(conf); - Admin admin = conn.getAdmin(); - Table table = conn.getTable(getTablename()); - RegionLocator regionLocator = conn.getRegionLocator(getTablename())) { + try (Connection conn = ConnectionFactory.createConnection(conf); Admin admin = conn.getAdmin(); + Table table = conn.getTable(getTablename()); + RegionLocator regionLocator = conn.getRegionLocator(getTablename())) { // Create a new loader. LoadIncrementalHFiles loader = new LoadIncrementalHFiles(conf); @@ -180,7 +170,6 @@ public void runLinkedListSparkJob(int iteration) throws Exception { loader.doBulkLoad(output, admin, table, regionLocator); } - // Delete the files. util.getTestFileSystem().delete(output, true); jsc.close(); @@ -188,8 +177,8 @@ public void runLinkedListSparkJob(int iteration) throws Exception { // See mapreduce.IntegrationTestBulkLoad#LinkedListCreationMapper // Used to generate test data - public static class LinkedListCreationMapper implements - Function2, Iterator>> { + public static class LinkedListCreationMapper + implements Function2, Iterator>> { SerializableWritable swConfig = null; private Random rand = new Random(); @@ -210,7 +199,6 @@ public Iterator> call(Integer v1, Iterator v2) throws Exception { int iterationsCur = config.getInt(CURRENT_ROUND_NUM, 0); List> res = new LinkedList<>(); - long tempId = partitionId + iterationsCur * partitionNum; long totalPartitionNum = partitionNum * iterationsNum; long chainId = Math.abs(rand.nextLong()); @@ -219,7 +207,7 @@ public Iterator> call(Integer v1, Iterator v2) throws Exception { byte[] chainIdArray = Bytes.toBytes(chainId); long currentRow = 0; long nextRow = getNextRow(0, chainLength); - for(long i = 0; i < chainLength; i++) { + for (long i = 0; i < chainLength; i++) { byte[] rk = Bytes.toBytes(currentRow); // Insert record into a list List tmp1 = Arrays.asList(rk, CHAIN_FAM, chainIdArray, Bytes.toBytes(nextRow)); @@ -230,7 +218,7 @@ public Iterator> call(Integer v1, Iterator v2) throws Exception { res.add(tmp3); currentRow = nextRow; - nextRow = getNextRow(i+1, chainLength); + nextRow = getNextRow(i + 1, chainLength); } return res.iterator(); } @@ -246,10 +234,8 @@ private long getNextRow(long index, long chainLength) { } } - - - public static class ListToKeyValueFunc implements - Function, Pair> { + public static class ListToKeyValueFunc + implements Function, Pair> { @Override public Pair call(List v1) throws Exception { if (v1 == null || v1.size() != 4) { @@ -266,8 +252,8 @@ public Pair call(List v1) throws Exception { */ public void runCheck() throws Exception { LOG.info("Running check"); - String jobName = IntegrationTestSparkBulkLoad.class.getSimpleName() + "_check" + - EnvironmentEdgeManager.currentTime(); + String jobName = IntegrationTestSparkBulkLoad.class.getSimpleName() + "_check" + + EnvironmentEdgeManager.currentTime(); SparkConf sparkConf = new SparkConf().setAppName(jobName).setMaster("local"); Configuration hbaseConf = new Configuration(getConf()); @@ -291,10 +277,10 @@ public void runCheck() throws Exception { // 4. Group LinkKey if they have same chainId, and repartition RDD by NaturalKeyPartitioner // 5. Check LinkList in each Partition using LinkedListCheckingFlatMapFunc hbaseContext.hbaseRDD(getTablename(), scan).flatMapToPair(new LinkedListCheckingFlatMapFunc()) - .sortByKey() - .combineByKey(new createCombinerFunc(), new mergeValueFunc(), new mergeCombinersFunc(), - new NaturalKeyPartitioner(new SerializableWritable<>(hbaseConf))) - .foreach(new LinkedListCheckingForeachFunc(new SerializableWritable<>(hbaseConf))); + .sortByKey() + .combineByKey(new createCombinerFunc(), new mergeValueFunc(), new mergeCombinersFunc(), + new NaturalKeyPartitioner(new SerializableWritable<>(hbaseConf))) + .foreach(new LinkedListCheckingForeachFunc(new SerializableWritable<>(hbaseConf))); jsc.close(); } @@ -315,11 +301,11 @@ private void runCheckWithRetry() throws Exception { * {@code Tuple}. */ public static class LinkedListCheckingFlatMapFunc implements - PairFlatMapFunction, SparkLinkKey, SparkLinkChain> { + PairFlatMapFunction, SparkLinkKey, SparkLinkChain> { @Override - public Iterator> call(Tuple2 v) throws Exception { + public Iterator> + call(Tuple2 v) throws Exception { Result value = v._2(); long longRk = Bytes.toLong(value.getRow()); List> list = new LinkedList<>(); @@ -330,15 +316,14 @@ public Iterator> call(Tuple2 tuple2 = - new Tuple2<>(new SparkLinkKey(chainId, order), new SparkLinkChain(longRk, next)); + new Tuple2<>(new SparkLinkKey(chainId, order), new SparkLinkChain(longRk, next)); list.add(tuple2); } return list.iterator(); } } - public static class createCombinerFunc implements - Function> { + public static class createCombinerFunc implements Function> { @Override public List call(SparkLinkChain v1) throws Exception { List list = new LinkedList<>(); @@ -347,8 +332,8 @@ public List call(SparkLinkChain v1) throws Exception { } } - public static class mergeValueFunc implements - Function2, SparkLinkChain, List> { + public static class mergeValueFunc + implements Function2, SparkLinkChain, List> { @Override public List call(List v1, SparkLinkChain v2) throws Exception { if (v1 == null) { @@ -360,23 +345,24 @@ public List call(List v1, SparkLinkChain v2) thr } } - public static class mergeCombinersFunc implements - Function2, List, List> { + public static class mergeCombinersFunc + implements Function2, List, List> { @Override public List call(List v1, List v2) - throws Exception { + throws Exception { v1.addAll(v2); return v1; } } /** - * Class to figure out what partition to send a link in the chain to. This is based upon - * the linkKey's ChainId. + * Class to figure out what partition to send a link in the chain to. This is based upon the + * linkKey's ChainId. */ public static class NaturalKeyPartitioner extends Partitioner { private int numPartions = 0; + public NaturalKeyPartitioner(SerializableWritable swConf) { Configuration hbaseConf = (Configuration) swConf.value(); numPartions = hbaseConf.getInt(BULKLOAD_PARTITIONS_NUM, DEFAULT_BULKLOAD_PARTITIONS_NUM); @@ -404,9 +390,9 @@ public int getPartition(Object key) { * Sort all LinkChain for one LinkKey, and test {@code List}. */ public static class LinkedListCheckingForeachFunc - implements VoidFunction>> { + implements VoidFunction>> { - private SerializableWritable swConf = null; + private SerializableWritable swConf = null; public LinkedListCheckingForeachFunc(SerializableWritable conf) { swConf = conf; @@ -425,16 +411,16 @@ public void call(Tuple2> v1) throws Exception if (next == -1) { if (lc.getRk() != 0L) { - String msg = "Chains should all start at rk 0, but read rk " + lc.getRk() - + ". Chain:" + key.getChainId() + ", order:" + key.getOrder(); + String msg = "Chains should all start at rk 0, but read rk " + lc.getRk() + ". Chain:" + + key.getChainId() + ", order:" + key.getOrder(); throw new RuntimeException(msg); } next = lc.getNext(); } else { if (next != lc.getRk()) { - String msg = "Missing a link in the chain. Prev rk " + prev + " was, expecting " - + next + " but got " + lc.getRk() + ". Chain:" + key.getChainId() - + ", order:" + key.getOrder(); + String msg = "Missing a link in the chain. Prev rk " + prev + " was, expecting " + next + + " but got " + lc.getRk() + ". Chain:" + key.getChainId() + ", order:" + + key.getOrder(); throw new RuntimeException(msg); } prev = lc.getRk(); @@ -446,16 +432,15 @@ public void call(Tuple2> v1) throws Exception int expectedChainLen = hbaseConf.getInt(BULKLOAD_CHAIN_LENGTH, DEFAULT_BULKLOAD_CHAIN_LENGTH); if (count != expectedChainLen) { String msg = "Chain wasn't the correct length. Expected " + expectedChainLen + " got " - + count + ". Chain:" + key.getChainId() + ", order:" + key.getOrder(); + + count + ". Chain:" + key.getChainId() + ", order:" + key.getOrder(); throw new RuntimeException(msg); } } } /** - * Writable class used as the key to group links in the linked list. - * - * Used as the key emited from a pass over the table. + * Writable class used as the key to group links in the linked list. Used as the key emited from a + * pass over the table. */ public static class SparkLinkKey implements java.io.Serializable, Comparable { @@ -505,7 +490,7 @@ public int compareTo(SparkLinkKey other) { /** * Writable used as the value emitted from a pass over the hbase table. */ - public static class SparkLinkChain implements java.io.Serializable, Comparable{ + public static class SparkLinkChain implements java.io.Serializable, Comparable { public Long getNext() { return next; @@ -515,7 +500,6 @@ public Long getRk() { return rk; } - public SparkLinkChain(Long rk, Long next) { this.rk = rk; this.next = next; @@ -549,11 +533,10 @@ public boolean equals(Object other) { } } - /** - * Allow the scan to go to replica, this would not affect the runCheck() - * Since data are BulkLoaded from HFile into table - * @throws IOException if an HBase operation fails + * Allow the scan to go to replica, this would not affect the runCheck() Since data are BulkLoaded + * from HFile into table + * @throws IOException if an HBase operation fails * @throws InterruptedException if modifying the table fails */ private void installSlowingCoproc() throws IOException, InterruptedException { @@ -577,7 +560,6 @@ public void testBulkLoad() throws Exception { runCheckWithRetry(); } - private byte[][] getSplits(int numRegions) { RegionSplitter.UniformSplit split = new RegionSplitter.UniformSplit(); split.setFirstRow(Bytes.toBytes(0L)); @@ -590,11 +572,7 @@ private void setupTable() throws IOException, InterruptedException { util.deleteTable(getTablename()); } - util.createTable( - getTablename(), - new byte[][]{CHAIN_FAM, SORT_FAM, DATA_FAM}, - getSplits(16) - ); + util.createTable(getTablename(), new byte[][] { CHAIN_FAM, SORT_FAM, DATA_FAM }, getSplits(16)); int replicaCount = conf.getInt(NUM_REPLICA_COUNT_KEY, DEFAULT_NUM_REPLICA_COUNT); @@ -618,7 +596,7 @@ public void setUpCluster() throws Exception { // Scale this up on a real cluster if (util.isDistributedCluster()) { util.getConfiguration().setIfUnset(BULKLOAD_PARTITIONS_NUM, - String.valueOf(DEFAULT_BULKLOAD_PARTITIONS_NUM)); + String.valueOf(DEFAULT_BULKLOAD_PARTITIONS_NUM)); util.getConfiguration().setIfUnset(BULKLOAD_IMPORT_ROUNDS, "1"); } else { util.startMiniMapReduceCluster(); @@ -663,14 +641,14 @@ public static TableName getTableName(Configuration conf) { @Override protected Set getColumnFamilies() { - return Sets.newHashSet(Bytes.toString(CHAIN_FAM) , Bytes.toString(DATA_FAM), - Bytes.toString(SORT_FAM)); + return Sets.newHashSet(Bytes.toString(CHAIN_FAM), Bytes.toString(DATA_FAM), + Bytes.toString(SORT_FAM)); } public static void main(String[] args) throws Exception { Configuration conf = HBaseConfiguration.create(); IntegrationTestingUtility.setUseDistributedCluster(conf); - int status = ToolRunner.run(conf, new IntegrationTestSparkBulkLoad(), args); + int status = ToolRunner.run(conf, new IntegrationTestSparkBulkLoad(), args); System.exit(status); } } diff --git a/spark/hbase-spark-protocol-shaded/pom.xml b/spark/hbase-spark-protocol-shaded/pom.xml index a429f30f..a17f6a99 100644 --- a/spark/hbase-spark-protocol-shaded/pom.xml +++ b/spark/hbase-spark-protocol-shaded/pom.xml @@ -1,4 +1,4 @@ - + @@ -110,8 +110,8 @@ org.apache.spark spark-streaming_${scala.binary.version} ${spark.version} - test-jar tests + test-jar test @@ -130,10 +130,10 @@ 3.0.5 test - - org.scala-lang - scala-library - + + org.scala-lang + scala-library + @@ -172,7 +172,7 @@ scala-reflect ${scala.version} - + org.apache.spark spark-unsafe_${scala.binary.version} ${spark.version} @@ -200,10 +200,10 @@ add-source - validate add-source + validate src/main/scala @@ -212,10 +212,10 @@ add-test-source - validate add-test-source + validate src/test/scala @@ -452,15 +452,13 @@ 1.5 - validate execute + validate - + @@ -484,18 +482,18 @@ scala-compile-first - process-resources add-source compile + process-resources scala-test-compile - process-test-resources testCompile + process-test-resources @@ -512,10 +510,10 @@ test - test test + test -Xmx1536m -XX:ReservedCodeCacheSize=512m false diff --git a/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java b/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java index c74db6b9..02034c11 100644 --- a/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java +++ b/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -22,7 +23,6 @@ import java.util.List; import java.util.Map; import java.util.Objects; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.filter.Filter.ReturnCode; @@ -35,28 +35,25 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import scala.collection.mutable.MutableList; import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException; /** - * This filter will push down all qualifier logic given to us - * by SparkSQL so that we have make the filters at the region server level - * and avoid sending the data back to the client to be filtered. + * This filter will push down all qualifier logic given to us by SparkSQL so that we have make the + * filters at the region server level and avoid sending the data back to the client to be filtered. */ @InterfaceAudience.Private -public class SparkSQLPushDownFilter extends FilterBase{ +public class SparkSQLPushDownFilter extends FilterBase { protected static final Logger log = LoggerFactory.getLogger(SparkSQLPushDownFilter.class); - //The following values are populated with protobuffer + // The following values are populated with protobuffer DynamicLogicExpression dynamicLogicExpression; byte[][] valueFromQueryArray; - HashMap> - currentCellToColumnIndexMap; + HashMap> currentCellToColumnIndexMap; - //The following values are transient + // The following values are transient HashMap columnToCurrentRowValueMap = null; static final byte[] rowKeyFamily = new byte[0]; @@ -65,10 +62,9 @@ public class SparkSQLPushDownFilter extends FilterBase{ String encoderClassName; public SparkSQLPushDownFilter(DynamicLogicExpression dynamicLogicExpression, - byte[][] valueFromQueryArray, - HashMap> - currentCellToColumnIndexMap, String encoderClassName) { + byte[][] valueFromQueryArray, + HashMap> currentCellToColumnIndexMap, + String encoderClassName) { this.dynamicLogicExpression = dynamicLogicExpression; this.valueFromQueryArray = valueFromQueryArray; this.currentCellToColumnIndexMap = currentCellToColumnIndexMap; @@ -76,25 +72,23 @@ public SparkSQLPushDownFilter(DynamicLogicExpression dynamicLogicExpression, } public SparkSQLPushDownFilter(DynamicLogicExpression dynamicLogicExpression, - byte[][] valueFromQueryArray, - MutableList fields, String encoderClassName) { + byte[][] valueFromQueryArray, MutableList fields, String encoderClassName) { this.dynamicLogicExpression = dynamicLogicExpression; this.valueFromQueryArray = valueFromQueryArray; this.encoderClassName = encoderClassName; - //generate family qualifier to index mapping - this.currentCellToColumnIndexMap = - new HashMap<>(); + // generate family qualifier to index mapping + this.currentCellToColumnIndexMap = new HashMap<>(); for (int i = 0; i < fields.size(); i++) { Field field = fields.apply(i); byte[] cfBytes = field.cfBytes(); ByteArrayComparable familyByteComparable = - new ByteArrayComparable(cfBytes, 0, cfBytes.length); + new ByteArrayComparable(cfBytes, 0, cfBytes.length); HashMap qualifierIndexMap = - currentCellToColumnIndexMap.get(familyByteComparable); + currentCellToColumnIndexMap.get(familyByteComparable); if (qualifierIndexMap == null) { qualifierIndexMap = new HashMap<>(); @@ -102,7 +96,7 @@ public SparkSQLPushDownFilter(DynamicLogicExpression dynamicLogicExpression, } byte[] qBytes = field.colBytes(); ByteArrayComparable qualifierByteComparable = - new ByteArrayComparable(qBytes, 0, qBytes.length); + new ByteArrayComparable(qBytes, 0, qBytes.length); qualifierIndexMap.put(qualifierByteComparable, field.colName()); } @@ -111,64 +105,51 @@ public SparkSQLPushDownFilter(DynamicLogicExpression dynamicLogicExpression, @Override public ReturnCode filterCell(final Cell c) throws IOException { - //If the map RowValueMap is empty then we need to populate + // If the map RowValueMap is empty then we need to populate // the row key if (columnToCurrentRowValueMap == null) { columnToCurrentRowValueMap = new HashMap<>(); - HashMap qualifierColumnMap = - currentCellToColumnIndexMap.get( - new ByteArrayComparable(rowKeyFamily, 0, rowKeyFamily.length)); + HashMap qualifierColumnMap = currentCellToColumnIndexMap + .get(new ByteArrayComparable(rowKeyFamily, 0, rowKeyFamily.length)); if (qualifierColumnMap != null) { - String rowKeyColumnName = - qualifierColumnMap.get( - new ByteArrayComparable(rowKeyQualifier, 0, - rowKeyQualifier.length)); - //Make sure that the rowKey is part of the where clause + String rowKeyColumnName = qualifierColumnMap + .get(new ByteArrayComparable(rowKeyQualifier, 0, rowKeyQualifier.length)); + // Make sure that the rowKey is part of the where clause if (rowKeyColumnName != null) { columnToCurrentRowValueMap.put(rowKeyColumnName, - new ByteArrayComparable(c.getRowArray(), - c.getRowOffset(), c.getRowLength())); + new ByteArrayComparable(c.getRowArray(), c.getRowOffset(), c.getRowLength())); } } } - //Always populate the column value into the RowValueMap + // Always populate the column value into the RowValueMap ByteArrayComparable currentFamilyByteComparable = - new ByteArrayComparable(c.getFamilyArray(), - c.getFamilyOffset(), - c.getFamilyLength()); + new ByteArrayComparable(c.getFamilyArray(), c.getFamilyOffset(), c.getFamilyLength()); HashMap qualifierColumnMap = - currentCellToColumnIndexMap.get( - currentFamilyByteComparable); + currentCellToColumnIndexMap.get(currentFamilyByteComparable); if (qualifierColumnMap != null) { - String columnName = - qualifierColumnMap.get( - new ByteArrayComparable(c.getQualifierArray(), - c.getQualifierOffset(), - c.getQualifierLength())); + String columnName = qualifierColumnMap.get(new ByteArrayComparable(c.getQualifierArray(), + c.getQualifierOffset(), c.getQualifierLength())); if (columnName != null) { columnToCurrentRowValueMap.put(columnName, - new ByteArrayComparable(c.getValueArray(), - c.getValueOffset(), c.getValueLength())); + new ByteArrayComparable(c.getValueArray(), c.getValueOffset(), c.getValueLength())); } } return ReturnCode.INCLUDE; } - @Override public boolean filterRow() throws IOException { try { boolean result = - dynamicLogicExpression.execute(columnToCurrentRowValueMap, - valueFromQueryArray); + dynamicLogicExpression.execute(columnToCurrentRowValueMap, valueFromQueryArray); columnToCurrentRowValueMap = null; return !result; } catch (Throwable e) { @@ -177,7 +158,6 @@ public boolean filterRow() throws IOException { return false; } - /** * @param pbBytes A pb serialized instance * @return An instance of SparkSQLPushDownFilter @@ -185,7 +165,7 @@ public boolean filterRow() throws IOException { */ @SuppressWarnings("unused") public static SparkSQLPushDownFilter parseFrom(final byte[] pbBytes) - throws DeserializationException { + throws DeserializationException { SparkFilterProtos.SQLPredicatePushDownFilter proto; try { @@ -197,81 +177,73 @@ public static SparkSQLPushDownFilter parseFrom(final byte[] pbBytes) String encoder = proto.getEncoderClassName(); BytesEncoder enc = JavaBytesEncoder.create(encoder); - //Load DynamicLogicExpression + // Load DynamicLogicExpression DynamicLogicExpression dynamicLogicExpression = - DynamicLogicExpressionBuilder.build(proto.getDynamicLogicExpression(), enc); + DynamicLogicExpressionBuilder.build(proto.getDynamicLogicExpression(), enc); - //Load valuesFromQuery + // Load valuesFromQuery final List valueFromQueryArrayList = proto.getValueFromQueryArrayList(); byte[][] valueFromQueryArray = new byte[valueFromQueryArrayList.size()][]; for (int i = 0; i < valueFromQueryArrayList.size(); i++) { valueFromQueryArray[i] = valueFromQueryArrayList.get(i).toByteArray(); } - //Load mapping from HBase family/qualifier to Spark SQL columnName - HashMap> - currentCellToColumnIndexMap = new HashMap<>(); + // Load mapping from HBase family/qualifier to Spark SQL columnName + HashMap> currentCellToColumnIndexMap = + new HashMap<>(); - for (SparkFilterProtos.SQLPredicatePushDownCellToColumnMapping - sqlPredicatePushDownCellToColumnMapping : - proto.getCellToColumnMappingList()) { + for (SparkFilterProtos.SQLPredicatePushDownCellToColumnMapping sqlPredicatePushDownCellToColumnMapping : proto + .getCellToColumnMappingList()) { - byte[] familyArray = - sqlPredicatePushDownCellToColumnMapping.getColumnFamily().toByteArray(); + byte[] familyArray = sqlPredicatePushDownCellToColumnMapping.getColumnFamily().toByteArray(); ByteArrayComparable familyByteComparable = - new ByteArrayComparable(familyArray, 0, familyArray.length); + new ByteArrayComparable(familyArray, 0, familyArray.length); HashMap qualifierMap = - currentCellToColumnIndexMap.get(familyByteComparable); + currentCellToColumnIndexMap.get(familyByteComparable); if (qualifierMap == null) { qualifierMap = new HashMap<>(); currentCellToColumnIndexMap.put(familyByteComparable, qualifierMap); } - byte[] qualifierArray = - sqlPredicatePushDownCellToColumnMapping.getQualifier().toByteArray(); + byte[] qualifierArray = sqlPredicatePushDownCellToColumnMapping.getQualifier().toByteArray(); ByteArrayComparable qualifierByteComparable = - new ByteArrayComparable(qualifierArray, 0 ,qualifierArray.length); + new ByteArrayComparable(qualifierArray, 0, qualifierArray.length); qualifierMap.put(qualifierByteComparable, - sqlPredicatePushDownCellToColumnMapping.getColumnName()); + sqlPredicatePushDownCellToColumnMapping.getColumnName()); } - return new SparkSQLPushDownFilter(dynamicLogicExpression, - valueFromQueryArray, currentCellToColumnIndexMap, encoder); + return new SparkSQLPushDownFilter(dynamicLogicExpression, valueFromQueryArray, + currentCellToColumnIndexMap, encoder); } - /** - * @return The filter serialized using pb - */ + /** Returns The filter serialized using pb */ public byte[] toByteArray() { SparkFilterProtos.SQLPredicatePushDownFilter.Builder builder = - SparkFilterProtos.SQLPredicatePushDownFilter.newBuilder(); + SparkFilterProtos.SQLPredicatePushDownFilter.newBuilder(); SparkFilterProtos.SQLPredicatePushDownCellToColumnMapping.Builder columnMappingBuilder = - SparkFilterProtos.SQLPredicatePushDownCellToColumnMapping.newBuilder(); + SparkFilterProtos.SQLPredicatePushDownCellToColumnMapping.newBuilder(); builder.setDynamicLogicExpression(dynamicLogicExpression.toExpressionString()); - for (byte[] valueFromQuery: valueFromQueryArray) { + for (byte[] valueFromQuery : valueFromQueryArray) { builder.addValueFromQueryArray(ByteString.copyFrom(valueFromQuery)); } - for (Map.Entry> - familyEntry : currentCellToColumnIndexMap.entrySet()) { - for (Map.Entry qualifierEntry : - familyEntry.getValue().entrySet()) { - columnMappingBuilder.setColumnFamily( - ByteString.copyFrom(familyEntry.getKey().bytes())); - columnMappingBuilder.setQualifier( - ByteString.copyFrom(qualifierEntry.getKey().bytes())); + for (Map.Entry> familyEntry : currentCellToColumnIndexMap.entrySet()) { + for (Map.Entry qualifierEntry : familyEntry.getValue() + .entrySet()) { + columnMappingBuilder.setColumnFamily(ByteString.copyFrom(familyEntry.getKey().bytes())); + columnMappingBuilder.setQualifier(ByteString.copyFrom(qualifierEntry.getKey().bytes())); columnMappingBuilder.setColumnName(qualifierEntry.getValue()); builder.addCellToColumnMapping(columnMappingBuilder.build()); } } builder.setEncoderClassName(encoderClassName); - return builder.build().toByteArray(); } @@ -294,9 +266,9 @@ public boolean equals(Object obj) { } i++; } - return this.dynamicLogicExpression.equals(f.dynamicLogicExpression) && - this.currentCellToColumnIndexMap.equals(f.currentCellToColumnIndexMap) && - this.encoderClassName.equals(f.encoderClassName); + return this.dynamicLogicExpression.equals(f.dynamicLogicExpression) + && this.currentCellToColumnIndexMap.equals(f.currentCellToColumnIndexMap) + && this.encoderClassName.equals(f.encoderClassName); } @Override diff --git a/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkDeleteExample.java b/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkDeleteExample.java index 8cf2c7fe..8be0acc9 100644 --- a/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkDeleteExample.java +++ b/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkDeleteExample.java @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -31,13 +32,13 @@ import org.apache.yetus.audience.InterfaceAudience; /** - * This is a simple example of deleting records in HBase - * with the bulkDelete function. + * This is a simple example of deleting records in HBase with the bulkDelete function. */ @InterfaceAudience.Private final public class JavaHBaseBulkDeleteExample { - private JavaHBaseBulkDeleteExample() {} + private JavaHBaseBulkDeleteExample() { + } public static void main(String[] args) { if (args.length < 1) { @@ -64,8 +65,7 @@ public static void main(String[] args) { JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, conf); - hbaseContext.bulkDelete(rdd, - TableName.valueOf(tableName), new DeleteFunction(), 4); + hbaseContext.bulkDelete(rdd, TableName.valueOf(tableName), new DeleteFunction(), 4); } finally { jsc.stop(); } @@ -74,6 +74,7 @@ public static void main(String[] args) { public static class DeleteFunction implements Function { private static final long serialVersionUID = 1L; + public Delete call(byte[] v) throws Exception { return new Delete(v); } diff --git a/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkGetExample.java b/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkGetExample.java index b5143de0..8ff21ea4 100644 --- a/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkGetExample.java +++ b/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkGetExample.java @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -34,13 +35,13 @@ import org.apache.yetus.audience.InterfaceAudience; /** - * This is a simple example of getting records in HBase - * with the bulkGet function. + * This is a simple example of getting records in HBase with the bulkGet function. */ @InterfaceAudience.Private final public class JavaHBaseBulkGetExample { - private JavaHBaseBulkGetExample() {} + private JavaHBaseBulkGetExample() { + } public static void main(String[] args) { if (args.length < 1) { @@ -68,7 +69,7 @@ public static void main(String[] args) { JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, conf); hbaseContext.bulkGet(TableName.valueOf(tableName), 2, rdd, new GetFunction(), - new ResultFunction()); + new ResultFunction()); } finally { jsc.stop(); } @@ -97,17 +98,11 @@ public String call(Result result) throws Exception { Cell cell = it.next(); String q = Bytes.toString(cell.getQualifierArray()); if (q.equals("counter")) { - b.append("(") - .append(Bytes.toString(cell.getQualifierArray())) - .append(",") - .append(Bytes.toLong(cell.getValueArray())) - .append(")"); + b.append("(").append(Bytes.toString(cell.getQualifierArray())).append(",") + .append(Bytes.toLong(cell.getValueArray())).append(")"); } else { - b.append("(") - .append(Bytes.toString(cell.getQualifierArray())) - .append(",") - .append(Bytes.toString(cell.getValueArray())) - .append(")"); + b.append("(").append(Bytes.toString(cell.getQualifierArray())).append(",") + .append(Bytes.toString(cell.getValueArray())).append(")"); } } return b.toString(); diff --git a/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkLoadExample.java b/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkLoadExample.java index 6738059c..44f1b339 100644 --- a/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkLoadExample.java +++ b/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkLoadExample.java @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -35,19 +36,17 @@ import org.apache.yetus.audience.InterfaceAudience; /** - * Run this example using command below: - * - * SPARK_HOME/bin/spark-submit --master local[2] - * --class org.apache.hadoop.hbase.spark.example.hbasecontext.JavaHBaseBulkLoadExample - * path/to/hbase-spark.jar {path/to/output/HFiles} - * - * This example will output put hfiles in {path/to/output/HFiles}, and user can run - * 'hbase org.apache.hadoop.hbase.tool.LoadIncrementalHFiles' to load the HFiles into table to - * verify this example. + * Run this example using command below: SPARK_HOME/bin/spark-submit --master local[2] --class + * org.apache.hadoop.hbase.spark.example.hbasecontext.JavaHBaseBulkLoadExample + * path/to/hbase-spark.jar {path/to/output/HFiles} This example will output put hfiles in + * {path/to/output/HFiles}, and user can run 'hbase + * org.apache.hadoop.hbase.tool.LoadIncrementalHFiles' to load the HFiles into table to verify this + * example. */ @InterfaceAudience.Private final public class JavaHBaseBulkLoadExample { - private JavaHBaseBulkLoadExample() {} + private JavaHBaseBulkLoadExample() { + } public static void main(String[] args) { if (args.length < 1) { @@ -63,7 +62,7 @@ public static void main(String[] args) { JavaSparkContext jsc = new JavaSparkContext(sparkConf); try { - List list= new ArrayList(); + List list = new ArrayList(); // row1 list.add("1," + columnFamily1 + ",b,1"); // row3 @@ -79,17 +78,15 @@ public static void main(String[] args) { Configuration conf = HBaseConfiguration.create(); JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, conf); - - - hbaseContext.bulkLoad(rdd, TableName.valueOf(tableName),new BulkLoadFunction(), args[0], - new HashMap(), false, HConstants.DEFAULT_MAX_FILE_SIZE); + hbaseContext.bulkLoad(rdd, TableName.valueOf(tableName), new BulkLoadFunction(), args[0], + new HashMap(), false, HConstants.DEFAULT_MAX_FILE_SIZE); } finally { jsc.stop(); } } public static class BulkLoadFunction - implements Function> { + implements Function> { @Override public Pair call(String v1) throws Exception { if (v1 == null) { @@ -97,12 +94,12 @@ public Pair call(String v1) throws Exception { } String[] strs = v1.split(","); - if(strs.length != 4) { + if (strs.length != 4) { return null; } KeyFamilyQualifier kfq = new KeyFamilyQualifier(Bytes.toBytes(strs[0]), - Bytes.toBytes(strs[1]), Bytes.toBytes(strs[2])); + Bytes.toBytes(strs[1]), Bytes.toBytes(strs[2])); return new Pair(kfq, Bytes.toBytes(strs[3])); } } diff --git a/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkPutExample.java b/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkPutExample.java index 4a80b96a..5f685a14 100644 --- a/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkPutExample.java +++ b/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkPutExample.java @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -31,18 +32,17 @@ import org.apache.yetus.audience.InterfaceAudience; /** - * This is a simple example of putting records in HBase - * with the bulkPut function. + * This is a simple example of putting records in HBase with the bulkPut function. */ @InterfaceAudience.Private final public class JavaHBaseBulkPutExample { - private JavaHBaseBulkPutExample() {} + private JavaHBaseBulkPutExample() { + } public static void main(String[] args) { if (args.length < 2) { - System.out.println("JavaHBaseBulkPutExample " + - "{tableName} {columnFamily}"); + System.out.println("JavaHBaseBulkPutExample " + "{tableName} {columnFamily}"); return; } @@ -66,9 +66,7 @@ public static void main(String[] args) { JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, conf); - hbaseContext.bulkPut(rdd, - TableName.valueOf(tableName), - new PutFunction()); + hbaseContext.bulkPut(rdd, TableName.valueOf(tableName), new PutFunction()); } finally { jsc.stop(); } @@ -82,8 +80,7 @@ public Put call(String v) throws Exception { String[] cells = v.split(","); Put put = new Put(Bytes.toBytes(cells[0])); - put.addColumn(Bytes.toBytes(cells[1]), Bytes.toBytes(cells[2]), - Bytes.toBytes(cells[3])); + put.addColumn(Bytes.toBytes(cells[1]), Bytes.toBytes(cells[2]), Bytes.toBytes(cells[3])); return put; } diff --git a/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseDistributedScan.java b/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseDistributedScan.java index 0d4f6806..76c7f6d8 100644 --- a/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseDistributedScan.java +++ b/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseDistributedScan.java @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -33,13 +34,13 @@ import scala.Tuple2; /** - * This is a simple example of scanning records from HBase - * with the hbaseRDD function. + * This is a simple example of scanning records from HBase with the hbaseRDD function. */ @InterfaceAudience.Private final public class JavaHBaseDistributedScan { - private JavaHBaseDistributedScan() {} + private JavaHBaseDistributedScan() { + } public static void main(String[] args) { if (args.length < 1) { @@ -61,7 +62,7 @@ public static void main(String[] args) { scan.setCaching(100); JavaRDD> javaRdd = - hbaseContext.hbaseRDD(TableName.valueOf(tableName), scan); + hbaseContext.hbaseRDD(TableName.valueOf(tableName), scan); List results = javaRdd.map(new ScanConvertFunction()).collect(); @@ -71,8 +72,8 @@ public static void main(String[] args) { } } - private static class ScanConvertFunction implements - Function, String> { + private static class ScanConvertFunction + implements Function, String> { @Override public String call(Tuple2 v1) throws Exception { return Bytes.toString(v1._1().copyBytes()); diff --git a/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseMapGetPutExample.java b/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseMapGetPutExample.java index a55d8532..c516ab35 100644 --- a/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseMapGetPutExample.java +++ b/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseMapGetPutExample.java @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -39,13 +40,13 @@ import scala.Tuple2; /** - * This is a simple example of using the foreachPartition - * method with a HBase connection + * This is a simple example of using the foreachPartition method with a HBase connection */ @InterfaceAudience.Private final public class JavaHBaseMapGetPutExample { - private JavaHBaseMapGetPutExample() {} + private JavaHBaseMapGetPutExample() { + } public static void main(String[] args) { if (args.length < 1) { @@ -71,26 +72,24 @@ public static void main(String[] args) { JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, conf); - hbaseContext.foreachPartition(rdd, - new VoidFunction, Connection>>() { - public void call(Tuple2, Connection> t) - throws Exception { - Table table = t._2().getTable(TableName.valueOf(tableName)); - BufferedMutator mutator = t._2().getBufferedMutator(TableName.valueOf(tableName)); + hbaseContext.foreachPartition(rdd, new VoidFunction, Connection>>() { + public void call(Tuple2, Connection> t) throws Exception { + Table table = t._2().getTable(TableName.valueOf(tableName)); + BufferedMutator mutator = t._2().getBufferedMutator(TableName.valueOf(tableName)); - while (t._1().hasNext()) { - byte[] b = t._1().next(); - Result r = table.get(new Get(b)); - if (r.getExists()) { - mutator.mutate(new Put(b)); - } + while (t._1().hasNext()) { + byte[] b = t._1().next(); + Result r = table.get(new Get(b)); + if (r.getExists()) { + mutator.mutate(new Put(b)); } - - mutator.flush(); - mutator.close(); - table.close(); } - }); + + mutator.flush(); + mutator.close(); + table.close(); + } + }); } finally { jsc.stop(); } @@ -98,6 +97,7 @@ public void call(Tuple2, Connection> t) public static class GetFunction implements Function { private static final long serialVersionUID = 1L; + public Get call(byte[] v) throws Exception { return new Get(v); } diff --git a/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseStreamingBulkPutExample.java b/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseStreamingBulkPutExample.java index 74fadc65..dc07a562 100644 --- a/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseStreamingBulkPutExample.java +++ b/spark/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseStreamingBulkPutExample.java @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -36,12 +37,12 @@ @InterfaceAudience.Private final public class JavaHBaseStreamingBulkPutExample { - private JavaHBaseStreamingBulkPutExample() {} + private JavaHBaseStreamingBulkPutExample() { + } public static void main(String[] args) { if (args.length < 4) { - System.out.println("JavaHBaseBulkPutExample " + - "{host} {port} {tableName}"); + System.out.println("JavaHBaseBulkPutExample " + "{host} {port} {tableName}"); return; } @@ -49,26 +50,22 @@ public static void main(String[] args) { String port = args[1]; String tableName = args[2]; - SparkConf sparkConf = - new SparkConf().setAppName("JavaHBaseStreamingBulkPutExample " + - tableName + ":" + port + ":" + tableName); + SparkConf sparkConf = new SparkConf() + .setAppName("JavaHBaseStreamingBulkPutExample " + tableName + ":" + port + ":" + tableName); JavaSparkContext jsc = new JavaSparkContext(sparkConf); try { - JavaStreamingContext jssc = - new JavaStreamingContext(jsc, new Duration(1000)); + JavaStreamingContext jssc = new JavaStreamingContext(jsc, new Duration(1000)); JavaReceiverInputDStream javaDstream = - jssc.socketTextStream(host, Integer.parseInt(port)); + jssc.socketTextStream(host, Integer.parseInt(port)); Configuration conf = HBaseConfiguration.create(); JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, conf); - hbaseContext.streamBulkPut(javaDstream, - TableName.valueOf(tableName), - new PutFunction()); + hbaseContext.streamBulkPut(javaDstream, TableName.valueOf(tableName), new PutFunction()); } finally { jsc.stop(); } @@ -82,9 +79,7 @@ public Put call(String v) throws Exception { String[] part = v.split(","); Put put = new Put(Bytes.toBytes(part[0])); - put.addColumn(Bytes.toBytes(part[1]), - Bytes.toBytes(part[2]), - Bytes.toBytes(part[3])); + put.addColumn(Bytes.toBytes(part[1]), Bytes.toBytes(part[2]), Bytes.toBytes(part[3])); return put; } diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/BulkLoadPartitioner.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/BulkLoadPartitioner.scala index 9442c50b..17316638 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/BulkLoadPartitioner.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/BulkLoadPartitioner.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,15 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark import java.util import java.util.Comparator - -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.util.Bytes import org.apache.spark.Partitioner +import org.apache.yetus.audience.InterfaceAudience /** * A Partitioner implementation that will separate records to different @@ -31,8 +30,7 @@ import org.apache.spark.Partitioner * @param startKeys The start keys for the given table */ @InterfaceAudience.Public -class BulkLoadPartitioner(startKeys:Array[Array[Byte]]) - extends Partitioner { +class BulkLoadPartitioner(startKeys: Array[Array[Byte]]) extends Partitioner { // when table not exist, startKeys = Byte[0][] override def numPartitions: Int = if (startKeys.length == 0) 1 else startKeys.length @@ -44,7 +42,7 @@ class BulkLoadPartitioner(startKeys:Array[Array[Byte]]) } } - val rowKey:Array[Byte] = + val rowKey: Array[Byte] = key match { case qualifier: KeyFamilyQualifier => qualifier.rowKey diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/ByteArrayComparable.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/ByteArrayComparable.scala index 2d0be38c..78cd3abc 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/ByteArrayComparable.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/ByteArrayComparable.scala @@ -1,5 +1,4 @@ /* - * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -16,15 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.util.Bytes +import org.apache.yetus.audience.InterfaceAudience @InterfaceAudience.Public -class ByteArrayComparable(val bytes:Array[Byte], val offset:Int = 0, var length:Int = -1) - extends Comparable[ByteArrayComparable] { +class ByteArrayComparable(val bytes: Array[Byte], val offset: Int = 0, var length: Int = -1) + extends Comparable[ByteArrayComparable] { if (length == -1) { length = bytes.length @@ -38,7 +36,7 @@ class ByteArrayComparable(val bytes:Array[Byte], val offset:Int = 0, var length: Bytes.hashCode(bytes, offset, length) } - override def equals (obj: Any): Boolean = { + override def equals(obj: Any): Boolean = { obj match { case b: ByteArrayComparable => Bytes.equals(bytes, offset, length, b.bytes, b.offset, b.length) diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/ByteArrayWrapper.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/ByteArrayWrapper.scala index 738fa45a..a774838e 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/ByteArrayWrapper.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/ByteArrayWrapper.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -17,9 +18,8 @@ package org.apache.hadoop.hbase.spark import java.io.Serializable - -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.util.Bytes +import org.apache.yetus.audience.InterfaceAudience /** * This is a wrapper over a byte array so it can work as @@ -28,10 +28,11 @@ import org.apache.hadoop.hbase.util.Bytes * @param value The Byte Array value */ @InterfaceAudience.Public -class ByteArrayWrapper (var value:Array[Byte]) - extends Comparable[ByteArrayWrapper] with Serializable { +class ByteArrayWrapper(var value: Array[Byte]) + extends Comparable[ByteArrayWrapper] + with Serializable { override def compareTo(valueOther: ByteArrayWrapper): Int = { - Bytes.compareTo(value,valueOther.value) + Bytes.compareTo(value, valueOther.value) } override def equals(o2: Any): Boolean = { o2 match { @@ -41,7 +42,7 @@ class ByteArrayWrapper (var value:Array[Byte]) false } } - override def hashCode():Int = { + override def hashCode(): Int = { Bytes.hashCode(value) } } diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/ColumnFamilyQualifierMapKeyWrapper.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/ColumnFamilyQualifierMapKeyWrapper.scala index 3037001b..84883992 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/ColumnFamilyQualifierMapKeyWrapper.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/ColumnFamilyQualifierMapKeyWrapper.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,11 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.util.Bytes +import org.apache.yetus.audience.InterfaceAudience /** * A wrapper class that will allow both columnFamily and qualifier to @@ -32,42 +32,45 @@ import org.apache.hadoop.hbase.util.Bytes * @param qualifierLength Length of the qualifier value with in the array */ @InterfaceAudience.Public -class ColumnFamilyQualifierMapKeyWrapper(val columnFamily:Array[Byte], - val columnFamilyOffSet:Int, - val columnFamilyLength:Int, - val qualifier:Array[Byte], - val qualifierOffSet:Int, - val qualifierLength:Int) - extends Serializable{ +class ColumnFamilyQualifierMapKeyWrapper( + val columnFamily: Array[Byte], + val columnFamilyOffSet: Int, + val columnFamilyLength: Int, + val qualifier: Array[Byte], + val qualifierOffSet: Int, + val qualifierLength: Int) + extends Serializable { - override def equals(other:Any): Boolean = { + override def equals(other: Any): Boolean = { val otherWrapper = other.asInstanceOf[ColumnFamilyQualifierMapKeyWrapper] - Bytes.compareTo(columnFamily, + Bytes.compareTo( + columnFamily, columnFamilyOffSet, columnFamilyLength, otherWrapper.columnFamily, otherWrapper.columnFamilyOffSet, - otherWrapper.columnFamilyLength) == 0 && Bytes.compareTo(qualifier, - qualifierOffSet, - qualifierLength, - otherWrapper.qualifier, - otherWrapper.qualifierOffSet, - otherWrapper.qualifierLength) == 0 + otherWrapper.columnFamilyLength) == 0 && Bytes.compareTo( + qualifier, + qualifierOffSet, + qualifierLength, + otherWrapper.qualifier, + otherWrapper.qualifierOffSet, + otherWrapper.qualifierLength) == 0 } - override def hashCode():Int = { + override def hashCode(): Int = { Bytes.hashCode(columnFamily, columnFamilyOffSet, columnFamilyLength) + Bytes.hashCode(qualifier, qualifierOffSet, qualifierLength) } - def cloneColumnFamily():Array[Byte] = { + def cloneColumnFamily(): Array[Byte] = { val resultArray = new Array[Byte](columnFamilyLength) System.arraycopy(columnFamily, columnFamilyOffSet, resultArray, 0, columnFamilyLength) resultArray } - def cloneQualifier():Array[Byte] = { + def cloneQualifier(): Array[Byte] = { val resultArray = new Array[Byte](qualifierLength) System.arraycopy(qualifier, qualifierOffSet, resultArray, 0, qualifierLength) resultArray diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala index 1a6cde21..7005f658 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,30 +15,27 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark import java.util import java.util.concurrent.ConcurrentLinkedQueue - -import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hadoop.hbase.CellUtil +import org.apache.hadoop.hbase.HBaseConfiguration +import org.apache.hadoop.hbase.HColumnDescriptor +import org.apache.hadoop.hbase.HTableDescriptor +import org.apache.hadoop.hbase.TableName import org.apache.hadoop.hbase.client._ import org.apache.hadoop.hbase.io.ImmutableBytesWritable import org.apache.hadoop.hbase.mapred.TableOutputFormat import org.apache.hadoop.hbase.spark.datasources._ import org.apache.hadoop.hbase.types._ import org.apache.hadoop.hbase.util.{Bytes, PositionedByteRange, SimplePositionedMutableByteRange} -import org.apache.hadoop.hbase.HBaseConfiguration -import org.apache.hadoop.hbase.HTableDescriptor -import org.apache.hadoop.hbase.HColumnDescriptor -import org.apache.hadoop.hbase.TableName -import org.apache.hadoop.hbase.CellUtil import org.apache.hadoop.mapred.JobConf import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, SaveMode, Row, SQLContext} +import org.apache.spark.sql.{DataFrame, Row, SaveMode, SQLContext} import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ - +import org.apache.yetus.audience.InterfaceAudience import scala.collection.mutable /** @@ -55,21 +53,21 @@ import scala.collection.mutable * Through the HBase Bytes object commands. */ @InterfaceAudience.Private -class DefaultSource extends RelationProvider with CreatableRelationProvider with Logging { +class DefaultSource extends RelationProvider with CreatableRelationProvider with Logging { + /** * Is given input from SparkSQL to construct a BaseRelation - * - * @param sqlContext SparkSQL context + * + * @param sqlContext SparkSQL context * @param parameters Parameters given to us from SparkSQL * @return A BaseRelation Object */ - override def createRelation(sqlContext: SQLContext, - parameters: Map[String, String]): - BaseRelation = { + override def createRelation( + sqlContext: SQLContext, + parameters: Map[String, String]): BaseRelation = { new HBaseRelation(parameters, None)(sqlContext) } - override def createRelation( sqlContext: SQLContext, mode: SaveMode, @@ -85,54 +83,70 @@ class DefaultSource extends RelationProvider with CreatableRelationProvider wit /** * Implementation of Spark BaseRelation that will build up our scan logic * , do the scan pruning, filter push down, and value conversions - * - * @param sqlContext SparkSQL context + * + * @param sqlContext SparkSQL context */ @InterfaceAudience.Private -case class HBaseRelation ( +case class HBaseRelation( @transient parameters: Map[String, String], - userSpecifiedSchema: Option[StructType] - )(@transient val sqlContext: SQLContext) - extends BaseRelation with PrunedFilteredScan with InsertableRelation with Logging { + userSpecifiedSchema: Option[StructType])(@transient val sqlContext: SQLContext) + extends BaseRelation + with PrunedFilteredScan + with InsertableRelation + with Logging { val timestamp = parameters.get(HBaseSparkConf.TIMESTAMP).map(_.toLong) val minTimestamp = parameters.get(HBaseSparkConf.TIMERANGE_START).map(_.toLong) val maxTimestamp = parameters.get(HBaseSparkConf.TIMERANGE_END).map(_.toLong) val maxVersions = parameters.get(HBaseSparkConf.MAX_VERSIONS).map(_.toInt) - val encoderClsName = parameters.get(HBaseSparkConf.QUERY_ENCODER).getOrElse(HBaseSparkConf.DEFAULT_QUERY_ENCODER) + val encoderClsName = + parameters.get(HBaseSparkConf.QUERY_ENCODER).getOrElse(HBaseSparkConf.DEFAULT_QUERY_ENCODER) @transient val encoder = JavaBytesEncoder.create(encoderClsName) val catalog = HBaseTableCatalog(parameters) def tableName = s"${catalog.namespace}:${catalog.name}" val configResources = parameters.get(HBaseSparkConf.HBASE_CONFIG_LOCATION) - val useHBaseContext = parameters.get(HBaseSparkConf.USE_HBASECONTEXT).map(_.toBoolean).getOrElse(HBaseSparkConf.DEFAULT_USE_HBASECONTEXT) - val usePushDownColumnFilter = parameters.get(HBaseSparkConf.PUSHDOWN_COLUMN_FILTER) - .map(_.toBoolean).getOrElse(HBaseSparkConf.DEFAULT_PUSHDOWN_COLUMN_FILTER) + val useHBaseContext = parameters + .get(HBaseSparkConf.USE_HBASECONTEXT) + .map(_.toBoolean) + .getOrElse(HBaseSparkConf.DEFAULT_USE_HBASECONTEXT) + val usePushDownColumnFilter = parameters + .get(HBaseSparkConf.PUSHDOWN_COLUMN_FILTER) + .map(_.toBoolean) + .getOrElse(HBaseSparkConf.DEFAULT_PUSHDOWN_COLUMN_FILTER) // The user supplied per table parameter will overwrite global ones in SparkConf - val blockCacheEnable = parameters.get(HBaseSparkConf.QUERY_CACHEBLOCKS).map(_.toBoolean) - .getOrElse( - sqlContext.sparkContext.getConf.getBoolean( - HBaseSparkConf.QUERY_CACHEBLOCKS, HBaseSparkConf.DEFAULT_QUERY_CACHEBLOCKS)) - val cacheSize = parameters.get(HBaseSparkConf.QUERY_CACHEDROWS).map(_.toInt) - .getOrElse( - sqlContext.sparkContext.getConf.getInt( - HBaseSparkConf.QUERY_CACHEDROWS, -1)) - val batchNum = parameters.get(HBaseSparkConf.QUERY_BATCHSIZE).map(_.toInt) - .getOrElse(sqlContext.sparkContext.getConf.getInt( - HBaseSparkConf.QUERY_BATCHSIZE, -1)) - - val bulkGetSize = parameters.get(HBaseSparkConf.BULKGET_SIZE).map(_.toInt) - .getOrElse(sqlContext.sparkContext.getConf.getInt( - HBaseSparkConf.BULKGET_SIZE, HBaseSparkConf.DEFAULT_BULKGET_SIZE)) - - //create or get latest HBaseContext - val hbaseContext:HBaseContext = if (useHBaseContext) { + val blockCacheEnable = parameters + .get(HBaseSparkConf.QUERY_CACHEBLOCKS) + .map(_.toBoolean) + .getOrElse(sqlContext.sparkContext.getConf + .getBoolean(HBaseSparkConf.QUERY_CACHEBLOCKS, HBaseSparkConf.DEFAULT_QUERY_CACHEBLOCKS)) + val cacheSize = parameters + .get(HBaseSparkConf.QUERY_CACHEDROWS) + .map(_.toInt) + .getOrElse(sqlContext.sparkContext.getConf.getInt(HBaseSparkConf.QUERY_CACHEDROWS, -1)) + val batchNum = parameters + .get(HBaseSparkConf.QUERY_BATCHSIZE) + .map(_.toInt) + .getOrElse(sqlContext.sparkContext.getConf.getInt(HBaseSparkConf.QUERY_BATCHSIZE, -1)) + + val bulkGetSize = parameters + .get(HBaseSparkConf.BULKGET_SIZE) + .map(_.toInt) + .getOrElse(sqlContext.sparkContext.getConf + .getInt(HBaseSparkConf.BULKGET_SIZE, HBaseSparkConf.DEFAULT_BULKGET_SIZE)) + + // create or get latest HBaseContext + val hbaseContext: HBaseContext = if (useHBaseContext) { LatestHBaseContextCache.latest } else { val hadoopConfig = sqlContext.sparkContext.hadoopConfiguration val config = HBaseConfiguration.create(hadoopConfig) - configResources.map(resource => resource.split(",").foreach(r => config.addResource(r))) + configResources.map( + resource => + resource + .split(",") + .foreach(r => config.addResource(r))) new HBaseContext(sqlContext.sparkContext, config) } @@ -147,15 +161,18 @@ case class HBaseRelation ( */ override val schema: StructType = userSpecifiedSchema.getOrElse(catalog.toDataType) - - def createTable() { - val numReg = parameters.get(HBaseTableCatalog.newTable).map(x => x.toInt).getOrElse(0) - val startKey = Bytes.toBytes( - parameters.get(HBaseTableCatalog.regionStart) + val numReg = parameters + .get(HBaseTableCatalog.newTable) + .map(x => x.toInt) + .getOrElse(0) + val startKey = Bytes.toBytes( + parameters + .get(HBaseTableCatalog.regionStart) .getOrElse(HBaseTableCatalog.defaultRegionStart)) val endKey = Bytes.toBytes( - parameters.get(HBaseTableCatalog.regionEnd) + parameters + .get(HBaseTableCatalog.regionEnd) .getOrElse(HBaseTableCatalog.defaultRegionEnd)) if (numReg > 3) { val tName = TableName.valueOf(tableName) @@ -167,67 +184,71 @@ case class HBaseRelation ( try { if (!admin.tableExists(tName)) { val tableDesc = new HTableDescriptor(tName) - cfs.foreach { x => - val cf = new HColumnDescriptor(x.getBytes()) - logDebug(s"add family $x to ${tableName}") - tableDesc.addFamily(cf) + cfs.foreach { + x => + val cf = new HColumnDescriptor(x.getBytes()) + logDebug(s"add family $x to ${tableName}") + tableDesc.addFamily(cf) } val splitKeys = Bytes.split(startKey, endKey, numReg); admin.createTable(tableDesc, splitKeys) } - }finally { + } finally { admin.close() connection.close() } } else { - logInfo( - s"""${HBaseTableCatalog.newTable} + logInfo(s"""${HBaseTableCatalog.newTable} |is not defined or no larger than 3, skip the create table""".stripMargin) } } /** - * - * @param data - * @param overwrite - */ + * @param data + * @param overwrite + */ override def insert(data: DataFrame, overwrite: Boolean): Unit = { val jobConfig: JobConf = new JobConf(hbaseConf, this.getClass) jobConfig.setOutputFormat(classOf[TableOutputFormat]) jobConfig.set(TableOutputFormat.OUTPUT_TABLE, tableName) var count = 0 val rkFields = catalog.getRowKey - val rkIdxedFields = rkFields.map{ case x => - (schema.fieldIndex(x.colName), x) + val rkIdxedFields = rkFields.map { + case x => + (schema.fieldIndex(x.colName), x) } - val colsIdxedFields = schema - .fieldNames - .partition( x => rkFields.map(_.colName).contains(x)) - ._2.map(x => (schema.fieldIndex(x), catalog.getField(x))) + val colsIdxedFields = schema.fieldNames + .partition(x => rkFields.map(_.colName).contains(x)) + ._2 + .map(x => (schema.fieldIndex(x), catalog.getField(x))) val rdd = data.rdd def convertToPut(row: Row) = { // construct bytes for row key - val rowBytes = rkIdxedFields.map { case (x, y) => - Utils.toBytes(row(x), y) + val rowBytes = rkIdxedFields.map { + case (x, y) => + Utils.toBytes(row(x), y) } - val rLen = rowBytes.foldLeft(0) { case (x, y) => - x + y.length + val rLen = rowBytes.foldLeft(0) { + case (x, y) => + x + y.length } val rBytes = new Array[Byte](rLen) var offset = 0 - rowBytes.foreach { x => - System.arraycopy(x, 0, rBytes, offset, x.length) - offset += x.length + rowBytes.foreach { + x => + System.arraycopy(x, 0, rBytes, offset, x.length) + offset += x.length } val put = timestamp.fold(new Put(rBytes))(new Put(rBytes, _)) - colsIdxedFields.foreach { case (x, y) => - val r = row(x) - if (r != null) { - val b = Utils.toBytes(r, y) - put.addColumn(Bytes.toBytes(y.cf), Bytes.toBytes(y.col), b) - } + colsIdxedFields.foreach { + case (x, y) => + val r = row(x) + if (r != null) { + val b = Utils.toBytes(r, y) + put.addColumn(Bytes.toBytes(y.cf), Bytes.toBytes(y.col), b) + } } count += 1 (new ImmutableBytesWritable, put) @@ -239,58 +260,70 @@ case class HBaseRelation ( requiredColumns.map(catalog.sMap.getField(_)).zipWithIndex } - /** - * Takes a HBase Row object and parses all of the fields from it. - * This is independent of which fields were requested from the key - * Because we have all the data it's less complex to parse everything. - * - * @param row the retrieved row from hbase. - * @param keyFields all of the fields in the row key, ORDERED by their order in the row key. - */ + * Takes a HBase Row object and parses all of the fields from it. + * This is independent of which fields were requested from the key + * Because we have all the data it's less complex to parse everything. + * + * @param row the retrieved row from hbase. + * @param keyFields all of the fields in the row key, ORDERED by their order in the row key. + */ def parseRowKey(row: Array[Byte], keyFields: Seq[Field]): Map[Field, Any] = { - keyFields.foldLeft((0, Seq[(Field, Any)]()))((state, field) => { - val idx = state._1 - val parsed = state._2 - if (field.length != -1) { - val value = Utils.hbaseFieldToScalaType(field, row, idx, field.length) - // Return the new index and appended value - (idx + field.length, parsed ++ Seq((field, value))) - } else { - field.dt match { - case StringType => - val pos = row.indexOf(HBaseTableCatalog.delimiter, idx) - if (pos == -1 || pos > row.length) { - // this is at the last dimension - val value = Utils.hbaseFieldToScalaType(field, row, idx, row.length) - (row.length + 1, parsed ++ Seq((field, value))) - } else { - val value = Utils.hbaseFieldToScalaType(field, row, idx, pos - idx) - (pos, parsed ++ Seq((field, value))) + keyFields + .foldLeft((0, Seq[(Field, Any)]()))( + (state, field) => { + val idx = state._1 + val parsed = state._2 + if (field.length != -1) { + val value = Utils.hbaseFieldToScalaType(field, row, idx, field.length) + // Return the new index and appended value + (idx + field.length, parsed ++ Seq((field, value))) + } else { + field.dt match { + case StringType => + val pos = row.indexOf(HBaseTableCatalog.delimiter, idx) + if (pos == -1 || pos > row.length) { + // this is at the last dimension + val value = Utils.hbaseFieldToScalaType(field, row, idx, row.length) + (row.length + 1, parsed ++ Seq((field, value))) + } else { + val value = Utils.hbaseFieldToScalaType(field, row, idx, pos - idx) + (pos, parsed ++ Seq((field, value))) + } + // We don't know the length, assume it extends to the end of the rowkey. + case _ => + ( + row.length + 1, + parsed ++ Seq((field, Utils.hbaseFieldToScalaType(field, row, idx, row.length)))) } - // We don't know the length, assume it extends to the end of the rowkey. - case _ => (row.length + 1, parsed ++ Seq((field, Utils.hbaseFieldToScalaType(field, row, idx, row.length)))) - } - } - })._2.toMap + } + }) + ._2 + .toMap } def buildRow(fields: Seq[Field], result: Result): Row = { val r = result.getRow val keySeq = parseRowKey(r, catalog.getRowKey) - val valueSeq = fields.filter(!_.isRowKey).map { x => - val kv = result.getColumnLatestCell(Bytes.toBytes(x.cf), Bytes.toBytes(x.col)) - if (kv == null || kv.getValueLength == 0) { - (x, null) - } else { - val v = CellUtil.cloneValue(kv) - (x, x.dt match { - // Here, to avoid arraycopy, return v directly instead of calling hbaseFieldToScalaType - case BinaryType => v - case _ => Utils.hbaseFieldToScalaType(x, v, 0, v.length) - }) + val valueSeq = fields + .filter(!_.isRowKey) + .map { + x => + val kv = result.getColumnLatestCell(Bytes.toBytes(x.cf), Bytes.toBytes(x.col)) + if (kv == null || kv.getValueLength == 0) { + (x, null) + } else { + val v = CellUtil.cloneValue(kv) + ( + x, + x.dt match { + // Here, to avoid arraycopy, return v directly instead of calling hbaseFieldToScalaType + case BinaryType => v + case _ => Utils.hbaseFieldToScalaType(x, v, 0, v.length) + }) + } } - }.toMap + .toMap val unionedRow = keySeq ++ valueSeq // Return the row ordered by the requested order Row.fromSeq(fields.map(unionedRow.get(_).getOrElse(null))) @@ -321,50 +354,65 @@ case class HBaseRelation ( logDebug("pushDownRowKeyFilter: " + pushDownRowKeyFilter.ranges) if (pushDownDynamicLogicExpression != null) { - logDebug("pushDownDynamicLogicExpression: " + - pushDownDynamicLogicExpression.toExpressionString) + logDebug( + "pushDownDynamicLogicExpression: " + + pushDownDynamicLogicExpression.toExpressionString) } logDebug("valueArray: " + valueArray.length) val requiredQualifierDefinitionList = new mutable.MutableList[Field] - requiredColumns.foreach( c => { - val field = catalog.getField(c) - requiredQualifierDefinitionList += field - }) + requiredColumns.foreach( + c => { + val field = catalog.getField(c) + requiredQualifierDefinitionList += field + }) - //retain the information for unit testing checks - DefaultSourceStaticUtils.populateLatestExecutionRules(pushDownRowKeyFilter, + // retain the information for unit testing checks + DefaultSourceStaticUtils.populateLatestExecutionRules( + pushDownRowKeyFilter, pushDownDynamicLogicExpression) val getList = new util.ArrayList[Get]() val rddList = new util.ArrayList[RDD[Row]]() - //add points to getList - pushDownRowKeyFilter.points.foreach(p => { - val get = new Get(p) - requiredQualifierDefinitionList.foreach( d => { - if (d.isRowKey) - get.addColumn(d.cfBytes, d.colBytes) + // add points to getList + pushDownRowKeyFilter.points.foreach( + p => { + val get = new Get(p) + requiredQualifierDefinitionList.foreach( + d => { + if (d.isRowKey) + get.addColumn(d.cfBytes, d.colBytes) + }) + getList.add(get) }) - getList.add(get) - }) - val pushDownFilterJava = if (usePushDownColumnFilter && pushDownDynamicLogicExpression != null) { - Some(new SparkSQLPushDownFilter(pushDownDynamicLogicExpression, - valueArray, requiredQualifierDefinitionList, encoderClsName)) - } else { - None - } - val hRdd = new HBaseTableScanRDD(this, hbaseContext, pushDownFilterJava, requiredQualifierDefinitionList.seq) + val pushDownFilterJava = + if (usePushDownColumnFilter && pushDownDynamicLogicExpression != null) { + Some( + new SparkSQLPushDownFilter( + pushDownDynamicLogicExpression, + valueArray, + requiredQualifierDefinitionList, + encoderClsName)) + } else { + None + } + val hRdd = new HBaseTableScanRDD( + this, + hbaseContext, + pushDownFilterJava, + requiredQualifierDefinitionList.seq) pushDownRowKeyFilter.points.foreach(hRdd.addPoint(_)) pushDownRowKeyFilter.ranges.foreach(hRdd.addRange(_)) var resultRDD: RDD[Row] = { - val tmp = hRdd.map{ r => - val indexedFields = getIndexedProjections(requiredColumns).map(_._1) - buildRow(indexedFields, r) + val tmp = hRdd.map { + r => + val indexedFields = getIndexedProjections(requiredColumns).map(_._1) + buildRow(indexedFields, r) } if (tmp.partitions.size > 0) { @@ -379,37 +427,40 @@ case class HBaseRelation ( scan.setCacheBlocks(blockCacheEnable) scan.setBatch(batchNum) scan.setCaching(cacheSize) - requiredQualifierDefinitionList.foreach( d => - scan.addColumn(d.cfBytes, d.colBytes)) - - val rdd = hbaseContext.hbaseRDD(TableName.valueOf(tableName), scan).map(r => { - val indexedFields = getIndexedProjections(requiredColumns).map(_._1) - buildRow(indexedFields, r._2) - }) - resultRDD=rdd + requiredQualifierDefinitionList.foreach(d => scan.addColumn(d.cfBytes, d.colBytes)) + + val rdd = hbaseContext + .hbaseRDD(TableName.valueOf(tableName), scan) + .map( + r => { + val indexedFields = getIndexedProjections(requiredColumns).map(_._1) + buildRow(indexedFields, r._2) + }) + resultRDD = rdd } resultRDD } - def buildPushDownPredicatesResource(filters: Array[Filter]): - (RowKeyFilter, DynamicLogicExpression, Array[Array[Byte]]) = { - var superRowKeyFilter:RowKeyFilter = null + def buildPushDownPredicatesResource( + filters: Array[Filter]): (RowKeyFilter, DynamicLogicExpression, Array[Array[Byte]]) = { + var superRowKeyFilter: RowKeyFilter = null val queryValueList = new mutable.MutableList[Array[Byte]] var superDynamicLogicExpression: DynamicLogicExpression = null - filters.foreach( f => { - val rowKeyFilter = new RowKeyFilter() - val logicExpression = transverseFilterTree(rowKeyFilter, queryValueList, f) - if (superDynamicLogicExpression == null) { - superDynamicLogicExpression = logicExpression - superRowKeyFilter = rowKeyFilter - } else { - superDynamicLogicExpression = - new AndLogicExpression(superDynamicLogicExpression, logicExpression) - superRowKeyFilter.mergeIntersect(rowKeyFilter) - } + filters.foreach( + f => { + val rowKeyFilter = new RowKeyFilter() + val logicExpression = transverseFilterTree(rowKeyFilter, queryValueList, f) + if (superDynamicLogicExpression == null) { + superDynamicLogicExpression = logicExpression + superRowKeyFilter = rowKeyFilter + } else { + superDynamicLogicExpression = + new AndLogicExpression(superDynamicLogicExpression, logicExpression) + superRowKeyFilter.mergeIntersect(rowKeyFilter) + } - }) + }) val queryValueArray = queryValueList.toArray @@ -421,27 +472,27 @@ case class HBaseRelation ( } /** - * For some codec, the order may be inconsistent between java primitive - * type and its byte array. We may have to split the predicates on some - * of the java primitive type into multiple predicates. The encoder will take - * care of it and returning the concrete ranges. - * - * For example in naive codec, some of the java primitive types have to be split into multiple - * predicates, and union these predicates together to make the predicates be performed correctly. - * For example, if we have "COLUMN < 2", we will transform it into - * "0 <= COLUMN < 2 OR Integer.MIN_VALUE <= COLUMN <= -1" - */ - - def transverseFilterTree(parentRowKeyFilter:RowKeyFilter, - valueArray:mutable.MutableList[Array[Byte]], - filter:Filter): DynamicLogicExpression = { + * For some codec, the order may be inconsistent between java primitive + * type and its byte array. We may have to split the predicates on some + * of the java primitive type into multiple predicates. The encoder will take + * care of it and returning the concrete ranges. + * + * For example in naive codec, some of the java primitive types have to be split into multiple + * predicates, and union these predicates together to make the predicates be performed correctly. + * For example, if we have "COLUMN < 2", we will transform it into + * "0 <= COLUMN < 2 OR Integer.MIN_VALUE <= COLUMN <= -1" + */ + + def transverseFilterTree( + parentRowKeyFilter: RowKeyFilter, + valueArray: mutable.MutableList[Array[Byte]], + filter: Filter): DynamicLogicExpression = { filter match { case EqualTo(attr, value) => val field = catalog.getField(attr) if (field != null) { if (field.isRowKey) { - parentRowKeyFilter.mergeIntersect(new RowKeyFilter( - Utils.toBytes(value, field), null)) + parentRowKeyFilter.mergeIntersect(new RowKeyFilter(Utils.toBytes(value, field), null)) } val byteValue = Utils.toBytes(value, field) valueArray += byteValue @@ -449,34 +500,30 @@ case class HBaseRelation ( new EqualLogicExpression(attr, valueArray.length - 1, false) /** - * encoder may split the predicates into multiple byte array boundaries. - * Each boundaries is mapped into the RowKeyFilter and then is unioned by the reduce - * operation. If the data type is not supported, b will be None, and there is - * no operation happens on the parentRowKeyFilter. - * - * Note that because LessThan is not inclusive, thus the first bound should be exclusive, - * which is controlled by inc. - * - * The other predicates, i.e., GreaterThan/LessThanOrEqual/GreaterThanOrEqual follows - * the similar logic. - */ + * encoder may split the predicates into multiple byte array boundaries. + * Each boundaries is mapped into the RowKeyFilter and then is unioned by the reduce + * operation. If the data type is not supported, b will be None, and there is + * no operation happens on the parentRowKeyFilter. + * + * Note that because LessThan is not inclusive, thus the first bound should be exclusive, + * which is controlled by inc. + * + * The other predicates, i.e., GreaterThan/LessThanOrEqual/GreaterThanOrEqual follows + * the similar logic. + */ case LessThan(attr, value) => val field = catalog.getField(attr) if (field != null) { if (field.isRowKey) { val b = encoder.ranges(value) var inc = false - b.map(_.less.map { x => - val r = new RowKeyFilter(null, - new ScanRange(x.upper, inc, x.low, true) - ) - inc = true - r - }).map { x => - x.reduce { (i, j) => - i.mergeUnion(j) - } - }.map(parentRowKeyFilter.mergeIntersect(_)) + b.map(_.less.map { + x => + val r = new RowKeyFilter(null, new ScanRange(x.upper, inc, x.low, true)) + inc = true + r + }).map { x => x.reduce { (i, j) => i.mergeUnion(j) } } + .map(parentRowKeyFilter.mergeIntersect(_)) } val byteValue = encoder.encode(field.dt, value) valueArray += byteValue @@ -488,16 +535,13 @@ case class HBaseRelation ( if (field.isRowKey) { val b = encoder.ranges(value) var inc = false - b.map(_.greater.map{x => - val r = new RowKeyFilter(null, - new ScanRange(x.upper, true, x.low, inc)) - inc = true - r - }).map { x => - x.reduce { (i, j) => - i.mergeUnion(j) - } - }.map(parentRowKeyFilter.mergeIntersect(_)) + b.map(_.greater.map { + x => + val r = new RowKeyFilter(null, new ScanRange(x.upper, true, x.low, inc)) + inc = true + r + }).map { x => x.reduce { (i, j) => i.mergeUnion(j) } } + .map(parentRowKeyFilter.mergeIntersect(_)) } val byteValue = encoder.encode(field.dt, value) valueArray += byteValue @@ -508,14 +552,10 @@ case class HBaseRelation ( if (field != null) { if (field.isRowKey) { val b = encoder.ranges(value) - b.map(_.less.map(x => - new RowKeyFilter(null, - new ScanRange(x.upper, true, x.low, true)))) - .map { x => - x.reduce{ (i, j) => - i.mergeUnion(j) - } - }.map(parentRowKeyFilter.mergeIntersect(_)) + b.map( + _.less.map(x => new RowKeyFilter(null, new ScanRange(x.upper, true, x.low, true)))) + .map { x => x.reduce { (i, j) => i.mergeUnion(j) } } + .map(parentRowKeyFilter.mergeIntersect(_)) } val byteValue = encoder.encode(field.dt, value) valueArray += byteValue @@ -526,14 +566,10 @@ case class HBaseRelation ( if (field != null) { if (field.isRowKey) { val b = encoder.ranges(value) - b.map(_.greater.map(x => - new RowKeyFilter(null, - new ScanRange(x.upper, true, x.low, true)))) - .map { x => - x.reduce { (i, j) => - i.mergeUnion(j) - } - }.map(parentRowKeyFilter.mergeIntersect(_)) + b.map( + _.greater.map(x => new RowKeyFilter(null, new ScanRange(x.upper, true, x.low, true)))) + .map { x => x.reduce { (i, j) => i.mergeUnion(j) } } + .map(parentRowKeyFilter.mergeIntersect(_)) } val byteValue = encoder.encode(field.dt, value) valueArray += byteValue @@ -545,7 +581,8 @@ case class HBaseRelation ( if (field.isRowKey) { val p = Utils.toBytes(value, field) val endRange = Utils.incrementByteArray(p) - parentRowKeyFilter.mergeIntersect(new RowKeyFilter(null, new ScanRange(endRange, false, p, true))) + parentRowKeyFilter.mergeIntersect( + new RowKeyFilter(null, new ScanRange(endRange, false, p, true))) } val byteValue = Utils.toBytes(value, field) valueArray += byteValue @@ -560,7 +597,6 @@ case class HBaseRelation ( new OrLogicExpression(leftExpression, rightExpression) case And(left, right) => - val leftExpression = transverseFilterTree(parentRowKeyFilter, valueArray, left) val rightSideRowKeyFilter = new RowKeyFilter val rightExpression = transverseFilterTree(rightSideRowKeyFilter, valueArray, right) @@ -588,53 +624,62 @@ case class HBaseRelation ( * @param isLowerBoundEqualTo Include lower bound value in the results */ @InterfaceAudience.Private -class ScanRange(var upperBound:Array[Byte], var isUpperBoundEqualTo:Boolean, - var lowerBound:Array[Byte], var isLowerBoundEqualTo:Boolean) - extends Serializable { +class ScanRange( + var upperBound: Array[Byte], + var isUpperBoundEqualTo: Boolean, + var lowerBound: Array[Byte], + var isLowerBoundEqualTo: Boolean) + extends Serializable { /** * Function to merge another scan object through a AND operation - * - * @param other Other scan object + * + * @param other Other scan object */ - def mergeIntersect(other:ScanRange): Unit = { + def mergeIntersect(other: ScanRange): Unit = { val upperBoundCompare = compareRange(upperBound, other.upperBound) val lowerBoundCompare = compareRange(lowerBound, other.lowerBound) - upperBound = if (upperBoundCompare <0) upperBound else other.upperBound - lowerBound = if (lowerBoundCompare >0) lowerBound else other.lowerBound + upperBound = if (upperBoundCompare < 0) upperBound else other.upperBound + lowerBound = if (lowerBoundCompare > 0) lowerBound else other.lowerBound - isLowerBoundEqualTo = if (lowerBoundCompare == 0) - isLowerBoundEqualTo && other.isLowerBoundEqualTo - else if (lowerBoundCompare < 0) other.isLowerBoundEqualTo - else isLowerBoundEqualTo + isLowerBoundEqualTo = + if (lowerBoundCompare == 0) + isLowerBoundEqualTo && other.isLowerBoundEqualTo + else if (lowerBoundCompare < 0) other.isLowerBoundEqualTo + else isLowerBoundEqualTo - isUpperBoundEqualTo = if (upperBoundCompare == 0) - isUpperBoundEqualTo && other.isUpperBoundEqualTo - else if (upperBoundCompare < 0) isUpperBoundEqualTo - else other.isUpperBoundEqualTo + isUpperBoundEqualTo = + if (upperBoundCompare == 0) + isUpperBoundEqualTo && other.isUpperBoundEqualTo + else if (upperBoundCompare < 0) isUpperBoundEqualTo + else other.isUpperBoundEqualTo } /** * Function to merge another scan object through a OR operation - * - * @param other Other scan object + * + * @param other Other scan object */ - def mergeUnion(other:ScanRange): Unit = { + def mergeUnion(other: ScanRange): Unit = { val upperBoundCompare = compareRange(upperBound, other.upperBound) val lowerBoundCompare = compareRange(lowerBound, other.lowerBound) - upperBound = if (upperBoundCompare >0) upperBound else other.upperBound - lowerBound = if (lowerBoundCompare <0) lowerBound else other.lowerBound + upperBound = if (upperBoundCompare > 0) upperBound else other.upperBound + lowerBound = if (lowerBoundCompare < 0) lowerBound else other.lowerBound - isLowerBoundEqualTo = if (lowerBoundCompare == 0) - isLowerBoundEqualTo || other.isLowerBoundEqualTo - else if (lowerBoundCompare < 0) isLowerBoundEqualTo else other.isLowerBoundEqualTo + isLowerBoundEqualTo = + if (lowerBoundCompare == 0) + isLowerBoundEqualTo || other.isLowerBoundEqualTo + else if (lowerBoundCompare < 0) isLowerBoundEqualTo + else other.isLowerBoundEqualTo - isUpperBoundEqualTo = if (upperBoundCompare == 0) - isUpperBoundEqualTo || other.isUpperBoundEqualTo - else if (upperBoundCompare < 0) other.isUpperBoundEqualTo else isUpperBoundEqualTo + isUpperBoundEqualTo = + if (upperBoundCompare == 0) + isUpperBoundEqualTo || other.isUpperBoundEqualTo + else if (upperBoundCompare < 0) other.isUpperBoundEqualTo + else isUpperBoundEqualTo } /** @@ -657,9 +702,9 @@ class ScanRange(var upperBound:Array[Byte], var isUpperBoundEqualTo:Boolean, * @param other Other scan object * @return True is overlap false is not overlap */ - def getOverLapScanRange(other:ScanRange): ScanRange = { - var leftRange:ScanRange = null - var rightRange:ScanRange = null + def getOverLapScanRange(other: ScanRange): ScanRange = { + var leftRange: ScanRange = null + var rightRange: ScanRange = null // First identify the Left range // Also lower bound can't be null @@ -682,13 +727,13 @@ class ScanRange(var upperBound:Array[Byte], var isUpperBoundEqualTo:Boolean, } /** - * The leftRange.upperBound has to be larger than the rightRange's lowerBound. - * Otherwise, there is no overlap. - * - * @param left: The range with the smaller lowBound - * @param right: The range with the larger lowBound - * @return Whether two ranges have overlap. - */ + * The leftRange.upperBound has to be larger than the rightRange's lowerBound. + * Otherwise, there is no overlap. + * + * @param left: The range with the smaller lowBound + * @param right: The range with the larger lowBound + * @return Whether two ranges have overlap. + */ def hasOverlap(left: ScanRange, right: ScanRange): Boolean = { compareRange(left.upperBound, right.lowerBound) >= 0 @@ -702,7 +747,7 @@ class ScanRange(var upperBound:Array[Byte], var isUpperBoundEqualTo:Boolean, * @param right Right byte array * @return 0 for equals 1 is left is greater and -1 is right is greater */ - def compareRange(left:Array[Byte], right:Array[Byte]): Int = { + def compareRange(left: Array[Byte], right: Array[Byte]): Int = { if (left == null && right == null) 0 else if (left == null && right != null) 1 else if (left != null && right == null) -1 @@ -710,20 +755,19 @@ class ScanRange(var upperBound:Array[Byte], var isUpperBoundEqualTo:Boolean, } /** - * * @return */ - def containsPoint(point:Array[Byte]): Boolean = { + def containsPoint(point: Array[Byte]): Boolean = { val lowerCompare = compareRange(point, lowerBound) val upperCompare = compareRange(point, upperBound) ((isLowerBoundEqualTo && lowerCompare >= 0) || (!isLowerBoundEqualTo && lowerCompare > 0)) && - ((isUpperBoundEqualTo && upperCompare <= 0) || - (!isUpperBoundEqualTo && upperCompare < 0)) + ((isUpperBoundEqualTo && upperCompare <= 0) || + (!isUpperBoundEqualTo && upperCompare < 0)) } - override def toString:String = { + override def toString: String = { "ScanRange:(upperBound:" + Bytes.toString(upperBound) + ",isUpperBoundEqualTo:" + isUpperBoundEqualTo + ",lowerBound:" + Bytes.toString(lowerBound) + ",isLowerBoundEqualTo:" + isLowerBoundEqualTo + ")" @@ -738,16 +782,16 @@ class ScanRange(var upperBound:Array[Byte], var isUpperBoundEqualTo:Boolean, * @param currentRange the initial scanRange when the filter is created */ @InterfaceAudience.Private -class ColumnFilter (currentPoint:Array[Byte] = null, - currentRange:ScanRange = null, - var points:mutable.MutableList[Array[Byte]] = - new mutable.MutableList[Array[Byte]](), - var ranges:mutable.MutableList[ScanRange] = - new mutable.MutableList[ScanRange]() ) extends Serializable { - //Collection of ranges - if (currentRange != null ) ranges.+=(currentRange) - - //Collection of points +class ColumnFilter( + currentPoint: Array[Byte] = null, + currentRange: ScanRange = null, + var points: mutable.MutableList[Array[Byte]] = new mutable.MutableList[Array[Byte]](), + var ranges: mutable.MutableList[ScanRange] = new mutable.MutableList[ScanRange]()) + extends Serializable { + // Collection of ranges + if (currentRange != null) ranges.+=(currentRange) + + // Collection of points if (currentPoint != null) points.+=(currentPoint) /** @@ -759,34 +803,47 @@ class ColumnFilter (currentPoint:Array[Byte] = null, * @param valueLength The length of the value * @return True is the value passes the filter false if not */ - def validate(value:Array[Byte], valueOffSet:Int, valueLength:Int):Boolean = { + def validate(value: Array[Byte], valueOffSet: Int, valueLength: Int): Boolean = { var result = false - points.foreach( p => { - if (Bytes.equals(p, 0, p.length, value, valueOffSet, valueLength)) { - result = true - } - }) - - ranges.foreach( r => { - val upperBoundPass = r.upperBound == null || - (r.isUpperBoundEqualTo && - Bytes.compareTo(r.upperBound, 0, r.upperBound.length, - value, valueOffSet, valueLength) >= 0) || - (!r.isUpperBoundEqualTo && - Bytes.compareTo(r.upperBound, 0, r.upperBound.length, - value, valueOffSet, valueLength) > 0) - - val lowerBoundPass = r.lowerBound == null || r.lowerBound.length == 0 + points.foreach( + p => { + if (Bytes.equals(p, 0, p.length, value, valueOffSet, valueLength)) { + result = true + } + }) + + ranges.foreach( + r => { + val upperBoundPass = r.upperBound == null || + (r.isUpperBoundEqualTo && + Bytes.compareTo( + r.upperBound, + 0, + r.upperBound.length, + value, + valueOffSet, + valueLength) >= 0) || + (!r.isUpperBoundEqualTo && + Bytes + .compareTo(r.upperBound, 0, r.upperBound.length, value, valueOffSet, valueLength) > 0) + + val lowerBoundPass = r.lowerBound == null || r.lowerBound.length == 0 (r.isLowerBoundEqualTo && - Bytes.compareTo(r.lowerBound, 0, r.lowerBound.length, - value, valueOffSet, valueLength) <= 0) || + Bytes + .compareTo( + r.lowerBound, + 0, + r.lowerBound.length, + value, + valueOffSet, + valueLength) <= 0) || (!r.isLowerBoundEqualTo && - Bytes.compareTo(r.lowerBound, 0, r.lowerBound.length, - value, valueOffSet, valueLength) < 0) + Bytes + .compareTo(r.lowerBound, 0, r.lowerBound.length, value, valueOffSet, valueLength) < 0) - result = result || (upperBoundPass && lowerBoundPass) - }) + result = result || (upperBoundPass && lowerBoundPass) + }) result } @@ -796,18 +853,21 @@ class ColumnFilter (currentPoint:Array[Byte] = null, * * @param other Filter to merge */ - def mergeUnion(other:ColumnFilter): Unit = { - other.points.foreach( p => points += p) - - other.ranges.foreach( otherR => { - var doesOverLap = false - ranges.foreach{ r => - if (r.getOverLapScanRange(otherR) != null) { - r.mergeUnion(otherR) - doesOverLap = true - }} - if (!doesOverLap) ranges.+=(otherR) - }) + def mergeUnion(other: ColumnFilter): Unit = { + other.points.foreach(p => points += p) + + other.ranges.foreach( + otherR => { + var doesOverLap = false + ranges.foreach { + r => + if (r.getOverLapScanRange(otherR) != null) { + r.mergeUnion(otherR) + doesOverLap = true + } + } + if (!doesOverLap) ranges.+=(otherR) + }) } /** @@ -816,46 +876,52 @@ class ColumnFilter (currentPoint:Array[Byte] = null, * * @param other Filter to merge */ - def mergeIntersect(other:ColumnFilter): Unit = { + def mergeIntersect(other: ColumnFilter): Unit = { val survivingPoints = new mutable.MutableList[Array[Byte]]() - points.foreach( p => { - other.points.foreach( otherP => { - if (Bytes.equals(p, otherP)) { - survivingPoints.+=(p) - } + points.foreach( + p => { + other.points.foreach( + otherP => { + if (Bytes.equals(p, otherP)) { + survivingPoints.+=(p) + } + }) }) - }) points = survivingPoints val survivingRanges = new mutable.MutableList[ScanRange]() - other.ranges.foreach( otherR => { - ranges.foreach( r => { - if (r.getOverLapScanRange(otherR) != null) { - r.mergeIntersect(otherR) - survivingRanges += r - } + other.ranges.foreach( + otherR => { + ranges.foreach( + r => { + if (r.getOverLapScanRange(otherR) != null) { + r.mergeIntersect(otherR) + survivingRanges += r + } + }) }) - }) ranges = survivingRanges } - override def toString:String = { + override def toString: String = { val strBuilder = new StringBuilder strBuilder.append("(points:(") var isFirst = true - points.foreach( p => { - if (isFirst) isFirst = false - else strBuilder.append(",") - strBuilder.append(Bytes.toString(p)) - }) + points.foreach( + p => { + if (isFirst) isFirst = false + else strBuilder.append(",") + strBuilder.append(Bytes.toString(p)) + }) strBuilder.append("),ranges:") isFirst = true - ranges.foreach( r => { - if (isFirst) isFirst = false - else strBuilder.append(",") - strBuilder.append(r) - }) + ranges.foreach( + r => { + if (isFirst) isFirst = false + else strBuilder.append(",") + strBuilder.append(r) + }) strBuilder.append("))") strBuilder.toString() } @@ -882,7 +948,7 @@ class ColumnFilterCollection { * @param column The column to be merged * @param other The other ColumnFilter object to merge */ - def mergeUnion(column:String, other:ColumnFilter): Unit = { + def mergeUnion(column: String, other: ColumnFilter): Unit = { val existingFilter = columnFilterMap.get(column) if (existingFilter.isEmpty) { columnFilterMap.+=((column, other)) @@ -898,10 +964,11 @@ class ColumnFilterCollection { * * @param other The other Column Filter Collection to be merged */ - def mergeUnion(other:ColumnFilterCollection): Unit = { - other.columnFilterMap.foreach( e => { - mergeUnion(e._1, e._2) - }) + def mergeUnion(other: ColumnFilterCollection): Unit = { + other.columnFilterMap.foreach( + e => { + mergeUnion(e._1, e._2) + }) } /** @@ -911,20 +978,21 @@ class ColumnFilterCollection { * * @param other The column filter from the other collection */ - def mergeIntersect(other:ColumnFilterCollection): Unit = { - other.columnFilterMap.foreach( e => { - val existingColumnFilter = columnFilterMap.get(e._1) - if (existingColumnFilter.isEmpty) { - columnFilterMap += e - } else { - existingColumnFilter.get.mergeIntersect(e._2) - } - }) + def mergeIntersect(other: ColumnFilterCollection): Unit = { + other.columnFilterMap.foreach( + e => { + val existingColumnFilter = columnFilterMap.get(e._1) + if (existingColumnFilter.isEmpty) { + columnFilterMap += e + } else { + existingColumnFilter.get.mergeIntersect(e._2) + } + }) } - override def toString:String = { + override def toString: String = { val strBuilder = new StringBuilder - columnFilterMap.foreach( e => strBuilder.append(e)) + columnFilterMap.foreach(e => strBuilder.append(e)) strBuilder.toString() } } @@ -948,12 +1016,11 @@ object DefaultSourceStaticUtils { getFreshByteRange(bytes, 0, bytes.length) } - def getFreshByteRange(bytes: Array[Byte], offset: Int = 0, length: Int): - PositionedByteRange = { + def getFreshByteRange(bytes: Array[Byte], offset: Int = 0, length: Int): PositionedByteRange = { byteRange.get().set(bytes).setLength(length).setOffset(offset) } - //This will contain the last 5 filters and required fields used in buildScan + // This will contain the last 5 filters and required fields used in buildScan // These values can be used in unit testing to make sure we are converting // The Spark SQL input correctly val lastFiveExecutionRules = @@ -966,10 +1033,11 @@ object DefaultSourceStaticUtils { * @param rowKeyFilter The rowKey Filter logic used in the last query * @param dynamicLogicExpression The dynamicLogicExpression used in the last query */ - def populateLatestExecutionRules(rowKeyFilter: RowKeyFilter, - dynamicLogicExpression: DynamicLogicExpression): Unit = { - lastFiveExecutionRules.add(new ExecutionRuleForUnitTesting( - rowKeyFilter, dynamicLogicExpression)) + def populateLatestExecutionRules( + rowKeyFilter: RowKeyFilter, + dynamicLogicExpression: DynamicLogicExpression): Unit = { + lastFiveExecutionRules.add( + new ExecutionRuleForUnitTesting(rowKeyFilter, dynamicLogicExpression)) while (lastFiveExecutionRules.size() > 5) { lastFiveExecutionRules.poll() } @@ -984,17 +1052,16 @@ object DefaultSourceStaticUtils { * @param currentRange the initial scanRange when the filter is created */ @InterfaceAudience.Private -class RowKeyFilter (currentPoint:Array[Byte] = null, - currentRange:ScanRange = - new ScanRange(null, true, new Array[Byte](0), true), - var points:mutable.MutableList[Array[Byte]] = - new mutable.MutableList[Array[Byte]](), - var ranges:mutable.MutableList[ScanRange] = - new mutable.MutableList[ScanRange]() ) extends Serializable { - //Collection of ranges - if (currentRange != null ) ranges.+=(currentRange) - - //Collection of points +class RowKeyFilter( + currentPoint: Array[Byte] = null, + currentRange: ScanRange = new ScanRange(null, true, new Array[Byte](0), true), + var points: mutable.MutableList[Array[Byte]] = new mutable.MutableList[Array[Byte]](), + var ranges: mutable.MutableList[ScanRange] = new mutable.MutableList[ScanRange]()) + extends Serializable { + // Collection of ranges + if (currentRange != null) ranges.+=(currentRange) + + // Collection of points if (currentPoint != null) points.+=(currentPoint) /** @@ -1006,34 +1073,47 @@ class RowKeyFilter (currentPoint:Array[Byte] = null, * @param valueLength The length of the value * @return True is the value passes the filter false if not */ - def validate(value:Array[Byte], valueOffSet:Int, valueLength:Int):Boolean = { + def validate(value: Array[Byte], valueOffSet: Int, valueLength: Int): Boolean = { var result = false - points.foreach( p => { - if (Bytes.equals(p, 0, p.length, value, valueOffSet, valueLength)) { - result = true - } - }) - - ranges.foreach( r => { - val upperBoundPass = r.upperBound == null || - (r.isUpperBoundEqualTo && - Bytes.compareTo(r.upperBound, 0, r.upperBound.length, - value, valueOffSet, valueLength) >= 0) || - (!r.isUpperBoundEqualTo && - Bytes.compareTo(r.upperBound, 0, r.upperBound.length, - value, valueOffSet, valueLength) > 0) - - val lowerBoundPass = r.lowerBound == null || r.lowerBound.length == 0 - (r.isLowerBoundEqualTo && - Bytes.compareTo(r.lowerBound, 0, r.lowerBound.length, - value, valueOffSet, valueLength) <= 0) || + points.foreach( + p => { + if (Bytes.equals(p, 0, p.length, value, valueOffSet, valueLength)) { + result = true + } + }) + + ranges.foreach( + r => { + val upperBoundPass = r.upperBound == null || + (r.isUpperBoundEqualTo && + Bytes.compareTo( + r.upperBound, + 0, + r.upperBound.length, + value, + valueOffSet, + valueLength) >= 0) || + (!r.isUpperBoundEqualTo && + Bytes + .compareTo(r.upperBound, 0, r.upperBound.length, value, valueOffSet, valueLength) > 0) + + val lowerBoundPass = r.lowerBound == null || r.lowerBound.length == 0 + (r.isLowerBoundEqualTo && + Bytes + .compareTo( + r.lowerBound, + 0, + r.lowerBound.length, + value, + valueOffSet, + valueLength) <= 0) || (!r.isLowerBoundEqualTo && - Bytes.compareTo(r.lowerBound, 0, r.lowerBound.length, - value, valueOffSet, valueLength) < 0) + Bytes + .compareTo(r.lowerBound, 0, r.lowerBound.length, value, valueOffSet, valueLength) < 0) - result = result || (upperBoundPass && lowerBoundPass) - }) + result = result || (upperBoundPass && lowerBoundPass) + }) result } @@ -1043,18 +1123,21 @@ class RowKeyFilter (currentPoint:Array[Byte] = null, * * @param other Filter to merge */ - def mergeUnion(other:RowKeyFilter): RowKeyFilter = { - other.points.foreach( p => points += p) - - other.ranges.foreach( otherR => { - var doesOverLap = false - ranges.foreach{ r => - if (r.getOverLapScanRange(otherR) != null) { - r.mergeUnion(otherR) - doesOverLap = true - }} - if (!doesOverLap) ranges.+=(otherR) - }) + def mergeUnion(other: RowKeyFilter): RowKeyFilter = { + other.points.foreach(p => points += p) + + other.ranges.foreach( + otherR => { + var doesOverLap = false + ranges.foreach { + r => + if (r.getOverLapScanRange(otherR) != null) { + r.mergeUnion(otherR) + doesOverLap = true + } + } + if (!doesOverLap) ranges.+=(otherR) + }) this } @@ -1064,76 +1147,86 @@ class RowKeyFilter (currentPoint:Array[Byte] = null, * * @param other Filter to merge */ - def mergeIntersect(other:RowKeyFilter): RowKeyFilter = { + def mergeIntersect(other: RowKeyFilter): RowKeyFilter = { val survivingPoints = new mutable.MutableList[Array[Byte]]() val didntSurviveFirstPassPoints = new mutable.MutableList[Array[Byte]]() if (points == null || points.length == 0) { - other.points.foreach( otherP => { - didntSurviveFirstPassPoints += otherP - }) + other.points.foreach( + otherP => { + didntSurviveFirstPassPoints += otherP + }) } else { - points.foreach(p => { - if (other.points.length == 0) { - didntSurviveFirstPassPoints += p - } else { - other.points.foreach(otherP => { - if (Bytes.equals(p, otherP)) { - survivingPoints += p - } else { - didntSurviveFirstPassPoints += p - } - }) - } - }) + points.foreach( + p => { + if (other.points.length == 0) { + didntSurviveFirstPassPoints += p + } else { + other.points.foreach( + otherP => { + if (Bytes.equals(p, otherP)) { + survivingPoints += p + } else { + didntSurviveFirstPassPoints += p + } + }) + } + }) } val survivingRanges = new mutable.MutableList[ScanRange]() if (ranges.length == 0) { - didntSurviveFirstPassPoints.foreach(p => { + didntSurviveFirstPassPoints.foreach( + p => { survivingPoints += p - }) - } else { - ranges.foreach(r => { - other.ranges.foreach(otherR => { - val overLapScanRange = r.getOverLapScanRange(otherR) - if (overLapScanRange != null) { - survivingRanges += overLapScanRange - } }) - didntSurviveFirstPassPoints.foreach(p => { - if (r.containsPoint(p)) { - survivingPoints += p - } + } else { + ranges.foreach( + r => { + other.ranges.foreach( + otherR => { + val overLapScanRange = r.getOverLapScanRange(otherR) + if (overLapScanRange != null) { + survivingRanges += overLapScanRange + } + }) + didntSurviveFirstPassPoints.foreach( + p => { + if (r.containsPoint(p)) { + survivingPoints += p + } + }) }) - }) } points = survivingPoints ranges = survivingRanges this } - override def toString:String = { + override def toString: String = { val strBuilder = new StringBuilder strBuilder.append("(points:(") var isFirst = true - points.foreach( p => { - if (isFirst) isFirst = false - else strBuilder.append(",") - strBuilder.append(Bytes.toString(p)) - }) + points.foreach( + p => { + if (isFirst) isFirst = false + else strBuilder.append(",") + strBuilder.append(Bytes.toString(p)) + }) strBuilder.append("),ranges:") isFirst = true - ranges.foreach( r => { - if (isFirst) isFirst = false - else strBuilder.append(",") - strBuilder.append(r) - }) + ranges.foreach( + r => { + if (isFirst) isFirst = false + else strBuilder.append(",") + strBuilder.append(r) + }) strBuilder.append("))") strBuilder.toString() } } @InterfaceAudience.Private -class ExecutionRuleForUnitTesting(val rowKeyFilter: RowKeyFilter, - val dynamicLogicExpression: DynamicLogicExpression) +class ExecutionRuleForUnitTesting( + val rowKeyFilter: RowKeyFilter, + val dynamicLogicExpression: DynamicLogicExpression) diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DynamicLogicExpression.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DynamicLogicExpression.scala index dab311e2..398093a5 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DynamicLogicExpression.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DynamicLogicExpression.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,15 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark import java.util - -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.spark.datasources.{BytesEncoder, JavaBytesEncoder} import org.apache.hadoop.hbase.spark.datasources.JavaBytesEncoder.JavaBytesEncoder import org.apache.hadoop.hbase.util.Bytes +import org.apache.yetus.audience.InterfaceAudience /** * Dynamic logic for SQL push down logic there is an instance for most @@ -31,12 +30,12 @@ import org.apache.hadoop.hbase.util.Bytes * Logic can be nested with And or Or operators. * * A logic tree can be written out as a string and reconstructed from that string - * */ @InterfaceAudience.Private trait DynamicLogicExpression { - def execute(columnToCurrentRowValueMap: util.HashMap[String, ByteArrayComparable], - valueFromQueryValueArray:Array[Array[Byte]]): Boolean + def execute( + columnToCurrentRowValueMap: util.HashMap[String, ByteArrayComparable], + valueFromQueryValueArray: Array[Array[Byte]]): Boolean def toExpressionString: String = { val strBuilder = new StringBuilder appendToExpression(strBuilder) @@ -44,7 +43,7 @@ trait DynamicLogicExpression { } def filterOps: JavaBytesEncoder = JavaBytesEncoder.Unknown - def appendToExpression(strBuilder:StringBuilder) + def appendToExpression(strBuilder: StringBuilder) var encoder: BytesEncoder = _ @@ -59,26 +58,33 @@ trait CompareTrait { self: DynamicLogicExpression => def columnName: String def valueFromQueryIndex: Int - def execute(columnToCurrentRowValueMap: - util.HashMap[String, ByteArrayComparable], - valueFromQueryValueArray:Array[Array[Byte]]): Boolean = { + def execute( + columnToCurrentRowValueMap: util.HashMap[String, ByteArrayComparable], + valueFromQueryValueArray: Array[Array[Byte]]): Boolean = { val currentRowValue = columnToCurrentRowValueMap.get(columnName) val valueFromQuery = valueFromQueryValueArray(valueFromQueryIndex) currentRowValue != null && - encoder.filter(currentRowValue.bytes, currentRowValue.offset, currentRowValue.length, - valueFromQuery, 0, valueFromQuery.length, filterOps) + encoder.filter( + currentRowValue.bytes, + currentRowValue.offset, + currentRowValue.length, + valueFromQuery, + 0, + valueFromQuery.length, + filterOps) } } @InterfaceAudience.Private -class AndLogicExpression (val leftExpression:DynamicLogicExpression, - val rightExpression:DynamicLogicExpression) - extends DynamicLogicExpression{ - override def execute(columnToCurrentRowValueMap: - util.HashMap[String, ByteArrayComparable], - valueFromQueryValueArray:Array[Array[Byte]]): Boolean = { +class AndLogicExpression( + val leftExpression: DynamicLogicExpression, + val rightExpression: DynamicLogicExpression) + extends DynamicLogicExpression { + override def execute( + columnToCurrentRowValueMap: util.HashMap[String, ByteArrayComparable], + valueFromQueryValueArray: Array[Array[Byte]]): Boolean = { leftExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray) && - rightExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray) + rightExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray) } override def appendToExpression(strBuilder: StringBuilder): Unit = { @@ -91,14 +97,15 @@ class AndLogicExpression (val leftExpression:DynamicLogicExpression, } @InterfaceAudience.Private -class OrLogicExpression (val leftExpression:DynamicLogicExpression, - val rightExpression:DynamicLogicExpression) - extends DynamicLogicExpression{ - override def execute(columnToCurrentRowValueMap: - util.HashMap[String, ByteArrayComparable], - valueFromQueryValueArray:Array[Array[Byte]]): Boolean = { +class OrLogicExpression( + val leftExpression: DynamicLogicExpression, + val rightExpression: DynamicLogicExpression) + extends DynamicLogicExpression { + override def execute( + columnToCurrentRowValueMap: util.HashMap[String, ByteArrayComparable], + valueFromQueryValueArray: Array[Array[Byte]]): Boolean = { leftExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray) || - rightExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray) + rightExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray) } override def appendToExpression(strBuilder: StringBuilder): Unit = { strBuilder.append("( ") @@ -110,19 +117,22 @@ class OrLogicExpression (val leftExpression:DynamicLogicExpression, } @InterfaceAudience.Private -class EqualLogicExpression (val columnName:String, - val valueFromQueryIndex:Int, - val isNot:Boolean) extends DynamicLogicExpression{ - override def execute(columnToCurrentRowValueMap: - util.HashMap[String, ByteArrayComparable], - valueFromQueryValueArray:Array[Array[Byte]]): Boolean = { +class EqualLogicExpression(val columnName: String, val valueFromQueryIndex: Int, val isNot: Boolean) + extends DynamicLogicExpression { + override def execute( + columnToCurrentRowValueMap: util.HashMap[String, ByteArrayComparable], + valueFromQueryValueArray: Array[Array[Byte]]): Boolean = { val currentRowValue = columnToCurrentRowValueMap.get(columnName) val valueFromQuery = valueFromQueryValueArray(valueFromQueryIndex) currentRowValue != null && - Bytes.equals(valueFromQuery, - 0, valueFromQuery.length, currentRowValue.bytes, - currentRowValue.offset, currentRowValue.length) != isNot + Bytes.equals( + valueFromQuery, + 0, + valueFromQuery.length, + currentRowValue.bytes, + currentRowValue.offset, + currentRowValue.length) != isNot } override def appendToExpression(strBuilder: StringBuilder): Unit = { val command = if (isNot) "!=" else "==" @@ -131,17 +141,22 @@ class EqualLogicExpression (val columnName:String, } @InterfaceAudience.Private -class StartsWithLogicExpression (val columnName:String, - val valueFromQueryIndex:Int) extends DynamicLogicExpression{ - override def execute(columnToCurrentRowValueMap: - util.HashMap[String, ByteArrayComparable], - valueFromQueryValueArray:Array[Array[Byte]]): Boolean = { +class StartsWithLogicExpression(val columnName: String, val valueFromQueryIndex: Int) + extends DynamicLogicExpression { + override def execute( + columnToCurrentRowValueMap: util.HashMap[String, ByteArrayComparable], + valueFromQueryValueArray: Array[Array[Byte]]): Boolean = { val currentRowValue = columnToCurrentRowValueMap.get(columnName) val valueFromQuery = valueFromQueryValueArray(valueFromQueryIndex) currentRowValue != null && valueFromQuery != null && currentRowValue.length >= valueFromQuery.length && - Bytes.equals(valueFromQuery,0, valueFromQuery.length, currentRowValue.bytes, - currentRowValue.offset, valueFromQuery.length) + Bytes.equals( + valueFromQuery, + 0, + valueFromQuery.length, + currentRowValue.bytes, + currentRowValue.offset, + valueFromQuery.length) } override def appendToExpression(strBuilder: StringBuilder): Unit = { strBuilder.append(columnName + " startsWith " + valueFromQueryIndex) @@ -149,11 +164,11 @@ class StartsWithLogicExpression (val columnName:String, } @InterfaceAudience.Private -class IsNullLogicExpression (val columnName:String, - val isNot:Boolean) extends DynamicLogicExpression{ - override def execute(columnToCurrentRowValueMap: - util.HashMap[String, ByteArrayComparable], - valueFromQueryValueArray:Array[Array[Byte]]): Boolean = { +class IsNullLogicExpression(val columnName: String, val isNot: Boolean) + extends DynamicLogicExpression { + override def execute( + columnToCurrentRowValueMap: util.HashMap[String, ByteArrayComparable], + valueFromQueryValueArray: Array[Array[Byte]]): Boolean = { val currentRowValue = columnToCurrentRowValueMap.get(columnName) (currentRowValue == null) != isNot @@ -165,9 +180,11 @@ class IsNullLogicExpression (val columnName:String, } @InterfaceAudience.Private -class GreaterThanLogicExpression (override val columnName:String, - override val valueFromQueryIndex:Int) - extends DynamicLogicExpression with CompareTrait{ +class GreaterThanLogicExpression( + override val columnName: String, + override val valueFromQueryIndex: Int) + extends DynamicLogicExpression + with CompareTrait { override val filterOps = JavaBytesEncoder.Greater override def appendToExpression(strBuilder: StringBuilder): Unit = { strBuilder.append(columnName + " > " + valueFromQueryIndex) @@ -175,9 +192,11 @@ class GreaterThanLogicExpression (override val columnName:String, } @InterfaceAudience.Private -class GreaterThanOrEqualLogicExpression (override val columnName:String, - override val valueFromQueryIndex:Int) - extends DynamicLogicExpression with CompareTrait{ +class GreaterThanOrEqualLogicExpression( + override val columnName: String, + override val valueFromQueryIndex: Int) + extends DynamicLogicExpression + with CompareTrait { override val filterOps = JavaBytesEncoder.GreaterEqual override def appendToExpression(strBuilder: StringBuilder): Unit = { strBuilder.append(columnName + " >= " + valueFromQueryIndex) @@ -185,9 +204,11 @@ class GreaterThanOrEqualLogicExpression (override val columnName:String, } @InterfaceAudience.Private -class LessThanLogicExpression (override val columnName:String, - override val valueFromQueryIndex:Int) - extends DynamicLogicExpression with CompareTrait { +class LessThanLogicExpression( + override val columnName: String, + override val valueFromQueryIndex: Int) + extends DynamicLogicExpression + with CompareTrait { override val filterOps = JavaBytesEncoder.Less override def appendToExpression(strBuilder: StringBuilder): Unit = { strBuilder.append(columnName + " < " + valueFromQueryIndex) @@ -195,9 +216,9 @@ class LessThanLogicExpression (override val columnName:String, } @InterfaceAudience.Private -class LessThanOrEqualLogicExpression (val columnName:String, - val valueFromQueryIndex:Int) - extends DynamicLogicExpression with CompareTrait{ +class LessThanOrEqualLogicExpression(val columnName: String, val valueFromQueryIndex: Int) + extends DynamicLogicExpression + with CompareTrait { override val filterOps = JavaBytesEncoder.LessEqual override def appendToExpression(strBuilder: StringBuilder): Unit = { strBuilder.append(columnName + " <= " + valueFromQueryIndex) @@ -206,9 +227,9 @@ class LessThanOrEqualLogicExpression (val columnName:String, @InterfaceAudience.Private class PassThroughLogicExpression() extends DynamicLogicExpression { - override def execute(columnToCurrentRowValueMap: - util.HashMap[String, ByteArrayComparable], - valueFromQueryValueArray: Array[Array[Byte]]): Boolean = true + override def execute( + columnToCurrentRowValueMap: util.HashMap[String, ByteArrayComparable], + valueFromQueryValueArray: Array[Array[Byte]]): Boolean = true override def appendToExpression(strBuilder: StringBuilder): Unit = { // Fix the offset bug by add dummy to avoid crash the region server. @@ -227,8 +248,10 @@ object DynamicLogicExpressionBuilder { expressionAndOffset._1 } - private def build(expressionArray:Array[String], - offSet:Int, encoder: BytesEncoder): (DynamicLogicExpression, Int) = { + private def build( + expressionArray: Array[String], + offSet: Int, + encoder: BytesEncoder): (DynamicLogicExpression, Int) = { val expr = { if (expressionArray(offSet).equals("(")) { val left = build(expressionArray, offSet + 1, encoder) @@ -243,26 +266,47 @@ object DynamicLogicExpressionBuilder { } else { val command = expressionArray(offSet + 1) if (command.equals("<")) { - (new LessThanLogicExpression(expressionArray(offSet), - expressionArray(offSet + 2).toInt), offSet + 3) + ( + new LessThanLogicExpression(expressionArray(offSet), expressionArray(offSet + 2).toInt), + offSet + 3) } else if (command.equals("<=")) { - (new LessThanOrEqualLogicExpression(expressionArray(offSet), - expressionArray(offSet + 2).toInt), offSet + 3) + ( + new LessThanOrEqualLogicExpression( + expressionArray(offSet), + expressionArray(offSet + 2).toInt), + offSet + 3) } else if (command.equals(">")) { - (new GreaterThanLogicExpression(expressionArray(offSet), - expressionArray(offSet + 2).toInt), offSet + 3) + ( + new GreaterThanLogicExpression( + expressionArray(offSet), + expressionArray(offSet + 2).toInt), + offSet + 3) } else if (command.equals(">=")) { - (new GreaterThanOrEqualLogicExpression(expressionArray(offSet), - expressionArray(offSet + 2).toInt), offSet + 3) + ( + new GreaterThanOrEqualLogicExpression( + expressionArray(offSet), + expressionArray(offSet + 2).toInt), + offSet + 3) } else if (command.equals("==")) { - (new EqualLogicExpression(expressionArray(offSet), - expressionArray(offSet + 2).toInt, false), offSet + 3) + ( + new EqualLogicExpression( + expressionArray(offSet), + expressionArray(offSet + 2).toInt, + false), + offSet + 3) } else if (command.equals("!=")) { - (new EqualLogicExpression(expressionArray(offSet), - expressionArray(offSet + 2).toInt, true), offSet + 3) + ( + new EqualLogicExpression( + expressionArray(offSet), + expressionArray(offSet + 2).toInt, + true), + offSet + 3) } else if (command.equals("startsWith")) { - (new StartsWithLogicExpression(expressionArray(offSet), - expressionArray(offSet + 2).toInt), offSet + 3) + ( + new StartsWithLogicExpression( + expressionArray(offSet), + expressionArray(offSet + 2).toInt), + offSet + 3) } else if (command.equals("isNull")) { (new IsNullLogicExpression(expressionArray(offSet), false), offSet + 2) } else if (command.equals("isNotNull")) { diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/FamiliesQualifiersValues.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/FamiliesQualifiersValues.scala index 7a651e1f..93410a42 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/FamiliesQualifiersValues.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/FamiliesQualifiersValues.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -17,7 +18,6 @@ package org.apache.hadoop.hbase.spark import java.util - import org.apache.yetus.audience.InterfaceAudience; /** @@ -26,14 +26,13 @@ import org.apache.yetus.audience.InterfaceAudience; */ @InterfaceAudience.Public class FamiliesQualifiersValues extends Serializable { - //Tree maps are used because we need the results to + // Tree maps are used because we need the results to // be sorted when we read them - val familyMap = new util.TreeMap[ByteArrayWrapper, - util.TreeMap[ByteArrayWrapper, Array[Byte]]]() + val familyMap = new util.TreeMap[ByteArrayWrapper, util.TreeMap[ByteArrayWrapper, Array[Byte]]]() - //normally in a row there are more columns then - //column families this wrapper is reused for column - //family look ups + // normally in a row there are more columns then + // column families this wrapper is reused for column + // family look ups val reusableWrapper = new ByteArrayWrapper(null) /** @@ -42,7 +41,7 @@ class FamiliesQualifiersValues extends Serializable { * @param qualifier HBase column qualifier * @param value HBase cell value */ - def += (family: Array[Byte], qualifier: Array[Byte], value: Array[Byte]): Unit = { + def +=(family: Array[Byte], qualifier: Array[Byte], value: Array[Byte]): Unit = { reusableWrapper.value = family @@ -57,11 +56,11 @@ class FamiliesQualifiersValues extends Serializable { } /** - * A wrapper for "+=" method above, can be used by Java - * @param family HBase column family - * @param qualifier HBase column qualifier - * @param value HBase cell value - */ + * A wrapper for "+=" method above, can be used by Java + * @param family HBase column family + * @param qualifier HBase column qualifier + * @param value HBase cell value + */ def add(family: Array[Byte], qualifier: Array[Byte], value: Array[Byte]): Unit = { this += (family, qualifier, value) } diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/FamilyHFileWriteOptions.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/FamilyHFileWriteOptions.scala index 9ee9291f..05117c51 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/FamilyHFileWriteOptions.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/FamilyHFileWriteOptions.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,11 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark import java.io.Serializable - import org.apache.yetus.audience.InterfaceAudience; /** @@ -32,7 +31,9 @@ import org.apache.yetus.audience.InterfaceAudience; * in the HFile */ @InterfaceAudience.Public -class FamilyHFileWriteOptions( val compression:String, - val bloomType: String, - val blockSize: Int, - val dataBlockEncoding: String) extends Serializable +class FamilyHFileWriteOptions( + val compression: String, + val bloomType: String, + val blockSize: Int, + val dataBlockEncoding: String) + extends Serializable diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseConnectionCache.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseConnectionCache.scala index 138c2247..812975f3 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseConnectionCache.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseConnectionCache.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,11 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark import java.io.IOException import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hbase.HConstants +import org.apache.hadoop.hbase.TableName import org.apache.hadoop.hbase.client.Admin import org.apache.hadoop.hbase.client.Connection import org.apache.hadoop.hbase.client.ConnectionFactory @@ -28,8 +30,6 @@ import org.apache.hadoop.hbase.ipc.RpcControllerFactory import org.apache.hadoop.hbase.security.User import org.apache.hadoop.hbase.security.UserProvider import org.apache.hadoop.hbase.spark.datasources.HBaseSparkConf -import org.apache.hadoop.hbase.HConstants -import org.apache.hadoop.hbase.TableName import org.apache.yetus.audience.InterfaceAudience import scala.collection.mutable @@ -53,8 +53,8 @@ private[spark] object HBaseConnectionCache extends Logging { Thread.sleep(timeout) } catch { case e: InterruptedException => - // setTimeout() and close() may interrupt the sleep and it's safe - // to ignore the exception + // setTimeout() and close() may interrupt the sleep and it's safe + // to ignore the exception } if (closed) return @@ -91,20 +91,21 @@ private[spark] object HBaseConnectionCache extends Logging { val tsNow: Long = System.currentTimeMillis() connectionMap.synchronized { connectionMap.foreach { - x => { - if(x._2.refCount < 0) { - logError(s"Bug to be fixed: negative refCount of connection ${x._2}") - } + x => + { + if (x._2.refCount < 0) { + logError(s"Bug to be fixed: negative refCount of connection ${x._2}") + } - if(forceClean || ((x._2.refCount <= 0) && (tsNow - x._2.timestamp > timeout))) { - try{ - x._2.connection.close() - } catch { - case e: IOException => logWarning(s"Fail to close connection ${x._2}", e) + if (forceClean || ((x._2.refCount <= 0) && (tsNow - x._2.timestamp > timeout))) { + try { + x._2.connection.close() + } catch { + case e: IOException => logWarning(s"Fail to close connection ${x._2}", e) + } + connectionMap.remove(x._1) } - connectionMap.remove(x._1) } - } } } } @@ -115,8 +116,11 @@ private[spark] object HBaseConnectionCache extends Logging { if (closed) return null cacheStat.numTotalRequests += 1 - val sc = connectionMap.getOrElseUpdate(key, {cacheStat.numActualConnectionsCreated += 1 - new SmartConnection(conn)}) + val sc = connectionMap.getOrElseUpdate( + key, { + cacheStat.numActualConnectionsCreated += 1 + new SmartConnection(conn) + }) sc.refCount += 1 sc } @@ -126,7 +130,7 @@ private[spark] object HBaseConnectionCache extends Logging { getConnection(new HBaseConnectionKey(conf), ConnectionFactory.createConnection(conf)) // For testing purpose only - def setTimeout(to: Long): Unit = { + def setTimeout(to: Long): Unit = { connectionMap.synchronized { if (closed) return @@ -137,8 +141,10 @@ private[spark] object HBaseConnectionCache extends Logging { } @InterfaceAudience.Private -private[hbase] case class SmartConnection ( - connection: Connection, var refCount: Int = 0, var timestamp: Long = 0) { +private[hbase] case class SmartConnection( + connection: Connection, + var refCount: Int = 0, + var timestamp: Long = 0) { def getTable(tableName: TableName): Table = connection.getTable(tableName) def getRegionLocator(tableName: TableName): RegionLocator = connection.getRegionLocator(tableName) def isClosed: Boolean = connection.isClosed @@ -146,7 +152,7 @@ private[hbase] case class SmartConnection ( def close() = { HBaseConnectionCache.connectionMap.synchronized { refCount -= 1 - if(refCount <= 0) + if (refCount <= 0) timestamp = System.currentTimeMillis() } } @@ -158,7 +164,6 @@ private[hbase] case class SmartConnection ( * * In essence, this class captures the properties in Configuration * that may be used in the process of establishing a connection. - * */ @InterfaceAudience.Private class HBaseConnectionKey(c: Configuration) extends Logging { @@ -191,8 +196,7 @@ class HBaseConnectionKey(c: Configuration) extends Logging { if (currentUser != null) { username = currentUser.getName } - } - catch { + } catch { case e: IOException => { logWarning("Error obtaining current user, skipping username in HBaseConnectionKey", e) } @@ -223,16 +227,14 @@ class HBaseConnectionKey(c: Configuration) extends Logging { val that: HBaseConnectionKey = obj.asInstanceOf[HBaseConnectionKey] if (this.username != null && !(this.username == that.username)) { return false - } - else if (this.username == null && that.username != null) { + } else if (this.username == null && that.username != null) { return false } if (this.properties == null) { if (that.properties != null) { return false } - } - else { + } else { if (that.properties == null) { return false } @@ -242,7 +244,7 @@ class HBaseConnectionKey(c: Configuration) extends Logging { val thatValue: Option[String] = that.properties.get(property) flag = true if (thisValue eq thatValue) { - flag = false //continue, so make flag be false + flag = false // continue, so make flag be false } if (flag && (thisValue == null || !(thisValue == thatValue))) { return false @@ -265,6 +267,7 @@ class HBaseConnectionKey(c: Configuration) extends Logging { * @param numActiveConnections number of current alive HBase connections the cache is holding */ @InterfaceAudience.Private -case class HBaseConnectionCacheStat(var numTotalRequests: Long, - var numActualConnectionsCreated: Long, - var numActiveConnections: Long) +case class HBaseConnectionCacheStat( + var numTotalRequests: Long, + var numActualConnectionsCreated: Long, + var numActiveConnections: Long) diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala index 091e78eb..c055bbc4 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,57 +15,56 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark +import java.io._ import java.net.InetSocketAddress import java.util import java.util.UUID import javax.management.openmbean.KeyAlreadyExistsException - -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.fs.HFileSystem +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileAlreadyExistsException, FileSystem, Path} import org.apache.hadoop.hbase._ +import org.apache.hadoop.hbase.client._ +import org.apache.hadoop.hbase.fs.HFileSystem +import org.apache.hadoop.hbase.io.ImmutableBytesWritable import org.apache.hadoop.hbase.io.compress.Compression import org.apache.hadoop.hbase.io.compress.Compression.Algorithm import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding -import org.apache.hadoop.hbase.io.hfile.{HFile, CacheConfig, HFileContextBuilder, HFileWriterImpl} -import org.apache.hadoop.hbase.regionserver.{HStoreFile, StoreFileWriter, StoreUtils, BloomType} +import org.apache.hadoop.hbase.io.hfile.{CacheConfig, HFile, HFileContextBuilder, HFileWriterImpl} +import org.apache.hadoop.hbase.mapreduce.{IdentityTableMapper, TableInputFormat, TableMapReduceUtil} +import org.apache.hadoop.hbase.regionserver.{BloomType, HStoreFile, StoreFileWriter, StoreUtils} +import org.apache.hadoop.hbase.spark.HBaseRDDFunctions._ import org.apache.hadoop.hbase.util.{Bytes, ChecksumType} import org.apache.hadoop.mapred.JobConf -import org.apache.spark.broadcast.Broadcast -import org.apache.spark.rdd.RDD -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.hbase.spark.HBaseRDDFunctions._ -import org.apache.hadoop.hbase.client._ -import scala.reflect.ClassTag -import org.apache.spark.{SerializableWritable, SparkContext} -import org.apache.hadoop.hbase.mapreduce.{TableMapReduceUtil, -TableInputFormat, IdentityTableMapper} -import org.apache.hadoop.hbase.io.ImmutableBytesWritable import org.apache.hadoop.mapreduce.Job -import org.apache.spark.streaming.dstream.DStream -import java.io._ import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod -import org.apache.hadoop.fs.{Path, FileAlreadyExistsException, FileSystem} +import org.apache.spark.{SerializableWritable, SparkContext} +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.dstream.DStream +import org.apache.yetus.audience.InterfaceAudience import scala.collection.mutable +import scala.reflect.ClassTag /** - * HBaseContext is a façade for HBase operations - * like bulk put, get, increment, delete, and scan - * - * HBaseContext will take the responsibilities - * of disseminating the configuration information - * to the working and managing the life cycle of Connections. + * HBaseContext is a façade for HBase operations + * like bulk put, get, increment, delete, and scan + * + * HBaseContext will take the responsibilities + * of disseminating the configuration information + * to the working and managing the life cycle of Connections. */ @InterfaceAudience.Public -class HBaseContext(@transient val sc: SparkContext, - @transient val config: Configuration, - val tmpHdfsConfgFile: String = null) - extends Serializable with Logging { - - @transient var tmpHdfsConfiguration:Configuration = config +class HBaseContext( + @transient val sc: SparkContext, + @transient val config: Configuration, + val tmpHdfsConfgFile: String = null) + extends Serializable + with Logging { + + @transient var tmpHdfsConfiguration: Configuration = config @transient var appliedCredentials = false @transient val job = Job.getInstance(config) TableMapReduceUtil.initCredentials(job) @@ -97,10 +97,8 @@ class HBaseContext(@transient val sc: SparkContext, * the RDD values and a Connection object to interact * with HBase */ - def foreachPartition[T](rdd: RDD[T], - f: (Iterator[T], Connection) => Unit):Unit = { - rdd.foreachPartition( - it => hbaseForeachPartition(broadcastedConf, it, f)) + def foreachPartition[T](rdd: RDD[T], f: (Iterator[T], Connection) => Unit): Unit = { + rdd.foreachPartition(it => hbaseForeachPartition(broadcastedConf, it, f)) } /** @@ -116,11 +114,11 @@ class HBaseContext(@transient val sc: SparkContext, * the DStream values and a Connection object to * interact with HBase */ - def foreachPartition[T](dstream: DStream[T], - f: (Iterator[T], Connection) => Unit):Unit = { - dstream.foreachRDD((rdd, time) => { - foreachPartition(rdd, f) - }) + def foreachPartition[T](dstream: DStream[T], f: (Iterator[T], Connection) => Unit): Unit = { + dstream.foreachRDD( + (rdd, time) => { + foreachPartition(rdd, f) + }) } /** @@ -138,12 +136,11 @@ class HBaseContext(@transient val sc: SparkContext, * @return Returns a new RDD generated by the user definition * function just like normal mapPartition */ - def mapPartitions[T, R: ClassTag](rdd: RDD[T], - mp: (Iterator[T], Connection) => Iterator[R]): RDD[R] = { + def mapPartitions[T, R: ClassTag]( + rdd: RDD[T], + mp: (Iterator[T], Connection) => Iterator[R]): RDD[R] = { - rdd.mapPartitions[R](it => hbaseMapPartition[T, R](broadcastedConf, - it, - mp)) + rdd.mapPartitions[R](it => hbaseMapPartition[T, R](broadcastedConf, it, mp)) } @@ -167,8 +164,7 @@ class HBaseContext(@transient val sc: SparkContext, * @return Returns a new DStream generated by the user * definition function just like normal mapPartition */ - def streamForeachPartition[T](dstream: DStream[T], - f: (Iterator[T], Connection) => Unit): Unit = { + def streamForeachPartition[T](dstream: DStream[T], f: (Iterator[T], Connection) => Unit): Unit = { dstream.foreachRDD(rdd => this.foreachPartition(rdd, f)) } @@ -193,13 +189,10 @@ class HBaseContext(@transient val sc: SparkContext, * @return Returns a new DStream generated by the user * definition function just like normal mapPartition */ - def streamMapPartitions[T, U: ClassTag](dstream: DStream[T], - f: (Iterator[T], Connection) => Iterator[U]): - DStream[U] = { - dstream.mapPartitions(it => hbaseMapPartition[T, U]( - broadcastedConf, - it, - f)) + def streamMapPartitions[T, U: ClassTag]( + dstream: DStream[T], + f: (Iterator[T], Connection) => Iterator[U]): DStream[U] = { + dstream.mapPartitions(it => hbaseMapPartition[T, U](broadcastedConf, it, f)) } /** @@ -218,18 +211,19 @@ class HBaseContext(@transient val sc: SparkContext, val tName = tableName.getName rdd.foreachPartition( - it => hbaseForeachPartition[T]( - broadcastedConf, - it, - (iterator, connection) => { - val m = connection.getBufferedMutator(TableName.valueOf(tName)) - iterator.foreach(T => m.mutate(f(T))) - m.flush() - m.close() - })) + it => + hbaseForeachPartition[T]( + broadcastedConf, + it, + (iterator, connection) => { + val m = connection.getBufferedMutator(TableName.valueOf(tName)) + iterator.foreach(T => m.mutate(f(T))) + m.flush() + m.close() + })) } - def applyCreds[T] (){ + def applyCreds[T]() { if (!appliedCredentials) { appliedCredentials = true @@ -253,13 +247,12 @@ class HBaseContext(@transient val sc: SparkContext, * @param f Function to convert a value in * the DStream to a HBase Put */ - def streamBulkPut[T](dstream: DStream[T], - tableName: TableName, - f: (T) => Put) = { + def streamBulkPut[T](dstream: DStream[T], tableName: TableName, f: (T) => Put) = { val tName = tableName.getName - dstream.foreachRDD((rdd, time) => { - bulkPut(rdd, TableName.valueOf(tName), f) - }) + dstream.foreachRDD( + (rdd, time) => { + bulkPut(rdd, TableName.valueOf(tName), f) + }) } /** @@ -275,8 +268,7 @@ class HBaseContext(@transient val sc: SparkContext, * HBase Deletes * @param batchSize The number of delete to batch before sending to HBase */ - def bulkDelete[T](rdd: RDD[T], tableName: TableName, - f: (T) => Delete, batchSize: Integer) { + def bulkDelete[T](rdd: RDD[T], tableName: TableName, f: (T) => Delete, batchSize: Integer) { bulkMutation(rdd, tableName, f, batchSize) } @@ -295,10 +287,11 @@ class HBaseContext(@transient val sc: SparkContext, * HBase Delete * @param batchSize The number of deletes to batch before sending to HBase */ - def streamBulkDelete[T](dstream: DStream[T], - tableName: TableName, - f: (T) => Delete, - batchSize: Integer) = { + def streamBulkDelete[T]( + dstream: DStream[T], + tableName: TableName, + f: (T) => Delete, + batchSize: Integer) = { streamBulkMutation(dstream, tableName, f, batchSize) } @@ -307,30 +300,35 @@ class HBaseContext(@transient val sc: SparkContext, * * May be opened up if requested */ - private def bulkMutation[T](rdd: RDD[T], tableName: TableName, - f: (T) => Mutation, batchSize: Integer) { + private def bulkMutation[T]( + rdd: RDD[T], + tableName: TableName, + f: (T) => Mutation, + batchSize: Integer) { val tName = tableName.getName rdd.foreachPartition( - it => hbaseForeachPartition[T]( - broadcastedConf, - it, - (iterator, connection) => { - val table = connection.getTable(TableName.valueOf(tName)) - val mutationList = new java.util.ArrayList[Mutation] - iterator.foreach(T => { - mutationList.add(f(T)) - if (mutationList.size >= batchSize) { + it => + hbaseForeachPartition[T]( + broadcastedConf, + it, + (iterator, connection) => { + val table = connection.getTable(TableName.valueOf(tName)) + val mutationList = new java.util.ArrayList[Mutation] + iterator.foreach( + T => { + mutationList.add(f(T)) + if (mutationList.size >= batchSize) { + table.batch(mutationList, null) + mutationList.clear() + } + }) + if (mutationList.size() > 0) { table.batch(mutationList, null) mutationList.clear() } - }) - if (mutationList.size() > 0) { - table.batch(mutationList, null) - mutationList.clear() - } - table.close() - })) + table.close() + })) } /** @@ -338,14 +336,16 @@ class HBaseContext(@transient val sc: SparkContext, * * May be opened up if requested */ - private def streamBulkMutation[T](dstream: DStream[T], - tableName: TableName, - f: (T) => Mutation, - batchSize: Integer) = { + private def streamBulkMutation[T]( + dstream: DStream[T], + tableName: TableName, + f: (T) => Mutation, + batchSize: Integer) = { val tName = tableName.getName - dstream.foreachRDD((rdd, time) => { - bulkMutation(rdd, TableName.valueOf(tName), f, batchSize) - }) + dstream.foreachRDD( + (rdd, time) => { + bulkMutation(rdd, TableName.valueOf(tName), f, batchSize) + }) } /** @@ -363,22 +363,16 @@ class HBaseContext(@transient val sc: SparkContext, * RDD * return new RDD that is created by the Get to HBase */ - def bulkGet[T, U: ClassTag](tableName: TableName, - batchSize: Integer, - rdd: RDD[T], - makeGet: (T) => Get, - convertResult: (Result) => U): RDD[U] = { - - val getMapPartition = new GetMapPartition(tableName, - batchSize, - makeGet, - convertResult) - - rdd.mapPartitions[U](it => - hbaseMapPartition[T, U]( - broadcastedConf, - it, - getMapPartition.run)) + def bulkGet[T, U: ClassTag]( + tableName: TableName, + batchSize: Integer, + rdd: RDD[T], + makeGet: (T) => Get, + convertResult: (Result) => U): RDD[U] = { + + val getMapPartition = new GetMapPartition(tableName, batchSize, makeGet, convertResult) + + rdd.mapPartitions[U](it => hbaseMapPartition[T, U](broadcastedConf, it, getMapPartition.run)) } /** @@ -398,21 +392,17 @@ class HBaseContext(@transient val sc: SparkContext, * DStream * @return A new DStream that is created by the Get to HBase */ - def streamBulkGet[T, U: ClassTag](tableName: TableName, - batchSize: Integer, - dStream: DStream[T], - makeGet: (T) => Get, - convertResult: (Result) => U): DStream[U] = { - - val getMapPartition = new GetMapPartition(tableName, - batchSize, - makeGet, - convertResult) - - dStream.mapPartitions[U](it => hbaseMapPartition[T, U]( - broadcastedConf, - it, - getMapPartition.run)) + def streamBulkGet[T, U: ClassTag]( + tableName: TableName, + batchSize: Integer, + dStream: DStream[T], + makeGet: (T) => Get, + convertResult: (Result) => U): DStream[U] = { + + val getMapPartition = new GetMapPartition(tableName, batchSize, makeGet, convertResult) + + dStream.mapPartitions[U]( + it => hbaseMapPartition[T, U](broadcastedConf, it, getMapPartition.run)) } /** @@ -425,21 +415,29 @@ class HBaseContext(@transient val sc: SparkContext, * what the user wants in the final generated RDD * @return new RDD with results from scan */ - def hbaseRDD[U: ClassTag](tableName: TableName, scan: Scan, - f: ((ImmutableBytesWritable, Result)) => U): RDD[U] = { + def hbaseRDD[U: ClassTag]( + tableName: TableName, + scan: Scan, + f: ((ImmutableBytesWritable, Result)) => U): RDD[U] = { val job: Job = Job.getInstance(getConf(broadcastedConf)) TableMapReduceUtil.initCredentials(job) - TableMapReduceUtil.initTableMapperJob(tableName, scan, - classOf[IdentityTableMapper], null, null, job) + TableMapReduceUtil.initTableMapperJob( + tableName, + scan, + classOf[IdentityTableMapper], + null, + null, + job) val jconf = new JobConf(job.getConfiguration) val jobCreds = jconf.getCredentials() UserGroupInformation.setConfiguration(sc.hadoopConfiguration) jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) - new NewHBaseRDD(sc, + new NewHBaseRDD( + sc, classOf[TableInputFormat], classOf[ImmutableBytesWritable], classOf[Result], @@ -454,10 +452,8 @@ class HBaseContext(@transient val sc: SparkContext, * @param tableName the name of the table to scan * @param scans the HBase scan object to use to read data from HBase * @return New RDD with results from scan - * */ - def hbaseRDD(tableName: TableName, scans: Scan): - RDD[(ImmutableBytesWritable, Result)] = { + def hbaseRDD(tableName: TableName, scans: Scan): RDD[(ImmutableBytesWritable, Result)] = { hbaseRDD[(ImmutableBytesWritable, Result)]( tableName, @@ -468,10 +464,10 @@ class HBaseContext(@transient val sc: SparkContext, /** * underlining wrapper all foreach functions in HBaseContext */ - private def hbaseForeachPartition[T](configBroadcast: - Broadcast[SerializableWritable[Configuration]], - it: Iterator[T], - f: (Iterator[T], Connection) => Unit) = { + private def hbaseForeachPartition[T]( + configBroadcast: Broadcast[SerializableWritable[Configuration]], + it: Iterator[T], + f: (Iterator[T], Connection) => Unit) = { val config = getConf(configBroadcast) @@ -485,8 +481,8 @@ class HBaseContext(@transient val sc: SparkContext, } } - private def getConf(configBroadcast: Broadcast[SerializableWritable[Configuration]]): - Configuration = { + private def getConf( + configBroadcast: Broadcast[SerializableWritable[Configuration]]): Configuration = { if (tmpHdfsConfiguration == null && tmpHdfsConfgFile != null) { val fs = FileSystem.newInstance(sc.hadoopConfiguration) @@ -508,14 +504,11 @@ class HBaseContext(@transient val sc: SparkContext, /** * underlining wrapper all mapPartition functions in HBaseContext - * */ private def hbaseMapPartition[K, U]( - configBroadcast: - Broadcast[SerializableWritable[Configuration]], - it: Iterator[K], - mp: (Iterator[K], Connection) => - Iterator[U]): Iterator[U] = { + configBroadcast: Broadcast[SerializableWritable[Configuration]], + it: Iterator[K], + mp: (Iterator[K], Connection) => Iterator[U]): Iterator[U] = { val config = getConf(configBroadcast) applyCreds @@ -531,11 +524,12 @@ class HBaseContext(@transient val sc: SparkContext, /** * underlining wrapper all get mapPartition functions in HBaseContext */ - private class GetMapPartition[T, U](tableName: TableName, - batchSize: Integer, - makeGet: (T) => Get, - convertResult: (Result) => U) - extends Serializable { + private class GetMapPartition[T, U]( + tableName: TableName, + batchSize: Integer, + makeGet: (T) => Get, + convertResult: (Result) => U) + extends Serializable { val tName = tableName.getName @@ -577,8 +571,7 @@ class HBaseContext(@transient val sc: SparkContext, * For instance, an Array of AnyRef can hold any type T, but may lose primitive * specialization. */ - private[spark] - def fakeClassTag[T]: ClassTag[T] = ClassTag.AnyRef.asInstanceOf[ClassTag[T]] + private[spark] def fakeClassTag[T]: ClassTag[T] = ClassTag.AnyRef.asInstanceOf[ClassTag[T]] /** * Spark Implementation of HBase Bulk load for wide rows or when @@ -609,17 +602,16 @@ class HBaseContext(@transient val sc: SparkContext, * @param nowTimeStamp Version timestamp * @tparam T The Type of values in the original RDD */ - def bulkLoad[T](rdd:RDD[T], - tableName: TableName, - flatMap: (T) => Iterator[(KeyFamilyQualifier, Array[Byte])], - stagingDir:String, - familyHFileWriteOptionsMap: - util.Map[Array[Byte], FamilyHFileWriteOptions] = - new util.HashMap[Array[Byte], FamilyHFileWriteOptions], - compactionExclude: Boolean = false, - maxSize:Long = HConstants.DEFAULT_MAX_FILE_SIZE, - nowTimeStamp:Long = System.currentTimeMillis()): - Unit = { + def bulkLoad[T]( + rdd: RDD[T], + tableName: TableName, + flatMap: (T) => Iterator[(KeyFamilyQualifier, Array[Byte])], + stagingDir: String, + familyHFileWriteOptionsMap: util.Map[Array[Byte], FamilyHFileWriteOptions] = + new util.HashMap[Array[Byte], FamilyHFileWriteOptions], + compactionExclude: Boolean = false, + maxSize: Long = HConstants.DEFAULT_MAX_FILE_SIZE, + nowTimeStamp: Long = System.currentTimeMillis()): Unit = { val stagingPath = new Path(stagingDir) val fs = stagingPath.getFileSystem(config) if (fs.exists(stagingPath)) { @@ -632,8 +624,8 @@ class HBaseContext(@transient val sc: SparkContext, if (startKeys.length == 0) { logInfo("Table " + tableName.toString + " was not found") } - val defaultCompressionStr = config.get("hfile.compression", - Compression.Algorithm.NONE.getName) + val defaultCompressionStr = + config.get("hfile.compression", Compression.Algorithm.NONE.getName) val hfileCompression = HFileWriterImpl .compressionByName(defaultCompressionStr) val tableRawName = tableName.getName @@ -651,63 +643,62 @@ class HBaseContext(@transient val sc: SparkContext, val regionSplitPartitioner = new BulkLoadPartitioner(startKeys) - //This is where all the magic happens - //Here we are going to do the following things + // This is where all the magic happens + // Here we are going to do the following things // 1. FlapMap every row in the RDD into key column value tuples // 2. Then we are going to repartition sort and shuffle // 3. Finally we are going to write out our HFiles - rdd.flatMap( r => flatMap(r)). - repartitionAndSortWithinPartitions(regionSplitPartitioner). - hbaseForeachPartition(this, (it, conn) => { - - val conf = broadcastedConf.value.value - val fs = new Path(stagingDir).getFileSystem(conf) - val writerMap = new mutable.HashMap[ByteArrayWrapper, WriterLength] - var previousRow:Array[Byte] = HConstants.EMPTY_BYTE_ARRAY - var rollOverRequested = false - val localTableName = TableName.valueOf(tableRawName) - - //Here is where we finally iterate through the data in this partition of the - //RDD that has been sorted and partitioned - it.foreach{ case (keyFamilyQualifier, cellValue:Array[Byte]) => - - val wl = writeValueToHFile(keyFamilyQualifier.rowKey, - keyFamilyQualifier.family, - keyFamilyQualifier.qualifier, - cellValue, - nowTimeStamp, - fs, - conn, - localTableName, - conf, - familyHFileWriteOptionsMapInternal, - hfileCompression, - writerMap, - stagingDir) - - rollOverRequested = rollOverRequested || wl.written > maxSize - - //This will only roll if we have at least one column family file that is - //bigger then maxSize and we have finished a given row key - if (rollOverRequested && Bytes.compareTo(previousRow, keyFamilyQualifier.rowKey) != 0) { - rollWriters(fs, writerMap, - regionSplitPartitioner, - previousRow, - compactionExclude) - rollOverRequested = false - } + rdd + .flatMap(r => flatMap(r)) + .repartitionAndSortWithinPartitions(regionSplitPartitioner) + .hbaseForeachPartition( + this, + (it, conn) => { + + val conf = broadcastedConf.value.value + val fs = new Path(stagingDir).getFileSystem(conf) + val writerMap = new mutable.HashMap[ByteArrayWrapper, WriterLength] + var previousRow: Array[Byte] = HConstants.EMPTY_BYTE_ARRAY + var rollOverRequested = false + val localTableName = TableName.valueOf(tableRawName) + + // Here is where we finally iterate through the data in this partition of the + // RDD that has been sorted and partitioned + it.foreach { + case (keyFamilyQualifier, cellValue: Array[Byte]) => + val wl = writeValueToHFile( + keyFamilyQualifier.rowKey, + keyFamilyQualifier.family, + keyFamilyQualifier.qualifier, + cellValue, + nowTimeStamp, + fs, + conn, + localTableName, + conf, + familyHFileWriteOptionsMapInternal, + hfileCompression, + writerMap, + stagingDir) - previousRow = keyFamilyQualifier.rowKey - } - //We have finished all the data so lets close up the writers - rollWriters(fs, writerMap, - regionSplitPartitioner, - previousRow, - compactionExclude) - rollOverRequested = false - }) + rollOverRequested = rollOverRequested || wl.written > maxSize + + // This will only roll if we have at least one column family file that is + // bigger then maxSize and we have finished a given row key + if (rollOverRequested && Bytes + .compareTo(previousRow, keyFamilyQualifier.rowKey) != 0) { + rollWriters(fs, writerMap, regionSplitPartitioner, previousRow, compactionExclude) + rollOverRequested = false + } + + previousRow = keyFamilyQualifier.rowKey + } + // We have finished all the data so lets close up the writers + rollWriters(fs, writerMap, regionSplitPartitioner, previousRow, compactionExclude) + rollOverRequested = false + }) } finally { - if(null != conn) conn.close() + if (null != conn) conn.close() } } @@ -743,17 +734,15 @@ class HBaseContext(@transient val sc: SparkContext, * @param maxSize Max size for the HFiles before they roll * @tparam T The Type of values in the original RDD */ - def bulkLoadThinRows[T](rdd:RDD[T], - tableName: TableName, - mapFunction: (T) => - (ByteArrayWrapper, FamiliesQualifiersValues), - stagingDir:String, - familyHFileWriteOptionsMap: - util.Map[Array[Byte], FamilyHFileWriteOptions] = - new util.HashMap[Array[Byte], FamilyHFileWriteOptions], - compactionExclude: Boolean = false, - maxSize:Long = HConstants.DEFAULT_MAX_FILE_SIZE): - Unit = { + def bulkLoadThinRows[T]( + rdd: RDD[T], + tableName: TableName, + mapFunction: (T) => (ByteArrayWrapper, FamiliesQualifiersValues), + stagingDir: String, + familyHFileWriteOptionsMap: util.Map[Array[Byte], FamilyHFileWriteOptions] = + new util.HashMap[Array[Byte], FamilyHFileWriteOptions], + compactionExclude: Boolean = false, + maxSize: Long = HConstants.DEFAULT_MAX_FILE_SIZE): Unit = { val stagingPath = new Path(stagingDir) val fs = stagingPath.getFileSystem(config) if (fs.exists(stagingPath)) { @@ -766,8 +755,8 @@ class HBaseContext(@transient val sc: SparkContext, if (startKeys.length == 0) { logInfo("Table " + tableName.toString + " was not found") } - val defaultCompressionStr = config.get("hfile.compression", - Compression.Algorithm.NONE.getName) + val defaultCompressionStr = + config.get("hfile.compression", Compression.Algorithm.NONE.getName) val defaultCompression = HFileWriterImpl .compressionByName(defaultCompressionStr) val nowTimeStamp = System.currentTimeMillis() @@ -786,94 +775,97 @@ class HBaseContext(@transient val sc: SparkContext, val regionSplitPartitioner = new BulkLoadPartitioner(startKeys) - //This is where all the magic happens - //Here we are going to do the following things + // This is where all the magic happens + // Here we are going to do the following things // 1. FlapMap every row in the RDD into key column value tuples // 2. Then we are going to repartition sort and shuffle // 3. Finally we are going to write out our HFiles - rdd.map( r => mapFunction(r)). - repartitionAndSortWithinPartitions(regionSplitPartitioner). - hbaseForeachPartition(this, (it, conn) => { - - val conf = broadcastedConf.value.value - val fs = new Path(stagingDir).getFileSystem(conf) - val writerMap = new mutable.HashMap[ByteArrayWrapper, WriterLength] - var previousRow:Array[Byte] = HConstants.EMPTY_BYTE_ARRAY - var rollOverRequested = false - val localTableName = TableName.valueOf(tableRawName) - - //Here is where we finally iterate through the data in this partition of the - //RDD that has been sorted and partitioned - it.foreach{ case (rowKey:ByteArrayWrapper, - familiesQualifiersValues:FamiliesQualifiersValues) => - - - if (Bytes.compareTo(previousRow, rowKey.value) == 0) { - throw new KeyAlreadyExistsException("The following key was sent to the " + - "HFile load more then one: " + Bytes.toString(previousRow)) - } - - //The family map is a tree map so the families will be sorted - val familyIt = familiesQualifiersValues.familyMap.entrySet().iterator() - while (familyIt.hasNext) { - val familyEntry = familyIt.next() - - val family = familyEntry.getKey.value - - val qualifierIt = familyEntry.getValue.entrySet().iterator() - - //The qualifier map is a tree map so the families will be sorted - while (qualifierIt.hasNext) { - - val qualifierEntry = qualifierIt.next() - val qualifier = qualifierEntry.getKey - val cellValue = qualifierEntry.getValue - - writeValueToHFile(rowKey.value, - family, - qualifier.value, // qualifier - cellValue, // value - nowTimeStamp, - fs, - conn, - localTableName, - conf, - familyHFileWriteOptionsMapInternal, - defaultCompression, - writerMap, - stagingDir) - - previousRow = rowKey.value - } - - writerMap.values.foreach( wl => { - rollOverRequested = rollOverRequested || wl.written > maxSize + rdd + .map(r => mapFunction(r)) + .repartitionAndSortWithinPartitions(regionSplitPartitioner) + .hbaseForeachPartition( + this, + (it, conn) => { + + val conf = broadcastedConf.value.value + val fs = new Path(stagingDir).getFileSystem(conf) + val writerMap = new mutable.HashMap[ByteArrayWrapper, WriterLength] + var previousRow: Array[Byte] = HConstants.EMPTY_BYTE_ARRAY + var rollOverRequested = false + val localTableName = TableName.valueOf(tableRawName) + + // Here is where we finally iterate through the data in this partition of the + // RDD that has been sorted and partitioned + it.foreach { + case (rowKey: ByteArrayWrapper, familiesQualifiersValues: FamiliesQualifiersValues) => + if (Bytes.compareTo(previousRow, rowKey.value) == 0) { + throw new KeyAlreadyExistsException( + "The following key was sent to the " + + "HFile load more then one: " + Bytes.toString(previousRow)) + } - //This will only roll if we have at least one column family file that is - //bigger then maxSize and we have finished a given row key - if (rollOverRequested) { - rollWriters(fs, writerMap, - regionSplitPartitioner, - previousRow, - compactionExclude) - rollOverRequested = false + // The family map is a tree map so the families will be sorted + val familyIt = familiesQualifiersValues.familyMap.entrySet().iterator() + while (familyIt.hasNext) { + val familyEntry = familyIt.next() + + val family = familyEntry.getKey.value + + val qualifierIt = familyEntry.getValue.entrySet().iterator() + + // The qualifier map is a tree map so the families will be sorted + while (qualifierIt.hasNext) { + + val qualifierEntry = qualifierIt.next() + val qualifier = qualifierEntry.getKey + val cellValue = qualifierEntry.getValue + + writeValueToHFile( + rowKey.value, + family, + qualifier.value, // qualifier + cellValue, // value + nowTimeStamp, + fs, + conn, + localTableName, + conf, + familyHFileWriteOptionsMapInternal, + defaultCompression, + writerMap, + stagingDir) + + previousRow = rowKey.value + } + + writerMap.values.foreach( + wl => { + rollOverRequested = rollOverRequested || wl.written > maxSize + + // This will only roll if we have at least one column family file that is + // bigger then maxSize and we have finished a given row key + if (rollOverRequested) { + rollWriters( + fs, + writerMap, + regionSplitPartitioner, + previousRow, + compactionExclude) + rollOverRequested = false + } + }) } - }) } - } - //This will get a writer for the column family - //If there is no writer for a given column family then - //it will get created here. - //We have finished all the data so lets close up the writers - rollWriters(fs, writerMap, - regionSplitPartitioner, - previousRow, - compactionExclude) - rollOverRequested = false - }) + // This will get a writer for the column family + // If there is no writer for a given column family then + // it will get created here. + // We have finished all the data so lets close up the writers + rollWriters(fs, writerMap, regionSplitPartitioner, previousRow, compactionExclude) + rollOverRequested = false + }) } finally { - if(null != conn) conn.close() + if (null != conn) conn.close() } } @@ -886,20 +878,23 @@ class HBaseContext(@transient val sc: SparkContext, * @param fs FileSystem object where we will be writing the HFiles to * @return WriterLength object */ - private def getNewHFileWriter(family: Array[Byte], conf: Configuration, - favoredNodes: Array[InetSocketAddress], - fs:FileSystem, - familydir:Path, - familyHFileWriteOptionsMapInternal: - util.HashMap[ByteArrayWrapper, FamilyHFileWriteOptions], - defaultCompression:Compression.Algorithm): WriterLength = { - + private def getNewHFileWriter( + family: Array[Byte], + conf: Configuration, + favoredNodes: Array[InetSocketAddress], + fs: FileSystem, + familydir: Path, + familyHFileWriteOptionsMapInternal: util.HashMap[ByteArrayWrapper, FamilyHFileWriteOptions], + defaultCompression: Compression.Algorithm): WriterLength = { var familyOptions = familyHFileWriteOptionsMapInternal.get(new ByteArrayWrapper(family)) if (familyOptions == null) { - familyOptions = new FamilyHFileWriteOptions(defaultCompression.toString, - BloomType.NONE.toString, HConstants.DEFAULT_BLOCKSIZE, DataBlockEncoding.NONE.toString) + familyOptions = new FamilyHFileWriteOptions( + defaultCompression.toString, + BloomType.NONE.toString, + HConstants.DEFAULT_BLOCKSIZE, + DataBlockEncoding.NONE.toString) familyHFileWriteOptionsMapInternal.put(new ByteArrayWrapper(family), familyOptions) } @@ -912,30 +907,32 @@ class HBaseContext(@transient val sc: SparkContext, val contextBuilder = new HFileContextBuilder() .withCompression(Algorithm.valueOf(familyOptions.compression)) // ChecksumType.nameToType is still an IA.Private Utils, but it's unlikely to be changed. - .withChecksumType(ChecksumType - .nameToType(conf.get(HConstants.CHECKSUM_TYPE_NAME, - ChecksumType.getDefaultChecksumType.getName))) + .withChecksumType( + ChecksumType + .nameToType( + conf.get(HConstants.CHECKSUM_TYPE_NAME, ChecksumType.getDefaultChecksumType.getName))) .withCellComparator(CellComparator.getInstance()) - .withBytesPerCheckSum(conf.getInt(HConstants.BYTES_PER_CHECKSUM, - HFile.DEFAULT_BYTES_PER_CHECKSUM)) + .withBytesPerCheckSum( + conf.getInt(HConstants.BYTES_PER_CHECKSUM, HFile.DEFAULT_BYTES_PER_CHECKSUM)) .withBlockSize(familyOptions.blockSize) if (HFile.getFormatVersion(conf) >= HFile.MIN_FORMAT_VERSION_WITH_TAGS) { contextBuilder.withIncludesTags(true) } - contextBuilder.withDataBlockEncoding(DataBlockEncoding. - valueOf(familyOptions.dataBlockEncoding)) + contextBuilder.withDataBlockEncoding(DataBlockEncoding.valueOf(familyOptions.dataBlockEncoding)) val hFileContext = contextBuilder.build() - //Add a '_' to the file name because this is a unfinished file. A rename will happen + // Add a '_' to the file name because this is a unfinished file. A rename will happen // to remove the '_' when the file is closed. - new WriterLength(0, + new WriterLength( + 0, new StoreFileWriter.Builder(conf, new CacheConfig(tempConf), new HFileSystem(fs)) .withBloomType(BloomType.valueOf(familyOptions.bloomType)) .withFileContext(hFileContext) .withFilePath(new Path(familydir, "_" + UUID.randomUUID.toString.replaceAll("-", ""))) - .withFavoredNodes(favoredNodes).build()) + .withFavoredNodes(favoredNodes) + .build()) } @@ -959,88 +956,91 @@ class HBaseContext(@transient val sc: SparkContext, * @return The writer for the given HFile that was writen * too */ - private def writeValueToHFile(rowKey: Array[Byte], - family: Array[Byte], - qualifier: Array[Byte], - cellValue:Array[Byte], - nowTimeStamp: Long, - fs: FileSystem, - conn: Connection, - tableName: TableName, - conf: Configuration, - familyHFileWriteOptionsMapInternal: - util.HashMap[ByteArrayWrapper, FamilyHFileWriteOptions], - hfileCompression:Compression.Algorithm, - writerMap:mutable.HashMap[ByteArrayWrapper, WriterLength], - stagingDir: String - ): WriterLength = { - - val wl = writerMap.getOrElseUpdate(new ByteArrayWrapper(family), { - val familyDir = new Path(stagingDir, Bytes.toString(family)) - - familyDir.getFileSystem(conf).mkdirs(familyDir); - - val loc:HRegionLocation = { - try { - val locator = - conn.getRegionLocator(tableName) - locator.getRegionLocation(rowKey) - } catch { - case e: Throwable => - logWarning("there's something wrong when locating rowkey: " + - Bytes.toString(rowKey)) - null - } - } - if (null == loc) { - if (log.isTraceEnabled) { - logTrace("failed to get region location, so use default writer: " + - Bytes.toString(rowKey)) - } - getNewHFileWriter(family = family, - conf = conf, - favoredNodes = null, - fs = fs, - familydir = familyDir, - familyHFileWriteOptionsMapInternal, - hfileCompression) - } else { - if (log.isDebugEnabled) { - logDebug("first rowkey: [" + Bytes.toString(rowKey) + "]") + private def writeValueToHFile( + rowKey: Array[Byte], + family: Array[Byte], + qualifier: Array[Byte], + cellValue: Array[Byte], + nowTimeStamp: Long, + fs: FileSystem, + conn: Connection, + tableName: TableName, + conf: Configuration, + familyHFileWriteOptionsMapInternal: util.HashMap[ByteArrayWrapper, FamilyHFileWriteOptions], + hfileCompression: Compression.Algorithm, + writerMap: mutable.HashMap[ByteArrayWrapper, WriterLength], + stagingDir: String): WriterLength = { + + val wl = writerMap.getOrElseUpdate( + new ByteArrayWrapper(family), { + val familyDir = new Path(stagingDir, Bytes.toString(family)) + + familyDir.getFileSystem(conf).mkdirs(familyDir); + + val loc: HRegionLocation = { + try { + val locator = + conn.getRegionLocator(tableName) + locator.getRegionLocation(rowKey) + } catch { + case e: Throwable => + logWarning( + "there's something wrong when locating rowkey: " + + Bytes.toString(rowKey)) + null + } } - val initialIsa = - new InetSocketAddress(loc.getHostname, loc.getPort) - if (initialIsa.isUnresolved) { + if (null == loc) { if (log.isTraceEnabled) { - logTrace("failed to resolve bind address: " + loc.getHostname + ":" - + loc.getPort + ", so use default writer") + logTrace( + "failed to get region location, so use default writer: " + + Bytes.toString(rowKey)) } - getNewHFileWriter(family, - conf, - null, - fs, - familyDir, + getNewHFileWriter( + family = family, + conf = conf, + favoredNodes = null, + fs = fs, + familydir = familyDir, familyHFileWriteOptionsMapInternal, hfileCompression) } else { - if(log.isDebugEnabled) { - logDebug("use favored nodes writer: " + initialIsa.getHostString) + if (log.isDebugEnabled) { + logDebug("first rowkey: [" + Bytes.toString(rowKey) + "]") + } + val initialIsa = + new InetSocketAddress(loc.getHostname, loc.getPort) + if (initialIsa.isUnresolved) { + if (log.isTraceEnabled) { + logTrace( + "failed to resolve bind address: " + loc.getHostname + ":" + + loc.getPort + ", so use default writer") + } + getNewHFileWriter( + family, + conf, + null, + fs, + familyDir, + familyHFileWriteOptionsMapInternal, + hfileCompression) + } else { + if (log.isDebugEnabled) { + logDebug("use favored nodes writer: " + initialIsa.getHostString) + } + getNewHFileWriter( + family, + conf, + Array[InetSocketAddress](initialIsa), + fs, + familyDir, + familyHFileWriteOptionsMapInternal, + hfileCompression) } - getNewHFileWriter(family, - conf, - Array[InetSocketAddress](initialIsa), - fs, - familyDir, - familyHFileWriteOptionsMapInternal, - hfileCompression) } - } - }) + }) - val keyValue =new KeyValue(rowKey, - family, - qualifier, - nowTimeStamp,cellValue) + val keyValue = new KeyValue(rowKey, family, qualifier, nowTimeStamp, cellValue) wl.writer.append(keyValue) wl.written += keyValue.getLength @@ -1057,21 +1057,21 @@ class HBaseContext(@transient val sc: SparkContext, * @param previousRow The last row to fill the HFile ending range metadata * @param compactionExclude The exclude compaction metadata flag for the HFile */ - private def rollWriters(fs:FileSystem, - writerMap:mutable.HashMap[ByteArrayWrapper, WriterLength], - regionSplitPartitioner: BulkLoadPartitioner, - previousRow: Array[Byte], - compactionExclude: Boolean): Unit = { - writerMap.values.foreach( wl => { - if (wl.writer != null) { - logDebug("Writer=" + wl.writer.getPath + - (if (wl.written == 0) "" else ", wrote=" + wl.written)) - closeHFileWriter(fs, wl.writer, - regionSplitPartitioner, - previousRow, - compactionExclude) - } - }) + private def rollWriters( + fs: FileSystem, + writerMap: mutable.HashMap[ByteArrayWrapper, WriterLength], + regionSplitPartitioner: BulkLoadPartitioner, + previousRow: Array[Byte], + compactionExclude: Boolean): Unit = { + writerMap.values.foreach( + wl => { + if (wl.writer != null) { + logDebug( + "Writer=" + wl.writer.getPath + + (if (wl.written == 0) "" else ", wrote=" + wl.written)) + closeHFileWriter(fs, wl.writer, regionSplitPartitioner, previousRow, compactionExclude) + } + }) writerMap.clear() } @@ -1085,33 +1085,35 @@ class HBaseContext(@transient val sc: SparkContext, * @param previousRow The last row to fill the HFile ending range metadata * @param compactionExclude The exclude compaction metadata flag for the HFile */ - private def closeHFileWriter(fs:FileSystem, - w: StoreFileWriter, - regionSplitPartitioner: BulkLoadPartitioner, - previousRow: Array[Byte], - compactionExclude: Boolean): Unit = { + private def closeHFileWriter( + fs: FileSystem, + w: StoreFileWriter, + regionSplitPartitioner: BulkLoadPartitioner, + previousRow: Array[Byte], + compactionExclude: Boolean): Unit = { if (w != null) { - w.appendFileInfo(HStoreFile.BULKLOAD_TIME_KEY, - Bytes.toBytes(System.currentTimeMillis())) - w.appendFileInfo(HStoreFile.BULKLOAD_TASK_KEY, + w.appendFileInfo(HStoreFile.BULKLOAD_TIME_KEY, Bytes.toBytes(System.currentTimeMillis())) + w.appendFileInfo( + HStoreFile.BULKLOAD_TASK_KEY, Bytes.toBytes(regionSplitPartitioner.getPartition(previousRow))) - w.appendFileInfo(HStoreFile.MAJOR_COMPACTION_KEY, - Bytes.toBytes(true)) - w.appendFileInfo(HStoreFile.EXCLUDE_FROM_MINOR_COMPACTION_KEY, + w.appendFileInfo(HStoreFile.MAJOR_COMPACTION_KEY, Bytes.toBytes(true)) + w.appendFileInfo( + HStoreFile.EXCLUDE_FROM_MINOR_COMPACTION_KEY, Bytes.toBytes(compactionExclude)) w.appendTrackedTimestampsToMetadata() w.close() val srcPath = w.getPath - //In the new path you will see that we are using substring. This is to + // In the new path you will see that we are using substring. This is to // remove the '_' character in front of the HFile name. '_' is a character // that will tell HBase that this file shouldn't be included in the bulk load // This feature is to protect for unfinished HFiles being submitted to HBase val newPath = new Path(w.getPath.getParent, w.getPath.getName.substring(1)) if (!fs.rename(srcPath, newPath)) { - throw new IOException("Unable to rename '" + srcPath + - "' to " + newPath) + throw new IOException( + "Unable to rename '" + srcPath + + "' to " + newPath) } } } @@ -1123,10 +1125,10 @@ class HBaseContext(@transient val sc: SparkContext, * @param written The writer to be wrapped * @param writer The number of bytes written to the writer */ - class WriterLength(var written:Long, val writer:StoreFileWriter) + class WriterLength(var written: Long, val writer: StoreFileWriter) } @InterfaceAudience.Private object LatestHBaseContextCache { - var latest:HBaseContext = null + var latest: HBaseContext = null } diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseDStreamFunctions.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseDStreamFunctions.scala index 4edde442..0ca5038c 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseDStreamFunctions.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseDStreamFunctions.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -17,11 +18,10 @@ package org.apache.hadoop.hbase.spark import org.apache.hadoop.hbase.TableName -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client._ import org.apache.hadoop.hbase.io.ImmutableBytesWritable import org.apache.spark.streaming.dstream.DStream - +import org.apache.yetus.audience.InterfaceAudience import scala.reflect.ClassTag /** @@ -50,9 +50,7 @@ object HBaseDStreamFunctions { * @param f The function that will turn the DStream values * into HBase Put objects. */ - def hbaseBulkPut(hc: HBaseContext, - tableName: TableName, - f: (T) => Put): Unit = { + def hbaseBulkPut(hc: HBaseContext, tableName: TableName, f: (T) => Put): Unit = { hc.streamBulkPut(dStream, tableName, f) } @@ -75,10 +73,12 @@ object HBaseDStreamFunctions { * out of the resulting DStream * @return A resulting DStream with type R objects */ - def hbaseBulkGet[R: ClassTag](hc: HBaseContext, - tableName: TableName, - batchSize:Int, f: (T) => Get, convertResult: (Result) => R): - DStream[R] = { + def hbaseBulkGet[R: ClassTag]( + hc: HBaseContext, + tableName: TableName, + batchSize: Int, + f: (T) => Get, + convertResult: (Result) => R): DStream[R] = { hc.streamBulkGet[T, R](tableName, batchSize, dStream, f, convertResult) } @@ -96,12 +96,17 @@ object HBaseDStreamFunctions { * in HBase Get objects * @return A resulting DStream with type R objects */ - def hbaseBulkGet(hc: HBaseContext, - tableName: TableName, batchSize:Int, - f: (T) => Get): DStream[(ImmutableBytesWritable, Result)] = { - hc.streamBulkGet[T, (ImmutableBytesWritable, Result)]( - tableName, batchSize, dStream, f, - result => (new ImmutableBytesWritable(result.getRow), result)) + def hbaseBulkGet( + hc: HBaseContext, + tableName: TableName, + batchSize: Int, + f: (T) => Get): DStream[(ImmutableBytesWritable, Result)] = { + hc.streamBulkGet[T, (ImmutableBytesWritable, Result)]( + tableName, + batchSize, + dStream, + f, + result => (new ImmutableBytesWritable(result.getRow), result)) } /** @@ -115,9 +120,11 @@ object HBaseDStreamFunctions { * a HBase Delete Object * @param batchSize The number of Deletes to be sent in a single batch */ - def hbaseBulkDelete(hc: HBaseContext, - tableName: TableName, - f:(T) => Delete, batchSize:Int): Unit = { + def hbaseBulkDelete( + hc: HBaseContext, + tableName: TableName, + f: (T) => Delete, + batchSize: Int): Unit = { hc.streamBulkDelete(dStream, tableName, f, batchSize) } @@ -132,8 +139,7 @@ object HBaseDStreamFunctions { * @param f This function will get an iterator for a Partition of an * DStream along with a connection object to HBase */ - def hbaseForeachPartition(hc: HBaseContext, - f: (Iterator[T], Connection) => Unit): Unit = { + def hbaseForeachPartition(hc: HBaseContext, f: (Iterator[T], Connection) => Unit): Unit = { hc.streamForeachPartition(dStream, f) } @@ -151,9 +157,9 @@ object HBaseDStreamFunctions { * DStream * @return A resulting DStream of type R */ - def hbaseMapPartitions[R: ClassTag](hc: HBaseContext, - f: (Iterator[T], Connection) => Iterator[R]): - DStream[R] = { + def hbaseMapPartitions[R: ClassTag]( + hc: HBaseContext, + f: (Iterator[T], Connection) => Iterator[R]): DStream[R] = { hc.streamMapPartitions(dStream, f) } } diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseRDDFunctions.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseRDDFunctions.scala index 2469c8e9..9e3da565 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseRDDFunctions.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseRDDFunctions.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,17 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark import java.util - import org.apache.hadoop.hbase.{HConstants, TableName} -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client._ import org.apache.hadoop.hbase.io.ImmutableBytesWritable import org.apache.spark.rdd.RDD - +import org.apache.yetus.audience.InterfaceAudience import scala.reflect.ClassTag /** @@ -32,8 +30,7 @@ import scala.reflect.ClassTag * applied to a Spark RDD so that we can easily interact with HBase */ @InterfaceAudience.Public -object HBaseRDDFunctions -{ +object HBaseRDDFunctions { /** * These are implicit methods for a RDD that contains any type of @@ -54,9 +51,7 @@ object HBaseRDDFunctions * @param f The function that will turn the RDD values * into HBase Put objects. */ - def hbaseBulkPut(hc: HBaseContext, - tableName: TableName, - f: (T) => Put): Unit = { + def hbaseBulkPut(hc: HBaseContext, tableName: TableName, f: (T) => Put): Unit = { hc.bulkPut(rdd, tableName, f) } @@ -79,9 +74,12 @@ object HBaseRDDFunctions * out of the resulting RDD * @return A resulting RDD with type R objects */ - def hbaseBulkGet[R: ClassTag](hc: HBaseContext, - tableName: TableName, batchSize:Int, - f: (T) => Get, convertResult: (Result) => R): RDD[R] = { + def hbaseBulkGet[R: ClassTag]( + hc: HBaseContext, + tableName: TableName, + batchSize: Int, + f: (T) => Get, + convertResult: (Result) => R): RDD[R] = { hc.bulkGet[T, R](tableName, batchSize, rdd, f, convertResult) } @@ -99,16 +97,22 @@ object HBaseRDDFunctions * in HBase Get objects * @return A resulting RDD with type R objects */ - def hbaseBulkGet(hc: HBaseContext, - tableName: TableName, batchSize:Int, - f: (T) => Get): RDD[(ImmutableBytesWritable, Result)] = { - hc.bulkGet[T, (ImmutableBytesWritable, Result)](tableName, - batchSize, rdd, f, - result => if (result != null && result.getRow != null) { - (new ImmutableBytesWritable(result.getRow), result) - } else { - null - }) + def hbaseBulkGet( + hc: HBaseContext, + tableName: TableName, + batchSize: Int, + f: (T) => Get): RDD[(ImmutableBytesWritable, Result)] = { + hc.bulkGet[T, (ImmutableBytesWritable, Result)]( + tableName, + batchSize, + rdd, + f, + result => + if (result != null && result.getRow != null) { + (new ImmutableBytesWritable(result.getRow), result) + } else { + null + }) } /** @@ -122,8 +126,11 @@ object HBaseRDDFunctions * a HBase Delete Object * @param batchSize The number of Deletes to be sent in a single batch */ - def hbaseBulkDelete(hc: HBaseContext, - tableName: TableName, f:(T) => Delete, batchSize:Int): Unit = { + def hbaseBulkDelete( + hc: HBaseContext, + tableName: TableName, + f: (T) => Delete, + batchSize: Int): Unit = { hc.bulkDelete(rdd, tableName, f, batchSize) } @@ -138,8 +145,7 @@ object HBaseRDDFunctions * @param f This function will get an iterator for a Partition of an * RDD along with a connection object to HBase */ - def hbaseForeachPartition(hc: HBaseContext, - f: (Iterator[T], Connection) => Unit): Unit = { + def hbaseForeachPartition(hc: HBaseContext, f: (Iterator[T], Connection) => Unit): Unit = { hc.foreachPartition(rdd, f) } @@ -157,10 +163,10 @@ object HBaseRDDFunctions * RDD * @return A resulting RDD of type R */ - def hbaseMapPartitions[R: ClassTag](hc: HBaseContext, - f: (Iterator[T], Connection) => Iterator[R]): - RDD[R] = { - hc.mapPartitions[T,R](rdd, f) + def hbaseMapPartitions[R: ClassTag]( + hc: HBaseContext, + f: (Iterator[T], Connection) => Iterator[R]): RDD[R] = { + hc.mapPartitions[T, R](rdd, f) } /** @@ -190,18 +196,23 @@ object HBaseRDDFunctions * @param compactionExclude Compaction excluded for the HFiles * @param maxSize Max size for the HFiles before they roll */ - def hbaseBulkLoad(hc: HBaseContext, - tableName: TableName, - flatMap: (T) => Iterator[(KeyFamilyQualifier, Array[Byte])], - stagingDir:String, - familyHFileWriteOptionsMap: - util.Map[Array[Byte], FamilyHFileWriteOptions] = - new util.HashMap[Array[Byte], FamilyHFileWriteOptions](), - compactionExclude: Boolean = false, - maxSize:Long = HConstants.DEFAULT_MAX_FILE_SIZE):Unit = { - hc.bulkLoad(rdd, tableName, - flatMap, stagingDir, familyHFileWriteOptionsMap, - compactionExclude, maxSize) + def hbaseBulkLoad( + hc: HBaseContext, + tableName: TableName, + flatMap: (T) => Iterator[(KeyFamilyQualifier, Array[Byte])], + stagingDir: String, + familyHFileWriteOptionsMap: util.Map[Array[Byte], FamilyHFileWriteOptions] = + new util.HashMap[Array[Byte], FamilyHFileWriteOptions](), + compactionExclude: Boolean = false, + maxSize: Long = HConstants.DEFAULT_MAX_FILE_SIZE): Unit = { + hc.bulkLoad( + rdd, + tableName, + flatMap, + stagingDir, + familyHFileWriteOptionsMap, + compactionExclude, + maxSize) } /** @@ -235,19 +246,23 @@ object HBaseRDDFunctions * @param compactionExclude Compaction excluded for the HFiles * @param maxSize Max size for the HFiles before they roll */ - def hbaseBulkLoadThinRows(hc: HBaseContext, - tableName: TableName, - mapFunction: (T) => - (ByteArrayWrapper, FamiliesQualifiersValues), - stagingDir:String, - familyHFileWriteOptionsMap: - util.Map[Array[Byte], FamilyHFileWriteOptions] = - new util.HashMap[Array[Byte], FamilyHFileWriteOptions](), - compactionExclude: Boolean = false, - maxSize:Long = HConstants.DEFAULT_MAX_FILE_SIZE):Unit = { - hc.bulkLoadThinRows(rdd, tableName, - mapFunction, stagingDir, familyHFileWriteOptionsMap, - compactionExclude, maxSize) + def hbaseBulkLoadThinRows( + hc: HBaseContext, + tableName: TableName, + mapFunction: (T) => (ByteArrayWrapper, FamiliesQualifiersValues), + stagingDir: String, + familyHFileWriteOptionsMap: util.Map[Array[Byte], FamilyHFileWriteOptions] = + new util.HashMap[Array[Byte], FamilyHFileWriteOptions](), + compactionExclude: Boolean = false, + maxSize: Long = HConstants.DEFAULT_MAX_FILE_SIZE): Unit = { + hc.bulkLoadThinRows( + rdd, + tableName, + mapFunction, + stagingDir, + familyHFileWriteOptionsMap, + compactionExclude, + maxSize) } } } diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/JavaHBaseContext.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/JavaHBaseContext.scala index be6581a3..2bf96ba7 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/JavaHBaseContext.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/JavaHBaseContext.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,23 +15,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark +import java.lang.Iterable import java.util.Map - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.hbase.TableName -import org.apache.hadoop.hbase.util.Pair -import org.apache.yetus.audience.InterfaceAudience import org.apache.hadoop.hbase.client.{Connection, Delete, Get, Put, Result, Scan} import org.apache.hadoop.hbase.io.ImmutableBytesWritable +import org.apache.hadoop.hbase.util.Pair import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.api.java.function.{FlatMapFunction, Function, VoidFunction} import org.apache.spark.streaming.api.java.JavaDStream - -import java.lang.Iterable - +import org.apache.yetus.audience.InterfaceAudience import scala.collection.JavaConversions._ import scala.reflect.ClassTag @@ -43,8 +40,8 @@ import scala.reflect.ClassTag * @param config This is the config information to out HBase cluster */ @InterfaceAudience.Public -class JavaHBaseContext(@transient val jsc: JavaSparkContext, - @transient val config: Configuration) extends Serializable { +class JavaHBaseContext(@transient val jsc: JavaSparkContext, @transient val config: Configuration) + extends Serializable { val hbaseContext = new HBaseContext(jsc.sc, config) /** @@ -60,10 +57,12 @@ class JavaHBaseContext(@transient val jsc: JavaSparkContext, * the RDD values and a Connection object to interact * with HBase */ - def foreachPartition[T](javaRdd: JavaRDD[T], - f: VoidFunction[(java.util.Iterator[T], Connection)]) = { + def foreachPartition[T]( + javaRdd: JavaRDD[T], + f: VoidFunction[(java.util.Iterator[T], Connection)]) = { - hbaseContext.foreachPartition(javaRdd.rdd, + hbaseContext.foreachPartition( + javaRdd.rdd, (it: Iterator[T], conn: Connection) => { f.call((it, conn)) }) @@ -82,9 +81,11 @@ class JavaHBaseContext(@transient val jsc: JavaSparkContext, * the JavaDStream values and a Connection object to * interact with HBase */ - def foreachPartition[T](javaDstream: JavaDStream[T], - f: VoidFunction[(Iterator[T], Connection)]) = { - hbaseContext.foreachPartition(javaDstream.dstream, + def foreachPartition[T]( + javaDstream: JavaDStream[T], + f: VoidFunction[(Iterator[T], Connection)]) = { + hbaseContext.foreachPartition( + javaDstream.dstream, (it: Iterator[T], conn: Connection) => f.call(it, conn)) } @@ -106,12 +107,13 @@ class JavaHBaseContext(@transient val jsc: JavaSparkContext, * @return Returns a new RDD generated by the user definition * function just like normal mapPartition */ - def mapPartitions[T, R](javaRdd: JavaRDD[T], - f: FlatMapFunction[(java.util.Iterator[T], - Connection), R]): JavaRDD[R] = { - JavaRDD.fromRDD(hbaseContext.mapPartitions(javaRdd.rdd, - (it: Iterator[T], conn: Connection) => - f.call(it, conn))(fakeClassTag[R]))(fakeClassTag[R]) + def mapPartitions[T, R]( + javaRdd: JavaRDD[T], + f: FlatMapFunction[(java.util.Iterator[T], Connection), R]): JavaRDD[R] = { + JavaRDD.fromRDD( + hbaseContext.mapPartitions( + javaRdd.rdd, + (it: Iterator[T], conn: Connection) => f.call(it, conn))(fakeClassTag[R]))(fakeClassTag[R]) } /** @@ -134,12 +136,13 @@ class JavaHBaseContext(@transient val jsc: JavaSparkContext, * @return Returns a new JavaDStream generated by the user * definition function just like normal mapPartition */ - def streamMap[T, U](javaDstream: JavaDStream[T], - mp: Function[(Iterator[T], Connection), Iterator[U]]): - JavaDStream[U] = { - JavaDStream.fromDStream(hbaseContext.streamMapPartitions(javaDstream.dstream, - (it: Iterator[T], conn: Connection) => - mp.call(it, conn))(fakeClassTag[U]))(fakeClassTag[U]) + def streamMap[T, U]( + javaDstream: JavaDStream[T], + mp: Function[(Iterator[T], Connection), Iterator[U]]): JavaDStream[U] = { + JavaDStream.fromDStream( + hbaseContext.streamMapPartitions( + javaDstream.dstream, + (it: Iterator[T], conn: Connection) => mp.call(it, conn))(fakeClassTag[U]))(fakeClassTag[U]) } /** @@ -155,9 +158,7 @@ class JavaHBaseContext(@transient val jsc: JavaSparkContext, * @param f Function to convert a value in the JavaRDD * to a HBase Put */ - def bulkPut[T](javaRdd: JavaRDD[T], - tableName: TableName, - f: Function[(T), Put]) { + def bulkPut[T](javaRdd: JavaRDD[T], tableName: TableName, f: Function[(T), Put]) { hbaseContext.bulkPut(javaRdd.rdd, tableName, (t: T) => f.call(t)) } @@ -176,12 +177,8 @@ class JavaHBaseContext(@transient val jsc: JavaSparkContext, * @param f Function to convert a value in * the JavaDStream to a HBase Put */ - def streamBulkPut[T](javaDstream: JavaDStream[T], - tableName: TableName, - f: Function[T, Put]) = { - hbaseContext.streamBulkPut(javaDstream.dstream, - tableName, - (t: T) => f.call(t)) + def streamBulkPut[T](javaDstream: JavaDStream[T], tableName: TableName, f: Function[T, Put]) = { + hbaseContext.streamBulkPut(javaDstream.dstream, tableName, (t: T) => f.call(t)) } /** @@ -199,8 +196,11 @@ class JavaHBaseContext(@transient val jsc: JavaSparkContext, * HBase Deletes * @param batchSize The number of deletes to batch before sending to HBase */ - def bulkDelete[T](javaRdd: JavaRDD[T], tableName: TableName, - f: Function[T, Delete], batchSize: Integer) { + def bulkDelete[T]( + javaRdd: JavaRDD[T], + tableName: TableName, + f: Function[T, Delete], + batchSize: Integer) { hbaseContext.bulkDelete(javaRdd.rdd, tableName, (t: T) => f.call(t), batchSize) } @@ -219,13 +219,12 @@ class JavaHBaseContext(@transient val jsc: JavaSparkContext, * HBase Delete * @param batchSize The number of deletes to be sent at once */ - def streamBulkDelete[T](javaDStream: JavaDStream[T], - tableName: TableName, - f: Function[T, Delete], - batchSize: Integer) = { - hbaseContext.streamBulkDelete(javaDStream.dstream, tableName, - (t: T) => f.call(t), - batchSize) + def streamBulkDelete[T]( + javaDStream: JavaDStream[T], + tableName: TableName, + f: Function[T, Delete], + batchSize: Integer) = { + hbaseContext.streamBulkDelete(javaDStream.dstream, tableName, (t: T) => f.call(t), batchSize) } /** @@ -244,19 +243,22 @@ class JavaHBaseContext(@transient val jsc: JavaSparkContext, * JavaRDD * @return New JavaRDD that is created by the Get to HBase */ - def bulkGet[T, U](tableName: TableName, - batchSize: Integer, - javaRdd: JavaRDD[T], - makeGet: Function[T, Get], - convertResult: Function[Result, U]): JavaRDD[U] = { + def bulkGet[T, U]( + tableName: TableName, + batchSize: Integer, + javaRdd: JavaRDD[T], + makeGet: Function[T, Get], + convertResult: Function[Result, U]): JavaRDD[U] = { - JavaRDD.fromRDD(hbaseContext.bulkGet[T, U](tableName, - batchSize, - javaRdd.rdd, - (t: T) => makeGet.call(t), - (r: Result) => { - convertResult.call(r) - })(fakeClassTag[U]))(fakeClassTag[U]) + JavaRDD.fromRDD( + hbaseContext.bulkGet[T, U]( + tableName, + batchSize, + javaRdd.rdd, + (t: T) => makeGet.call(t), + (r: Result) => { + convertResult.call(r) + })(fakeClassTag[U]))(fakeClassTag[U]) } @@ -277,77 +279,93 @@ class JavaHBaseContext(@transient val jsc: JavaSparkContext, * JavaDStream * @return New JavaDStream that is created by the Get to HBase */ - def streamBulkGet[T, U](tableName: TableName, - batchSize: Integer, - javaDStream: JavaDStream[T], - makeGet: Function[T, Get], - convertResult: Function[Result, U]): JavaDStream[U] = { - JavaDStream.fromDStream(hbaseContext.streamBulkGet(tableName, - batchSize, - javaDStream.dstream, - (t: T) => makeGet.call(t), - (r: Result) => convertResult.call(r))(fakeClassTag[U]))(fakeClassTag[U]) + def streamBulkGet[T, U]( + tableName: TableName, + batchSize: Integer, + javaDStream: JavaDStream[T], + makeGet: Function[T, Get], + convertResult: Function[Result, U]): JavaDStream[U] = { + JavaDStream.fromDStream( + hbaseContext.streamBulkGet( + tableName, + batchSize, + javaDStream.dstream, + (t: T) => makeGet.call(t), + (r: Result) => convertResult.call(r))(fakeClassTag[U]))(fakeClassTag[U]) } /** - * A simple abstraction over the HBaseContext.bulkLoad method. - * It allow addition support for a user to take a JavaRDD and - * convert into new JavaRDD[Pair] based on MapFunction, - * and HFiles will be generated in stagingDir for bulk load - * - * @param javaRdd The javaRDD we are bulk loading from - * @param tableName The HBase table we are loading into - * @param mapFunc A Function that will convert a value in JavaRDD - * to Pair(KeyFamilyQualifier, Array[Byte]) - * @param stagingDir The location on the FileSystem to bulk load into - * @param familyHFileWriteOptionsMap Options that will define how the HFile for a - * column family is written - * @param compactionExclude Compaction excluded for the HFiles - * @param maxSize Max size for the HFiles before they roll - */ - def bulkLoad[T](javaRdd: JavaRDD[T], - tableName: TableName, - mapFunc : Function[T, Pair[KeyFamilyQualifier, Array[Byte]]], - stagingDir: String, - familyHFileWriteOptionsMap: Map[Array[Byte], FamilyHFileWriteOptions], - compactionExclude: Boolean, - maxSize: Long): - Unit = { - hbaseContext.bulkLoad[Pair[KeyFamilyQualifier, Array[Byte]]](javaRdd.map(mapFunc).rdd, tableName, t => { - val keyFamilyQualifier = t.getFirst - val value = t.getSecond - Seq((keyFamilyQualifier, value)).iterator - }, stagingDir, familyHFileWriteOptionsMap, compactionExclude, maxSize) + * A simple abstraction over the HBaseContext.bulkLoad method. + * It allow addition support for a user to take a JavaRDD and + * convert into new JavaRDD[Pair] based on MapFunction, + * and HFiles will be generated in stagingDir for bulk load + * + * @param javaRdd The javaRDD we are bulk loading from + * @param tableName The HBase table we are loading into + * @param mapFunc A Function that will convert a value in JavaRDD + * to Pair(KeyFamilyQualifier, Array[Byte]) + * @param stagingDir The location on the FileSystem to bulk load into + * @param familyHFileWriteOptionsMap Options that will define how the HFile for a + * column family is written + * @param compactionExclude Compaction excluded for the HFiles + * @param maxSize Max size for the HFiles before they roll + */ + def bulkLoad[T]( + javaRdd: JavaRDD[T], + tableName: TableName, + mapFunc: Function[T, Pair[KeyFamilyQualifier, Array[Byte]]], + stagingDir: String, + familyHFileWriteOptionsMap: Map[Array[Byte], FamilyHFileWriteOptions], + compactionExclude: Boolean, + maxSize: Long): Unit = { + hbaseContext.bulkLoad[Pair[KeyFamilyQualifier, Array[Byte]]]( + javaRdd.map(mapFunc).rdd, + tableName, + t => { + val keyFamilyQualifier = t.getFirst + val value = t.getSecond + Seq((keyFamilyQualifier, value)).iterator + }, + stagingDir, + familyHFileWriteOptionsMap, + compactionExclude, + maxSize) } /** - * A simple abstraction over the HBaseContext.bulkLoadThinRows method. - * It allow addition support for a user to take a JavaRDD and - * convert into new JavaRDD[Pair] based on MapFunction, - * and HFiles will be generated in stagingDir for bulk load - * - * @param javaRdd The javaRDD we are bulk loading from - * @param tableName The HBase table we are loading into - * @param mapFunc A Function that will convert a value in JavaRDD - * to Pair(ByteArrayWrapper, FamiliesQualifiersValues) - * @param stagingDir The location on the FileSystem to bulk load into - * @param familyHFileWriteOptionsMap Options that will define how the HFile for a - * column family is written - * @param compactionExclude Compaction excluded for the HFiles - * @param maxSize Max size for the HFiles before they roll - */ - def bulkLoadThinRows[T](javaRdd: JavaRDD[T], - tableName: TableName, - mapFunc : Function[T, Pair[ByteArrayWrapper, FamiliesQualifiersValues]], - stagingDir: String, - familyHFileWriteOptionsMap: Map[Array[Byte], FamilyHFileWriteOptions], - compactionExclude: Boolean, - maxSize: Long): - Unit = { - hbaseContext.bulkLoadThinRows[Pair[ByteArrayWrapper, FamiliesQualifiersValues]](javaRdd.map(mapFunc).rdd, - tableName, t => { - (t.getFirst, t.getSecond) - }, stagingDir, familyHFileWriteOptionsMap, compactionExclude, maxSize) + * A simple abstraction over the HBaseContext.bulkLoadThinRows method. + * It allow addition support for a user to take a JavaRDD and + * convert into new JavaRDD[Pair] based on MapFunction, + * and HFiles will be generated in stagingDir for bulk load + * + * @param javaRdd The javaRDD we are bulk loading from + * @param tableName The HBase table we are loading into + * @param mapFunc A Function that will convert a value in JavaRDD + * to Pair(ByteArrayWrapper, FamiliesQualifiersValues) + * @param stagingDir The location on the FileSystem to bulk load into + * @param familyHFileWriteOptionsMap Options that will define how the HFile for a + * column family is written + * @param compactionExclude Compaction excluded for the HFiles + * @param maxSize Max size for the HFiles before they roll + */ + def bulkLoadThinRows[T]( + javaRdd: JavaRDD[T], + tableName: TableName, + mapFunc: Function[T, Pair[ByteArrayWrapper, FamiliesQualifiersValues]], + stagingDir: String, + familyHFileWriteOptionsMap: Map[Array[Byte], FamilyHFileWriteOptions], + compactionExclude: Boolean, + maxSize: Long): Unit = { + hbaseContext.bulkLoadThinRows[Pair[ByteArrayWrapper, FamiliesQualifiersValues]]( + javaRdd.map(mapFunc).rdd, + tableName, + t => { + (t.getFirst, t.getSecond) + }, + stagingDir, + familyHFileWriteOptionsMap, + compactionExclude, + maxSize) } /** @@ -360,15 +378,14 @@ class JavaHBaseContext(@transient val jsc: JavaSparkContext, * What the user wants in the final generated JavaRDD * @return New JavaRDD with results from scan */ - def hbaseRDD[U](tableName: TableName, - scans: Scan, - f: Function[(ImmutableBytesWritable, Result), U]): - JavaRDD[U] = { + def hbaseRDD[U]( + tableName: TableName, + scans: Scan, + f: Function[(ImmutableBytesWritable, Result), U]): JavaRDD[U] = { JavaRDD.fromRDD( - hbaseContext.hbaseRDD[U](tableName, - scans, - (v: (ImmutableBytesWritable, Result)) => - f.call(v._1, v._2))(fakeClassTag[U]))(fakeClassTag[U]) + hbaseContext + .hbaseRDD[U](tableName, scans, (v: (ImmutableBytesWritable, Result)) => f.call(v._1, v._2))( + fakeClassTag[U]))(fakeClassTag[U]) } /** @@ -379,9 +396,7 @@ class JavaHBaseContext(@transient val jsc: JavaSparkContext, * @param scans The HBase scan object to use to read data from HBase * @return New JavaRDD with results from scan */ - def hbaseRDD(tableName: TableName, - scans: Scan): - JavaRDD[(ImmutableBytesWritable, Result)] = { + def hbaseRDD(tableName: TableName, scans: Scan): JavaRDD[(ImmutableBytesWritable, Result)] = { JavaRDD.fromRDD(hbaseContext.hbaseRDD(tableName, scans)) } @@ -398,7 +413,6 @@ class JavaHBaseContext(@transient val jsc: JavaSparkContext, * but may lose primitive * specialization. */ - private[spark] - def fakeClassTag[T]: ClassTag[T] = ClassTag.AnyRef.asInstanceOf[ClassTag[T]] + private[spark] def fakeClassTag[T]: ClassTag[T] = ClassTag.AnyRef.asInstanceOf[ClassTag[T]] } diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/KeyFamilyQualifier.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/KeyFamilyQualifier.scala index 7fd5a629..d71e31a5 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/KeyFamilyQualifier.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/KeyFamilyQualifier.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,13 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark import java.io.Serializable - -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.util.Bytes +import org.apache.yetus.audience.InterfaceAudience /** * This is the key to be used for sorting and shuffling. @@ -32,8 +31,12 @@ import org.apache.hadoop.hbase.util.Bytes * @param qualifier Cell Qualifier */ @InterfaceAudience.Public -class KeyFamilyQualifier(val rowKey:Array[Byte], val family:Array[Byte], val qualifier:Array[Byte]) - extends Comparable[KeyFamilyQualifier] with Serializable { +class KeyFamilyQualifier( + val rowKey: Array[Byte], + val family: Array[Byte], + val qualifier: Array[Byte]) + extends Comparable[KeyFamilyQualifier] + with Serializable { override def compareTo(o: KeyFamilyQualifier): Int = { var result = Bytes.compareTo(rowKey, o.rowKey) if (result == 0) { diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/Logging.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/Logging.scala index a92f4e0c..18636313 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/Logging.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/Logging.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,13 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark import org.apache.yetus.audience.InterfaceAudience -import org.slf4j.impl.StaticLoggerBinder import org.slf4j.Logger import org.slf4j.LoggerFactory +import org.slf4j.impl.StaticLoggerBinder /** * Utility trait for classes that want to log data. Creates a SLF4J logger for the class and allows diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/NewHBaseRDD.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/NewHBaseRDD.scala index 7088ce90..aeb502d9 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/NewHBaseRDD.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/NewHBaseRDD.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,22 +15,23 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark import org.apache.hadoop.conf.Configuration -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.mapreduce.InputFormat -import org.apache.spark.rdd.NewHadoopRDD import org.apache.spark.{InterruptibleIterator, Partition, SparkContext, TaskContext} +import org.apache.spark.rdd.NewHadoopRDD +import org.apache.yetus.audience.InterfaceAudience @InterfaceAudience.Public -class NewHBaseRDD[K,V](@transient val sc : SparkContext, - @transient val inputFormatClass: Class[_ <: InputFormat[K, V]], - @transient val keyClass: Class[K], - @transient val valueClass: Class[V], - @transient private val __conf: Configuration, - val hBaseContext: HBaseContext) extends NewHadoopRDD(sc, inputFormatClass, keyClass, valueClass, __conf) { +class NewHBaseRDD[K, V]( + @transient val sc: SparkContext, + @transient val inputFormatClass: Class[_ <: InputFormat[K, V]], + @transient val keyClass: Class[K], + @transient val valueClass: Class[V], + @transient private val __conf: Configuration, + val hBaseContext: HBaseContext) + extends NewHadoopRDD(sc, inputFormatClass, keyClass, valueClass, __conf) { override def compute(theSplit: Partition, context: TaskContext): InterruptibleIterator[(K, V)] = { hBaseContext.applyCreds() diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/Bound.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/Bound.scala index af372eaf..ef0d287c 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/Bound.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/Bound.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,11 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark.datasources -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.spark.hbase._ +import org.apache.yetus.audience.InterfaceAudience /** * The Bound represent the boudary for the scan @@ -36,11 +36,13 @@ case class Range(lower: Option[Bound], upper: Option[Bound]) @InterfaceAudience.Private object Range { def apply(region: HBaseRegion): Range = { - Range(region.start.map(Bound(_, true)), if (region.end.get.size == 0) { - None - } else { - region.end.map((Bound(_, false))) - }) + Range( + region.start.map(Bound(_, true)), + if (region.end.get.size == 0) { + None + } else { + region.end.map((Bound(_, false))) + }) } } @@ -50,46 +52,65 @@ object Ranges { // 1. r.lower.inc is true, and r.upper.inc is false // 2. for each range in rs, its upper.inc is false def and(r: Range, rs: Seq[Range]): Seq[Range] = { - rs.flatMap{ s => - val lower = s.lower.map { x => - // the scan has lower bound - r.lower.map { y => - // the region has lower bound - if (ord.compare(x.b, y.b) < 0) { - // scan lower bound is smaller than region server lower bound - Some(y) - } else { - // scan low bound is greater or equal to region server lower bound - Some(x) + rs.flatMap { + s => + val lower = s.lower + .map { + x => + // the scan has lower bound + r.lower + .map { + y => + // the region has lower bound + if (ord.compare(x.b, y.b) < 0) { + // scan lower bound is smaller than region server lower bound + Some(y) + } else { + // scan low bound is greater or equal to region server lower bound + Some(x) + } + } + .getOrElse(Some(x)) } - }.getOrElse(Some(x)) - }.getOrElse(r.lower) + .getOrElse(r.lower) - val upper = s.upper.map { x => - // the scan has upper bound - r.upper.map { y => - // the region has upper bound - if (ord.compare(x.b, y.b) >= 0) { - // scan upper bound is larger than server upper bound - // but region server scan stop is exclusive. It is OK here. - Some(y) - } else { - // scan upper bound is less or equal to region server upper bound - Some(x) + val upper = s.upper + .map { + x => + // the scan has upper bound + r.upper + .map { + y => + // the region has upper bound + if (ord.compare(x.b, y.b) >= 0) { + // scan upper bound is larger than server upper bound + // but region server scan stop is exclusive. It is OK here. + Some(y) + } else { + // scan upper bound is less or equal to region server upper bound + Some(x) + } + } + .getOrElse(Some(x)) } - }.getOrElse(Some(x)) - }.getOrElse(r.upper) + .getOrElse(r.upper) - val c = lower.map { case x => - upper.map { case y => - ord.compare(x.b, y.b) - }.getOrElse(-1) - }.getOrElse(-1) - if (c < 0) { - Some(Range(lower, upper)) - } else { - None - } + val c = lower + .map { + case x => + upper + .map { + case y => + ord.compare(x.b, y.b) + } + .getOrElse(-1) + } + .getOrElse(-1) + if (c < 0) { + Some(Range(lower, upper)) + } else { + None + } }.seq } } @@ -97,24 +118,25 @@ object Ranges { @InterfaceAudience.Private object Points { def and(r: Range, ps: Seq[Array[Byte]]): Seq[Array[Byte]] = { - ps.flatMap { p => - if (ord.compare(r.lower.get.b, p) <= 0) { - // if region lower bound is less or equal to the point - if (r.upper.isDefined) { - // if region upper bound is defined - if (ord.compare(r.upper.get.b, p) > 0) { - // if the upper bound is greater than the point (because upper bound is exclusive) - Some(p) + ps.flatMap { + p => + if (ord.compare(r.lower.get.b, p) <= 0) { + // if region lower bound is less or equal to the point + if (r.upper.isDefined) { + // if region upper bound is defined + if (ord.compare(r.upper.get.b, p) > 0) { + // if the upper bound is greater than the point (because upper bound is exclusive) + Some(p) + } else { + None + } } else { - None + // if the region upper bound is not defined (infinity) + Some(p) } } else { - // if the region upper bound is not defined (infinity) - Some(p) + None } - } else { - None - } } } } diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/DataTypeParserWrapper.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/DataTypeParserWrapper.scala index c0ccc928..936276da 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/DataTypeParserWrapper.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/DataTypeParserWrapper.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark.datasources import org.apache.spark.sql.catalyst.parser.CatalystSqlParser @@ -27,6 +27,6 @@ trait DataTypeParser { } @InterfaceAudience.Private -object DataTypeParserWrapper extends DataTypeParser{ +object DataTypeParserWrapper extends DataTypeParser { def parse(dataTypeString: String): DataType = CatalystSqlParser.parseDataType(dataTypeString) } diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseResources.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseResources.scala index 0f467a78..a179d514 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseResources.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseResources.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,14 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark.datasources -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.TableName import org.apache.hadoop.hbase.client._ -import org.apache.hadoop.hbase.spark.{HBaseConnectionKey, SmartConnection, - HBaseConnectionCache, HBaseRelation} +import org.apache.hadoop.hbase.spark.{HBaseConnectionCache, HBaseConnectionKey, HBaseRelation, SmartConnection} +import org.apache.yetus.audience.InterfaceAudience import scala.language.implicitConversions // Resource and ReferencedResources are defined for extensibility, @@ -125,13 +124,16 @@ case class RegionResource(relation: HBaseRelation) extends ReferencedResource { var rl: RegionLocator = _ val regions = releaseOnException { val keys = rl.getStartEndKeys - keys.getFirst.zip(keys.getSecond) + keys.getFirst + .zip(keys.getSecond) .zipWithIndex - .map(x => - HBaseRegion(x._2, - Some(x._1._1), - Some(x._1._2), - Some(rl.getRegionLocation(x._1._1).getHostname))) + .map( + x => + HBaseRegion( + x._2, + Some(x._1._1), + Some(x._1._2), + Some(rl.getRegionLocation(x._1._1).getHostname))) } override def init(): Unit = { @@ -152,7 +154,7 @@ case class RegionResource(relation: HBaseRelation) extends ReferencedResource { } @InterfaceAudience.Private -object HBaseResources{ +object HBaseResources { implicit def ScanResToScan(sr: ScanResource): ResultScanner = { sr.rs } diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseSparkConf.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseSparkConf.scala index dc497f94..77c15316 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseSparkConf.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseSparkConf.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark.datasources import org.apache.yetus.audience.InterfaceAudience; @@ -25,38 +25,52 @@ import org.apache.yetus.audience.InterfaceAudience; * set in SparkConf. If not set, the default value will take effect. */ @InterfaceAudience.Public -object HBaseSparkConf{ - /** Set to false to disable server-side caching of blocks for this scan, +object HBaseSparkConf { + + /** + * Set to false to disable server-side caching of blocks for this scan, * false by default, since full table scans generate too much BC churn. */ val QUERY_CACHEBLOCKS = "hbase.spark.query.cacheblocks" val DEFAULT_QUERY_CACHEBLOCKS = false + /** The number of rows for caching that will be passed to scan. */ val QUERY_CACHEDROWS = "hbase.spark.query.cachedrows" + /** Set the maximum number of values to return for each call to next() in scan. */ val QUERY_BATCHSIZE = "hbase.spark.query.batchsize" + /** The number of BulkGets send to HBase. */ val BULKGET_SIZE = "hbase.spark.bulkget.size" val DEFAULT_BULKGET_SIZE = 1000 + /** Set to specify the location of hbase configuration file. */ val HBASE_CONFIG_LOCATION = "hbase.spark.config.location" - /** Set to specify whether create or use latest cached HBaseContext*/ + + /** Set to specify whether create or use latest cached HBaseContext */ val USE_HBASECONTEXT = "hbase.spark.use.hbasecontext" val DEFAULT_USE_HBASECONTEXT = true + /** Pushdown the filter to data source engine to increase the performance of queries. */ val PUSHDOWN_COLUMN_FILTER = "hbase.spark.pushdown.columnfilter" - val DEFAULT_PUSHDOWN_COLUMN_FILTER= true + val DEFAULT_PUSHDOWN_COLUMN_FILTER = true + /** Class name of the encoder, which encode data types from Spark to HBase bytes. */ val QUERY_ENCODER = "hbase.spark.query.encoder" val DEFAULT_QUERY_ENCODER = classOf[NaiveEncoder].getCanonicalName + /** The timestamp used to filter columns with a specific timestamp. */ val TIMESTAMP = "hbase.spark.query.timestamp" + /** The starting timestamp used to filter columns with a specific range of versions. */ val TIMERANGE_START = "hbase.spark.query.timerange.start" + /** The ending timestamp used to filter columns with a specific range of versions. */ - val TIMERANGE_END = "hbase.spark.query.timerange.end" + val TIMERANGE_END = "hbase.spark.query.timerange.end" + /** The maximum number of version to return. */ val MAX_VERSIONS = "hbase.spark.query.maxVersions" + /** Delayed time to close hbase-spark connection when no reference to this connection, in milliseconds. */ val DEFAULT_CONNECTION_CLOSE_DELAY = 10 * 60 * 1000 } diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableCatalog.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableCatalog.scala index 6ac0454c..d76a2453 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableCatalog.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableCatalog.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,15 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark.datasources import org.apache.avro.Schema -import org.apache.yetus.audience.InterfaceAudience import org.apache.hadoop.hbase.spark.{Logging, SchemaConverters} import org.apache.hadoop.hbase.util.Bytes import org.apache.spark.sql.types._ - +import org.apache.yetus.audience.InterfaceAudience import scala.collection.mutable import scala.util.parsing.json.JSON @@ -35,15 +34,17 @@ case class Field( col: String, sType: Option[String] = None, avroSchema: Option[String] = None, - serdes: Option[SerDes]= None, - len: Int = -1) extends Logging { + serdes: Option[SerDes] = None, + len: Int = -1) + extends Logging { override def toString = s"$colName $cf $col" val isRowKey = cf == HBaseTableCatalog.rowKey var start: Int = _ - def schema: Option[Schema] = avroSchema.map { x => - logDebug(s"avro: $x") - val p = new Schema.Parser - p.parse(x) + def schema: Option[Schema] = avroSchema.map { + x => + logDebug(s"avro: $x") + val p = new Schema.Parser + p.parse(x) } lazy val exeSchema = schema @@ -54,7 +55,7 @@ case class Field( } // converter from catalyst to avro - lazy val catalystToAvro: (Any) => Any ={ + lazy val catalystToAvro: (Any) => Any = { SchemaConverters.createConverterToAvro(dt, colName, "recordNamespace") } @@ -74,10 +75,8 @@ case class Field( } val dt = { - sType.map(DataTypeParserWrapper.parse(_)).getOrElse{ - schema.map{ x=> - SchemaConverters.toSqlType(x).dataType - }.get + sType.map(DataTypeParserWrapper.parse(_)).getOrElse { + schema.map { x => SchemaConverters.toSqlType(x).dataType }.get } } @@ -118,8 +117,9 @@ case class RowKey(k: String) { if (varLength) { -1 } else { - fields.foldLeft(0){case (x, y) => - x + y.length + fields.foldLeft(0) { + case (x, y) => + x + y.length } } } @@ -127,8 +127,9 @@ case class RowKey(k: String) { // The map between the column presented to Spark and the HBase field @InterfaceAudience.Private case class SchemaMap(map: mutable.HashMap[String, Field]) { - def toFields = map.map { case (name, field) => - StructField(name, field.dt) + def toFields = map.map { + case (name, field) => + StructField(name, field.dt) }.toSeq def fields = map.values @@ -136,19 +137,19 @@ case class SchemaMap(map: mutable.HashMap[String, Field]) { def getField(name: String) = map(name) } - // The definition of HBase and Relation relation schema @InterfaceAudience.Private case class HBaseTableCatalog( - namespace: String, - name: String, - row: RowKey, - sMap: SchemaMap, - @transient params: Map[String, String]) extends Logging { + namespace: String, + name: String, + row: RowKey, + sMap: SchemaMap, + @transient params: Map[String, String]) + extends Logging { def toDataType = StructType(sMap.toFields) def getField(name: String) = sMap.getField(name) def getRowKey: Seq[Field] = row.fields - def getPrimaryKey= row.keys(0) + def getPrimaryKey = row.keys(0) def getColumnFamilies = { sMap.fields.map(_.cf).filter(_ != HBaseTableCatalog.rowKey).toSeq.distinct } @@ -158,30 +159,31 @@ case class HBaseTableCatalog( // Setup the start and length for each dimension of row key at runtime. def dynSetupRowKey(rowKey: Array[Byte]) { logDebug(s"length: ${rowKey.length}") - if(row.varLength) { + if (row.varLength) { var start = 0 - row.fields.foreach { f => - logDebug(s"start: $start") - f.start = start - f.length = { - // If the length is not defined - if (f.length == -1) { - f.dt match { - case StringType => - var pos = rowKey.indexOf(HBaseTableCatalog.delimiter, start) - if (pos == -1 || pos > rowKey.length) { - // this is at the last dimension - pos = rowKey.length - } - pos - start - // We don't know the length, assume it extend to the end of the rowkey. - case _ => rowKey.length - start + row.fields.foreach { + f => + logDebug(s"start: $start") + f.start = start + f.length = { + // If the length is not defined + if (f.length == -1) { + f.dt match { + case StringType => + var pos = rowKey.indexOf(HBaseTableCatalog.delimiter, start) + if (pos == -1 || pos > rowKey.length) { + // this is at the last dimension + pos = rowKey.length + } + pos - start + // We don't know the length, assume it extend to the end of the rowkey. + case _ => rowKey.length - start + } + } else { + f.length } - } else { - f.length } - } - start += f.length + start += f.length } } } @@ -192,9 +194,10 @@ case class HBaseTableCatalog( // The length is determined at run time if it is string or binary and the length is undefined. if (row.fields.filter(_.length == -1).isEmpty) { var start = 0 - row.fields.foreach { f => - f.start = start - start += f.length + row.fields.foreach { + f => + f.start = start + start += f.length } } else { row.varLength = true @@ -233,12 +236,12 @@ object HBaseTableCatalog { val length = "length" /** - * User provide table schema definition - * {"tablename":"name", "rowkey":"key1:key2", - * "columns":{"col1":{"cf":"cf1", "col":"col1", "type":"type1"}, - * "col2":{"cf":"cf2", "col":"col2", "type":"type2"}}} - * Note that any col in the rowKey, there has to be one corresponding col defined in columns - */ + * User provide table schema definition + * {"tablename":"name", "rowkey":"key1:key2", + * "columns":{"col1":{"cf":"cf1", "col":"col1", "type":"type1"}, + * "col2":{"cf":"cf2", "col":"col2", "type":"type2"}}} + * Note that any col in the rowKey, there has to be one corresponding col defined in columns + */ def apply(params: Map[String, String]): HBaseTableCatalog = { val parameters = convert(params) // println(jString) @@ -249,19 +252,25 @@ object HBaseTableCatalog { val tName = tableMeta.get(tableName).get.asInstanceOf[String] val cIter = map.get(columns).get.asInstanceOf[Map[String, Map[String, String]]].toIterator val schemaMap = mutable.HashMap.empty[String, Field] - cIter.foreach { case (name, column) => - val sd = { - column.get(serdes).asInstanceOf[Option[String]].map(n => - Class.forName(n).newInstance().asInstanceOf[SerDes] - ) - } - val len = column.get(length).map(_.toInt).getOrElse(-1) - val sAvro = column.get(avro).map(parameters(_)) - val f = Field(name, column.getOrElse(cf, rowKey), - column.get(col).get, - column.get(`type`), - sAvro, sd, len) - schemaMap.+=((name, f)) + cIter.foreach { + case (name, column) => + val sd = { + column + .get(serdes) + .asInstanceOf[Option[String]] + .map(n => Class.forName(n).newInstance().asInstanceOf[SerDes]) + } + val len = column.get(length).map(_.toInt).getOrElse(-1) + val sAvro = column.get(avro).map(parameters(_)) + val f = Field( + name, + column.getOrElse(cf, rowKey), + column.get(col).get, + column.get(`type`), + sAvro, + sd, + len) + schemaMap.+=((name, f)) } val rKey = RowKey(map.get(rowKey).get.asInstanceOf[String]) HBaseTableCatalog(nSpace, tName, rKey, SchemaMap(schemaMap), parameters) @@ -294,23 +303,28 @@ object HBaseTableCatalog { if (nsTableName == null) return parameters val tableParts = nsTableName.trim.split(':') val tableNamespace = if (tableParts.length == 1) { - "default" + "default" } else if (tableParts.length == 2) { - tableParts(0) + tableParts(0) } else { - throw new IllegalArgumentException("Invalid table name '" + nsTableName + - "' should be ':' or '' ") + throw new IllegalArgumentException( + "Invalid table name '" + nsTableName + + "' should be ':' or '' ") } val tableName = tableParts(tableParts.length - 1) val schemaMappingString = parameters.getOrElse(SCHEMA_COLUMNS_MAPPING_KEY, "") import scala.collection.JavaConverters._ - val schemaMap = generateSchemaMappingMap(schemaMappingString).asScala.map(_._2.asInstanceOf[SchemaQualifierDefinition]) + val schemaMap = generateSchemaMappingMap(schemaMappingString).asScala.map( + _._2.asInstanceOf[SchemaQualifierDefinition]) - val rowkey = schemaMap.filter { - _.columnFamily == "rowkey" - }.map(_.columnName) - val cols = schemaMap.map { x => - s""""${x.columnName}":{"cf":"${x.columnFamily}", "col":"${x.qualifier}", "type":"${x.colType}"}""".stripMargin + val rowkey = schemaMap + .filter { + _.columnFamily == "rowkey" + } + .map(_.columnName) + val cols = schemaMap.map { + x => + s""""${x.columnName}":{"cf":"${x.columnFamily}", "col":"${x.qualifier}", "type":"${x.colType}"}""".stripMargin } val jsonCatalog = s"""{ @@ -321,62 +335,72 @@ object HBaseTableCatalog { |} |} """.stripMargin - parameters ++ Map(HBaseTableCatalog.tableCatalog->jsonCatalog) + parameters ++ Map(HBaseTableCatalog.tableCatalog -> jsonCatalog) } /** - * Reads the SCHEMA_COLUMNS_MAPPING_KEY and converts it to a map of - * SchemaQualifierDefinitions with the original sql column name as the key - * - * @param schemaMappingString The schema mapping string from the SparkSQL map - * @return A map of definitions keyed by the SparkSQL column name - */ + * Reads the SCHEMA_COLUMNS_MAPPING_KEY and converts it to a map of + * SchemaQualifierDefinitions with the original sql column name as the key + * + * @param schemaMappingString The schema mapping string from the SparkSQL map + * @return A map of definitions keyed by the SparkSQL column name + */ @InterfaceAudience.Private - def generateSchemaMappingMap(schemaMappingString:String): - java.util.HashMap[String, SchemaQualifierDefinition] = { + def generateSchemaMappingMap( + schemaMappingString: String): java.util.HashMap[String, SchemaQualifierDefinition] = { println(schemaMappingString) try { val columnDefinitions = schemaMappingString.split(',') val resultingMap = new java.util.HashMap[String, SchemaQualifierDefinition]() - columnDefinitions.map(cd => { - val parts = cd.trim.split(' ') + columnDefinitions.map( + cd => { + val parts = cd.trim.split(' ') - //Make sure we get three parts - // - if (parts.length == 3) { - val hbaseDefinitionParts = if (parts(2).charAt(0) == ':') { - Array[String]("rowkey", parts(0)) + // Make sure we get three parts + // + if (parts.length == 3) { + val hbaseDefinitionParts = if (parts(2).charAt(0) == ':') { + Array[String]("rowkey", parts(0)) + } else { + parts(2).split(':') + } + resultingMap.put( + parts(0), + new SchemaQualifierDefinition( + parts(0), + parts(1), + hbaseDefinitionParts(0), + hbaseDefinitionParts(1))) } else { - parts(2).split(':') + throw new IllegalArgumentException( + "Invalid value for schema mapping '" + cd + + "' should be ' :' " + + "for columns and ' :' for rowKeys") } - resultingMap.put(parts(0), new SchemaQualifierDefinition(parts(0), - parts(1), hbaseDefinitionParts(0), hbaseDefinitionParts(1))) - } else { - throw new IllegalArgumentException("Invalid value for schema mapping '" + cd + - "' should be ' :' " + - "for columns and ' :' for rowKeys") - } - }) + }) resultingMap } catch { - case e:Exception => throw - new IllegalArgumentException("Invalid value for " + SCHEMA_COLUMNS_MAPPING_KEY + - " '" + - schemaMappingString + "'", e ) + case e: Exception => + throw new IllegalArgumentException( + "Invalid value for " + SCHEMA_COLUMNS_MAPPING_KEY + + " '" + + schemaMappingString + "'", + e) } } } /** - * Construct to contains column data that spend SparkSQL and HBase - * - * @param columnName SparkSQL column name - * @param colType SparkSQL column type - * @param columnFamily HBase column family - * @param qualifier HBase qualifier name - */ + * Construct to contains column data that spend SparkSQL and HBase + * + * @param columnName SparkSQL column name + * @param colType SparkSQL column type + * @param columnFamily HBase column family + * @param qualifier HBase qualifier name + */ @InterfaceAudience.Private -case class SchemaQualifierDefinition(columnName:String, - colType:String, - columnFamily:String, - qualifier:String) +case class SchemaQualifierDefinition( + columnName: String, + colType: String, + columnFamily: String, + qualifier: String) diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala index c334076f..0d7ec0d0 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,29 +15,26 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark.datasources import java.util.ArrayList - -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client._ import org.apache.hadoop.hbase.spark._ -import org.apache.hadoop.hbase.spark.hbase._ import org.apache.hadoop.hbase.spark.datasources.HBaseResources._ +import org.apache.hadoop.hbase.spark.hbase._ import org.apache.hadoop.hbase.util.ShutdownHookManager -import org.apache.spark.{SparkEnv, TaskContext, Partition} +import org.apache.spark.{Partition, SparkEnv, TaskContext} import org.apache.spark.rdd.RDD - +import org.apache.yetus.audience.InterfaceAudience import scala.collection.mutable @InterfaceAudience.Private -class HBaseTableScanRDD(relation: HBaseRelation, - val hbaseContext: HBaseContext, - @transient val filter: Option[SparkSQLPushDownFilter] = None, - val columns: Seq[Field] = Seq.empty - ) extends RDD[Result](relation.sqlContext.sparkContext, Nil) - { +class HBaseTableScanRDD( + relation: HBaseRelation, + val hbaseContext: HBaseContext, + @transient val filter: Option[SparkSQLPushDownFilter] = None, + val columns: Seq[Field] = Seq.empty) + extends RDD[Result](relation.sqlContext.sparkContext, Nil) { private def sparkConf = SparkEnv.get.conf @transient var ranges = Seq.empty[Range] @transient var points = Seq.empty[Array[Byte]] @@ -60,7 +58,7 @@ class HBaseTableScanRDD(relation: HBaseRelation, val newArray = new Array[Byte](r.upperBound.length + 1) System.arraycopy(r.upperBound, 0, newArray, 0, r.upperBound.length) - //New Max Bytes + // New Max Bytes newArray(r.upperBound.length) = ByteMin Some(Bound(newArray, false)) } @@ -74,36 +72,50 @@ class HBaseTableScanRDD(relation: HBaseRelation, val regions = RegionResource(relation) var idx = 0 logDebug(s"There are ${regions.size} regions") - val ps = regions.flatMap { x => - val rs = Ranges.and(Range(x), ranges) - val ps = Points.and(Range(x), points) - if (rs.size > 0 || ps.size > 0) { - if(log.isDebugEnabled) { - rs.foreach(x => logDebug(x.toString)) + val ps = regions.flatMap { + x => + val rs = Ranges.and(Range(x), ranges) + val ps = Points.and(Range(x), points) + if (rs.size > 0 || ps.size > 0) { + if (log.isDebugEnabled) { + rs.foreach(x => logDebug(x.toString)) + } + idx += 1 + Some( + HBaseScanPartition( + idx - 1, + x, + rs, + ps, + SerializedFilter.toSerializedTypedFilter(filter))) + } else { + None } - idx += 1 - Some(HBaseScanPartition(idx - 1, x, rs, ps, SerializedFilter.toSerializedTypedFilter(filter))) - } else { - None - } }.toArray if (log.isDebugEnabled) { logDebug(s"Partitions: ${ps.size}"); ps.foreach(x => logDebug(x.toString)) } regions.release() - ShutdownHookManager.affixShutdownHook( new Thread() { - override def run() { - HBaseConnectionCache.close() - } - }, 0) + ShutdownHookManager.affixShutdownHook( + new Thread() { + override def run() { + HBaseConnectionCache.close() + } + }, + 0) ps.asInstanceOf[Array[Partition]] } override def getPreferredLocations(split: Partition): Seq[String] = { - split.asInstanceOf[HBaseScanPartition].regions.server.map { - identity - }.toSeq + split + .asInstanceOf[HBaseScanPartition] + .regions + .server + .map { + identity + } + .toSeq } private def buildGets( @@ -112,27 +124,30 @@ class HBaseTableScanRDD(relation: HBaseRelation, filter: Option[SparkSQLPushDownFilter], columns: Seq[Field], hbaseContext: HBaseContext): Iterator[Result] = { - g.grouped(relation.bulkGetSize).flatMap{ x => - val gets = new ArrayList[Get](x.size) - val rowkeySet = new mutable.HashSet[String]() - x.foreach{ y => - if (!rowkeySet.contains(y.mkString("Array(", ", ", ")"))) { - val g = new Get(y) - handleTimeSemantics(g) - columns.foreach { d => - if (!d.isRowKey) { - g.addColumn(d.cfBytes, d.colBytes) + g.grouped(relation.bulkGetSize).flatMap { + x => + val gets = new ArrayList[Get](x.size) + val rowkeySet = new mutable.HashSet[String]() + x.foreach { + y => + if (!rowkeySet.contains(y.mkString("Array(", ", ", ")"))) { + val g = new Get(y) + handleTimeSemantics(g) + columns.foreach { + d => + if (!d.isRowKey) { + g.addColumn(d.cfBytes, d.colBytes) + } + } + filter.foreach(g.setFilter(_)) + gets.add(g) + rowkeySet.add(y.mkString("Array(", ", ", ")")) } - } - filter.foreach(g.setFilter(_)) - gets.add(g) - rowkeySet.add(y.mkString("Array(", ", ", ")")) } - } - hbaseContext.applyCreds() - val tmp = tbr.get(gets) - rddResources.addResource(tmp) - toResultIterator(tmp) + hbaseContext.applyCreds() + val tmp = tbr.get(gets) + rddResources.addResource(tmp) + toResultIterator(tmp) } } @@ -141,7 +156,7 @@ class HBaseTableScanRDD(relation: HBaseRelation, var idx = 0 var cur: Option[Result] = None override def hasNext: Boolean = { - while(idx < result.length && cur.isEmpty) { + while (idx < result.length && cur.isEmpty) { val r = result(idx) idx += 1 if (!r.isEmpty) { @@ -163,7 +178,8 @@ class HBaseTableScanRDD(relation: HBaseRelation, iterator } - private def buildScan(range: Range, + private def buildScan( + range: Range, filter: Option[SparkSQLPushDownFilter], columns: Seq[Field]): Scan = { val scan = (range.lower, range.upper) match { @@ -174,10 +190,11 @@ class HBaseTableScanRDD(relation: HBaseRelation, } handleTimeSemantics(scan) - columns.foreach { d => - if (!d.isRowKey) { - scan.addColumn(d.cfBytes, d.colBytes) - } + columns.foreach { + d => + if (!d.isRowKey) { + scan.addColumn(d.cfBytes, d.colBytes) + } } scan.setCacheBlocks(relation.blockCacheEnable) scan.setBatch(relation.batchNum) @@ -223,7 +240,7 @@ class HBaseTableScanRDD(relation: HBaseRelation, val tableResource = TableResource(relation) context.addTaskCompletionListener[Unit](context => close()) val points = partition.points - val gIt: Iterator[Result] = { + val gIt: Iterator[Result] = { if (points.isEmpty) { Iterator.empty: Iterator[Result] } else { @@ -231,37 +248,44 @@ class HBaseTableScanRDD(relation: HBaseRelation, } } val rIts = scans.par - .map { scan => - hbaseContext.applyCreds() - val scanner = tableResource.getScanner(scan) - rddResources.addResource(scanner) - scanner - }.map(toResultIterator(_)) - .fold(Iterator.empty: Iterator[Result]){ case (x, y) => - x ++ y - } ++ gIt - ShutdownHookManager.affixShutdownHook( new Thread() { - override def run() { - HBaseConnectionCache.close() + .map { + scan => + hbaseContext.applyCreds() + val scanner = tableResource.getScanner(scan) + rddResources.addResource(scanner) + scanner } - }, 0) + .map(toResultIterator(_)) + .fold(Iterator.empty: Iterator[Result]) { + case (x, y) => + x ++ y + } ++ gIt + ShutdownHookManager.affixShutdownHook( + new Thread() { + override def run() { + HBaseConnectionCache.close() + } + }, + 0) rIts } private def handleTimeSemantics(query: Query): Unit = { // Set timestamp related values if present - (query, relation.timestamp, relation.minTimestamp, relation.maxTimestamp) match { + (query, relation.timestamp, relation.minTimestamp, relation.maxTimestamp) match { case (q: Scan, Some(ts), None, None) => q.setTimeStamp(ts) case (q: Get, Some(ts), None, None) => q.setTimeStamp(ts) - case (q:Scan, None, Some(minStamp), Some(maxStamp)) => q.setTimeRange(minStamp, maxStamp) - case (q:Get, None, Some(minStamp), Some(maxStamp)) => q.setTimeRange(minStamp, maxStamp) + case (q: Scan, None, Some(minStamp), Some(maxStamp)) => q.setTimeRange(minStamp, maxStamp) + case (q: Get, None, Some(minStamp), Some(maxStamp)) => q.setTimeRange(minStamp, maxStamp) case (q, None, None, None) => - case _ => throw new IllegalArgumentException(s"Invalid combination of query/timestamp/time range provided. " + - s"timeStamp is: ${relation.timestamp.get}, minTimeStamp is: ${relation.minTimestamp.get}, " + - s"maxTimeStamp is: ${relation.maxTimestamp.get}") + case _ => + throw new IllegalArgumentException( + s"Invalid combination of query/timestamp/time range provided. " + + s"timeStamp is: ${relation.timestamp.get}, minTimeStamp is: ${relation.minTimestamp.get}, " + + s"maxTimeStamp is: ${relation.maxTimestamp.get}") } if (relation.maxVersions.isDefined) { query match { @@ -291,7 +315,8 @@ private[hbase] case class HBaseRegion( override val index: Int, val start: Option[HBaseType] = None, val end: Option[HBaseType] = None, - val server: Option[String] = None) extends Partition + val server: Option[String] = None) + extends Partition @InterfaceAudience.Private private[hbase] case class HBaseScanPartition( @@ -299,7 +324,8 @@ private[hbase] case class HBaseScanPartition( val regions: HBaseRegion, val scanRanges: Seq[Range], val points: Seq[Array[Byte]], - val sf: SerializedFilter) extends Partition + val sf: SerializedFilter) + extends Partition @InterfaceAudience.Private case class RDDResources(set: mutable.HashSet[Resource]) { diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/JavaBytesEncoder.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/JavaBytesEncoder.scala index a8d804d6..eac4feb6 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/JavaBytesEncoder.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/JavaBytesEncoder.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,95 +15,99 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark.datasources import org.apache.hadoop.hbase.HBaseInterfaceAudience import org.apache.hadoop.hbase.spark.Logging import org.apache.hadoop.hbase.spark.datasources.JavaBytesEncoder.JavaBytesEncoder +import org.apache.spark.sql.types._ import org.apache.yetus.audience.InterfaceAudience import org.apache.yetus.audience.InterfaceStability -import org.apache.spark.sql.types._ /** - * The ranges for the data type whose size is known. Whether the bound is inclusive - * or exclusive is undefind, and upper to the caller to decide. - * - * @param low: the lower bound of the range. - * @param upper: the upper bound of the range. - */ + * The ranges for the data type whose size is known. Whether the bound is inclusive + * or exclusive is undefind, and upper to the caller to decide. + * + * @param low: the lower bound of the range. + * @param upper: the upper bound of the range. + */ @InterfaceAudience.LimitedPrivate(Array(HBaseInterfaceAudience.SPARK)) @InterfaceStability.Evolving -case class BoundRange(low: Array[Byte],upper: Array[Byte]) +case class BoundRange(low: Array[Byte], upper: Array[Byte]) /** - * The class identifies the ranges for a java primitive type. The caller needs - * to decide the bound is either inclusive or exclusive on its own. - * information - * - * @param less: the set of ranges for LessThan/LessOrEqualThan - * @param greater: the set of ranges for GreaterThan/GreaterThanOrEqualTo - * @param value: the byte array of the original value - */ + * The class identifies the ranges for a java primitive type. The caller needs + * to decide the bound is either inclusive or exclusive on its own. + * information + * + * @param less: the set of ranges for LessThan/LessOrEqualThan + * @param greater: the set of ranges for GreaterThan/GreaterThanOrEqualTo + * @param value: the byte array of the original value + */ @InterfaceAudience.LimitedPrivate(Array(HBaseInterfaceAudience.SPARK)) @InterfaceStability.Evolving case class BoundRanges(less: Array[BoundRange], greater: Array[BoundRange], value: Array[Byte]) /** - * The trait to support plugin architecture for different encoder/decoder. - * encode is used for serializing the data type to byte array and the filter is - * used to filter out the unnecessary records. - */ + * The trait to support plugin architecture for different encoder/decoder. + * encode is used for serializing the data type to byte array and the filter is + * used to filter out the unnecessary records. + */ @InterfaceAudience.LimitedPrivate(Array(HBaseInterfaceAudience.SPARK)) @InterfaceStability.Evolving trait BytesEncoder { def encode(dt: DataType, value: Any): Array[Byte] /** - * The function performing real filtering operations. The format of filterBytes depends on the - * implementation of the BytesEncoder. - * - * @param input: the current input byte array that needs to be filtered out - * @param offset1: the starting offset of the input byte array. - * @param length1: the length of the input byte array. - * @param filterBytes: the byte array provided by query condition. - * @param offset2: the starting offset in the filterBytes. - * @param length2: the length of the bytes in the filterBytes - * @param ops: The operation of the filter operator. - * @return true: the record satisfies the predicates - * false: the record does not satisfy the predicates. - */ - def filter(input: Array[Byte], offset1: Int, length1: Int, - filterBytes: Array[Byte], offset2: Int, length2: Int, - ops: JavaBytesEncoder): Boolean + * The function performing real filtering operations. The format of filterBytes depends on the + * implementation of the BytesEncoder. + * + * @param input: the current input byte array that needs to be filtered out + * @param offset1: the starting offset of the input byte array. + * @param length1: the length of the input byte array. + * @param filterBytes: the byte array provided by query condition. + * @param offset2: the starting offset in the filterBytes. + * @param length2: the length of the bytes in the filterBytes + * @param ops: The operation of the filter operator. + * @return true: the record satisfies the predicates + * false: the record does not satisfy the predicates. + */ + def filter( + input: Array[Byte], + offset1: Int, + length1: Int, + filterBytes: Array[Byte], + offset2: Int, + length2: Int, + ops: JavaBytesEncoder): Boolean /** - * Currently, it is used for partition pruning. - * As for some codec, the order may be inconsistent between java primitive - * type and its byte array. We may have to split the predicates on some - * of the java primitive type into multiple predicates. - * - * For example in naive codec, some of the java primitive types have to be - * split into multiple predicates, and union these predicates together to - * make the predicates be performed correctly. - * For example, if we have "COLUMN < 2", we will transform it into - * "0 <= COLUMN < 2 OR Integer.MIN_VALUE <= COLUMN <= -1" - */ + * Currently, it is used for partition pruning. + * As for some codec, the order may be inconsistent between java primitive + * type and its byte array. We may have to split the predicates on some + * of the java primitive type into multiple predicates. + * + * For example in naive codec, some of the java primitive types have to be + * split into multiple predicates, and union these predicates together to + * make the predicates be performed correctly. + * For example, if we have "COLUMN < 2", we will transform it into + * "0 <= COLUMN < 2 OR Integer.MIN_VALUE <= COLUMN <= -1" + */ def ranges(in: Any): Option[BoundRanges] } @InterfaceAudience.LimitedPrivate(Array(HBaseInterfaceAudience.SPARK)) @InterfaceStability.Evolving -object JavaBytesEncoder extends Enumeration with Logging{ +object JavaBytesEncoder extends Enumeration with Logging { type JavaBytesEncoder = Value val Greater, GreaterEqual, Less, LessEqual, Equal, Unknown = Value /** - * create the encoder/decoder - * - * @param clsName: the class name of the encoder/decoder class - * @return the instance of the encoder plugin. - */ + * create the encoder/decoder + * + * @param clsName: the class name of the encoder/decoder class + * @return the instance of the encoder plugin. + */ def create(clsName: String): BytesEncoder = { try { Class.forName(clsName).newInstance.asInstanceOf[BytesEncoder] diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/NaiveEncoder.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/NaiveEncoder.scala index 67328696..b54d2797 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/NaiveEncoder.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/NaiveEncoder.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.hadoop.hbase.spark.datasources /* * Licensed to the Apache Software Foundation (ASF) under one or more @@ -16,24 +33,23 @@ package org.apache.hadoop.hbase.spark.datasources * limitations under the License. */ -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.spark.datasources.JavaBytesEncoder.JavaBytesEncoder import org.apache.hadoop.hbase.spark.Logging +import org.apache.hadoop.hbase.spark.datasources.JavaBytesEncoder.JavaBytesEncoder import org.apache.hadoop.hbase.spark.hbase._ import org.apache.hadoop.hbase.util.Bytes import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String - +import org.apache.yetus.audience.InterfaceAudience /** - * This is the naive non-order preserving encoder/decoder. - * Due to the inconsistency of the order between java primitive types - * and their bytearray. The data type has to be passed in so that the filter - * can work correctly, which is done by wrapping the type into the first byte - * of the serialized array. - */ + * This is the naive non-order preserving encoder/decoder. + * Due to the inconsistency of the order between java primitive types + * and their bytearray. The data type has to be passed in so that the filter + * can work correctly, which is done by wrapping the type into the first byte + * of the serialized array. + */ @InterfaceAudience.Private -class NaiveEncoder extends BytesEncoder with Logging{ +class NaiveEncoder extends BytesEncoder with Logging { var code = 0 def nextCode: Byte = { code += 1 @@ -50,128 +66,148 @@ class NaiveEncoder extends BytesEncoder with Logging{ val TimestampEnc = nextCode val UnknownEnc = nextCode - /** - * Evaluate the java primitive type and return the BoundRanges. For one value, it may have - * multiple output ranges because of the inconsistency of order between java primitive type - * and its byte array order. - * - * For short, integer, and long, the order of number is consistent with byte array order - * if two number has the same sign bit. But the negative number is larger than positive - * number in byte array. - * - * For double and float, the order of positive number is consistent with its byte array order. - * But the order of negative number is the reverse order of byte array. Please refer to IEEE-754 - * and https://en.wikipedia.org/wiki/Single-precision_floating-point_format - */ + * Evaluate the java primitive type and return the BoundRanges. For one value, it may have + * multiple output ranges because of the inconsistency of order between java primitive type + * and its byte array order. + * + * For short, integer, and long, the order of number is consistent with byte array order + * if two number has the same sign bit. But the negative number is larger than positive + * number in byte array. + * + * For double and float, the order of positive number is consistent with its byte array order. + * But the order of negative number is the reverse order of byte array. Please refer to IEEE-754 + * and https://en.wikipedia.org/wiki/Single-precision_floating-point_format + */ def ranges(in: Any): Option[BoundRanges] = in match { case a: Integer => - val b = Bytes.toBytes(a) + val b = Bytes.toBytes(a) if (a >= 0) { logDebug(s"range is 0 to $a and ${Integer.MIN_VALUE} to -1") - Some(BoundRanges( - Array(BoundRange(Bytes.toBytes(0: Int), b), - BoundRange(Bytes.toBytes(Integer.MIN_VALUE), Bytes.toBytes(-1: Int))), - Array(BoundRange(b, Bytes.toBytes(Integer.MAX_VALUE))), b)) + Some( + BoundRanges( + Array( + BoundRange(Bytes.toBytes(0: Int), b), + BoundRange(Bytes.toBytes(Integer.MIN_VALUE), Bytes.toBytes(-1: Int))), + Array(BoundRange(b, Bytes.toBytes(Integer.MAX_VALUE))), + b)) } else { - Some(BoundRanges( - Array(BoundRange(Bytes.toBytes(Integer.MIN_VALUE), b)), - Array(BoundRange(b, Bytes.toBytes(-1: Integer)), - BoundRange(Bytes.toBytes(0: Int), Bytes.toBytes(Integer.MAX_VALUE))), b)) + Some( + BoundRanges( + Array(BoundRange(Bytes.toBytes(Integer.MIN_VALUE), b)), + Array( + BoundRange(b, Bytes.toBytes(-1: Integer)), + BoundRange(Bytes.toBytes(0: Int), Bytes.toBytes(Integer.MAX_VALUE))), + b)) } case a: Long => - val b = Bytes.toBytes(a) + val b = Bytes.toBytes(a) if (a >= 0) { - Some(BoundRanges( - Array(BoundRange(Bytes.toBytes(0: Long), b), - BoundRange(Bytes.toBytes(Long.MinValue), Bytes.toBytes(-1: Long))), - Array(BoundRange(b, Bytes.toBytes(Long.MaxValue))), b)) + Some( + BoundRanges( + Array( + BoundRange(Bytes.toBytes(0: Long), b), + BoundRange(Bytes.toBytes(Long.MinValue), Bytes.toBytes(-1: Long))), + Array(BoundRange(b, Bytes.toBytes(Long.MaxValue))), + b)) } else { - Some(BoundRanges( - Array(BoundRange(Bytes.toBytes(Long.MinValue), b)), - Array(BoundRange(b, Bytes.toBytes(-1: Long)), - BoundRange(Bytes.toBytes(0: Long), Bytes.toBytes(Long.MaxValue))), b)) + Some( + BoundRanges( + Array(BoundRange(Bytes.toBytes(Long.MinValue), b)), + Array( + BoundRange(b, Bytes.toBytes(-1: Long)), + BoundRange(Bytes.toBytes(0: Long), Bytes.toBytes(Long.MaxValue))), + b)) } case a: Short => - val b = Bytes.toBytes(a) + val b = Bytes.toBytes(a) if (a >= 0) { - Some(BoundRanges( - Array(BoundRange(Bytes.toBytes(0: Short), b), - BoundRange(Bytes.toBytes(Short.MinValue), Bytes.toBytes(-1: Short))), - Array(BoundRange(b, Bytes.toBytes(Short.MaxValue))), b)) + Some( + BoundRanges( + Array( + BoundRange(Bytes.toBytes(0: Short), b), + BoundRange(Bytes.toBytes(Short.MinValue), Bytes.toBytes(-1: Short))), + Array(BoundRange(b, Bytes.toBytes(Short.MaxValue))), + b)) } else { - Some(BoundRanges( - Array(BoundRange(Bytes.toBytes(Short.MinValue), b)), - Array(BoundRange(b, Bytes.toBytes(-1: Short)), - BoundRange(Bytes.toBytes(0: Short), Bytes.toBytes(Short.MaxValue))), b)) + Some( + BoundRanges( + Array(BoundRange(Bytes.toBytes(Short.MinValue), b)), + Array( + BoundRange(b, Bytes.toBytes(-1: Short)), + BoundRange(Bytes.toBytes(0: Short), Bytes.toBytes(Short.MaxValue))), + b)) } case a: Double => - val b = Bytes.toBytes(a) + val b = Bytes.toBytes(a) if (a >= 0.0f) { - Some(BoundRanges( - Array(BoundRange(Bytes.toBytes(0.0d), b), - BoundRange(Bytes.toBytes(-0.0d), Bytes.toBytes(Double.MinValue))), - Array(BoundRange(b, Bytes.toBytes(Double.MaxValue))), b)) + Some( + BoundRanges( + Array( + BoundRange(Bytes.toBytes(0.0d), b), + BoundRange(Bytes.toBytes(-0.0d), Bytes.toBytes(Double.MinValue))), + Array(BoundRange(b, Bytes.toBytes(Double.MaxValue))), + b)) } else { - Some(BoundRanges( - Array(BoundRange(b, Bytes.toBytes(Double.MinValue))), - Array(BoundRange(Bytes.toBytes(-0.0d), b), - BoundRange(Bytes.toBytes(0.0d), Bytes.toBytes(Double.MaxValue))), b)) + Some( + BoundRanges( + Array(BoundRange(b, Bytes.toBytes(Double.MinValue))), + Array( + BoundRange(Bytes.toBytes(-0.0d), b), + BoundRange(Bytes.toBytes(0.0d), Bytes.toBytes(Double.MaxValue))), + b)) } case a: Float => - val b = Bytes.toBytes(a) + val b = Bytes.toBytes(a) if (a >= 0.0f) { - Some(BoundRanges( - Array(BoundRange(Bytes.toBytes(0.0f), b), - BoundRange(Bytes.toBytes(-0.0f), Bytes.toBytes(Float.MinValue))), - Array(BoundRange(b, Bytes.toBytes(Float.MaxValue))), b)) + Some( + BoundRanges( + Array( + BoundRange(Bytes.toBytes(0.0f), b), + BoundRange(Bytes.toBytes(-0.0f), Bytes.toBytes(Float.MinValue))), + Array(BoundRange(b, Bytes.toBytes(Float.MaxValue))), + b)) } else { - Some(BoundRanges( - Array(BoundRange(b, Bytes.toBytes(Float.MinValue))), - Array(BoundRange(Bytes.toBytes(-0.0f), b), - BoundRange(Bytes.toBytes(0.0f), Bytes.toBytes(Float.MaxValue))), b)) + Some( + BoundRanges( + Array(BoundRange(b, Bytes.toBytes(Float.MinValue))), + Array( + BoundRange(Bytes.toBytes(-0.0f), b), + BoundRange(Bytes.toBytes(0.0f), Bytes.toBytes(Float.MaxValue))), + b)) } case a: Array[Byte] => - Some(BoundRanges( - Array(BoundRange(bytesMin, a)), - Array(BoundRange(a, bytesMax)), a)) + Some(BoundRanges(Array(BoundRange(bytesMin, a)), Array(BoundRange(a, bytesMax)), a)) case a: Byte => - val b = Array(a) - Some(BoundRanges( - Array(BoundRange(bytesMin, b)), - Array(BoundRange(b, bytesMax)), b)) + val b = Array(a) + Some(BoundRanges(Array(BoundRange(bytesMin, b)), Array(BoundRange(b, bytesMax)), b)) case a: String => - val b = Bytes.toBytes(a) - Some(BoundRanges( - Array(BoundRange(bytesMin, b)), - Array(BoundRange(b, bytesMax)), b)) + val b = Bytes.toBytes(a) + Some(BoundRanges(Array(BoundRange(bytesMin, b)), Array(BoundRange(b, bytesMax)), b)) case a: UTF8String => val b = a.getBytes - Some(BoundRanges( - Array(BoundRange(bytesMin, b)), - Array(BoundRange(b, bytesMax)), b)) + Some(BoundRanges(Array(BoundRange(bytesMin, b)), Array(BoundRange(b, bytesMax)), b)) case _ => None } def compare(c: Int, ops: JavaBytesEncoder): Boolean = { ops match { - case JavaBytesEncoder.Greater => c > 0 - case JavaBytesEncoder.GreaterEqual => c >= 0 - case JavaBytesEncoder.Less => c < 0 - case JavaBytesEncoder.LessEqual => c <= 0 + case JavaBytesEncoder.Greater => c > 0 + case JavaBytesEncoder.GreaterEqual => c >= 0 + case JavaBytesEncoder.Less => c < 0 + case JavaBytesEncoder.LessEqual => c <= 0 } } /** - * encode the data type into byte array. Note that it is a naive implementation with the - * data type byte appending to the head of the serialized byte array. - * - * @param dt: The data type of the input - * @param value: the value of the input - * @return the byte array with the first byte indicating the data type. - */ - override def encode(dt: DataType, - value: Any): Array[Byte] = { + * encode the data type into byte array. Note that it is a naive implementation with the + * data type byte appending to the head of the serialized byte array. + * + * @param dt: The data type of the input + * @param value: the value of the input + * @return the byte array with the first byte indicating the data type. + */ + override def encode(dt: DataType, value: Any): Array[Byte] = { dt match { case BooleanType => val result = new Array[Byte](Bytes.SIZEOF_BOOLEAN + 1) @@ -191,7 +227,7 @@ class NaiveEncoder extends BytesEncoder with Logging{ result(0) = IntEnc Bytes.putInt(result, 1, value.asInstanceOf[Int]) result - case LongType|TimestampType => + case LongType | TimestampType => val result = new Array[Byte](Bytes.SIZEOF_LONG + 1) result(0) = LongEnc Bytes.putLong(result, 1, value.asInstanceOf[Long]) @@ -227,9 +263,14 @@ class NaiveEncoder extends BytesEncoder with Logging{ } } - override def filter(input: Array[Byte], offset1: Int, length1: Int, - filterBytes: Array[Byte], offset2: Int, length2: Int, - ops: JavaBytesEncoder): Boolean = { + override def filter( + input: Array[Byte], + offset1: Int, + length1: Int, + filterBytes: Array[Byte], + offset2: Int, + length2: Int, + ops: JavaBytesEncoder): Boolean = { filterBytes(offset2) match { case ShortEnc => val in = Bytes.toShort(input, offset1) @@ -255,7 +296,8 @@ class NaiveEncoder extends BytesEncoder with Logging{ // for String, Byte, Binary, Boolean and other types // we can use the order of byte array directly. compare( - Bytes.compareTo(input, offset1, length1, filterBytes, offset2 + 1, length2 - 1), ops) + Bytes.compareTo(input, offset1, length1, filterBytes, offset2 + 1, length2 - 1), + ops) } } } diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/SchemaConverters.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/SchemaConverters.scala index 9eeabc54..ebfd6648 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/SchemaConverters.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/SchemaConverters.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark import java.io.ByteArrayInputStream @@ -22,26 +22,22 @@ import java.nio.ByteBuffer import java.sql.Timestamp import java.util import java.util.HashMap - +import org.apache.avro.{Schema, SchemaBuilder} +import org.apache.avro.Schema.Type._ import org.apache.avro.SchemaBuilder.BaseFieldTypeBuilder import org.apache.avro.SchemaBuilder.BaseTypeBuilder import org.apache.avro.SchemaBuilder.FieldAssembler import org.apache.avro.SchemaBuilder.FieldDefault import org.apache.avro.SchemaBuilder.RecordBuilder +import org.apache.avro.generic.{GenericData, GenericDatumReader, GenericDatumWriter, GenericRecord} +import org.apache.avro.generic.GenericData.{Fixed, Record} import org.apache.avro.io._ import org.apache.commons.io.output.ByteArrayOutputStream -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.util.Bytes - -import scala.collection.JavaConversions._ - -import org.apache.avro.{SchemaBuilder, Schema} -import org.apache.avro.Schema.Type._ -import org.apache.avro.generic.GenericData.{Record, Fixed} -import org.apache.avro.generic.{GenericDatumReader, GenericDatumWriter, GenericData, GenericRecord} import org.apache.spark.sql.Row import org.apache.spark.sql.types._ - +import org.apache.yetus.audience.InterfaceAudience +import scala.collection.JavaConversions._ import scala.collection.immutable.Map @InterfaceAudience.Private @@ -50,22 +46,23 @@ abstract class AvroException(msg: String) extends Exception(msg) @InterfaceAudience.Private case class SchemaConversionException(msg: String) extends AvroException(msg) -/*** - * On top level, the converters provide three high level interface. - * 1. toSqlType: This function takes an avro schema and returns a sql schema. - * 2. createConverterToSQL: Returns a function that is used to convert avro types to their - * corresponding sparkSQL representations. - * 3. convertTypeToAvro: This function constructs converter function for a given sparkSQL - * datatype. This is used in writing Avro records out to disk - */ +/** + * * + * On top level, the converters provide three high level interface. + * 1. toSqlType: This function takes an avro schema and returns a sql schema. + * 2. createConverterToSQL: Returns a function that is used to convert avro types to their + * corresponding sparkSQL representations. + * 3. convertTypeToAvro: This function constructs converter function for a given sparkSQL + * datatype. This is used in writing Avro records out to disk + */ @InterfaceAudience.Private object SchemaConverters { case class SchemaType(dataType: DataType, nullable: Boolean) /** - * This function takes an avro schema and returns a sql schema. - */ + * This function takes an avro schema and returns a sql schema. + */ def toSqlType(avroSchema: Schema): SchemaType = { avroSchema.getType match { case INT => SchemaType(IntegerType, nullable = false) @@ -79,9 +76,10 @@ object SchemaConverters { case ENUM => SchemaType(StringType, nullable = false) case RECORD => - val fields = avroSchema.getFields.map { f => - val schemaType = toSqlType(f.schema()) - StructField(f.name, schemaType.dataType, schemaType.nullable) + val fields = avroSchema.getFields.map { + f => + val schemaType = toSqlType(f.schema()) + StructField(f.name, schemaType.dataType, schemaType.nullable) } SchemaType(StructType(fields), nullable = false) @@ -107,98 +105,113 @@ object SchemaConverters { } else { toSqlType(Schema.createUnion(remainingUnionTypes)).copy(nullable = true) } - } else avroSchema.getTypes.map(_.getType) match { - case Seq(t1, t2) if Set(t1, t2) == Set(INT, LONG) => - SchemaType(LongType, nullable = false) - case Seq(t1, t2) if Set(t1, t2) == Set(FLOAT, DOUBLE) => - SchemaType(DoubleType, nullable = false) - case other => throw new SchemaConversionException( - s"This mix of union types is not supported: $other") - } + } else + avroSchema.getTypes.map(_.getType) match { + case Seq(t1, t2) if Set(t1, t2) == Set(INT, LONG) => + SchemaType(LongType, nullable = false) + case Seq(t1, t2) if Set(t1, t2) == Set(FLOAT, DOUBLE) => + SchemaType(DoubleType, nullable = false) + case other => + throw new SchemaConversionException( + s"This mix of union types is not supported: $other") + } case other => throw new SchemaConversionException(s"Unsupported type $other") } } /** - * This function converts sparkSQL StructType into avro schema. This method uses two other - * converter methods in order to do the conversion. - */ + * This function converts sparkSQL StructType into avro schema. This method uses two other + * converter methods in order to do the conversion. + */ private def convertStructToAvro[T]( - structType: StructType, - schemaBuilder: RecordBuilder[T], - recordNamespace: String): T = { + structType: StructType, + schemaBuilder: RecordBuilder[T], + recordNamespace: String): T = { val fieldsAssembler: FieldAssembler[T] = schemaBuilder.fields() - structType.fields.foreach { field => - val newField = fieldsAssembler.name(field.name).`type`() - - if (field.nullable) { - convertFieldTypeToAvro(field.dataType, newField.nullable(), field.name, recordNamespace) - .noDefault - } else { - convertFieldTypeToAvro(field.dataType, newField, field.name, recordNamespace) - .noDefault - } + structType.fields.foreach { + field => + val newField = fieldsAssembler.name(field.name).`type`() + + if (field.nullable) { + convertFieldTypeToAvro( + field.dataType, + newField.nullable(), + field.name, + recordNamespace).noDefault + } else { + convertFieldTypeToAvro(field.dataType, newField, field.name, recordNamespace).noDefault + } } fieldsAssembler.endRecord() } /** - * Returns a function that is used to convert avro types to their - * corresponding sparkSQL representations. - */ + * Returns a function that is used to convert avro types to their + * corresponding sparkSQL representations. + */ def createConverterToSQL(schema: Schema): Any => Any = { schema.getType match { // Avro strings are in Utf8, so we have to call toString on them case STRING | ENUM => (item: Any) => if (item == null) null else item.toString case INT | BOOLEAN | DOUBLE | FLOAT | LONG => identity // Byte arrays are reused by avro, so we have to make a copy of them. - case FIXED => (item: Any) => if (item == null) { - null - } else { - item.asInstanceOf[Fixed].bytes().clone() - } - case BYTES => (item: Any) => if (item == null) { - null - } else { - val bytes = item.asInstanceOf[ByteBuffer] - val javaBytes = new Array[Byte](bytes.remaining) - bytes.get(javaBytes) - javaBytes - } + case FIXED => + (item: Any) => + if (item == null) { + null + } else { + item.asInstanceOf[Fixed].bytes().clone() + } + case BYTES => + (item: Any) => + if (item == null) { + null + } else { + val bytes = item.asInstanceOf[ByteBuffer] + val javaBytes = new Array[Byte](bytes.remaining) + bytes.get(javaBytes) + javaBytes + } case RECORD => val fieldConverters = schema.getFields.map(f => createConverterToSQL(f.schema)) - (item: Any) => if (item == null) { - null - } else { - val record = item.asInstanceOf[GenericRecord] - val converted = new Array[Any](fieldConverters.size) - var idx = 0 - while (idx < fieldConverters.size) { - converted(idx) = fieldConverters.apply(idx)(record.get(idx)) - idx += 1 + (item: Any) => + if (item == null) { + null + } else { + val record = item.asInstanceOf[GenericRecord] + val converted = new Array[Any](fieldConverters.size) + var idx = 0 + while (idx < fieldConverters.size) { + converted(idx) = fieldConverters.apply(idx)(record.get(idx)) + idx += 1 + } + Row.fromSeq(converted.toSeq) } - Row.fromSeq(converted.toSeq) - } case ARRAY => val elementConverter = createConverterToSQL(schema.getElementType) - (item: Any) => if (item == null) { - null - } else { - try { - item.asInstanceOf[GenericData.Array[Any]].map(elementConverter) - } catch { - case e: Throwable => - item.asInstanceOf[util.ArrayList[Any]].map(elementConverter) + (item: Any) => + if (item == null) { + null + } else { + try { + item.asInstanceOf[GenericData.Array[Any]].map(elementConverter) + } catch { + case e: Throwable => + item.asInstanceOf[util.ArrayList[Any]].map(elementConverter) + } } - } case MAP => val valueConverter = createConverterToSQL(schema.getValueType) - (item: Any) => if (item == null) { - null - } else { - item.asInstanceOf[HashMap[Any, Any]].map(x => (x._1.toString, valueConverter(x._2))).toMap - } + (item: Any) => + if (item == null) { + null + } else { + item + .asInstanceOf[HashMap[Any, Any]] + .map(x => (x._1.toString, valueConverter(x._2))) + .toMap + } case UNION => if (schema.getTypes.exists(_.getType == NULL)) { val remainingUnionTypes = schema.getTypes.filterNot(_.getType == NULL) @@ -207,39 +220,41 @@ object SchemaConverters { } else { createConverterToSQL(Schema.createUnion(remainingUnionTypes)) } - } else schema.getTypes.map(_.getType) match { - case Seq(t1, t2) if Set(t1, t2) == Set(INT, LONG) => - (item: Any) => { - item match { - case l: Long => l - case i: Int => i.toLong - case null => null + } else + schema.getTypes.map(_.getType) match { + case Seq(t1, t2) if Set(t1, t2) == Set(INT, LONG) => + (item: Any) => { + item match { + case l: Long => l + case i: Int => i.toLong + case null => null + } } - } - case Seq(t1, t2) if Set(t1, t2) == Set(FLOAT, DOUBLE) => - (item: Any) => { - item match { - case d: Double => d - case f: Float => f.toDouble - case null => null + case Seq(t1, t2) if Set(t1, t2) == Set(FLOAT, DOUBLE) => + (item: Any) => { + item match { + case d: Double => d + case f: Float => f.toDouble + case null => null + } } - } - case other => throw new SchemaConversionException( - s"This mix of union types is not supported (see README): $other") - } + case other => + throw new SchemaConversionException( + s"This mix of union types is not supported (see README): $other") + } case other => throw new SchemaConversionException(s"invalid avro type: $other") } } /** - * This function is used to convert some sparkSQL type to avro type. Note that this function won't - * be used to construct fields of avro record (convertFieldTypeToAvro is used for that). - */ + * This function is used to convert some sparkSQL type to avro type. Note that this function won't + * be used to construct fields of avro record (convertFieldTypeToAvro is used for that). + */ private def convertTypeToAvro[T]( - dataType: DataType, - schemaBuilder: BaseTypeBuilder[T], - structName: String, - recordNamespace: String): T = { + dataType: DataType, + schemaBuilder: BaseTypeBuilder[T], + structName: String, + recordNamespace: String): T = { dataType match { case ByteType => schemaBuilder.intType() case ShortType => schemaBuilder.intType() @@ -274,15 +289,15 @@ object SchemaConverters { } /** - * This function is used to construct fields of the avro record, where schema of the field is - * specified by avro representation of dataType. Since builders for record fields are different - * from those for everything else, we have to use a separate method. - */ + * This function is used to construct fields of the avro record, where schema of the field is + * specified by avro representation of dataType. Since builders for record fields are different + * from those for everything else, we have to use a separate method. + */ private def convertFieldTypeToAvro[T]( - dataType: DataType, - newFieldBuilder: BaseFieldTypeBuilder[T], - structName: String, - recordNamespace: String): FieldDefault[T, _] = { + dataType: DataType, + newFieldBuilder: BaseFieldTypeBuilder[T], + structName: String, + recordNamespace: String): FieldDefault[T, _] = { dataType match { case ByteType => newFieldBuilder.intType() case ShortType => newFieldBuilder.intType() @@ -323,24 +338,28 @@ object SchemaConverters { SchemaBuilder.builder() } } + /** - * This function constructs converter function for a given sparkSQL datatype. This is used in - * writing Avro records out to disk - */ + * This function constructs converter function for a given sparkSQL datatype. This is used in + * writing Avro records out to disk + */ def createConverterToAvro( - dataType: DataType, - structName: String, - recordNamespace: String): (Any) => Any = { + dataType: DataType, + structName: String, + recordNamespace: String): (Any) => Any = { dataType match { - case BinaryType => (item: Any) => item match { - case null => null - case bytes: Array[Byte] => ByteBuffer.wrap(bytes) - } - case ByteType | ShortType | IntegerType | LongType | - FloatType | DoubleType | StringType | BooleanType => identity + case BinaryType => + (item: Any) => + item match { + case null => null + case bytes: Array[Byte] => ByteBuffer.wrap(bytes) + } + case ByteType | ShortType | IntegerType | LongType | FloatType | DoubleType | StringType | + BooleanType => + identity case _: DecimalType => (item: Any) => if (item == null) null else item.toString - case TimestampType => (item: Any) => - if (item == null) null else item.asInstanceOf[Timestamp].getTime + case TimestampType => + (item: Any) => if (item == null) null else item.asInstanceOf[Timestamp].getTime case ArrayType(elementType, _) => val elementConverter = createConverterToAvro(elementType, structName, recordNamespace) (item: Any) => { @@ -365,18 +384,19 @@ object SchemaConverters { null } else { val javaMap = new HashMap[String, Any]() - item.asInstanceOf[Map[String, Any]].foreach { case (key, value) => - javaMap.put(key, valueConverter(value)) + item.asInstanceOf[Map[String, Any]].foreach { + case (key, value) => + javaMap.put(key, valueConverter(value)) } javaMap } } case structType: StructType => val builder = SchemaBuilder.record(structName).namespace(recordNamespace) - val schema: Schema = SchemaConverters.convertStructToAvro( - structType, builder, recordNamespace) - val fieldConverters = structType.fields.map(field => - createConverterToAvro(field.dataType, field.name, recordNamespace)) + val schema: Schema = + SchemaConverters.convertStructToAvro(structType, builder, recordNamespace) + val fieldConverters = structType.fields.map( + field => createConverterToAvro(field.dataType, field.name, recordNamespace)) (item: Any) => { if (item == null) { null @@ -400,10 +420,10 @@ object SchemaConverters { @InterfaceAudience.Private object AvroSerdes { // We only handle top level is record or primary type now - def serialize(input: Any, schema: Schema): Array[Byte]= { + def serialize(input: Any, schema: Schema): Array[Byte] = { schema.getType match { case BOOLEAN => Bytes.toBytes(input.asInstanceOf[Boolean]) - case BYTES | FIXED=> input.asInstanceOf[Array[Byte]] + case BYTES | FIXED => input.asInstanceOf[Array[Byte]] case DOUBLE => Bytes.toBytes(input.asInstanceOf[Double]) case FLOAT => Bytes.toBytes(input.asInstanceOf[Float]) case INT => Bytes.toBytes(input.asInstanceOf[Int]) @@ -416,7 +436,7 @@ object AvroSerdes { val encoder2: BinaryEncoder = EncoderFactory.get().directBinaryEncoder(bao2, null) writer2.write(gr, encoder2) bao2.toByteArray() - case _ => throw new Exception(s"unsupported data type ${schema.getType}") //TODO + case _ => throw new Exception(s"unsupported data type ${schema.getType}") // TODO } } diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/SerDes.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/SerDes.scala index 59e44f3f..c3d5c634 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/SerDes.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/SerDes.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark.datasources import org.apache.hadoop.hbase.util.Bytes diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/SerializableConfiguration.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/SerializableConfiguration.scala index 0e2b6f4c..f5b4c5a2 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/SerializableConfiguration.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/SerializableConfiguration.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,14 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark.datasources import java.io.{IOException, ObjectInputStream, ObjectOutputStream} - import org.apache.hadoop.conf.Configuration -import org.apache.yetus.audience.InterfaceAudience; - +import org.apache.yetus.audience.InterfaceAudience import scala.util.control.NonFatal @InterfaceAudience.Private diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/Utils.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/Utils.scala index 05d80d41..de7eec9a 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/Utils.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/Utils.scala @@ -1,13 +1,13 @@ - /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -15,11 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark.datasources import java.sql.{Date, Timestamp} - import org.apache.hadoop.hbase.spark.AvroSerdes import org.apache.hadoop.hbase.util.Bytes import org.apache.spark.sql.types._ @@ -30,21 +28,17 @@ import org.apache.yetus.audience.InterfaceAudience; object Utils { /** - * Parses the hbase field to it's corresponding - * scala type which can then be put into a Spark GenericRow - * which is then automatically converted by Spark. - */ - def hbaseFieldToScalaType( - f: Field, - src: Array[Byte], - offset: Int, - length: Int): Any = { + * Parses the hbase field to it's corresponding + * scala type which can then be put into a Spark GenericRow + * which is then automatically converted by Spark. + */ + def hbaseFieldToScalaType(f: Field, src: Array[Byte], offset: Int, length: Int): Any = { if (f.exeSchema.isDefined) { // If we have avro schema defined, use it to get record, and then convert them to catalyst data type val m = AvroSerdes.deserialize(src, f.exeSchema.get) val n = f.avroToCatalyst.map(_(m)) n.get - } else { + } else { // Fall back to atomic type f.dt match { case BooleanType => src(offset) != 0 diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/package.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/package.scala index 8f1f15c2..66e3897e 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/package.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/package.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,11 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark import org.apache.hadoop.hbase.util.Bytes - import scala.math.Ordering // TODO: add @InterfaceAudience.Private if https://issues.scala-lang.org/browse/SI-3600 is resolved @@ -33,7 +32,7 @@ package object hbase { return Bytes.compareTo(x, y) } } - //Do not use BinaryType.ordering + // Do not use BinaryType.ordering implicit val order: Ordering[HBaseType] = ord } diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/datasources/AvroSource.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/datasources/AvroSource.scala index 068b1af6..b0d50290 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/datasources/AvroSource.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/datasources/AvroSource.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,17 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark.example.datasources import org.apache.avro.Schema import org.apache.avro.generic.GenericData import org.apache.hadoop.hbase.spark.AvroSerdes import org.apache.hadoop.hbase.spark.datasources.HBaseTableCatalog -import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.SQLContext import org.apache.spark.SparkConf import org.apache.spark.SparkContext +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.SQLContext import org.apache.yetus.audience.InterfaceAudience /** @@ -32,8 +32,7 @@ import org.apache.yetus.audience.InterfaceAudience * @param col1 Column #1, Type is Array[Byte] */ @InterfaceAudience.Private -case class AvroHBaseRecord(col0: String, - col1: Array[Byte]) +case class AvroHBaseRecord(col0: String, col1: Array[Byte]) @InterfaceAudience.Private object AvroHBaseRecord { val schemaString = @@ -58,12 +57,13 @@ object AvroHBaseRecord { user.put("name", s"name${"%03d".format(i)}") user.put("favorite_number", i) user.put("favorite_color", s"color${"%03d".format(i)}") - val favoriteArray = new GenericData.Array[String](2, avroSchema.getField("favorite_array").schema()) + val favoriteArray = + new GenericData.Array[String](2, avroSchema.getField("favorite_array").schema()) favoriteArray.add(s"number${i}") - favoriteArray.add(s"number${i+1}") + favoriteArray.add(s"number${i + 1}") user.put("favorite_array", favoriteArray) import scala.collection.JavaConverters._ - val favoriteMap = Map[String, Int](("key1" -> i), ("key2" -> (i+1))).asJava + val favoriteMap = Map[String, Int](("key1" -> i), ("key2" -> (i + 1))).asJava user.put("favorite_map", favoriteMap) val avroByte = AvroSerdes.serialize(user, avroSchema) AvroHBaseRecord(s"name${"%03d".format(i)}", avroByte) @@ -107,19 +107,21 @@ object AvroSource { import sqlContext.implicits._ def withCatalog(cat: String): DataFrame = { - sqlContext - .read - .options(Map("avroSchema" -> AvroHBaseRecord.schemaString, HBaseTableCatalog.tableCatalog -> avroCatalog)) + sqlContext.read + .options( + Map( + "avroSchema" -> AvroHBaseRecord.schemaString, + HBaseTableCatalog.tableCatalog -> avroCatalog)) .format("org.apache.hadoop.hbase.spark") .load() } - val data = (0 to 255).map { i => - AvroHBaseRecord(i) - } + val data = (0 to 255).map { i => AvroHBaseRecord(i) } - sc.parallelize(data).toDF.write.options( - Map(HBaseTableCatalog.tableCatalog -> catalog, HBaseTableCatalog.newTable -> "5")) + sc.parallelize(data) + .toDF + .write + .options(Map(HBaseTableCatalog.tableCatalog -> catalog, HBaseTableCatalog.newTable -> "5")) .format("org.apache.hadoop.hbase.spark") .save() @@ -140,15 +142,18 @@ object AvroSource { throw new UserCustomizedSampleException("value invalid") } - df.write.options( - Map("avroSchema"->AvroHBaseRecord.schemaString, HBaseTableCatalog.tableCatalog->avroCatalogInsert, - HBaseTableCatalog.newTable -> "5")) + df.write + .options( + Map( + "avroSchema" -> AvroHBaseRecord.schemaString, + HBaseTableCatalog.tableCatalog -> avroCatalogInsert, + HBaseTableCatalog.newTable -> "5")) .format("org.apache.hadoop.hbase.spark") .save() val newDF = withCatalog(avroCatalogInsert) newDF.show() newDF.printSchema() - if(newDF.count() != 256) { + if (newDF.count() != 256) { throw new UserCustomizedSampleException("value invalid") } diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/datasources/DataType.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/datasources/DataType.scala index ac7e7766..d314dc8c 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/datasources/DataType.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/datasources/DataType.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,19 +15,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark.example.datasources import org.apache.hadoop.hbase.spark.datasources.HBaseTableCatalog -import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.SQLContext import org.apache.spark.SparkConf import org.apache.spark.SparkContext +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.SQLContext import org.apache.yetus.audience.InterfaceAudience @InterfaceAudience.Private -class UserCustomizedSampleException(message: String = null, cause: Throwable = null) extends - RuntimeException(UserCustomizedSampleException.message(message, cause), cause) +class UserCustomizedSampleException(message: String = null, cause: Throwable = null) + extends RuntimeException(UserCustomizedSampleException.message(message, cause), cause) @InterfaceAudience.Private object UserCustomizedSampleException { @@ -38,19 +38,20 @@ object UserCustomizedSampleException { @InterfaceAudience.Private case class IntKeyRecord( - col0: Integer, - col1: Boolean, - col2: Double, - col3: Float, - col4: Int, - col5: Long, - col6: Short, - col7: String, - col8: Byte) + col0: Integer, + col1: Boolean, + col2: Double, + col3: Float, + col4: Int, + col5: Long, + col6: Short, + col7: String, + col8: Byte) object IntKeyRecord { def apply(i: Int): IntKeyRecord = { - IntKeyRecord(if (i % 2 == 0) i else -i, + IntKeyRecord( + if (i % 2 == 0) i else -i, i % 2 == 0, i.toDouble, i.toFloat, @@ -80,7 +81,7 @@ object DataType { |} |}""".stripMargin - def main(args: Array[String]){ + def main(args: Array[String]) { val sparkConf = new SparkConf().setAppName("DataTypeExample") val sc = new SparkContext(sparkConf) val sqlContext = new SQLContext(sc) @@ -88,19 +89,18 @@ object DataType { import sqlContext.implicits._ def withCatalog(cat: String): DataFrame = { - sqlContext - .read - .options(Map(HBaseTableCatalog.tableCatalog->cat)) + sqlContext.read + .options(Map(HBaseTableCatalog.tableCatalog -> cat)) .format("org.apache.hadoop.hbase.spark") .load() } // test populate table - val data = (0 until 32).map { i => - IntKeyRecord(i) - } - sc.parallelize(data).toDF.write.options( - Map(HBaseTableCatalog.tableCatalog -> cat, HBaseTableCatalog.newTable -> "5")) + val data = (0 until 32).map { i => IntKeyRecord(i) } + sc.parallelize(data) + .toDF + .write + .options(Map(HBaseTableCatalog.tableCatalog -> cat, HBaseTableCatalog.newTable -> "5")) .format("org.apache.hadoop.hbase.spark") .save() @@ -108,35 +108,35 @@ object DataType { val df = withCatalog(cat) val s = df.filter($"col0" < 0) s.show() - if(s.count() != 16){ + if (s.count() != 16) { throw new UserCustomizedSampleException("value invalid") } - //test less or equal than -10. The number of results is 11 + // test less or equal than -10. The number of results is 11 val num1 = df.filter($"col0" <= -10) num1.show() val c1 = num1.count() println(s"test result count should be 11: $c1") - //test less or equal than -9. The number of results is 12 + // test less or equal than -9. The number of results is 12 val num2 = df.filter($"col0" <= -9) num2.show() val c2 = num2.count() println(s"test result count should be 12: $c2") - //test greater or equal than -9". The number of results is 21 + // test greater or equal than -9". The number of results is 21 val num3 = df.filter($"col0" >= -9) num3.show() val c3 = num3.count() println(s"test result count should be 21: $c3") - //test greater or equal than 0. The number of results is 16 + // test greater or equal than 0. The number of results is 16 val num4 = df.filter($"col0" >= 0) num4.show() val c4 = num4.count() println(s"test result count should be 16: $c4") - //test greater than 10. The number of results is 10 + // test greater than 10. The number of results is 10 val num5 = df.filter($"col0" > 10) num5.show() val c5 = num5.count() @@ -148,24 +148,24 @@ object DataType { val c6 = num6.count() println(s"test result count should be 11: $c6") - //test "or". The number of results is 21 + // test "or". The number of results is 21 val num7 = df.filter($"col0" <= -10 || $"col0" > 10) num7.show() val c7 = num7.count() println(s"test result count should be 21: $c7") - //test "all". The number of results is 32 + // test "all". The number of results is 32 val num8 = df.filter($"col0" >= -100) num8.show() val c8 = num8.count() println(s"test result count should be 32: $c8") - //test "full query" + // test "full query" val df1 = withCatalog(cat) df1.show() val c_df = df1.count() println(s"df count should be 32: $c_df") - if(c_df != 32){ + if (c_df != 32) { throw new UserCustomizedSampleException("value invalid") } } diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/datasources/HBaseSource.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/datasources/HBaseSource.scala index 6accae09..d7101e2e 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/datasources/HBaseSource.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/datasources/HBaseSource.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,33 +15,33 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark.example.datasources import org.apache.hadoop.hbase.spark.datasources.HBaseTableCatalog -import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.SQLContext import org.apache.spark.SparkConf import org.apache.spark.SparkContext +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.SQLContext import org.apache.yetus.audience.InterfaceAudience @InterfaceAudience.Private case class HBaseRecord( - col0: String, - col1: Boolean, - col2: Double, - col3: Float, - col4: Int, - col5: Long, - col6: Short, - col7: String, - col8: Byte) + col0: String, + col1: Boolean, + col2: Double, + col3: Float, + col4: Int, + col5: Long, + col6: Short, + col7: String, + col8: Byte) @InterfaceAudience.Private object HBaseRecord { def apply(i: Int): HBaseRecord = { val s = s"""row${"%03d".format(i)}""" - HBaseRecord(s, + HBaseRecord( + s, i % 2 == 0, i.toDouble, i.toFloat, @@ -78,30 +79,32 @@ object HBaseSource { import sqlContext.implicits._ def withCatalog(cat: String): DataFrame = { - sqlContext - .read - .options(Map(HBaseTableCatalog.tableCatalog->cat)) + sqlContext.read + .options(Map(HBaseTableCatalog.tableCatalog -> cat)) .format("org.apache.hadoop.hbase.spark") .load() } - val data = (0 to 255).map { i => - HBaseRecord(i) - } + val data = (0 to 255).map { i => HBaseRecord(i) } - sc.parallelize(data).toDF.write.options( - Map(HBaseTableCatalog.tableCatalog -> cat, HBaseTableCatalog.newTable -> "5")) + sc.parallelize(data) + .toDF + .write + .options(Map(HBaseTableCatalog.tableCatalog -> cat, HBaseTableCatalog.newTable -> "5")) .format("org.apache.hadoop.hbase.spark") .save() val df = withCatalog(cat) df.show() df.filter($"col0" <= "row005") - .select($"col0", $"col1").show + .select($"col0", $"col1") + .show df.filter($"col0" === "row005" || $"col0" <= "row005") - .select($"col0", $"col1").show + .select($"col0", $"col1") + .show df.filter($"col0" > "row250") - .select($"col0", $"col1").show + .select($"col0", $"col1") + .show df.registerTempTable("table1") val c = sqlContext.sql("select count(col1) from table1 where col0 < 'row050'") c.show() diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseBulkDeleteExample.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseBulkDeleteExample.scala index 506fd229..5f19003c 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseBulkDeleteExample.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseBulkDeleteExample.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,14 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark.example.hbasecontext +import org.apache.hadoop.hbase.HBaseConfiguration +import org.apache.hadoop.hbase.TableName import org.apache.hadoop.hbase.client.Delete import org.apache.hadoop.hbase.spark.HBaseContext import org.apache.hadoop.hbase.util.Bytes -import org.apache.hadoop.hbase.HBaseConfiguration -import org.apache.hadoop.hbase.TableName import org.apache.spark.SparkConf import org.apache.spark.SparkContext import org.apache.yetus.audience.InterfaceAudience @@ -43,19 +43,20 @@ object HBaseBulkDeleteExample { val sparkConf = new SparkConf().setAppName("HBaseBulkDeleteExample " + tableName) val sc = new SparkContext(sparkConf) try { - //[Array[Byte]] - val rdd = sc.parallelize(Array( - Bytes.toBytes("1"), - Bytes.toBytes("2"), - Bytes.toBytes("3"), - Bytes.toBytes("4"), - Bytes.toBytes("5") - )) + // [Array[Byte]] + val rdd = sc.parallelize( + Array( + Bytes.toBytes("1"), + Bytes.toBytes("2"), + Bytes.toBytes("3"), + Bytes.toBytes("4"), + Bytes.toBytes("5"))) val conf = HBaseConfiguration.create() val hbaseContext = new HBaseContext(sc, conf) - hbaseContext.bulkDelete[Array[Byte]](rdd, + hbaseContext.bulkDelete[Array[Byte]]( + rdd, TableName.valueOf(tableName), putRecord => new Delete(putRecord), 4) diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseBulkGetExample.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseBulkGetExample.scala index 58bc1d43..2ac7c96d 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseBulkGetExample.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseBulkGetExample.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,16 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark.example.hbasecontext +import org.apache.hadoop.hbase.CellUtil +import org.apache.hadoop.hbase.HBaseConfiguration +import org.apache.hadoop.hbase.TableName import org.apache.hadoop.hbase.client.Get import org.apache.hadoop.hbase.client.Result import org.apache.hadoop.hbase.spark.HBaseContext import org.apache.hadoop.hbase.util.Bytes -import org.apache.hadoop.hbase.CellUtil -import org.apache.hadoop.hbase.HBaseConfiguration -import org.apache.hadoop.hbase.TableName import org.apache.spark.SparkConf import org.apache.spark.SparkContext import org.apache.yetus.audience.InterfaceAudience @@ -47,15 +47,16 @@ object HBaseBulkGetExample { try { - //[(Array[Byte])] - val rdd = sc.parallelize(Array( - Bytes.toBytes("1"), - Bytes.toBytes("2"), - Bytes.toBytes("3"), - Bytes.toBytes("4"), - Bytes.toBytes("5"), - Bytes.toBytes("6"), - Bytes.toBytes("7"))) + // [(Array[Byte])] + val rdd = sc.parallelize( + Array( + Bytes.toBytes("1"), + Bytes.toBytes("2"), + Bytes.toBytes("3"), + Bytes.toBytes("4"), + Bytes.toBytes("5"), + Bytes.toBytes("6"), + Bytes.toBytes("7"))) val conf = HBaseConfiguration.create() @@ -88,7 +89,9 @@ object HBaseBulkGetExample { b.toString() }) - getRdd.collect().foreach(v => println(v)) + getRdd + .collect() + .foreach(v => println(v)) } finally { sc.stop() diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseBulkPutExample.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseBulkPutExample.scala index 0a6f379f..f8330398 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseBulkPutExample.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseBulkPutExample.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,14 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark.example.hbasecontext +import org.apache.hadoop.hbase.HBaseConfiguration +import org.apache.hadoop.hbase.TableName import org.apache.hadoop.hbase.client.Put import org.apache.hadoop.hbase.spark.HBaseContext import org.apache.hadoop.hbase.util.Bytes -import org.apache.hadoop.hbase.HBaseConfiguration -import org.apache.hadoop.hbase.TableName import org.apache.spark.SparkConf import org.apache.spark.SparkContext import org.apache.yetus.audience.InterfaceAudience @@ -41,34 +41,40 @@ object HBaseBulkPutExample { val tableName = args(0) val columnFamily = args(1) - val sparkConf = new SparkConf().setAppName("HBaseBulkPutExample " + - tableName + " " + columnFamily) + val sparkConf = new SparkConf().setAppName( + "HBaseBulkPutExample " + + tableName + " " + columnFamily) val sc = new SparkContext(sparkConf) try { - //[(Array[Byte], Array[(Array[Byte], Array[Byte], Array[Byte])])] - val rdd = sc.parallelize(Array( - (Bytes.toBytes("1"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("1")))), - (Bytes.toBytes("2"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("2")))), - (Bytes.toBytes("3"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("3")))), - (Bytes.toBytes("4"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("4")))), - (Bytes.toBytes("5"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("5")))) - )) + // [(Array[Byte], Array[(Array[Byte], Array[Byte], Array[Byte])])] + val rdd = sc.parallelize( + Array( + ( + Bytes.toBytes("1"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("1")))), + ( + Bytes.toBytes("2"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("2")))), + ( + Bytes.toBytes("3"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("3")))), + ( + Bytes.toBytes("4"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("4")))), + ( + Bytes.toBytes("5"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("5")))))) val conf = HBaseConfiguration.create() val hbaseContext = new HBaseContext(sc, conf) - hbaseContext.bulkPut[(Array[Byte], Array[(Array[Byte], Array[Byte], Array[Byte])])](rdd, + hbaseContext.bulkPut[(Array[Byte], Array[(Array[Byte], Array[Byte], Array[Byte])])]( + rdd, TableName.valueOf(tableName), (putRecord) => { val put = new Put(putRecord._1) - putRecord._2.foreach((putValue) => - put.addColumn(putValue._1, putValue._2, putValue._3)) + putRecord._2.foreach((putValue) => put.addColumn(putValue._1, putValue._2, putValue._3)) put }); } finally { diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseBulkPutExampleFromFile.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseBulkPutExampleFromFile.scala index 51ff0da6..e7895763 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseBulkPutExampleFromFile.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseBulkPutExampleFromFile.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,14 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark.example.hbasecontext +import org.apache.hadoop.hbase.HBaseConfiguration +import org.apache.hadoop.hbase.TableName import org.apache.hadoop.hbase.client.Put import org.apache.hadoop.hbase.spark.HBaseContext import org.apache.hadoop.hbase.util.Bytes -import org.apache.hadoop.hbase.HBaseConfiguration -import org.apache.hadoop.hbase.TableName import org.apache.hadoop.io.LongWritable import org.apache.hadoop.io.Text import org.apache.hadoop.mapred.TextInputFormat @@ -38,7 +38,8 @@ import org.apache.yetus.audience.InterfaceAudience object HBaseBulkPutExampleFromFile { def main(args: Array[String]) { if (args.length < 3) { - println("HBaseBulkPutExampleFromFile {tableName} {columnFamily} {inputFile} are missing an argument") + println( + "HBaseBulkPutExampleFromFile {tableName} {columnFamily} {inputFile} are missing an argument") return } @@ -46,30 +47,30 @@ object HBaseBulkPutExampleFromFile { val columnFamily = args(1) val inputFile = args(2) - val sparkConf = new SparkConf().setAppName("HBaseBulkPutExampleFromFile " + - tableName + " " + columnFamily + " " + inputFile) + val sparkConf = new SparkConf().setAppName( + "HBaseBulkPutExampleFromFile " + + tableName + " " + columnFamily + " " + inputFile) val sc = new SparkContext(sparkConf) try { - var rdd = sc.hadoopFile( - inputFile, - classOf[TextInputFormat], - classOf[LongWritable], - classOf[Text]).map(v => { - System.out.println("reading-" + v._2.toString) - v._2.toString - }) + var rdd = sc + .hadoopFile(inputFile, classOf[TextInputFormat], classOf[LongWritable], classOf[Text]) + .map( + v => { + System.out.println("reading-" + v._2.toString) + v._2.toString + }) val conf = HBaseConfiguration.create() val hbaseContext = new HBaseContext(sc, conf) - hbaseContext.bulkPut[String](rdd, + hbaseContext.bulkPut[String]( + rdd, TableName.valueOf(tableName), (putRecord) => { System.out.println("hbase-" + putRecord) val put = new Put(Bytes.toBytes("Value- " + putRecord)) - put.addColumn(Bytes.toBytes("c"), Bytes.toBytes("1"), - Bytes.toBytes(putRecord.length())) + put.addColumn(Bytes.toBytes("c"), Bytes.toBytes("1"), Bytes.toBytes(putRecord.length())) put }); } finally { diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseBulkPutTimestampExample.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseBulkPutTimestampExample.scala index 9bfcc2c5..7b882d26 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseBulkPutTimestampExample.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseBulkPutTimestampExample.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,15 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark.example.hbasecontext -import org.apache.hadoop.hbase.spark.HBaseContext -import org.apache.spark.SparkContext import org.apache.hadoop.hbase.{HBaseConfiguration, TableName} -import org.apache.hadoop.hbase.util.Bytes import org.apache.hadoop.hbase.client.Put +import org.apache.hadoop.hbase.spark.HBaseContext +import org.apache.hadoop.hbase.util.Bytes import org.apache.spark.SparkConf +import org.apache.spark.SparkContext import org.apache.yetus.audience.InterfaceAudience /** @@ -34,42 +34,51 @@ import org.apache.yetus.audience.InterfaceAudience object HBaseBulkPutTimestampExample { def main(args: Array[String]) { if (args.length < 2) { - System.out.println("HBaseBulkPutTimestampExample {tableName} {columnFamily} are missing an argument") + System.out.println( + "HBaseBulkPutTimestampExample {tableName} {columnFamily} are missing an argument") return } val tableName = args(0) val columnFamily = args(1) - val sparkConf = new SparkConf().setAppName("HBaseBulkPutTimestampExample " + - tableName + " " + columnFamily) + val sparkConf = new SparkConf().setAppName( + "HBaseBulkPutTimestampExample " + + tableName + " " + columnFamily) val sc = new SparkContext(sparkConf) try { - val rdd = sc.parallelize(Array( - (Bytes.toBytes("6"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("1")))), - (Bytes.toBytes("7"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("2")))), - (Bytes.toBytes("8"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("3")))), - (Bytes.toBytes("9"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("4")))), - (Bytes.toBytes("10"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("5")))))) + val rdd = sc.parallelize( + Array( + ( + Bytes.toBytes("6"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("1")))), + ( + Bytes.toBytes("7"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("2")))), + ( + Bytes.toBytes("8"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("3")))), + ( + Bytes.toBytes("9"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("4")))), + ( + Bytes.toBytes("10"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("5")))))) val conf = HBaseConfiguration.create() val timeStamp = System.currentTimeMillis() val hbaseContext = new HBaseContext(sc, conf) - hbaseContext.bulkPut[(Array[Byte], Array[(Array[Byte], Array[Byte], Array[Byte])])](rdd, + hbaseContext.bulkPut[(Array[Byte], Array[(Array[Byte], Array[Byte], Array[Byte])])]( + rdd, TableName.valueOf(tableName), (putRecord) => { val put = new Put(putRecord._1) - putRecord._2.foreach((putValue) => put.addColumn(putValue._1, putValue._2, - timeStamp, putValue._3)) + putRecord._2.foreach( + (putValue) => put.addColumn(putValue._1, putValue._2, timeStamp, putValue._3)) put }) } finally { diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseDistributedScanExample.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseDistributedScanExample.scala index 7d8643a9..2f3619b2 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseDistributedScanExample.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseDistributedScanExample.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -16,14 +17,15 @@ */ package org.apache.hadoop.hbase.spark.example.hbasecontext +import org.apache.hadoop.hbase.HBaseConfiguration +import org.apache.hadoop.hbase.TableName import org.apache.hadoop.hbase.client.Scan import org.apache.hadoop.hbase.spark.HBaseContext import org.apache.hadoop.hbase.util.Bytes -import org.apache.hadoop.hbase.HBaseConfiguration -import org.apache.hadoop.hbase.TableName import org.apache.spark.SparkConf import org.apache.spark.SparkContext import org.apache.yetus.audience.InterfaceAudience + /** * This is a simple example of scanning records from HBase * with the hbaseRDD function in Distributed fashion. @@ -38,7 +40,7 @@ object HBaseDistributedScanExample { val tableName = args(0) - val sparkConf = new SparkConf().setAppName("HBaseDistributedScanExample " + tableName ) + val sparkConf = new SparkConf().setAppName("HBaseDistributedScanExample " + tableName) val sc = new SparkContext(sparkConf) try { @@ -53,7 +55,11 @@ object HBaseDistributedScanExample { getRdd.foreach(v => println(Bytes.toString(v._1.get()))) - println("Length: " + getRdd.map(r => r._1.copyBytes()).collect().length); + println( + "Length: " + getRdd + .map(r => r._1.copyBytes()) + .collect() + .length); } finally { sc.stop() } diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseStreamingBulkPutExample.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseStreamingBulkPutExample.scala index 20a22f73..4774a066 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseStreamingBulkPutExample.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseStreamingBulkPutExample.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,14 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark.example.hbasecontext +import org.apache.hadoop.hbase.HBaseConfiguration +import org.apache.hadoop.hbase.TableName import org.apache.hadoop.hbase.client.Put import org.apache.hadoop.hbase.spark.HBaseContext import org.apache.hadoop.hbase.util.Bytes -import org.apache.hadoop.hbase.HBaseConfiguration -import org.apache.hadoop.hbase.TableName import org.apache.spark.SparkConf import org.apache.spark.SparkContext import org.apache.spark.streaming.Seconds @@ -35,8 +35,9 @@ import org.apache.yetus.audience.InterfaceAudience object HBaseStreamingBulkPutExample { def main(args: Array[String]) { if (args.length < 4) { - println("HBaseStreamingBulkPutExample " + - "{host} {port} {tableName} {columnFamily} are missing an argument") + println( + "HBaseStreamingBulkPutExample " + + "{host} {port} {tableName} {columnFamily} are missing an argument") return } @@ -45,8 +46,9 @@ object HBaseStreamingBulkPutExample { val tableName = args(2) val columnFamily = args(3) - val sparkConf = new SparkConf().setAppName("HBaseStreamingBulkPutExample " + - tableName + " " + columnFamily) + val sparkConf = new SparkConf().setAppName( + "HBaseStreamingBulkPutExample " + + tableName + " " + columnFamily) val sc = new SparkContext(sparkConf) try { val ssc = new StreamingContext(sc, Seconds(1)) @@ -57,7 +59,8 @@ object HBaseStreamingBulkPutExample { val hbaseContext = new HBaseContext(sc, conf) - hbaseContext.streamBulkPut[String](lines, + hbaseContext.streamBulkPut[String]( + lines, TableName.valueOf(tableName), (putRecord) => { if (putRecord.length() > 0) { diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/rdd/HBaseBulkDeleteExample.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/rdd/HBaseBulkDeleteExample.scala index 0ba4d1c4..4c6eab8a 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/rdd/HBaseBulkDeleteExample.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/rdd/HBaseBulkDeleteExample.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -16,12 +17,12 @@ */ package org.apache.hadoop.hbase.spark.example.rdd +import org.apache.hadoop.hbase.HBaseConfiguration +import org.apache.hadoop.hbase.TableName import org.apache.hadoop.hbase.client.Delete import org.apache.hadoop.hbase.spark.HBaseContext import org.apache.hadoop.hbase.spark.HBaseRDDFunctions._ import org.apache.hadoop.hbase.util.Bytes -import org.apache.hadoop.hbase.HBaseConfiguration -import org.apache.hadoop.hbase.TableName import org.apache.spark.SparkConf import org.apache.spark.SparkContext import org.apache.yetus.audience.InterfaceAudience @@ -43,20 +44,22 @@ object HBaseBulkDeleteExample { val sparkConf = new SparkConf().setAppName("HBaseBulkDeleteExample " + tableName) val sc = new SparkContext(sparkConf) try { - //[Array[Byte]] - val rdd = sc.parallelize(Array( - Bytes.toBytes("1"), - Bytes.toBytes("2"), - Bytes.toBytes("3"), - Bytes.toBytes("4"), - Bytes.toBytes("5") - )) + // [Array[Byte]] + val rdd = sc.parallelize( + Array( + Bytes.toBytes("1"), + Bytes.toBytes("2"), + Bytes.toBytes("3"), + Bytes.toBytes("4"), + Bytes.toBytes("5"))) val conf = HBaseConfiguration.create() val hbaseContext = new HBaseContext(sc, conf) - rdd.hbaseBulkDelete(hbaseContext, TableName.valueOf(tableName), + rdd.hbaseBulkDelete( + hbaseContext, + TableName.valueOf(tableName), putRecord => new Delete(putRecord), 4) diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/rdd/HBaseBulkGetExample.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/rdd/HBaseBulkGetExample.scala index 0736f6e9..27617fe6 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/rdd/HBaseBulkGetExample.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/rdd/HBaseBulkGetExample.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -16,14 +17,14 @@ */ package org.apache.hadoop.hbase.spark.example.rdd +import org.apache.hadoop.hbase.CellUtil +import org.apache.hadoop.hbase.HBaseConfiguration +import org.apache.hadoop.hbase.TableName import org.apache.hadoop.hbase.client.Get import org.apache.hadoop.hbase.client.Result import org.apache.hadoop.hbase.spark.HBaseContext import org.apache.hadoop.hbase.spark.HBaseRDDFunctions._ import org.apache.hadoop.hbase.util.Bytes -import org.apache.hadoop.hbase.CellUtil -import org.apache.hadoop.hbase.HBaseConfiguration -import org.apache.hadoop.hbase.TableName import org.apache.spark.SparkConf import org.apache.spark.SparkContext import org.apache.yetus.audience.InterfaceAudience @@ -47,21 +48,25 @@ object HBaseBulkGetExample { try { - //[(Array[Byte])] - val rdd = sc.parallelize(Array( - Bytes.toBytes("1"), - Bytes.toBytes("2"), - Bytes.toBytes("3"), - Bytes.toBytes("4"), - Bytes.toBytes("5"), - Bytes.toBytes("6"), - Bytes.toBytes("7"))) + // [(Array[Byte])] + val rdd = sc.parallelize( + Array( + Bytes.toBytes("1"), + Bytes.toBytes("2"), + Bytes.toBytes("3"), + Bytes.toBytes("4"), + Bytes.toBytes("5"), + Bytes.toBytes("6"), + Bytes.toBytes("7"))) val conf = HBaseConfiguration.create() val hbaseContext = new HBaseContext(sc, conf) - val getRdd = rdd.hbaseBulkGet[String](hbaseContext, TableName.valueOf(tableName), 2, + val getRdd = rdd.hbaseBulkGet[String]( + hbaseContext, + TableName.valueOf(tableName), + 2, record => { System.out.println("making Get") new Get(record) @@ -85,7 +90,9 @@ object HBaseBulkGetExample { b.toString() }) - getRdd.collect().foreach(v => println(v)) + getRdd + .collect() + .foreach(v => println(v)) } finally { sc.stop() diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/rdd/HBaseBulkPutExample.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/rdd/HBaseBulkPutExample.scala index 9f5885fa..e9ad0706 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/rdd/HBaseBulkPutExample.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/rdd/HBaseBulkPutExample.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,15 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark.example.rdd +import org.apache.hadoop.hbase.HBaseConfiguration +import org.apache.hadoop.hbase.TableName import org.apache.hadoop.hbase.client.Put import org.apache.hadoop.hbase.spark.HBaseContext import org.apache.hadoop.hbase.spark.HBaseRDDFunctions._ import org.apache.hadoop.hbase.util.Bytes -import org.apache.hadoop.hbase.HBaseConfiguration -import org.apache.hadoop.hbase.TableName import org.apache.spark.SparkConf import org.apache.spark.SparkContext import org.apache.yetus.audience.InterfaceAudience @@ -33,48 +33,55 @@ import org.apache.yetus.audience.InterfaceAudience */ @InterfaceAudience.Private object HBaseBulkPutExample { - def main(args: Array[String]) { - if (args.length < 2) { - println("HBaseBulkPutExample {tableName} {columnFamily} are missing an arguments") - return - } + def main(args: Array[String]) { + if (args.length < 2) { + println("HBaseBulkPutExample {tableName} {columnFamily} are missing an arguments") + return + } - val tableName = args(0) - val columnFamily = args(1) + val tableName = args(0) + val columnFamily = args(1) - val sparkConf = new SparkConf().setAppName("HBaseBulkPutExample " + - tableName + " " + columnFamily) - val sc = new SparkContext(sparkConf) + val sparkConf = new SparkConf().setAppName( + "HBaseBulkPutExample " + + tableName + " " + columnFamily) + val sc = new SparkContext(sparkConf) - try { - //[(Array[Byte], Array[(Array[Byte], Array[Byte], Array[Byte])])] - val rdd = sc.parallelize(Array( - (Bytes.toBytes("1"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("1")))), - (Bytes.toBytes("2"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("2")))), - (Bytes.toBytes("3"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("3")))), - (Bytes.toBytes("4"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("4")))), - (Bytes.toBytes("5"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("5")))) - )) + try { + // [(Array[Byte], Array[(Array[Byte], Array[Byte], Array[Byte])])] + val rdd = sc.parallelize( + Array( + ( + Bytes.toBytes("1"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("1")))), + ( + Bytes.toBytes("2"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("2")))), + ( + Bytes.toBytes("3"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("3")))), + ( + Bytes.toBytes("4"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("4")))), + ( + Bytes.toBytes("5"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("5")))))) - val conf = HBaseConfiguration.create() + val conf = HBaseConfiguration.create() - val hbaseContext = new HBaseContext(sc, conf) + val hbaseContext = new HBaseContext(sc, conf) - rdd.hbaseBulkPut(hbaseContext, TableName.valueOf(tableName), - (putRecord) => { - val put = new Put(putRecord._1) - putRecord._2.foreach((putValue) => put.addColumn(putValue._1, putValue._2, - putValue._3)) - put - }) + rdd.hbaseBulkPut( + hbaseContext, + TableName.valueOf(tableName), + (putRecord) => { + val put = new Put(putRecord._1) + putRecord._2.foreach((putValue) => put.addColumn(putValue._1, putValue._2, putValue._3)) + put + }) - } finally { - sc.stop() - } - } - } + } finally { + sc.stop() + } + } +} diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/rdd/HBaseForeachPartitionExample.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/rdd/HBaseForeachPartitionExample.scala index be257eef..f6f9f91d 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/rdd/HBaseForeachPartitionExample.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/rdd/HBaseForeachPartitionExample.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark.example.rdd import org.apache.hadoop.hbase.HBaseConfiguration @@ -42,40 +42,46 @@ object HBaseForeachPartitionExample { val tableName = args(0) val columnFamily = args(1) - val sparkConf = new SparkConf().setAppName("HBaseForeachPartitionExample " + - tableName + " " + columnFamily) + val sparkConf = new SparkConf().setAppName( + "HBaseForeachPartitionExample " + + tableName + " " + columnFamily) val sc = new SparkContext(sparkConf) try { - //[(Array[Byte], Array[(Array[Byte], Array[Byte], Array[Byte])])] - val rdd = sc.parallelize(Array( - (Bytes.toBytes("1"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("1")))), - (Bytes.toBytes("2"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("2")))), - (Bytes.toBytes("3"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("3")))), - (Bytes.toBytes("4"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("4")))), - (Bytes.toBytes("5"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("5")))) - )) + // [(Array[Byte], Array[(Array[Byte], Array[Byte], Array[Byte])])] + val rdd = sc.parallelize( + Array( + ( + Bytes.toBytes("1"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("1")))), + ( + Bytes.toBytes("2"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("2")))), + ( + Bytes.toBytes("3"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("3")))), + ( + Bytes.toBytes("4"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("4")))), + ( + Bytes.toBytes("5"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("5")))))) val conf = HBaseConfiguration.create() val hbaseContext = new HBaseContext(sc, conf) - - rdd.hbaseForeachPartition(hbaseContext, + rdd.hbaseForeachPartition( + hbaseContext, (it, connection) => { val m = connection.getBufferedMutator(TableName.valueOf(tableName)) - it.foreach(r => { - val put = new Put(r._1) - r._2.foreach((putValue) => - put.addColumn(putValue._1, putValue._2, putValue._3)) - m.mutate(put) - }) + it.foreach( + r => { + val put = new Put(r._1) + r._2.foreach((putValue) => put.addColumn(putValue._1, putValue._2, putValue._3)) + m.mutate(put) + }) m.flush() m.close() }) diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/rdd/HBaseMapPartitionExample.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/rdd/HBaseMapPartitionExample.scala index 07935240..02249921 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/rdd/HBaseMapPartitionExample.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/example/rdd/HBaseMapPartitionExample.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,15 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark.example.rdd +import org.apache.hadoop.hbase.HBaseConfiguration +import org.apache.hadoop.hbase.TableName import org.apache.hadoop.hbase.client.Get import org.apache.hadoop.hbase.spark.HBaseContext import org.apache.hadoop.hbase.spark.HBaseRDDFunctions._ import org.apache.hadoop.hbase.util.Bytes -import org.apache.hadoop.hbase.HBaseConfiguration -import org.apache.hadoop.hbase.TableName import org.apache.spark.SparkConf import org.apache.spark.SparkContext import org.apache.yetus.audience.InterfaceAudience @@ -46,45 +46,51 @@ object HBaseMapPartitionExample { try { - //[(Array[Byte])] - val rdd = sc.parallelize(Array( - Bytes.toBytes("1"), - Bytes.toBytes("2"), - Bytes.toBytes("3"), - Bytes.toBytes("4"), - Bytes.toBytes("5"), - Bytes.toBytes("6"), - Bytes.toBytes("7"))) + // [(Array[Byte])] + val rdd = sc.parallelize( + Array( + Bytes.toBytes("1"), + Bytes.toBytes("2"), + Bytes.toBytes("3"), + Bytes.toBytes("4"), + Bytes.toBytes("5"), + Bytes.toBytes("6"), + Bytes.toBytes("7"))) val conf = HBaseConfiguration.create() val hbaseContext = new HBaseContext(sc, conf) - val getRdd = rdd.hbaseMapPartitions[String](hbaseContext, (it, connection) => { - val table = connection.getTable(TableName.valueOf(tableName)) - it.map{r => - //batching would be faster. This is just an example - val result = table.get(new Get(r)) + val getRdd = rdd.hbaseMapPartitions[String]( + hbaseContext, + (it, connection) => { + val table = connection.getTable(TableName.valueOf(tableName)) + it.map { + r => + // batching would be faster. This is just an example + val result = table.get(new Get(r)) - val it = result.listCells().iterator() - val b = new StringBuilder + val it = result.listCells().iterator() + val b = new StringBuilder - b.append(Bytes.toString(result.getRow) + ":") + b.append(Bytes.toString(result.getRow) + ":") - while (it.hasNext) { - val cell = it.next() - val q = Bytes.toString(cell.getQualifierArray) - if (q.equals("counter")) { - b.append("(" + q + "," + Bytes.toLong(cell.getValueArray) + ")") - } else { - b.append("(" + q + "," + Bytes.toString(cell.getValueArray) + ")") - } + while (it.hasNext) { + val cell = it.next() + val q = Bytes.toString(cell.getQualifierArray) + if (q.equals("counter")) { + b.append("(" + q + "," + Bytes.toLong(cell.getValueArray) + ")") + } else { + b.append("(" + q + "," + Bytes.toString(cell.getValueArray) + ")") + } + } + b.toString() } - b.toString() - } - }) + }) - getRdd.collect().foreach(v => println(v)) + getRdd + .collect() + .foreach(v => println(v)) } finally { sc.stop() diff --git a/spark/hbase-spark/src/test/java/org/apache/hadoop/hbase/spark/TestJavaHBaseContext.java b/spark/hbase-spark/src/test/java/org/apache/hadoop/hbase/spark/TestJavaHBaseContext.java index 6601eb70..0f8c4aa7 100644 --- a/spark/hbase-spark/src/test/java/org/apache/hadoop/hbase/spark/TestJavaHBaseContext.java +++ b/spark/hbase-spark/src/test/java/org/apache/hadoop/hbase/spark/TestJavaHBaseContext.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -63,12 +63,12 @@ import org.slf4j.LoggerFactory; import scala.Tuple2; -@Category({MiscTests.class, MediumTests.class}) +@Category({ MiscTests.class, MediumTests.class }) public class TestJavaHBaseContext implements Serializable { @ClassRule public static final HBaseClassTestRule TIMEOUT = - HBaseClassTestRule.forClass(TestJavaHBaseContext.class); + HBaseClassTestRule.forClass(TestJavaHBaseContext.class); protected static transient JavaSparkContext JSC; private static HBaseTestingUtility TEST_UTIL; @@ -131,7 +131,7 @@ public void setUp() throws Exception { LOG.info(" - creating table {}", Bytes.toString(tableName)); TEST_UTIL.createTable(TableName.valueOf(tableName), - new byte[][]{columnFamily, columnFamily1}); + new byte[][] { columnFamily, columnFamily1 }); LOG.info(" - created table"); } @@ -167,9 +167,7 @@ public void testBulkPut() throws IOException { table.close(); } - HBASE_CONTEXT.bulkPut(rdd, - TableName.valueOf(tableName), - new PutFunction()); + HBASE_CONTEXT.bulkPut(rdd, TableName.valueOf(tableName), new PutFunction()); table = conn.getTable(TableName.valueOf(tableName)); @@ -203,8 +201,7 @@ public Put call(String v) throws Exception { String[] cells = v.split(","); Put put = new Put(Bytes.toBytes(cells[0])); - put.addColumn(Bytes.toBytes(cells[1]), Bytes.toBytes(cells[2]), - Bytes.toBytes(cells[3])); + put.addColumn(Bytes.toBytes(cells[1]), Bytes.toBytes(cells[2]), Bytes.toBytes(cells[3])); return put; } } @@ -223,14 +220,10 @@ public void testBulkDelete() throws IOException { populateTableWithMockData(conf, TableName.valueOf(tableName)); HBASE_CONTEXT.bulkDelete(rdd, TableName.valueOf(tableName), - new JavaHBaseBulkDeleteExample.DeleteFunction(), 2); + new JavaHBaseBulkDeleteExample.DeleteFunction(), 2); - - - try ( - Connection conn = ConnectionFactory.createConnection(conf); - Table table = conn.getTable(TableName.valueOf(tableName)) - ){ + try (Connection conn = ConnectionFactory.createConnection(conf); + Table table = conn.getTable(TableName.valueOf(tableName))) { Result result1 = table.get(new Get(Bytes.toBytes("1"))); Assert.assertNull("Row 1 should had been deleted", result1.getRow()); @@ -258,16 +251,15 @@ public void testDistributedScan() throws IOException { scan.setCaching(100); JavaRDD javaRdd = - HBASE_CONTEXT.hbaseRDD(TableName.valueOf(tableName), scan) - .map(new ScanConvertFunction()); + HBASE_CONTEXT.hbaseRDD(TableName.valueOf(tableName), scan).map(new ScanConvertFunction()); List results = javaRdd.collect(); Assert.assertEquals(results.size(), 5); } - private static class ScanConvertFunction implements - Function, String> { + private static class ScanConvertFunction + implements Function, String> { @Override public String call(Tuple2 v1) throws Exception { return Bytes.toString(v1._1().copyBytes()); @@ -289,10 +281,8 @@ public void testBulkGet() throws IOException { populateTableWithMockData(conf, TableName.valueOf(tableName)); - final JavaRDD stringJavaRDD = - HBASE_CONTEXT.bulkGet(TableName.valueOf(tableName), 2, rdd, - new GetFunction(), - new ResultFunction()); + final JavaRDD stringJavaRDD = HBASE_CONTEXT.bulkGet(TableName.valueOf(tableName), 2, + rdd, new GetFunction(), new ResultFunction()); Assert.assertEquals(stringJavaRDD.count(), 5); } @@ -302,14 +292,14 @@ public void testBulkLoad() throws Exception { Path output = TEST_UTIL.getDataTestDir("testBulkLoad"); // Add cell as String: "row,falmily,qualifier,value" - List list= new ArrayList(); + List list = new ArrayList(); // row1 list.add("1," + columnFamilyStr + ",b,1"); // row3 list.add("3," + columnFamilyStr + ",a,2"); list.add("3," + columnFamilyStr + ",b,1"); list.add("3," + columnFamilyStr1 + ",a,1"); - //row2 + // row2 list.add("2," + columnFamilyStr + ",a,3"); list.add("2," + columnFamilyStr + ",b,3"); @@ -318,18 +308,16 @@ public void testBulkLoad() throws Exception { Configuration conf = TEST_UTIL.getConfiguration(); HBASE_CONTEXT.bulkLoad(rdd, TableName.valueOf(tableName), new BulkLoadFunction(), - output.toUri().getPath(), new HashMap(), false, - HConstants.DEFAULT_MAX_FILE_SIZE); + output.toUri().getPath(), new HashMap(), false, + HConstants.DEFAULT_MAX_FILE_SIZE); try (Connection conn = ConnectionFactory.createConnection(conf); - Admin admin = conn.getAdmin()) { + Admin admin = conn.getAdmin()) { Table table = conn.getTable(TableName.valueOf(tableName)); // Do bulk load LoadIncrementalHFiles load = new LoadIncrementalHFiles(conf); load.doBulkLoad(output, admin, table, conn.getRegionLocator(TableName.valueOf(tableName))); - - // Check row1 List cell1 = table.get(new Get(Bytes.toBytes("1"))).listCells(); Assert.assertEquals(cell1.size(), 1); @@ -367,7 +355,7 @@ public void testBulkLoadThinRows() throws Exception { Path output = TEST_UTIL.getDataTestDir("testBulkLoadThinRows"); // because of the limitation of scala bulkLoadThinRows API // we need to provide data as - List> list= new ArrayList>(); + List> list = new ArrayList>(); // row1 List list1 = new ArrayList(); list1.add("1," + columnFamilyStr + ",b,1"); @@ -378,7 +366,7 @@ public void testBulkLoadThinRows() throws Exception { list3.add("3," + columnFamilyStr + ",b,1"); list3.add("3," + columnFamilyStr1 + ",a,1"); list.add(list3); - //row2 + // row2 List list2 = new ArrayList(); list2.add("2," + columnFamilyStr + ",a,3"); list2.add("2," + columnFamilyStr + ",b,3"); @@ -389,12 +377,11 @@ public void testBulkLoadThinRows() throws Exception { Configuration conf = TEST_UTIL.getConfiguration(); HBASE_CONTEXT.bulkLoadThinRows(rdd, TableName.valueOf(tableName), - new BulkLoadThinRowsFunction(), output.toString(), new HashMap<>(), false, - HConstants.DEFAULT_MAX_FILE_SIZE); - + new BulkLoadThinRowsFunction(), output.toString(), new HashMap<>(), false, + HConstants.DEFAULT_MAX_FILE_SIZE); try (Connection conn = ConnectionFactory.createConnection(conf); - Admin admin = conn.getAdmin()) { + Admin admin = conn.getAdmin()) { Table table = conn.getTable(TableName.valueOf(tableName)); // Do bulk load LoadIncrementalHFiles load = new LoadIncrementalHFiles(conf); @@ -432,27 +419,30 @@ public void testBulkLoadThinRows() throws Exception { } } + public static class BulkLoadFunction - implements Function> { - @Override public Pair call(String v1) throws Exception { + implements Function> { + @Override + public Pair call(String v1) throws Exception { if (v1 == null) { return null; } String[] strs = v1.split(","); - if(strs.length != 4) { + if (strs.length != 4) { return null; } KeyFamilyQualifier kfq = new KeyFamilyQualifier(Bytes.toBytes(strs[0]), - Bytes.toBytes(strs[1]), Bytes.toBytes(strs[2])); + Bytes.toBytes(strs[1]), Bytes.toBytes(strs[2])); return new Pair(kfq, Bytes.toBytes(strs[3])); } } public static class BulkLoadThinRowsFunction - implements Function, Pair> { - @Override public Pair call(List list) { + implements Function, Pair> { + @Override + public Pair call(List list) { if (list == null) { return null; } @@ -495,17 +485,11 @@ public String call(Result result) throws Exception { Cell cell = it.next(); String q = Bytes.toString(CellUtil.cloneQualifier(cell)); if ("counter".equals(q)) { - b.append("(") - .append(q) - .append(",") - .append(Bytes.toLong(CellUtil.cloneValue(cell))) - .append(")"); + b.append("(").append(q).append(",").append(Bytes.toLong(CellUtil.cloneValue(cell))) + .append(")"); } else { - b.append("(") - .append(q) - .append(",") - .append(Bytes.toString(CellUtil.cloneValue(cell))) - .append(")"); + b.append("(").append(q).append(",").append(Bytes.toString(CellUtil.cloneValue(cell))) + .append(")"); } } return b.toString(); @@ -513,11 +497,9 @@ public String call(Result result) throws Exception { } protected void populateTableWithMockData(Configuration conf, TableName tableName) - throws IOException { - try ( - Connection conn = ConnectionFactory.createConnection(conf); - Table table = conn.getTable(tableName); - Admin admin = conn.getAdmin()) { + throws IOException { + try (Connection conn = ConnectionFactory.createConnection(conf); + Table table = conn.getTable(tableName); Admin admin = conn.getAdmin()) { List puts = new ArrayList<>(5); diff --git a/spark/hbase-spark/src/test/java/org/apache/hadoop/hbase/spark/TestJavaHBaseContextForLargeRows.java b/spark/hbase-spark/src/test/java/org/apache/hadoop/hbase/spark/TestJavaHBaseContextForLargeRows.java index 81908ad5..824b53d4 100644 --- a/spark/hbase-spark/src/test/java/org/apache/hadoop/hbase/spark/TestJavaHBaseContextForLargeRows.java +++ b/spark/hbase-spark/src/test/java/org/apache/hadoop/hbase/spark/TestJavaHBaseContextForLargeRows.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -21,7 +21,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; @@ -41,20 +40,21 @@ @Category({ MiscTests.class, MediumTests.class }) public class TestJavaHBaseContextForLargeRows extends TestJavaHBaseContext { - @ClassRule public static final HBaseClassTestRule TIMEOUT = - HBaseClassTestRule.forClass(TestJavaHBaseContextForLargeRows.class); + @ClassRule + public static final HBaseClassTestRule TIMEOUT = + HBaseClassTestRule.forClass(TestJavaHBaseContextForLargeRows.class); - @BeforeClass public static void setUpBeforeClass() throws Exception { + @BeforeClass + public static void setUpBeforeClass() throws Exception { JSC = new JavaSparkContext("local", "JavaHBaseContextSuite"); init(); } protected void populateTableWithMockData(Configuration conf, TableName tableName) - throws IOException { + throws IOException { try (Connection conn = ConnectionFactory.createConnection(conf); - Table table = conn.getTable(tableName); - Admin admin = conn.getAdmin()) { + Table table = conn.getTable(tableName); Admin admin = conn.getAdmin()) { List puts = new ArrayList<>(5); diff --git a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/BulkLoadSuite.scala b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/BulkLoadSuite.scala index 1c148a90..7a9b0968 100644 --- a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/BulkLoadSuite.scala +++ b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/BulkLoadSuite.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,26 +15,23 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark +import java.io.File +import java.net.URI +import java.nio.file.Files import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.hbase.{CellUtil, HBaseTestingUtility, HConstants, TableName} import org.apache.hadoop.hbase.client.{ConnectionFactory, Get} import org.apache.hadoop.hbase.io.hfile.{CacheConfig, HFile} +import org.apache.hadoop.hbase.spark.HBaseRDDFunctions._ import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles -import org.apache.hadoop.hbase.{CellUtil, HBaseTestingUtility, HConstants, TableName} import org.apache.hadoop.hbase.util.Bytes -import org.apache.hadoop.hbase.spark.HBaseRDDFunctions._ import org.apache.spark.SparkContext import org.junit.rules.TemporaryFolder import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} -import java.io.File -import java.net.URI -import java.nio.file.Files - -class BulkLoadSuite extends FunSuite with -BeforeAndAfterEach with BeforeAndAfterAll with Logging { +class BulkLoadSuite extends FunSuite with BeforeAndAfterEach with BeforeAndAfterAll with Logging { @transient var sc: SparkContext = null var TEST_UTIL = new HBaseTestingUtility @@ -42,7 +40,6 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { val columnFamily2 = "f2" val testFolder = new TemporaryFolder() - override def beforeAll() { TEST_UTIL.startMiniCluster() logInfo(" - minicluster started") @@ -56,7 +53,7 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { logInfo(" - created table") - val envMap = Map[String,String](("Xmx", "512m")) + val envMap = Map[String, String](("Xmx", "512m")) sc = new SparkContext("local", "test", null, Nil, envMap) } @@ -69,11 +66,12 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { sc.stop() } - test ("Staging dir: Test usage of staging dir on a separate filesystem") { + test("Staging dir: Test usage of staging dir on a separate filesystem") { val config = TEST_UTIL.getConfiguration logInfo(" - creating table " + tableName) - TEST_UTIL.createTable(TableName.valueOf(tableName), + TEST_UTIL.createTable( + TableName.valueOf(tableName), Array(Bytes.toBytes(columnFamily1), Bytes.toBytes(columnFamily2))) // Test creates rdd with 2 column families and @@ -81,11 +79,14 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { // using bulkLoad functionality. We don't check the load functionality // due the limitations of the HBase Minicluster - val rdd = sc.parallelize(Array( - (Bytes.toBytes("1"), - (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo1"))), - (Bytes.toBytes("2"), - (Bytes.toBytes(columnFamily2), Bytes.toBytes("b"), Bytes.toBytes("bar.2"))))) + val rdd = sc.parallelize( + Array( + ( + Bytes.toBytes("1"), + (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo1"))), + ( + Bytes.toBytes("2"), + (Bytes.toBytes(columnFamily2), Bytes.toBytes("b"), Bytes.toBytes("bar.2"))))) val hbaseContext = new HBaseContext(sc, config) val uri = Files.createTempDirectory("tmpDirPrefix").toUri @@ -93,21 +94,22 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { val stagingFolder = new File(stagingUri) val fs = new Path(stagingUri.toString).getFileSystem(config) try { - hbaseContext.bulkLoad[(Array[Byte], (Array[Byte], Array[Byte], Array[Byte]))](rdd, - TableName.valueOf(tableName), - t => { - val rowKey = t._1 - val family: Array[Byte] = t._2._1 - val qualifier = t._2._2 - val value: Array[Byte] = t._2._3 + hbaseContext.bulkLoad[(Array[Byte], (Array[Byte], Array[Byte], Array[Byte]))]( + rdd, + TableName.valueOf(tableName), + t => { + val rowKey = t._1 + val family: Array[Byte] = t._2._1 + val qualifier = t._2._2 + val value: Array[Byte] = t._2._3 - val keyFamilyQualifier = new KeyFamilyQualifier(rowKey, family, qualifier) + val keyFamilyQualifier = new KeyFamilyQualifier(rowKey, family, qualifier) - Seq((keyFamilyQualifier, value)).iterator - }, - stagingUri.toString) + Seq((keyFamilyQualifier, value)).iterator + }, + stagingUri.toString) - assert(fs.listStatus(new Path(stagingFolder.getPath)).length == 2) + assert(fs.listStatus(new Path(stagingFolder.getPath)).length == 2) } finally { val admin = ConnectionFactory.createConnection(config).getAdmin @@ -124,56 +126,67 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { } } - test("Wide Row Bulk Load: Test multi family and multi column tests " + - "with all default HFile Configs.") { + test( + "Wide Row Bulk Load: Test multi family and multi column tests " + + "with all default HFile Configs.") { val config = TEST_UTIL.getConfiguration logInfo(" - creating table " + tableName) - TEST_UTIL.createTable(TableName.valueOf(tableName), + TEST_UTIL.createTable( + TableName.valueOf(tableName), Array(Bytes.toBytes(columnFamily1), Bytes.toBytes(columnFamily2))) - //There are a number of tests in here. + // There are a number of tests in here. // 1. Row keys are not in order // 2. Qualifiers are not in order // 3. Column Families are not in order // 4. There are tests for records in one column family and some in two column families // 5. There are records will a single qualifier and some with two - val rdd = sc.parallelize(Array( - (Bytes.toBytes("1"), - (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo1"))), - (Bytes.toBytes("3"), - (Bytes.toBytes(columnFamily2), Bytes.toBytes("b"), Bytes.toBytes("foo2.a"))), - (Bytes.toBytes("3"), - (Bytes.toBytes(columnFamily2), Bytes.toBytes("a"), Bytes.toBytes("foo2.b"))), - (Bytes.toBytes("3"), - (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo2.c"))), - (Bytes.toBytes("5"), - (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo3"))), - (Bytes.toBytes("4"), - (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo.1"))), - (Bytes.toBytes("4"), - (Bytes.toBytes(columnFamily2), Bytes.toBytes("b"), Bytes.toBytes("foo.2"))), - (Bytes.toBytes("2"), - (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("bar.1"))), - (Bytes.toBytes("2"), - (Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("bar.2"))))) - - + val rdd = sc.parallelize( + Array( + ( + Bytes.toBytes("1"), + (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo1"))), + ( + Bytes.toBytes("3"), + (Bytes.toBytes(columnFamily2), Bytes.toBytes("b"), Bytes.toBytes("foo2.a"))), + ( + Bytes.toBytes("3"), + (Bytes.toBytes(columnFamily2), Bytes.toBytes("a"), Bytes.toBytes("foo2.b"))), + ( + Bytes.toBytes("3"), + (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo2.c"))), + ( + Bytes.toBytes("5"), + (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo3"))), + ( + Bytes.toBytes("4"), + (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo.1"))), + ( + Bytes.toBytes("4"), + (Bytes.toBytes(columnFamily2), Bytes.toBytes("b"), Bytes.toBytes("foo.2"))), + ( + Bytes.toBytes("2"), + (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("bar.1"))), + ( + Bytes.toBytes("2"), + (Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("bar.2"))))) val hbaseContext = new HBaseContext(sc, config) testFolder.create() val stagingFolder = testFolder.newFolder() - hbaseContext.bulkLoad[(Array[Byte], (Array[Byte], Array[Byte], Array[Byte]))](rdd, + hbaseContext.bulkLoad[(Array[Byte], (Array[Byte], Array[Byte], Array[Byte]))]( + rdd, TableName.valueOf(tableName), t => { val rowKey = t._1 - val family:Array[Byte] = t._2._1 + val family: Array[Byte] = t._2._1 val qualifier = t._2._2 - val value:Array[Byte] = t._2._3 + val value: Array[Byte] = t._2._3 - val keyFamilyQualifier= new KeyFamilyQualifier(rowKey, family, qualifier) + val keyFamilyQualifier = new KeyFamilyQualifier(rowKey, family, qualifier) Seq((keyFamilyQualifier, value)).iterator }, @@ -187,7 +200,10 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { val load = new LoadIncrementalHFiles(config) val table = conn.getTable(TableName.valueOf(tableName)) try { - load.doBulkLoad(new Path(stagingFolder.getPath), conn.getAdmin, table, + load.doBulkLoad( + new Path(stagingFolder.getPath), + conn.getAdmin, + table, conn.getRegionLocator(TableName.valueOf(tableName))) val cells5 = table.get(new Get(Bytes.toBytes("5"))).listCells() @@ -217,7 +233,6 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { assert(Bytes.toString(CellUtil.cloneFamily(cells3.get(2))).equals("f2")) assert(Bytes.toString(CellUtil.cloneQualifier(cells3.get(2))).equals("b")) - val cells2 = table.get(new Get(Bytes.toBytes("2"))).listCells() assert(cells2.size == 2) assert(Bytes.toString(CellUtil.cloneValue(cells2.get(0))).equals("bar.1")) @@ -249,54 +264,67 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { } } - test("Wide Row Bulk Load: Test HBase client: Test Roll Over and " + - "using an implicit call to bulk load") { + test( + "Wide Row Bulk Load: Test HBase client: Test Roll Over and " + + "using an implicit call to bulk load") { val config = TEST_UTIL.getConfiguration logInfo(" - creating table " + tableName) - TEST_UTIL.createTable(TableName.valueOf(tableName), + TEST_UTIL.createTable( + TableName.valueOf(tableName), Array(Bytes.toBytes(columnFamily1), Bytes.toBytes(columnFamily2))) - //There are a number of tests in here. + // There are a number of tests in here. // 1. Row keys are not in order // 2. Qualifiers are not in order // 3. Column Families are not in order // 4. There are tests for records in one column family and some in two column families // 5. There are records will a single qualifier and some with two - val rdd = sc.parallelize(Array( - (Bytes.toBytes("1"), - (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo1"))), - (Bytes.toBytes("3"), - (Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("foo2.b"))), - (Bytes.toBytes("3"), - (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo2.a"))), - (Bytes.toBytes("3"), - (Bytes.toBytes(columnFamily1), Bytes.toBytes("c"), Bytes.toBytes("foo2.c"))), - (Bytes.toBytes("5"), - (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo3"))), - (Bytes.toBytes("4"), - (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo.1"))), - (Bytes.toBytes("4"), - (Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("foo.2"))), - (Bytes.toBytes("2"), - (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("bar.1"))), - (Bytes.toBytes("2"), - (Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("bar.2"))))) + val rdd = sc.parallelize( + Array( + ( + Bytes.toBytes("1"), + (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo1"))), + ( + Bytes.toBytes("3"), + (Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("foo2.b"))), + ( + Bytes.toBytes("3"), + (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo2.a"))), + ( + Bytes.toBytes("3"), + (Bytes.toBytes(columnFamily1), Bytes.toBytes("c"), Bytes.toBytes("foo2.c"))), + ( + Bytes.toBytes("5"), + (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo3"))), + ( + Bytes.toBytes("4"), + (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo.1"))), + ( + Bytes.toBytes("4"), + (Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("foo.2"))), + ( + Bytes.toBytes("2"), + (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("bar.1"))), + ( + Bytes.toBytes("2"), + (Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("bar.2"))))) val hbaseContext = new HBaseContext(sc, config) testFolder.create() val stagingFolder = testFolder.newFolder() - rdd.hbaseBulkLoad(hbaseContext, + rdd.hbaseBulkLoad( + hbaseContext, TableName.valueOf(tableName), t => { val rowKey = t._1 - val family:Array[Byte] = t._2._1 + val family: Array[Byte] = t._2._1 val qualifier = t._2._2 val value = t._2._3 - val keyFamilyQualifier= new KeyFamilyQualifier(rowKey, family, qualifier) + val keyFamilyQualifier = new KeyFamilyQualifier(rowKey, family, qualifier) Seq((keyFamilyQualifier, value)).iterator }, @@ -308,15 +336,18 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { val fs = FileSystem.get(config) assert(fs.listStatus(new Path(stagingFolder.getPath)).length == 1) - assert(fs.listStatus(new Path(stagingFolder.getPath+ "/f1")).length == 5) + assert(fs.listStatus(new Path(stagingFolder.getPath + "/f1")).length == 5) val conn = ConnectionFactory.createConnection(config) val load = new LoadIncrementalHFiles(config) val table = conn.getTable(TableName.valueOf(tableName)) try { - load.doBulkLoad(new Path(stagingFolder.getPath), - conn.getAdmin, table, conn.getRegionLocator(TableName.valueOf(tableName))) + load.doBulkLoad( + new Path(stagingFolder.getPath), + conn.getAdmin, + table, + conn.getRegionLocator(TableName.valueOf(tableName))) val cells5 = table.get(new Get(Bytes.toBytes("5"))).listCells() assert(cells5.size == 1) @@ -375,44 +406,56 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { } } - test("Wide Row Bulk Load: Test multi family and multi column tests" + - " with one column family with custom configs plus multi region") { + test( + "Wide Row Bulk Load: Test multi family and multi column tests" + + " with one column family with custom configs plus multi region") { val config = TEST_UTIL.getConfiguration - val splitKeys:Array[Array[Byte]] = new Array[Array[Byte]](2) + val splitKeys: Array[Array[Byte]] = new Array[Array[Byte]](2) splitKeys(0) = Bytes.toBytes("2") splitKeys(1) = Bytes.toBytes("4") logInfo(" - creating table " + tableName) - TEST_UTIL.createTable(TableName.valueOf(tableName), + TEST_UTIL.createTable( + TableName.valueOf(tableName), Array(Bytes.toBytes(columnFamily1), Bytes.toBytes(columnFamily2)), splitKeys) - //There are a number of tests in here. + // There are a number of tests in here. // 1. Row keys are not in order // 2. Qualifiers are not in order // 3. Column Families are not in order // 4. There are tests for records in one column family and some in two column families // 5. There are records will a single qualifier and some with two - val rdd = sc.parallelize(Array( - (Bytes.toBytes("1"), - (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo1"))), - (Bytes.toBytes("3"), - (Bytes.toBytes(columnFamily2), Bytes.toBytes("b"), Bytes.toBytes("foo2.a"))), - (Bytes.toBytes("3"), - (Bytes.toBytes(columnFamily2), Bytes.toBytes("a"), Bytes.toBytes("foo2.b"))), - (Bytes.toBytes("3"), - (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo2.c"))), - (Bytes.toBytes("5"), - (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo3"))), - (Bytes.toBytes("4"), - (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo.1"))), - (Bytes.toBytes("4"), - (Bytes.toBytes(columnFamily2), Bytes.toBytes("b"), Bytes.toBytes("foo.2"))), - (Bytes.toBytes("2"), - (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("bar.1"))), - (Bytes.toBytes("2"), - (Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("bar.2"))))) + val rdd = sc.parallelize( + Array( + ( + Bytes.toBytes("1"), + (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo1"))), + ( + Bytes.toBytes("3"), + (Bytes.toBytes(columnFamily2), Bytes.toBytes("b"), Bytes.toBytes("foo2.a"))), + ( + Bytes.toBytes("3"), + (Bytes.toBytes(columnFamily2), Bytes.toBytes("a"), Bytes.toBytes("foo2.b"))), + ( + Bytes.toBytes("3"), + (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo2.c"))), + ( + Bytes.toBytes("5"), + (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo3"))), + ( + Bytes.toBytes("4"), + (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo.1"))), + ( + Bytes.toBytes("4"), + (Bytes.toBytes(columnFamily2), Bytes.toBytes("b"), Bytes.toBytes("foo.2"))), + ( + Bytes.toBytes("2"), + (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("bar.1"))), + ( + Bytes.toBytes("2"), + (Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("bar.2"))))) val hbaseContext = new HBaseContext(sc, config) @@ -421,20 +464,20 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { val familyHBaseWriterOptions = new java.util.HashMap[Array[Byte], FamilyHFileWriteOptions] - val f1Options = new FamilyHFileWriteOptions("GZ", "ROW", 128, - "PREFIX") + val f1Options = new FamilyHFileWriteOptions("GZ", "ROW", 128, "PREFIX") familyHBaseWriterOptions.put(Bytes.toBytes(columnFamily1), f1Options) - hbaseContext.bulkLoad[(Array[Byte], (Array[Byte], Array[Byte], Array[Byte]))](rdd, + hbaseContext.bulkLoad[(Array[Byte], (Array[Byte], Array[Byte], Array[Byte]))]( + rdd, TableName.valueOf(tableName), t => { val rowKey = t._1 - val family:Array[Byte] = t._2._1 + val family: Array[Byte] = t._2._1 val qualifier = t._2._2 val value = t._2._3 - val keyFamilyQualifier= new KeyFamilyQualifier(rowKey, family, qualifier) + val keyFamilyQualifier = new KeyFamilyQualifier(rowKey, family, qualifier) Seq((keyFamilyQualifier, value)).iterator }, @@ -446,34 +489,36 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { val fs = FileSystem.get(config) assert(fs.listStatus(new Path(stagingFolder.getPath)).length == 2) - val f1FileList = fs.listStatus(new Path(stagingFolder.getPath +"/f1")) - for ( i <- 0 until f1FileList.length) { - val reader = HFile.createReader(fs, f1FileList(i).getPath, - new CacheConfig(config), true, config) + val f1FileList = fs.listStatus(new Path(stagingFolder.getPath + "/f1")) + for (i <- 0 until f1FileList.length) { + val reader = + HFile.createReader(fs, f1FileList(i).getPath, new CacheConfig(config), true, config) assert(reader.getTrailer.getCompressionCodec().getName.equals("gz")) assert(reader.getDataBlockEncoding.name().equals("PREFIX")) } - assert( 3 == f1FileList.length) + assert(3 == f1FileList.length) - val f2FileList = fs.listStatus(new Path(stagingFolder.getPath +"/f2")) - for ( i <- 0 until f2FileList.length) { - val reader = HFile.createReader(fs, f2FileList(i).getPath, - new CacheConfig(config), true, config) + val f2FileList = fs.listStatus(new Path(stagingFolder.getPath + "/f2")) + for (i <- 0 until f2FileList.length) { + val reader = + HFile.createReader(fs, f2FileList(i).getPath, new CacheConfig(config), true, config) assert(reader.getTrailer.getCompressionCodec().getName.equals("none")) assert(reader.getDataBlockEncoding.name().equals("NONE")) } - assert( 2 == f2FileList.length) - + assert(2 == f2FileList.length) val conn = ConnectionFactory.createConnection(config) val load = new LoadIncrementalHFiles(config) val table = conn.getTable(TableName.valueOf(tableName)) try { - load.doBulkLoad(new Path(stagingFolder.getPath), - conn.getAdmin, table, conn.getRegionLocator(TableName.valueOf(tableName))) + load.doBulkLoad( + new Path(stagingFolder.getPath), + conn.getAdmin, + table, + conn.getRegionLocator(TableName.valueOf(tableName))) val cells5 = table.get(new Get(Bytes.toBytes("5"))).listCells() assert(cells5.size == 1) @@ -502,7 +547,6 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { assert(Bytes.toString(CellUtil.cloneFamily(cells3.get(2))).equals("f2")) assert(Bytes.toString(CellUtil.cloneQualifier(cells3.get(2))).equals("b")) - val cells2 = table.get(new Get(Bytes.toBytes("2"))).listCells() assert(cells2.size == 2) assert(Bytes.toString(CellUtil.cloneValue(cells2.get(0))).equals("bar.1")) @@ -536,7 +580,7 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { test("Test partitioner") { - var splitKeys:Array[Array[Byte]] = new Array[Array[Byte]](3) + var splitKeys: Array[Array[Byte]] = new Array[Array[Byte]](3) splitKeys(0) = Bytes.toBytes("") splitKeys(1) = Bytes.toBytes("3") splitKeys(2) = Bytes.toBytes("7") @@ -552,7 +596,6 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { assert(2 == partitioner.getPartition(Bytes.toBytes("7"))) assert(2 == partitioner.getPartition(Bytes.toBytes("8"))) - splitKeys = new Array[Array[Byte]](1) splitKeys(0) = Bytes.toBytes("") @@ -593,59 +636,56 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { assert(6 == partitioner.getPartition(Bytes.toBytes("13"))) } - test("Thin Row Bulk Load: Test multi family and multi column tests " + - "with all default HFile Configs") { + test( + "Thin Row Bulk Load: Test multi family and multi column tests " + + "with all default HFile Configs") { val config = TEST_UTIL.getConfiguration logInfo(" - creating table " + tableName) - TEST_UTIL.createTable(TableName.valueOf(tableName), + TEST_UTIL.createTable( + TableName.valueOf(tableName), Array(Bytes.toBytes(columnFamily1), Bytes.toBytes(columnFamily2))) - //There are a number of tests in here. + // There are a number of tests in here. // 1. Row keys are not in order // 2. Qualifiers are not in order // 3. Column Families are not in order // 4. There are tests for records in one column family and some in two column families // 5. There are records will a single qualifier and some with two - val rdd = sc.parallelize(Array( - ("1", - (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo1"))), - ("3", - (Bytes.toBytes(columnFamily2), Bytes.toBytes("b"), Bytes.toBytes("foo2.a"))), - ("3", - (Bytes.toBytes(columnFamily2), Bytes.toBytes("a"), Bytes.toBytes("foo2.b"))), - ("3", - (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo2.c"))), - ("5", - (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo3"))), - ("4", - (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo.1"))), - ("4", - (Bytes.toBytes(columnFamily2), Bytes.toBytes("b"), Bytes.toBytes("foo.2"))), - ("2", - (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("bar.1"))), - ("2", - (Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("bar.2"))))). - groupByKey() + val rdd = sc + .parallelize( + Array( + ("1", (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo1"))), + ("3", (Bytes.toBytes(columnFamily2), Bytes.toBytes("b"), Bytes.toBytes("foo2.a"))), + ("3", (Bytes.toBytes(columnFamily2), Bytes.toBytes("a"), Bytes.toBytes("foo2.b"))), + ("3", (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo2.c"))), + ("5", (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo3"))), + ("4", (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo.1"))), + ("4", (Bytes.toBytes(columnFamily2), Bytes.toBytes("b"), Bytes.toBytes("foo.2"))), + ("2", (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("bar.1"))), + ("2", (Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("bar.2"))))) + .groupByKey() val hbaseContext = new HBaseContext(sc, config) testFolder.create() val stagingFolder = testFolder.newFolder() - hbaseContext.bulkLoadThinRows[(String, Iterable[(Array[Byte], Array[Byte], Array[Byte])])](rdd, + hbaseContext.bulkLoadThinRows[(String, Iterable[(Array[Byte], Array[Byte], Array[Byte])])]( + rdd, TableName.valueOf(tableName), t => { val rowKey = Bytes.toBytes(t._1) val familyQualifiersValues = new FamiliesQualifiersValues - t._2.foreach(f => { - val family:Array[Byte] = f._1 - val qualifier = f._2 - val value:Array[Byte] = f._3 - - familyQualifiersValues +=(family, qualifier, value) - }) + t._2.foreach( + f => { + val family: Array[Byte] = f._1 + val qualifier = f._2 + val value: Array[Byte] = f._3 + + familyQualifiersValues += (family, qualifier, value) + }) (new ByteArrayWrapper(rowKey), familyQualifiersValues) }, stagingFolder.getPath) @@ -658,7 +698,10 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { val load = new LoadIncrementalHFiles(config) val table = conn.getTable(TableName.valueOf(tableName)) try { - load.doBulkLoad(new Path(stagingFolder.getPath), conn.getAdmin, table, + load.doBulkLoad( + new Path(stagingFolder.getPath), + conn.getAdmin, + table, conn.getRegionLocator(TableName.valueOf(tableName))) val cells5 = table.get(new Get(Bytes.toBytes("5"))).listCells() @@ -688,7 +731,6 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { assert(Bytes.toString(CellUtil.cloneFamily(cells3.get(2))).equals("f2")) assert(Bytes.toString(CellUtil.cloneQualifier(cells3.get(2))).equals("b")) - val cells2 = table.get(new Get(Bytes.toBytes("2"))).listCells() assert(cells2.size == 2) assert(Bytes.toString(CellUtil.cloneValue(cells2.get(0))).equals("bar.1")) @@ -720,59 +762,56 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { } } - test("Thin Row Bulk Load: Test HBase client: Test Roll Over and " + - "using an implicit call to bulk load") { + test( + "Thin Row Bulk Load: Test HBase client: Test Roll Over and " + + "using an implicit call to bulk load") { val config = TEST_UTIL.getConfiguration logInfo(" - creating table " + tableName) - TEST_UTIL.createTable(TableName.valueOf(tableName), + TEST_UTIL.createTable( + TableName.valueOf(tableName), Array(Bytes.toBytes(columnFamily1), Bytes.toBytes(columnFamily2))) - //There are a number of tests in here. + // There are a number of tests in here. // 1. Row keys are not in order // 2. Qualifiers are not in order // 3. Column Families are not in order // 4. There are tests for records in one column family and some in two column families // 5. There are records will a single qualifier and some with two - val rdd = sc.parallelize(Array( - ("1", - (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo1"))), - ("3", - (Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("foo2.b"))), - ("3", - (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo2.a"))), - ("3", - (Bytes.toBytes(columnFamily1), Bytes.toBytes("c"), Bytes.toBytes("foo2.c"))), - ("5", - (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo3"))), - ("4", - (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo.1"))), - ("4", - (Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("foo.2"))), - ("2", - (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("bar.1"))), - ("2", - (Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("bar.2"))))). - groupByKey() + val rdd = sc + .parallelize( + Array( + ("1", (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo1"))), + ("3", (Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("foo2.b"))), + ("3", (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo2.a"))), + ("3", (Bytes.toBytes(columnFamily1), Bytes.toBytes("c"), Bytes.toBytes("foo2.c"))), + ("5", (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo3"))), + ("4", (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo.1"))), + ("4", (Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("foo.2"))), + ("2", (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("bar.1"))), + ("2", (Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("bar.2"))))) + .groupByKey() val hbaseContext = new HBaseContext(sc, config) testFolder.create() val stagingFolder = testFolder.newFolder() - rdd.hbaseBulkLoadThinRows(hbaseContext, + rdd.hbaseBulkLoadThinRows( + hbaseContext, TableName.valueOf(tableName), t => { val rowKey = t._1 val familyQualifiersValues = new FamiliesQualifiersValues - t._2.foreach(f => { - val family:Array[Byte] = f._1 - val qualifier = f._2 - val value:Array[Byte] = f._3 - - familyQualifiersValues +=(family, qualifier, value) - }) + t._2.foreach( + f => { + val family: Array[Byte] = f._1 + val qualifier = f._2 + val value: Array[Byte] = f._3 + + familyQualifiersValues += (family, qualifier, value) + }) (new ByteArrayWrapper(Bytes.toBytes(rowKey)), familyQualifiersValues) }, stagingFolder.getPath, @@ -783,15 +822,18 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { val fs = FileSystem.get(config) assert(fs.listStatus(new Path(stagingFolder.getPath)).length == 1) - assert(fs.listStatus(new Path(stagingFolder.getPath+ "/f1")).length == 5) + assert(fs.listStatus(new Path(stagingFolder.getPath + "/f1")).length == 5) val conn = ConnectionFactory.createConnection(config) val load = new LoadIncrementalHFiles(config) val table = conn.getTable(TableName.valueOf(tableName)) try { - load.doBulkLoad(new Path(stagingFolder.getPath), - conn.getAdmin, table, conn.getRegionLocator(TableName.valueOf(tableName))) + load.doBulkLoad( + new Path(stagingFolder.getPath), + conn.getAdmin, + table, + conn.getRegionLocator(TableName.valueOf(tableName))) val cells5 = table.get(new Get(Bytes.toBytes("5"))).listCells() assert(cells5.size == 1) @@ -850,45 +892,40 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { } } - test("Thin Row Bulk Load: Test multi family and multi column tests" + - " with one column family with custom configs plus multi region") { + test( + "Thin Row Bulk Load: Test multi family and multi column tests" + + " with one column family with custom configs plus multi region") { val config = TEST_UTIL.getConfiguration - val splitKeys:Array[Array[Byte]] = new Array[Array[Byte]](2) + val splitKeys: Array[Array[Byte]] = new Array[Array[Byte]](2) splitKeys(0) = Bytes.toBytes("2") splitKeys(1) = Bytes.toBytes("4") logInfo(" - creating table " + tableName) - TEST_UTIL.createTable(TableName.valueOf(tableName), + TEST_UTIL.createTable( + TableName.valueOf(tableName), Array(Bytes.toBytes(columnFamily1), Bytes.toBytes(columnFamily2)), splitKeys) - //There are a number of tests in here. + // There are a number of tests in here. // 1. Row keys are not in order // 2. Qualifiers are not in order // 3. Column Families are not in order // 4. There are tests for records in one column family and some in two column families // 5. There are records will a single qualifier and some with two - val rdd = sc.parallelize(Array( - ("1", - (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo1"))), - ("3", - (Bytes.toBytes(columnFamily2), Bytes.toBytes("b"), Bytes.toBytes("foo2.a"))), - ("3", - (Bytes.toBytes(columnFamily2), Bytes.toBytes("a"), Bytes.toBytes("foo2.b"))), - ("3", - (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo2.c"))), - ("5", - (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo3"))), - ("4", - (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo.1"))), - ("4", - (Bytes.toBytes(columnFamily2), Bytes.toBytes("b"), Bytes.toBytes("foo.2"))), - ("2", - (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("bar.1"))), - ("2", - (Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("bar.2"))))). - groupByKey() + val rdd = sc + .parallelize( + Array( + ("1", (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo1"))), + ("3", (Bytes.toBytes(columnFamily2), Bytes.toBytes("b"), Bytes.toBytes("foo2.a"))), + ("3", (Bytes.toBytes(columnFamily2), Bytes.toBytes("a"), Bytes.toBytes("foo2.b"))), + ("3", (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo2.c"))), + ("5", (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo3"))), + ("4", (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo.1"))), + ("4", (Bytes.toBytes(columnFamily2), Bytes.toBytes("b"), Bytes.toBytes("foo.2"))), + ("2", (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("bar.1"))), + ("2", (Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("bar.2"))))) + .groupByKey() val hbaseContext = new HBaseContext(sc, config) @@ -897,24 +934,25 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { val familyHBaseWriterOptions = new java.util.HashMap[Array[Byte], FamilyHFileWriteOptions] - val f1Options = new FamilyHFileWriteOptions("GZ", "ROW", 128, - "PREFIX") + val f1Options = new FamilyHFileWriteOptions("GZ", "ROW", 128, "PREFIX") familyHBaseWriterOptions.put(Bytes.toBytes(columnFamily1), f1Options) - hbaseContext.bulkLoadThinRows[(String, Iterable[(Array[Byte], Array[Byte], Array[Byte])])](rdd, + hbaseContext.bulkLoadThinRows[(String, Iterable[(Array[Byte], Array[Byte], Array[Byte])])]( + rdd, TableName.valueOf(tableName), t => { val rowKey = t._1 val familyQualifiersValues = new FamiliesQualifiersValues - t._2.foreach(f => { - val family:Array[Byte] = f._1 - val qualifier = f._2 - val value:Array[Byte] = f._3 - - familyQualifiersValues +=(family, qualifier, value) - }) + t._2.foreach( + f => { + val family: Array[Byte] = f._1 + val qualifier = f._2 + val value: Array[Byte] = f._3 + + familyQualifiersValues += (family, qualifier, value) + }) (new ByteArrayWrapper(Bytes.toBytes(rowKey)), familyQualifiersValues) }, stagingFolder.getPath, @@ -925,34 +963,36 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { val fs = FileSystem.get(config) assert(fs.listStatus(new Path(stagingFolder.getPath)).length == 2) - val f1FileList = fs.listStatus(new Path(stagingFolder.getPath +"/f1")) - for ( i <- 0 until f1FileList.length) { - val reader = HFile.createReader(fs, f1FileList(i).getPath, - new CacheConfig(config), true, config) + val f1FileList = fs.listStatus(new Path(stagingFolder.getPath + "/f1")) + for (i <- 0 until f1FileList.length) { + val reader = + HFile.createReader(fs, f1FileList(i).getPath, new CacheConfig(config), true, config) assert(reader.getTrailer.getCompressionCodec().getName.equals("gz")) assert(reader.getDataBlockEncoding.name().equals("PREFIX")) } - assert( 3 == f1FileList.length) + assert(3 == f1FileList.length) - val f2FileList = fs.listStatus(new Path(stagingFolder.getPath +"/f2")) - for ( i <- 0 until f2FileList.length) { - val reader = HFile.createReader(fs, f2FileList(i).getPath, - new CacheConfig(config), true, config) + val f2FileList = fs.listStatus(new Path(stagingFolder.getPath + "/f2")) + for (i <- 0 until f2FileList.length) { + val reader = + HFile.createReader(fs, f2FileList(i).getPath, new CacheConfig(config), true, config) assert(reader.getTrailer.getCompressionCodec().getName.equals("none")) assert(reader.getDataBlockEncoding.name().equals("NONE")) } - assert( 2 == f2FileList.length) - + assert(2 == f2FileList.length) val conn = ConnectionFactory.createConnection(config) val load = new LoadIncrementalHFiles(config) val table = conn.getTable(TableName.valueOf(tableName)) try { - load.doBulkLoad(new Path(stagingFolder.getPath), - conn.getAdmin, table, conn.getRegionLocator(TableName.valueOf(tableName))) + load.doBulkLoad( + new Path(stagingFolder.getPath), + conn.getAdmin, + table, + conn.getRegionLocator(TableName.valueOf(tableName))) val cells5 = table.get(new Get(Bytes.toBytes("5"))).listCells() assert(cells5.size == 1) @@ -981,7 +1021,6 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { assert(Bytes.toString(CellUtil.cloneFamily(cells3.get(2))).equals("f2")) assert(Bytes.toString(CellUtil.cloneQualifier(cells3.get(2))).equals("b")) - val cells2 = table.get(new Get(Bytes.toBytes("2"))).listCells() assert(cells2.size == 2) assert(Bytes.toString(CellUtil.cloneValue(cells2.get(0))).equals("bar.1")) diff --git a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DefaultSourceSuite.scala b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DefaultSourceSuite.scala index 0f402db8..60b5d3c1 100644 --- a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DefaultSourceSuite.scala +++ b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DefaultSourceSuite.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,38 +15,37 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark import java.sql.{Date, Timestamp} - import org.apache.avro.Schema import org.apache.avro.generic.GenericData +import org.apache.hadoop.hbase.{HBaseTestingUtility, TableName} import org.apache.hadoop.hbase.client.{ConnectionFactory, Put} import org.apache.hadoop.hbase.spark.datasources.{HBaseSparkConf, HBaseTableCatalog} import org.apache.hadoop.hbase.util.Bytes -import org.apache.hadoop.hbase.{HBaseTestingUtility, TableName} -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.{DataFrame, SQLContext} import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.functions._ import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} import org.xml.sax.SAXParseException case class HBaseRecord( - col0: String, - col1: Boolean, - col2: Double, - col3: Float, - col4: Int, - col5: Long, - col6: Short, - col7: String, - col8: Byte) + col0: String, + col1: Boolean, + col2: Double, + col3: Float, + col4: Int, + col5: Long, + col6: Short, + col7: String, + col8: Byte) object HBaseRecord { def apply(i: Int, t: String): HBaseRecord = { val s = s"""row${"%03d".format(i)}""" - HBaseRecord(s, + HBaseRecord( + s, i % 2 == 0, i.toDouble, i.toFloat, @@ -57,9 +57,7 @@ object HBaseRecord { } } - -case class AvroHBaseKeyRecord(col0: Array[Byte], - col1: Array[Byte]) +case class AvroHBaseKeyRecord(col0: Array[Byte], col1: Array[Byte]) object AvroHBaseKeyRecord { val schemaString = @@ -84,8 +82,11 @@ object AvroHBaseKeyRecord { } } -class DefaultSourceSuite extends FunSuite with -BeforeAndAfterEach with BeforeAndAfterAll with Logging { +class DefaultSourceSuite + extends FunSuite + with BeforeAndAfterEach + with BeforeAndAfterAll + with Logging { @transient var sc: SparkContext = null var TEST_UTIL: HBaseTestingUtility = new HBaseTestingUtility @@ -96,8 +97,8 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { val timestamp = 1234567890000L - var sqlContext:SQLContext = null - var df:DataFrame = null + var sqlContext: SQLContext = null + var df: DataFrame = null override def beforeAll() { @@ -135,7 +136,7 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { sparkConf.set(HBaseSparkConf.QUERY_BATCHSIZE, "100") sparkConf.set(HBaseSparkConf.QUERY_CACHEDROWS, "100") - sc = new SparkContext("local", "test", sparkConf) + sc = new SparkContext("local", "test", sparkConf) val connection = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration) try { @@ -211,16 +212,28 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { try { val put = new Put(Bytes.toBytes("row")) - put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("binary"), Array(1.toByte, 2.toByte, 3.toByte)) + put.addColumn( + Bytes.toBytes(columnFamily), + Bytes.toBytes("binary"), + Array(1.toByte, 2.toByte, 3.toByte)) put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("boolean"), Bytes.toBytes(true)) put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("byte"), Array(127.toByte)) - put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("short"), Bytes.toBytes(32767.toShort)) + put.addColumn( + Bytes.toBytes(columnFamily), + Bytes.toBytes("short"), + Bytes.toBytes(32767.toShort)) put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("int"), Bytes.toBytes(1000000)) - put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("long"), Bytes.toBytes(10000000000L)) + put.addColumn( + Bytes.toBytes(columnFamily), + Bytes.toBytes("long"), + Bytes.toBytes(10000000000L)) put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("float"), Bytes.toBytes(0.5f)) put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("double"), Bytes.toBytes(0.125)) put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("date"), Bytes.toBytes(timestamp)) - put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("timestamp"), Bytes.toBytes(timestamp)) + put.addColumn( + Bytes.toBytes(columnFamily), + Bytes.toBytes("timestamp"), + Bytes.toBytes(timestamp)) put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("string"), Bytes.toBytes("string")) t3Table.put(put) } finally { @@ -243,8 +256,9 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { new HBaseContext(sc, TEST_UTIL.getConfiguration) sqlContext = new SQLContext(sc) - df = sqlContext.load("org.apache.hadoop.hbase.spark", - Map(HBaseTableCatalog.tableCatalog->hbaseTable1Catalog)) + df = sqlContext.load( + "org.apache.hadoop.hbase.spark", + Map(HBaseTableCatalog.tableCatalog -> hbaseTable1Catalog)) df.registerTempTable("hbaseTable1") @@ -258,9 +272,9 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { |} |}""".stripMargin - - df = sqlContext.load("org.apache.hadoop.hbase.spark", - Map(HBaseTableCatalog.tableCatalog->hbaseTable2Catalog)) + df = sqlContext.load( + "org.apache.hadoop.hbase.spark", + Map(HBaseTableCatalog.tableCatalog -> hbaseTable2Catalog)) df.registerTempTable("hbaseTable2") } @@ -277,21 +291,24 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { DefaultSourceStaticUtils.lastFiveExecutionRules.clear() } - /** * A example of query three fields and also only using rowkey points for the filter */ test("Test rowKey point only rowKey query") { - val results = sqlContext.sql("SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseTable1 " + - "WHERE " + - "(KEY_FIELD = 'get1' or KEY_FIELD = 'get2' or KEY_FIELD = 'get3')").take(10) + val results = sqlContext + .sql( + "SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseTable1 " + + "WHERE " + + "(KEY_FIELD = 'get1' or KEY_FIELD = 'get2' or KEY_FIELD = 'get3')") + .take(10) val executionRules = DefaultSourceStaticUtils.lastFiveExecutionRules.poll() assert(results.length == 3) - assert(executionRules.dynamicLogicExpression.toExpressionString. - equals("( ( KEY_FIELD == 0 OR KEY_FIELD == 1 ) OR KEY_FIELD == 2 )")) + assert( + executionRules.dynamicLogicExpression.toExpressionString.equals( + "( ( KEY_FIELD == 0 OR KEY_FIELD == 1 ) OR KEY_FIELD == 2 )")) assert(executionRules.rowKeyFilter.points.size == 3) assert(executionRules.rowKeyFilter.ranges.size == 0) @@ -302,13 +319,17 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { * some rowkey points are duplicate. */ test("Test rowKey point only rowKey query, which contains duplicate rowkey") { - val results = sqlContext.sql("SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseTable1 " + - "WHERE " + - "(KEY_FIELD = 'get1' or KEY_FIELD = 'get2' or KEY_FIELD = 'get1')").take(10) + val results = sqlContext + .sql( + "SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseTable1 " + + "WHERE " + + "(KEY_FIELD = 'get1' or KEY_FIELD = 'get2' or KEY_FIELD = 'get1')") + .take(10) val executionRules = DefaultSourceStaticUtils.lastFiveExecutionRules.poll() assert(results.length == 2) - assert(executionRules.dynamicLogicExpression.toExpressionString. - equals("( KEY_FIELD == 0 OR KEY_FIELD == 1 )")) + assert( + executionRules.dynamicLogicExpression.toExpressionString.equals( + "( KEY_FIELD == 0 OR KEY_FIELD == 1 )")) assert(executionRules.rowKeyFilter.points.size == 2) assert(executionRules.rowKeyFilter.ranges.size == 0) } @@ -317,16 +338,20 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { * A example of query three fields and also only using cell points for the filter */ test("Test cell point only rowKey query") { - val results = sqlContext.sql("SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseTable1 " + - "WHERE " + - "(B_FIELD = '4' or B_FIELD = '10' or A_FIELD = 'foo1')").take(10) + val results = sqlContext + .sql( + "SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseTable1 " + + "WHERE " + + "(B_FIELD = '4' or B_FIELD = '10' or A_FIELD = 'foo1')") + .take(10) val executionRules = DefaultSourceStaticUtils.lastFiveExecutionRules.poll() assert(results.length == 3) - assert(executionRules.dynamicLogicExpression.toExpressionString. - equals("( ( B_FIELD == 0 OR B_FIELD == 1 ) OR A_FIELD == 2 )")) + assert( + executionRules.dynamicLogicExpression.toExpressionString.equals( + "( ( B_FIELD == 0 OR B_FIELD == 1 ) OR A_FIELD == 2 )")) } /** @@ -334,28 +359,32 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { * Also an example of less then and greater then */ test("Test two range rowKey query") { - val results = sqlContext.sql("SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseTable1 " + - "WHERE " + - "( KEY_FIELD < 'get2' or KEY_FIELD > 'get3')").take(10) + val results = sqlContext + .sql( + "SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseTable1 " + + "WHERE " + + "( KEY_FIELD < 'get2' or KEY_FIELD > 'get3')") + .take(10) val executionRules = DefaultSourceStaticUtils.lastFiveExecutionRules.poll() assert(results.length == 3) - assert(executionRules.dynamicLogicExpression.toExpressionString. - equals("( KEY_FIELD < 0 OR KEY_FIELD > 1 )")) + assert( + executionRules.dynamicLogicExpression.toExpressionString.equals( + "( KEY_FIELD < 0 OR KEY_FIELD > 1 )")) assert(executionRules.rowKeyFilter.points.size == 0) assert(executionRules.rowKeyFilter.ranges.size == 2) val scanRange1 = executionRules.rowKeyFilter.ranges.get(0).get - assert(Bytes.equals(scanRange1.lowerBound,Bytes.toBytes(""))) - assert(Bytes.equals(scanRange1.upperBound,Bytes.toBytes("get2"))) + assert(Bytes.equals(scanRange1.lowerBound, Bytes.toBytes(""))) + assert(Bytes.equals(scanRange1.upperBound, Bytes.toBytes("get2"))) assert(scanRange1.isLowerBoundEqualTo) assert(!scanRange1.isUpperBoundEqualTo) val scanRange2 = executionRules.rowKeyFilter.ranges.get(1).get - assert(Bytes.equals(scanRange2.lowerBound,Bytes.toBytes("get3"))) + assert(Bytes.equals(scanRange2.lowerBound, Bytes.toBytes("get3"))) assert(scanRange2.upperBound == null) assert(!scanRange2.isLowerBoundEqualTo) assert(scanRange2.isUpperBoundEqualTo) @@ -368,16 +397,18 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { * This example makes sure the code works for a int rowKey */ test("Test two range rowKey query where the rowKey is Int and there is a range over lap") { - val results = sqlContext.sql("SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseTable2 " + - "WHERE " + - "( KEY_FIELD < 4 or KEY_FIELD > 2)").take(10) + val results = sqlContext + .sql( + "SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseTable2 " + + "WHERE " + + "( KEY_FIELD < 4 or KEY_FIELD > 2)") + .take(10) val executionRules = DefaultSourceStaticUtils.lastFiveExecutionRules.poll() - - - assert(executionRules.dynamicLogicExpression.toExpressionString. - equals("( KEY_FIELD < 0 OR KEY_FIELD > 1 )")) + assert( + executionRules.dynamicLogicExpression.toExpressionString.equals( + "( KEY_FIELD < 0 OR KEY_FIELD > 1 )")) assert(executionRules.rowKeyFilter.points.size == 0) assert(executionRules.rowKeyFilter.ranges.size == 2) @@ -391,14 +422,18 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { * This example makes sure the code works for a int rowKey */ test("Test two range rowKey query where the rowKey is Int and the ranges don't over lap") { - val results = sqlContext.sql("SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseTable2 " + - "WHERE " + - "( KEY_FIELD < 2 or KEY_FIELD > 4)").take(10) + val results = sqlContext + .sql( + "SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseTable2 " + + "WHERE " + + "( KEY_FIELD < 2 or KEY_FIELD > 4)") + .take(10) val executionRules = DefaultSourceStaticUtils.lastFiveExecutionRules.poll() - assert(executionRules.dynamicLogicExpression.toExpressionString. - equals("( KEY_FIELD < 0 OR KEY_FIELD > 1 )")) + assert( + executionRules.dynamicLogicExpression.toExpressionString.equals( + "( KEY_FIELD < 0 OR KEY_FIELD > 1 )")) assert(executionRules.rowKeyFilter.points.size == 0) @@ -420,9 +455,12 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { * Also an example of less then and equal to and greater then and equal to */ test("Test one combined range rowKey query") { - val results = sqlContext.sql("SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseTable1 " + - "WHERE " + - "(KEY_FIELD <= 'get3' and KEY_FIELD >= 'get2')").take(10) + val results = sqlContext + .sql( + "SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseTable1 " + + "WHERE " + + "(KEY_FIELD <= 'get3' and KEY_FIELD >= 'get2')") + .take(10) val executionRules = DefaultSourceStaticUtils.lastFiveExecutionRules.poll() @@ -435,7 +473,7 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { assert(executionRules.rowKeyFilter.ranges.size == 1) val scanRange1 = executionRules.rowKeyFilter.ranges.get(0).get - assert(Bytes.equals(scanRange1.lowerBound,Bytes.toBytes("get2"))) + assert(Bytes.equals(scanRange1.lowerBound, Bytes.toBytes("get2"))) assert(Bytes.equals(scanRange1.upperBound, Bytes.toBytes("get3"))) assert(scanRange1.isLowerBoundEqualTo) assert(scanRange1.isUpperBoundEqualTo) @@ -461,27 +499,30 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { * rowKey and the a column */ test("Test SQL point and range combo") { - val results = sqlContext.sql("SELECT KEY_FIELD FROM hbaseTable1 " + - "WHERE " + - "(KEY_FIELD = 'get1' and B_FIELD < '3') or " + - "(KEY_FIELD >= 'get3' and B_FIELD = '8')").take(5) + val results = sqlContext + .sql( + "SELECT KEY_FIELD FROM hbaseTable1 " + + "WHERE " + + "(KEY_FIELD = 'get1' and B_FIELD < '3') or " + + "(KEY_FIELD >= 'get3' and B_FIELD = '8')") + .take(5) val executionRules = DefaultSourceStaticUtils.lastFiveExecutionRules.poll() - assert(executionRules.dynamicLogicExpression.toExpressionString. - equals("( ( KEY_FIELD == 0 AND B_FIELD < 1 ) OR " + - "( KEY_FIELD >= 2 AND B_FIELD == 3 ) )")) + assert( + executionRules.dynamicLogicExpression.toExpressionString.equals( + "( ( KEY_FIELD == 0 AND B_FIELD < 1 ) OR " + + "( KEY_FIELD >= 2 AND B_FIELD == 3 ) )")) assert(executionRules.rowKeyFilter.points.size == 1) assert(executionRules.rowKeyFilter.ranges.size == 1) val scanRange1 = executionRules.rowKeyFilter.ranges.get(0).get - assert(Bytes.equals(scanRange1.lowerBound,Bytes.toBytes("get3"))) + assert(Bytes.equals(scanRange1.lowerBound, Bytes.toBytes("get3"))) assert(scanRange1.upperBound == null) assert(scanRange1.isLowerBoundEqualTo) assert(scanRange1.isUpperBoundEqualTo) - assert(results.length == 3) } @@ -490,29 +531,32 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { */ test("Test two complete range non merge rowKey query") { - val results = sqlContext.sql("SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseTable2 " + - "WHERE " + - "( KEY_FIELD >= 1 and KEY_FIELD <= 2) or" + - "( KEY_FIELD > 3 and KEY_FIELD <= 5)").take(10) - + val results = sqlContext + .sql( + "SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseTable2 " + + "WHERE " + + "( KEY_FIELD >= 1 and KEY_FIELD <= 2) or" + + "( KEY_FIELD > 3 and KEY_FIELD <= 5)") + .take(10) assert(results.length == 4) val executionRules = DefaultSourceStaticUtils.lastFiveExecutionRules.poll() - assert(executionRules.dynamicLogicExpression.toExpressionString. - equals("( ( KEY_FIELD >= 0 AND KEY_FIELD <= 1 ) OR " + - "( KEY_FIELD > 2 AND KEY_FIELD <= 3 ) )")) + assert( + executionRules.dynamicLogicExpression.toExpressionString.equals( + "( ( KEY_FIELD >= 0 AND KEY_FIELD <= 1 ) OR " + + "( KEY_FIELD > 2 AND KEY_FIELD <= 3 ) )")) assert(executionRules.rowKeyFilter.points.size == 0) assert(executionRules.rowKeyFilter.ranges.size == 2) val scanRange1 = executionRules.rowKeyFilter.ranges.get(0).get - assert(Bytes.equals(scanRange1.lowerBound,Bytes.toBytes(1))) + assert(Bytes.equals(scanRange1.lowerBound, Bytes.toBytes(1))) assert(Bytes.equals(scanRange1.upperBound, Bytes.toBytes(2))) assert(scanRange1.isLowerBoundEqualTo) assert(scanRange1.isUpperBoundEqualTo) val scanRange2 = executionRules.rowKeyFilter.ranges.get(1).get - assert(Bytes.equals(scanRange2.lowerBound,Bytes.toBytes(3))) + assert(Bytes.equals(scanRange2.lowerBound, Bytes.toBytes(3))) assert(Bytes.equals(scanRange2.upperBound, Bytes.toBytes(5))) assert(!scanRange2.isLowerBoundEqualTo) assert(scanRange2.isUpperBoundEqualTo) @@ -523,24 +567,28 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { * A complex query with two complex ranges that does merge into one */ test("Test two complete range merge rowKey query") { - val results = sqlContext.sql("SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseTable1 " + - "WHERE " + - "( KEY_FIELD >= 'get1' and KEY_FIELD <= 'get2') or" + - "( KEY_FIELD > 'get3' and KEY_FIELD <= 'get5')").take(10) + val results = sqlContext + .sql( + "SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseTable1 " + + "WHERE " + + "( KEY_FIELD >= 'get1' and KEY_FIELD <= 'get2') or" + + "( KEY_FIELD > 'get3' and KEY_FIELD <= 'get5')") + .take(10) val executionRules = DefaultSourceStaticUtils.lastFiveExecutionRules.poll() assert(results.length == 4) - assert(executionRules.dynamicLogicExpression.toExpressionString. - equals("( ( KEY_FIELD >= 0 AND KEY_FIELD <= 1 ) OR " + - "( KEY_FIELD > 2 AND KEY_FIELD <= 3 ) )")) + assert( + executionRules.dynamicLogicExpression.toExpressionString.equals( + "( ( KEY_FIELD >= 0 AND KEY_FIELD <= 1 ) OR " + + "( KEY_FIELD > 2 AND KEY_FIELD <= 3 ) )")) assert(executionRules.rowKeyFilter.points.size == 0) assert(executionRules.rowKeyFilter.ranges.size == 2) val scanRange1 = executionRules.rowKeyFilter.ranges.get(0).get - assert(Bytes.equals(scanRange1.lowerBound,Bytes.toBytes("get1"))) + assert(Bytes.equals(scanRange1.lowerBound, Bytes.toBytes("get1"))) assert(Bytes.equals(scanRange1.upperBound, Bytes.toBytes("get2"))) assert(scanRange1.isLowerBoundEqualTo) assert(scanRange1.isUpperBoundEqualTo) @@ -554,51 +602,59 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { test("Test OR logic with a one RowKey and One column") { - val results = sqlContext.sql("SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseTable1 " + - "WHERE " + - "( KEY_FIELD >= 'get1' or A_FIELD <= 'foo2') or" + - "( KEY_FIELD > 'get3' or B_FIELD <= '4')").take(10) + val results = sqlContext + .sql( + "SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseTable1 " + + "WHERE " + + "( KEY_FIELD >= 'get1' or A_FIELD <= 'foo2') or" + + "( KEY_FIELD > 'get3' or B_FIELD <= '4')") + .take(10) val executionRules = DefaultSourceStaticUtils.lastFiveExecutionRules.poll() assert(results.length == 5) - assert(executionRules.dynamicLogicExpression.toExpressionString. - equals("( ( KEY_FIELD >= 0 OR A_FIELD <= 1 ) OR " + - "( KEY_FIELD > 2 OR B_FIELD <= 3 ) )")) + assert( + executionRules.dynamicLogicExpression.toExpressionString.equals( + "( ( KEY_FIELD >= 0 OR A_FIELD <= 1 ) OR " + + "( KEY_FIELD > 2 OR B_FIELD <= 3 ) )")) assert(executionRules.rowKeyFilter.points.size == 0) assert(executionRules.rowKeyFilter.ranges.size == 1) val scanRange1 = executionRules.rowKeyFilter.ranges.get(0).get - //This is the main test for 14406 - //Because the key is joined through a or with a qualifier - //There is no filter on the rowKey - assert(Bytes.equals(scanRange1.lowerBound,Bytes.toBytes(""))) + // This is the main test for 14406 + // Because the key is joined through a or with a qualifier + // There is no filter on the rowKey + assert(Bytes.equals(scanRange1.lowerBound, Bytes.toBytes(""))) assert(scanRange1.upperBound == null) assert(scanRange1.isLowerBoundEqualTo) assert(scanRange1.isUpperBoundEqualTo) } test("Test OR logic with a two columns") { - val results = sqlContext.sql("SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseTable1 " + - "WHERE " + - "( B_FIELD > '4' or A_FIELD <= 'foo2') or" + - "( A_FIELD > 'foo2' or B_FIELD < '4')").take(10) + val results = sqlContext + .sql( + "SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseTable1 " + + "WHERE " + + "( B_FIELD > '4' or A_FIELD <= 'foo2') or" + + "( A_FIELD > 'foo2' or B_FIELD < '4')") + .take(10) val executionRules = DefaultSourceStaticUtils.lastFiveExecutionRules.poll() assert(results.length == 5) - assert(executionRules.dynamicLogicExpression.toExpressionString. - equals("( ( B_FIELD > 0 OR A_FIELD <= 1 ) OR " + - "( A_FIELD > 2 OR B_FIELD < 3 ) )")) + assert( + executionRules.dynamicLogicExpression.toExpressionString.equals( + "( ( B_FIELD > 0 OR A_FIELD <= 1 ) OR " + + "( A_FIELD > 2 OR B_FIELD < 3 ) )")) assert(executionRules.rowKeyFilter.points.size == 0) assert(executionRules.rowKeyFilter.ranges.size == 1) val scanRange1 = executionRules.rowKeyFilter.ranges.get(0).get - assert(Bytes.equals(scanRange1.lowerBound,Bytes.toBytes(""))) + assert(Bytes.equals(scanRange1.lowerBound, Bytes.toBytes(""))) assert(scanRange1.upperBound == null) assert(scanRange1.isLowerBoundEqualTo) assert(scanRange1.isUpperBoundEqualTo) @@ -606,42 +662,50 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { } test("Test single RowKey Or Column logic") { - val results = sqlContext.sql("SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseTable1 " + - "WHERE " + - "( KEY_FIELD >= 'get4' or A_FIELD <= 'foo2' )").take(10) + val results = sqlContext + .sql( + "SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseTable1 " + + "WHERE " + + "( KEY_FIELD >= 'get4' or A_FIELD <= 'foo2' )") + .take(10) val executionRules = DefaultSourceStaticUtils.lastFiveExecutionRules.poll() assert(results.length == 4) - assert(executionRules.dynamicLogicExpression.toExpressionString. - equals("( KEY_FIELD >= 0 OR A_FIELD <= 1 )")) + assert( + executionRules.dynamicLogicExpression.toExpressionString.equals( + "( KEY_FIELD >= 0 OR A_FIELD <= 1 )")) assert(executionRules.rowKeyFilter.points.size == 0) assert(executionRules.rowKeyFilter.ranges.size == 1) val scanRange1 = executionRules.rowKeyFilter.ranges.get(0).get - assert(Bytes.equals(scanRange1.lowerBound,Bytes.toBytes(""))) + assert(Bytes.equals(scanRange1.lowerBound, Bytes.toBytes(""))) assert(scanRange1.upperBound == null) assert(scanRange1.isLowerBoundEqualTo) assert(scanRange1.isUpperBoundEqualTo) } test("Test Rowkey And with complex logic (HBASE-26863)") { - val results = sqlContext.sql("SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseTable1 " + - "WHERE " + - "( KEY_FIELD >= 'get1' AND KEY_FIELD <= 'get3' ) AND (A_FIELD = 'foo1' OR B_FIELD = '8')").take(10) + val results = sqlContext + .sql( + "SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseTable1 " + + "WHERE " + + "( KEY_FIELD >= 'get1' AND KEY_FIELD <= 'get3' ) AND (A_FIELD = 'foo1' OR B_FIELD = '8')") + .take(10) val executionRules = DefaultSourceStaticUtils.lastFiveExecutionRules.poll() assert(results.length == 2) - assert(executionRules.dynamicLogicExpression.toExpressionString - == "( ( ( KEY_FIELD isNotNull AND KEY_FIELD >= 0 ) AND KEY_FIELD <= 1 ) AND ( A_FIELD == 2 OR B_FIELD == 3 ) )") + assert( + executionRules.dynamicLogicExpression.toExpressionString + == "( ( ( KEY_FIELD isNotNull AND KEY_FIELD >= 0 ) AND KEY_FIELD <= 1 ) AND ( A_FIELD == 2 OR B_FIELD == 3 ) )") assert(executionRules.rowKeyFilter.points.size == 0) assert(executionRules.rowKeyFilter.ranges.size == 1) val scanRange1 = executionRules.rowKeyFilter.ranges.get(0).get - assert(Bytes.equals(scanRange1.lowerBound,Bytes.toBytes("get1"))) + assert(Bytes.equals(scanRange1.lowerBound, Bytes.toBytes("get1"))) assert(Bytes.equals(scanRange1.upperBound, Bytes.toBytes("get3"))) assert(scanRange1.isLowerBoundEqualTo) assert(scanRange1.isUpperBoundEqualTo) @@ -659,20 +723,23 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { |}""".stripMargin intercept[Exception] { - df = sqlContext.load("org.apache.hadoop.hbase.spark", - Map(HBaseTableCatalog.tableCatalog->catalog)) + df = sqlContext.load( + "org.apache.hadoop.hbase.spark", + Map(HBaseTableCatalog.tableCatalog -> catalog)) df.registerTempTable("hbaseNonExistingTmp") - sqlContext.sql("SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseNonExistingTmp " + - "WHERE " + - "( KEY_FIELD >= 'get1' and KEY_FIELD <= 'get3') or" + - "( KEY_FIELD > 'get3' and KEY_FIELD <= 'get5')").count() + sqlContext + .sql( + "SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseNonExistingTmp " + + "WHERE " + + "( KEY_FIELD >= 'get1' and KEY_FIELD <= 'get3') or" + + "( KEY_FIELD > 'get3' and KEY_FIELD <= 'get5')") + .count() } DefaultSourceStaticUtils.lastFiveExecutionRules.poll() } - test("Test table with column that doesn't exist") { val catalog = s"""{ |"table":{"namespace":"default", "name":"t1"}, @@ -684,13 +751,15 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { |"C_FIELD":{"cf":"c", "col":"c", "type":"string"} |} |}""".stripMargin - df = sqlContext.load("org.apache.hadoop.hbase.spark", - Map(HBaseTableCatalog.tableCatalog->catalog)) + df = sqlContext.load( + "org.apache.hadoop.hbase.spark", + Map(HBaseTableCatalog.tableCatalog -> catalog)) df.registerTempTable("hbaseFactColumnTmp") - val result = sqlContext.sql("SELECT KEY_FIELD, " + - "B_FIELD, A_FIELD FROM hbaseFactColumnTmp") + val result = sqlContext.sql( + "SELECT KEY_FIELD, " + + "B_FIELD, A_FIELD FROM hbaseFactColumnTmp") assert(result.count() == 5) @@ -710,13 +779,15 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { |"I_FIELD":{"cf":"c", "col":"i", "type":"int"} |} |}""".stripMargin - df = sqlContext.load("org.apache.hadoop.hbase.spark", - Map(HBaseTableCatalog.tableCatalog->catalog)) + df = sqlContext.load( + "org.apache.hadoop.hbase.spark", + Map(HBaseTableCatalog.tableCatalog -> catalog)) df.registerTempTable("hbaseIntTmp") - val result = sqlContext.sql("SELECT KEY_FIELD, B_FIELD, I_FIELD FROM hbaseIntTmp"+ - " where I_FIELD > 4 and I_FIELD < 10") + val result = sqlContext.sql( + "SELECT KEY_FIELD, B_FIELD, I_FIELD FROM hbaseIntTmp" + + " where I_FIELD > 4 and I_FIELD < 10") val localResult = result.take(5) @@ -741,13 +812,15 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { |"I_FIELD":{"cf":"c", "col":"i", "type":"string"} |} |}""".stripMargin - df = sqlContext.load("org.apache.hadoop.hbase.spark", - Map(HBaseTableCatalog.tableCatalog->catalog)) + df = sqlContext.load( + "org.apache.hadoop.hbase.spark", + Map(HBaseTableCatalog.tableCatalog -> catalog)) df.registerTempTable("hbaseIntWrongTypeTmp") - val result = sqlContext.sql("SELECT KEY_FIELD, " + - "B_FIELD, I_FIELD FROM hbaseIntWrongTypeTmp") + val result = sqlContext.sql( + "SELECT KEY_FIELD, " + + "B_FIELD, I_FIELD FROM hbaseIntWrongTypeTmp") val localResult = result.take(10) assert(localResult.length == 5) @@ -773,13 +846,15 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { |} |}""".stripMargin intercept[Exception] { - df = sqlContext.load("org.apache.hadoop.hbase.spark", - Map(HBaseTableCatalog.tableCatalog->catalog)) + df = sqlContext.load( + "org.apache.hadoop.hbase.spark", + Map(HBaseTableCatalog.tableCatalog -> catalog)) df.registerTempTable("hbaseIntWrongTypeTmp") - val result = sqlContext.sql("SELECT KEY_FIELD, " + - "B_FIELD, I_FIELD FROM hbaseIntWrongTypeTmp") + val result = sqlContext.sql( + "SELECT KEY_FIELD, " + + "B_FIELD, I_FIELD FROM hbaseIntWrongTypeTmp") val localResult = result.take(10) assert(localResult.length == 5) @@ -791,29 +866,44 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { } test("Test HBaseSparkConf matching") { - val df = sqlContext.load("org.apache.hadoop.hbase.spark.HBaseTestSource", - Map("cacheSize" -> "100", + val df = sqlContext.load( + "org.apache.hadoop.hbase.spark.HBaseTestSource", + Map( + "cacheSize" -> "100", "batchNum" -> "100", - "blockCacheingEnable" -> "true", "rowNum" -> "10")) + "blockCacheingEnable" -> "true", + "rowNum" -> "10")) assert(df.count() == 10) - val df1 = sqlContext.load("org.apache.hadoop.hbase.spark.HBaseTestSource", - Map("cacheSize" -> "1000", - "batchNum" -> "100", "blockCacheingEnable" -> "true", "rowNum" -> "10")) + val df1 = sqlContext.load( + "org.apache.hadoop.hbase.spark.HBaseTestSource", + Map( + "cacheSize" -> "1000", + "batchNum" -> "100", + "blockCacheingEnable" -> "true", + "rowNum" -> "10")) intercept[Exception] { assert(df1.count() == 10) } - val df2 = sqlContext.load("org.apache.hadoop.hbase.spark.HBaseTestSource", - Map("cacheSize" -> "100", - "batchNum" -> "1000", "blockCacheingEnable" -> "true", "rowNum" -> "10")) + val df2 = sqlContext.load( + "org.apache.hadoop.hbase.spark.HBaseTestSource", + Map( + "cacheSize" -> "100", + "batchNum" -> "1000", + "blockCacheingEnable" -> "true", + "rowNum" -> "10")) intercept[Exception] { assert(df2.count() == 10) } - val df3 = sqlContext.load("org.apache.hadoop.hbase.spark.HBaseTestSource", - Map("cacheSize" -> "100", - "batchNum" -> "100", "blockCacheingEnable" -> "false", "rowNum" -> "10")) + val df3 = sqlContext.load( + "org.apache.hadoop.hbase.spark.HBaseTestSource", + Map( + "cacheSize" -> "100", + "batchNum" -> "100", + "blockCacheingEnable" -> "false", + "rowNum" -> "10")) intercept[Exception] { assert(df3.count() == 10) } @@ -830,8 +920,9 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { |"Z_FIELD":{"cf":"c", "col":"z", "type":"string"} |} |}""".stripMargin - df = sqlContext.load("org.apache.hadoop.hbase.spark", - Map(HBaseTableCatalog.tableCatalog->catalog)) + df = sqlContext.load( + "org.apache.hadoop.hbase.spark", + Map(HBaseTableCatalog.tableCatalog -> catalog)) df.registerTempTable("hbaseZTmp") @@ -861,15 +952,20 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { |"Z_FIELD":{"cf":"c", "col":"z", "type":"string"} |} |}""".stripMargin - df = sqlContext.load("org.apache.hadoop.hbase.spark", - Map(HBaseTableCatalog.tableCatalog->catalog, + df = sqlContext.load( + "org.apache.hadoop.hbase.spark", + Map( + HBaseTableCatalog.tableCatalog -> catalog, HBaseSparkConf.PUSHDOWN_COLUMN_FILTER -> "false")) df.registerTempTable("hbaseNoPushDownTmp") - val results = sqlContext.sql("SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseNoPushDownTmp " + - "WHERE " + - "(KEY_FIELD <= 'get3' and KEY_FIELD >= 'get2')").take(10) + val results = sqlContext + .sql( + "SELECT KEY_FIELD, B_FIELD, A_FIELD FROM hbaseNoPushDownTmp " + + "WHERE " + + "(KEY_FIELD <= 'get3' and KEY_FIELD >= 'get2')") + .take(10) val executionRules = DefaultSourceStaticUtils.lastFiveExecutionRules.poll() @@ -897,15 +993,19 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { |"STRING_FIELD":{"cf":"c", "col":"string", "type":"string"} |} |}""".stripMargin - df = sqlContext.load("org.apache.hadoop.hbase.spark", - Map(HBaseTableCatalog.tableCatalog->catalog)) + df = sqlContext.load( + "org.apache.hadoop.hbase.spark", + Map(HBaseTableCatalog.tableCatalog -> catalog)) df.registerTempTable("hbaseTestMapping") - val results = sqlContext.sql("SELECT binary_field, boolean_field, " + - "byte_field, short_field, int_field, long_field, " + - "float_field, double_field, date_field, timestamp_field, " + - "string_field FROM hbaseTestMapping").collect() + val results = sqlContext + .sql( + "SELECT binary_field, boolean_field, " + + "byte_field, short_field, int_field, long_field, " + + "float_field, double_field, date_field, timestamp_field, " + + "string_field FROM hbaseTestMapping") + .collect() assert(results.length == 1) @@ -917,7 +1017,8 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { System.out.println("2: " + result.get(2)) System.out.println("3: " + result.get(3)) - assert(result.get(0).asInstanceOf[Array[Byte]].sameElements(Array(1.toByte, 2.toByte, 3.toByte))) + assert( + result.get(0).asInstanceOf[Array[Byte]].sameElements(Array(1.toByte, 2.toByte, 3.toByte))) assert(result.get(1) == true) assert(result.get(2) == 127) assert(result.get(3) == 32767) @@ -948,9 +1049,8 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { |}""".stripMargin def withCatalog(cat: String): DataFrame = { - sqlContext - .read - .options(Map(HBaseTableCatalog.tableCatalog->cat)) + sqlContext.read + .options(Map(HBaseTableCatalog.tableCatalog -> cat)) .format("org.apache.hadoop.hbase.spark") .load() } @@ -958,11 +1058,12 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { test("populate table") { val sql = sqlContext import sql.implicits._ - val data = (0 to 255).map { i => - HBaseRecord(i, "extra") - } - sc.parallelize(data).toDF.write.options( - Map(HBaseTableCatalog.tableCatalog -> writeCatalog, HBaseTableCatalog.newTable -> "5")) + val data = (0 to 255).map { i => HBaseRecord(i, "extra") } + sc.parallelize(data) + .toDF + .write + .options( + Map(HBaseTableCatalog.tableCatalog -> writeCatalog, HBaseTableCatalog.newTable -> "5")) .format("org.apache.hadoop.hbase.spark") .save() } @@ -984,7 +1085,8 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { val sql = sqlContext import sql.implicits._ val df = withCatalog(writeCatalog) - val s = df.filter($"col0" <= "row005") + val s = df + .filter($"col0" <= "row005") .select("col0", "col1") s.show() assert(s.count() == 6) @@ -994,7 +1096,8 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { val sql = sqlContext import sql.implicits._ val df = withCatalog(writeCatalog) - val s = df.filter(col("col0").startsWith("row00")) + val s = df + .filter(col("col0").startsWith("row00")) .select("col0", "col1") s.show() assert(s.count() == 10) @@ -1004,7 +1107,8 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { val sql = sqlContext import sql.implicits._ val df = withCatalog(writeCatalog) - val s = df.filter(col("col0").startsWith("row005")) + val s = df + .filter(col("col0").startsWith("row005")) .select("col0", "col1") s.show() assert(s.count() == 1) @@ -1014,7 +1118,8 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { val sql = sqlContext import sql.implicits._ val df = withCatalog(writeCatalog) - val s = df.filter(col("col0").startsWith("row")) + val s = df + .filter(col("col0").startsWith("row")) .select("col0", "col1") s.show() assert(s.count() == 256) @@ -1024,7 +1129,8 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { val sql = sqlContext import sql.implicits._ val df = withCatalog(writeCatalog) - val s = df.filter(col("col0").startsWith("row19")) + val s = df + .filter(col("col0").startsWith("row19")) .select("col0", "col1") s.show() assert(s.count() == 10) @@ -1034,7 +1140,8 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { val sql = sqlContext import sql.implicits._ val df = withCatalog(writeCatalog) - val s = df.filter(col("col0").startsWith("")) + val s = df + .filter(col("col0").startsWith("")) .select("col0", "col1") s.show() assert(s.count() == 256) @@ -1050,26 +1157,33 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { // and get an old view. val oldMs = 754869600000L val startMs = System.currentTimeMillis() - val oldData = (0 to 100).map { i => - HBaseRecord(i, "old") - } - val newData = (200 to 255).map { i => - HBaseRecord(i, "new") - } - - sc.parallelize(oldData).toDF.write.options( - Map(HBaseTableCatalog.tableCatalog -> writeCatalog, HBaseTableCatalog.tableName -> "5", - HBaseSparkConf.TIMESTAMP -> oldMs.toString)) + val oldData = (0 to 100).map { i => HBaseRecord(i, "old") } + val newData = (200 to 255).map { i => HBaseRecord(i, "new") } + + sc.parallelize(oldData) + .toDF + .write + .options( + Map( + HBaseTableCatalog.tableCatalog -> writeCatalog, + HBaseTableCatalog.tableName -> "5", + HBaseSparkConf.TIMESTAMP -> oldMs.toString)) .format("org.apache.hadoop.hbase.spark") .save() - sc.parallelize(newData).toDF.write.options( - Map(HBaseTableCatalog.tableCatalog -> writeCatalog, HBaseTableCatalog.tableName -> "5")) + sc.parallelize(newData) + .toDF + .write + .options( + Map(HBaseTableCatalog.tableCatalog -> writeCatalog, HBaseTableCatalog.tableName -> "5")) .format("org.apache.hadoop.hbase.spark") .save() // Test specific timestamp -- Full scan, Timestamp val individualTimestamp = sqlContext.read - .options(Map(HBaseTableCatalog.tableCatalog -> writeCatalog, HBaseSparkConf.TIMESTAMP -> oldMs.toString)) + .options( + Map( + HBaseTableCatalog.tableCatalog -> writeCatalog, + HBaseSparkConf.TIMESTAMP -> oldMs.toString)) .format("org.apache.hadoop.hbase.spark") .load() assert(individualTimestamp.count() == 101) @@ -1088,8 +1202,11 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { // Test Getting old stuff -- Full Scan, TimeRange val oldRange = sqlContext.read - .options(Map(HBaseTableCatalog.tableCatalog -> writeCatalog, HBaseSparkConf.TIMERANGE_START -> "0", - HBaseSparkConf.TIMERANGE_END -> (oldMs + 100).toString)) + .options( + Map( + HBaseTableCatalog.tableCatalog -> writeCatalog, + HBaseSparkConf.TIMERANGE_START -> "0", + HBaseSparkConf.TIMERANGE_END -> (oldMs + 100).toString)) .format("org.apache.hadoop.hbase.spark") .load() assert(oldRange.count() == 101) @@ -1099,17 +1216,20 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { // Test Getting middle stuff -- Full Scan, TimeRange val middleRange = sqlContext.read - .options(Map(HBaseTableCatalog.tableCatalog -> writeCatalog, HBaseSparkConf.TIMERANGE_START -> "0", - HBaseSparkConf.TIMERANGE_END -> (startMs + 100).toString)) + .options( + Map( + HBaseTableCatalog.tableCatalog -> writeCatalog, + HBaseSparkConf.TIMERANGE_START -> "0", + HBaseSparkConf.TIMERANGE_END -> (startMs + 100).toString)) .format("org.apache.hadoop.hbase.spark") .load() assert(middleRange.count() == 256) // Test Getting middle stuff -- Pruned Scan, TimeRange - val middleElement200 = middleRange.where(col("col0") === lit("row200")).select("col7").collect()(0)(0) + val middleElement200 = + middleRange.where(col("col0") === lit("row200")).select("col7").collect()(0)(0) assert(middleElement200 == "String200: extra") } - // catalog for insertion def avroWriteCatalog = s"""{ |"table":{"namespace":"default", "name":"avrotable"}, @@ -1141,25 +1261,25 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { |}""".stripMargin def withAvroCatalog(cat: String): DataFrame = { - sqlContext - .read - .options(Map("avroSchema"->AvroHBaseKeyRecord.schemaString, - HBaseTableCatalog.tableCatalog->avroCatalog)) + sqlContext.read + .options( + Map( + "avroSchema" -> AvroHBaseKeyRecord.schemaString, + HBaseTableCatalog.tableCatalog -> avroCatalog)) .format("org.apache.hadoop.hbase.spark") .load() } - test("populate avro table") { val sql = sqlContext import sql.implicits._ - val data = (0 to 255).map { i => - AvroHBaseKeyRecord(i) - } - sc.parallelize(data).toDF.write.options( - Map(HBaseTableCatalog.tableCatalog -> avroWriteCatalog, - HBaseTableCatalog.newTable -> "5")) + val data = (0 to 255).map { i => AvroHBaseKeyRecord(i) } + sc.parallelize(data) + .toDF + .write + .options( + Map(HBaseTableCatalog.tableCatalog -> avroWriteCatalog, HBaseTableCatalog.newTable -> "5")) .format("org.apache.hadoop.hbase.spark") .save() } @@ -1167,8 +1287,7 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { test("avro empty column") { val df = withAvroCatalog(avroCatalog) df.registerTempTable("avrotable") - val c = sqlContext.sql("select count(1) from avrotable") - .rdd.collect()(0)(0).asInstanceOf[Long] + val c = sqlContext.sql("select count(1) from avrotable").rdd.collect()(0)(0).asInstanceOf[Long] assert(c == 256) } @@ -1181,10 +1300,12 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { test("avro serialization and deserialization query") { val df = withAvroCatalog(avroCatalog) - df.write.options( - Map("avroSchema"->AvroHBaseKeyRecord.schemaString, - HBaseTableCatalog.tableCatalog->avroCatalogInsert, - HBaseTableCatalog.newTable -> "5")) + df.write + .options( + Map( + "avroSchema" -> AvroHBaseKeyRecord.schemaString, + HBaseTableCatalog.tableCatalog -> avroCatalogInsert, + HBaseTableCatalog.newTable -> "5")) .format("org.apache.hadoop.hbase.spark") .save() val newDF = withAvroCatalog(avroCatalogInsert) @@ -1197,7 +1318,8 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { val sql = sqlContext import sql.implicits._ val df = withAvroCatalog(avroCatalog) - val r = df.filter($"col1.name" === "name005" || $"col1.name" <= "name005") + val r = df + .filter($"col1.name" === "name005" || $"col1.name" <= "name005") .select("col0", "col1.favorite_color", "col1.favorite_number") r.show() assert(r.count() == 6) @@ -1207,7 +1329,8 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { val sql = sqlContext import sql.implicits._ val df = withAvroCatalog(avroCatalog) - val s = df.filter($"col1.name" <= "name005" || $"col1.name".contains("name007")) + val s = df + .filter($"col1.name" <= "name005" || $"col1.name".contains("name007")) .select("col0", "col1.favorite_color", "col1.favorite_number") s.show() assert(s.count() == 7) @@ -1224,10 +1347,12 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { |} |}""".stripMargin try { - HBaseRelation(Map(HBaseTableCatalog.tableCatalog -> catalog, - HBaseSparkConf.USE_HBASECONTEXT -> "false"), None)(sqlContext) + HBaseRelation( + Map(HBaseTableCatalog.tableCatalog -> catalog, HBaseSparkConf.USE_HBASECONTEXT -> "false"), + None)(sqlContext) } catch { - case e: Throwable => if(e.getCause.isInstanceOf[SAXParseException]) { + case e: Throwable => + if (e.getCause.isInstanceOf[SAXParseException]) { fail("SAXParseException due to configuration loading empty resource") } else { println("Failed due to some other exception, ignore " + e.getMessage) diff --git a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DynamicLogicExpressionSuite.scala b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DynamicLogicExpressionSuite.scala index e493c54a..3ada1ebe 100644 --- a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DynamicLogicExpressionSuite.scala +++ b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DynamicLogicExpressionSuite.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,18 +15,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark import java.util - import org.apache.hadoop.hbase.spark.datasources.{HBaseSparkConf, JavaBytesEncoder} import org.apache.hadoop.hbase.util.Bytes import org.apache.spark.sql.types._ import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} -class DynamicLogicExpressionSuite extends FunSuite with -BeforeAndAfterEach with BeforeAndAfterAll with Logging { +class DynamicLogicExpressionSuite + extends FunSuite + with BeforeAndAfterEach + with BeforeAndAfterAll + with Logging { val encoder = JavaBytesEncoder.create(HBaseSparkConf.DEFAULT_QUERY_ENCODER) @@ -137,34 +139,31 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { columnToCurrentRowValueMap.put("Col1", new ByteArrayComparable(Bytes.toBytes(10))) val valueFromQueryValueArray = new Array[Array[Byte]](1) - //great than + // great than valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10) assert(!greaterLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) valueFromQueryValueArray(0) = encoder.encode(IntegerType, 20) assert(!greaterLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) - //great than and equal + // great than and equal valueFromQueryValueArray(0) = encoder.encode(IntegerType, 5) - assert(greaterAndEqualLogic.execute(columnToCurrentRowValueMap, - valueFromQueryValueArray)) + assert(greaterAndEqualLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10) - assert(greaterAndEqualLogic.execute(columnToCurrentRowValueMap, - valueFromQueryValueArray)) + assert(greaterAndEqualLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) valueFromQueryValueArray(0) = encoder.encode(IntegerType, 20) - assert(!greaterAndEqualLogic.execute(columnToCurrentRowValueMap, - valueFromQueryValueArray)) + assert(!greaterAndEqualLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) - //less than + // less than valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10) assert(!lessLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) valueFromQueryValueArray(0) = encoder.encode(IntegerType, 5) assert(!lessLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) - //less than and equal + // less than and equal valueFromQueryValueArray(0) = encoder.encode(IntegerType, 20) assert(lessAndEqualLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) @@ -174,21 +173,21 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10) assert(lessAndEqualLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) - //equal too + // equal too valueFromQueryValueArray(0) = Bytes.toBytes(10) assert(equalLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) valueFromQueryValueArray(0) = Bytes.toBytes(5) assert(!equalLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) - //not equal too + // not equal too valueFromQueryValueArray(0) = Bytes.toBytes(10) assert(!notEqualLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) valueFromQueryValueArray(0) = Bytes.toBytes(5) assert(notEqualLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) - //pass through + // pass through valueFromQueryValueArray(0) = Bytes.toBytes(10) assert(passThrough.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) @@ -196,7 +195,6 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { assert(passThrough.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) } - test("Double Type") { val leftLogic = new LessThanLogicExpression("Col1", 0) leftLogic.setEncoder(encoder) @@ -295,34 +293,31 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { columnToCurrentRowValueMap.put("Col1", new ByteArrayComparable(Bytes.toBytes(10L))) val valueFromQueryValueArray = new Array[Array[Byte]](1) - //great than + // great than valueFromQueryValueArray(0) = encoder.encode(LongType, 10L) assert(!greaterLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) valueFromQueryValueArray(0) = encoder.encode(LongType, 20L) assert(!greaterLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) - //great than and equal + // great than and equal valueFromQueryValueArray(0) = encoder.encode(LongType, 5L) - assert(greaterAndEqualLogic.execute(columnToCurrentRowValueMap, - valueFromQueryValueArray)) + assert(greaterAndEqualLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) valueFromQueryValueArray(0) = encoder.encode(LongType, 10L) - assert(greaterAndEqualLogic.execute(columnToCurrentRowValueMap, - valueFromQueryValueArray)) + assert(greaterAndEqualLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) valueFromQueryValueArray(0) = encoder.encode(LongType, 20L) - assert(!greaterAndEqualLogic.execute(columnToCurrentRowValueMap, - valueFromQueryValueArray)) + assert(!greaterAndEqualLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) - //less than + // less than valueFromQueryValueArray(0) = encoder.encode(LongType, 10L) assert(!lessLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) valueFromQueryValueArray(0) = encoder.encode(LongType, 5L) assert(!lessLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) - //less than and equal + // less than and equal valueFromQueryValueArray(0) = encoder.encode(LongType, 20L) assert(lessAndEqualLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) @@ -332,14 +327,14 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { valueFromQueryValueArray(0) = encoder.encode(LongType, 10L) assert(lessAndEqualLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) - //equal too + // equal too valueFromQueryValueArray(0) = Bytes.toBytes(10L) assert(equalLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) valueFromQueryValueArray(0) = Bytes.toBytes(5L) assert(!equalLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) - //not equal too + // not equal too valueFromQueryValueArray(0) = Bytes.toBytes(10L) assert(!notEqualLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) diff --git a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseCatalogSuite.scala b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseCatalogSuite.scala index 326c2395..b5646503 100644 --- a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseCatalogSuite.scala +++ b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseCatalogSuite.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark import org.apache.hadoop.hbase.spark.datasources.{DataTypeParserWrapper, DoubleSerDes, HBaseTableCatalog} @@ -22,7 +22,11 @@ import org.apache.hadoop.hbase.util.Bytes import org.apache.spark.sql.types._ import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} -class HBaseCatalogSuite extends FunSuite with BeforeAndAfterEach with BeforeAndAfterAll with Logging { +class HBaseCatalogSuite + extends FunSuite + with BeforeAndAfterEach + with BeforeAndAfterAll + with Logging { val map = s"""MAP>""" val array = s"""array>""" @@ -35,7 +39,8 @@ class HBaseCatalogSuite extends FunSuite with BeforeAndAfterEach with BeforeAndA |"col2":{"cf":"rowkey", "col":"key2", "type":"double"}, |"col3":{"cf":"cf1", "col":"col2", "type":"binary"}, |"col4":{"cf":"cf1", "col":"col3", "type":"timestamp"}, - |"col5":{"cf":"cf1", "col":"col4", "type":"double", "serdes":"${classOf[DoubleSerDes].getName}"}, + |"col5":{"cf":"cf1", "col":"col4", "type":"double", "serdes":"${classOf[ + DoubleSerDes].getName}"}, |"col6":{"cf":"cf1", "col":"col5", "type":"$map"}, |"col7":{"cf":"cf1", "col":"col6", "type":"$array"}, |"col8":{"cf":"cf1", "col":"col7", "type":"$arrayMap"}, @@ -43,7 +48,7 @@ class HBaseCatalogSuite extends FunSuite with BeforeAndAfterEach with BeforeAndA |"col10":{"cf":"cf1", "col":"col9", "type":"timestamp"} |} |}""".stripMargin - val parameters = Map(HBaseTableCatalog.tableCatalog->catalog) + val parameters = Map(HBaseTableCatalog.tableCatalog -> catalog) val t = HBaseTableCatalog(parameters) def checkDataType(dataTypeString: String, expectedDataType: DataType): Unit = { @@ -69,28 +74,20 @@ class HBaseCatalogSuite extends FunSuite with BeforeAndAfterEach with BeforeAndA assert(t.getField("col10").dt == TimestampType) } - checkDataType( - map, - t.getField("col6").dt - ) + checkDataType(map, t.getField("col6").dt) - checkDataType( - array, - t.getField("col7").dt - ) + checkDataType(array, t.getField("col7").dt) - checkDataType( - arrayMap, - t.getField("col8").dt - ) + checkDataType(arrayMap, t.getField("col8").dt) test("convert") { - val m = Map("hbase.columns.mapping" -> - "KEY_FIELD STRING :key, A_FIELD STRING c:a, B_FIELD DOUBLE c:b, C_FIELD BINARY c:c,", + val m = Map( + "hbase.columns.mapping" -> + "KEY_FIELD STRING :key, A_FIELD STRING c:a, B_FIELD DOUBLE c:b, C_FIELD BINARY c:c,", "hbase.table" -> "NAMESPACE:TABLE") val map = HBaseTableCatalog.convert(m) val json = map.get(HBaseTableCatalog.tableCatalog).get - val parameters = Map(HBaseTableCatalog.tableCatalog->json) + val parameters = Map(HBaseTableCatalog.tableCatalog -> json) val t = HBaseTableCatalog(parameters) assert(t.namespace === "NAMESPACE") assert(t.name == "TABLE") @@ -102,8 +99,9 @@ class HBaseCatalogSuite extends FunSuite with BeforeAndAfterEach with BeforeAndA } test("compatibility") { - val m = Map("hbase.columns.mapping" -> - "KEY_FIELD STRING :key, A_FIELD STRING c:a, B_FIELD DOUBLE c:b, C_FIELD BINARY c:c,", + val m = Map( + "hbase.columns.mapping" -> + "KEY_FIELD STRING :key, A_FIELD STRING c:a, B_FIELD DOUBLE c:b, C_FIELD BINARY c:c,", "hbase.table" -> "t1") val t = HBaseTableCatalog(m) assert(t.namespace === "default") diff --git a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseConnectionCacheSuite.scala b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseConnectionCacheSuite.scala index f07b1d6e..66e5acd4 100644 --- a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseConnectionCacheSuite.scala +++ b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseConnectionCacheSuite.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,25 +15,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark import java.util.concurrent.ExecutorService -import scala.util.Random - -import org.apache.hadoop.hbase.client.{BufferedMutator, Table, RegionLocator, - Connection, BufferedMutatorParams, Admin, TableBuilder} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.hbase.TableName +import org.apache.hadoop.hbase.client.{Admin, BufferedMutator, BufferedMutatorParams, Connection, RegionLocator, Table, TableBuilder} import org.scalatest.FunSuite +import scala.util.Random -case class HBaseConnectionKeyMocker (confId: Int) extends HBaseConnectionKey (null) { +case class HBaseConnectionKeyMocker(confId: Int) extends HBaseConnectionKey(null) { override def hashCode: Int = { confId } override def equals(obj: Any): Boolean = { - if(!obj.isInstanceOf[HBaseConnectionKeyMocker]) + if (!obj.isInstanceOf[HBaseConnectionKeyMocker]) false else confId == obj.asInstanceOf[HBaseConnectionKeyMocker].confId @@ -42,12 +40,12 @@ case class HBaseConnectionKeyMocker (confId: Int) extends HBaseConnectionKey (nu class ConnectionMocker extends Connection { var isClosed: Boolean = false - def getRegionLocator (tableName: TableName): RegionLocator = null + def getRegionLocator(tableName: TableName): RegionLocator = null def getConfiguration: Configuration = null - override def getTable (tableName: TableName): Table = null + override def getTable(tableName: TableName): Table = null override def getTable(tableName: TableName, pool: ExecutorService): Table = null - def getBufferedMutator (params: BufferedMutatorParams): BufferedMutator = null - def getBufferedMutator (tableName: TableName): BufferedMutator = null + def getBufferedMutator(params: BufferedMutatorParams): BufferedMutator = null + def getBufferedMutator(tableName: TableName): BufferedMutator = null def getAdmin: Admin = null def getTableBuilder(tableName: TableName, pool: ExecutorService): TableBuilder = null @@ -61,7 +59,8 @@ class ConnectionMocker extends Connection { def abort(why: String, e: Throwable) = {} /* Without override, we can also compile it against HBase 2.1. */ - /* override */ def clearRegionLocationCache(): Unit = {} + /* override */ + def clearRegionLocationCache(): Unit = {} } class HBaseConnectionCacheSuite extends FunSuite with Logging { @@ -139,8 +138,12 @@ class HBaseConnectionCacheSuite extends FunSuite with Logging { Thread.sleep(3 * 1000) // Leave housekeeping thread enough time HBaseConnectionCache.connectionMap.synchronized { assert(HBaseConnectionCache.connectionMap.size === 1) - assert(HBaseConnectionCache.connectionMap.iterator.next()._1 - .asInstanceOf[HBaseConnectionKeyMocker].confId === 2) + assert( + HBaseConnectionCache.connectionMap.iterator + .next() + ._1 + .asInstanceOf[HBaseConnectionKeyMocker] + .confId === 2) assert(HBaseConnectionCache.getStat.numActiveConnections === 1) } @@ -154,7 +157,8 @@ class HBaseConnectionCacheSuite extends FunSuite with Logging { override def run() { for (i <- 0 to 999) { val c = HBaseConnectionCache.getConnection( - new HBaseConnectionKeyMocker(Random.nextInt(10)), new ConnectionMocker) + new HBaseConnectionKeyMocker(Random.nextInt(10)), + new ConnectionMocker) } } } @@ -178,16 +182,15 @@ class HBaseConnectionCacheSuite extends FunSuite with Logging { assert(HBaseConnectionCache.getStat.numActualConnectionsCreated === 10) assert(HBaseConnectionCache.getStat.numActiveConnections === 10) - var totalRc : Int = 0 - HBaseConnectionCache.connectionMap.foreach { - x => totalRc += x._2.refCount - } + var totalRc: Int = 0 + HBaseConnectionCache.connectionMap.foreach { x => totalRc += x._2.refCount } assert(totalRc === 100 * 1000) HBaseConnectionCache.connectionMap.foreach { - x => { - x._2.refCount = 0 - x._2.timestamp = System.currentTimeMillis() - 1000 - } + x => + { + x._2.refCount = 0 + x._2.timestamp = System.currentTimeMillis() - 1000 + } } } Thread.sleep(1000) @@ -203,7 +206,8 @@ class HBaseConnectionCacheSuite extends FunSuite with Logging { override def run() { for (i <- 0 to 999) { val c = HBaseConnectionCache.getConnection( - new HBaseConnectionKeyMocker(Random.nextInt(10)), new ConnectionMocker) + new HBaseConnectionKeyMocker(Random.nextInt(10)), + new ConnectionMocker) Thread.`yield`() c.close() } diff --git a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseContextSuite.scala b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseContextSuite.scala index 1b35b93f..4935b09d 100644 --- a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseContextSuite.scala +++ b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseContextSuite.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -16,15 +17,18 @@ */ package org.apache.hadoop.hbase.spark +import org.apache.hadoop.hbase.{CellUtil, HBaseTestingUtility, TableName} import org.apache.hadoop.hbase.client._ import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter import org.apache.hadoop.hbase.util.Bytes -import org.apache.hadoop.hbase.{ CellUtil, TableName, HBaseTestingUtility} -import org.apache.spark.{SparkException, SparkContext} +import org.apache.spark.{SparkContext, SparkException} import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} -class HBaseContextSuite extends FunSuite with -BeforeAndAfterEach with BeforeAndAfterAll with Logging { +class HBaseContextSuite + extends FunSuite + with BeforeAndAfterEach + with BeforeAndAfterAll + with Logging { @transient var sc: SparkContext = null var hbaseContext: HBaseContext = null @@ -47,7 +51,7 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { TEST_UTIL.createTable(TableName.valueOf(tableName), Bytes.toBytes(columnFamily)) logInfo(" - created table") - val envMap = Map[String,String](("Xmx", "512m")) + val envMap = Map[String, String](("Xmx", "512m")) sc = new SparkContext("local", "test", null, Nil, envMap) @@ -65,19 +69,26 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { test("bulkput to test HBase client") { val config = TEST_UTIL.getConfiguration - val rdd = sc.parallelize(Array( - (Bytes.toBytes("1"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1")))), - (Bytes.toBytes("2"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("b"), Bytes.toBytes("foo2")))), - (Bytes.toBytes("3"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("c"), Bytes.toBytes("foo3")))), - (Bytes.toBytes("4"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("d"), Bytes.toBytes("foo")))), - (Bytes.toBytes("5"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("e"), Bytes.toBytes("bar")))))) - - hbaseContext.bulkPut[(Array[Byte], Array[(Array[Byte], Array[Byte], Array[Byte])])](rdd, + val rdd = sc.parallelize( + Array( + ( + Bytes.toBytes("1"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1")))), + ( + Bytes.toBytes("2"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("b"), Bytes.toBytes("foo2")))), + ( + Bytes.toBytes("3"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("c"), Bytes.toBytes("foo3")))), + ( + Bytes.toBytes("4"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("d"), Bytes.toBytes("foo")))), + ( + Bytes.toBytes("5"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("e"), Bytes.toBytes("bar")))))) + + hbaseContext.bulkPut[(Array[Byte], Array[(Array[Byte], Array[Byte], Array[Byte])])]( + rdd, TableName.valueOf(tableName), (putRecord) => { val put = new Put(putRecord._1) @@ -89,24 +100,39 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { val table = connection.getTable(TableName.valueOf("t1")) try { - val foo1 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("1"))). - getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")))) + val foo1 = Bytes.toString( + CellUtil.cloneValue( + table + .get(new Get(Bytes.toBytes("1"))) + .getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")))) assert(foo1 == "foo1") - val foo2 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("2"))). - getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("b")))) + val foo2 = Bytes.toString( + CellUtil.cloneValue( + table + .get(new Get(Bytes.toBytes("2"))) + .getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("b")))) assert(foo2 == "foo2") - val foo3 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("3"))). - getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("c")))) + val foo3 = Bytes.toString( + CellUtil.cloneValue( + table + .get(new Get(Bytes.toBytes("3"))) + .getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("c")))) assert(foo3 == "foo3") - val foo4 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("4"))). - getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("d")))) + val foo4 = Bytes.toString( + CellUtil.cloneValue( + table + .get(new Get(Bytes.toBytes("4"))) + .getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("d")))) assert(foo4 == "foo") - val foo5 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("5"))). - getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("e")))) + val foo5 = Bytes.toString( + CellUtil.cloneValue( + table + .get(new Get(Bytes.toBytes("5"))) + .getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("e")))) assert(foo5 == "bar") } finally { @@ -131,21 +157,29 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo3")) table.put(put) - val rdd = sc.parallelize(Array( - Bytes.toBytes("delete1"), - Bytes.toBytes("delete3"))) + val rdd = sc.parallelize(Array(Bytes.toBytes("delete1"), Bytes.toBytes("delete3"))) - hbaseContext.bulkDelete[Array[Byte]](rdd, + hbaseContext.bulkDelete[Array[Byte]]( + rdd, TableName.valueOf(tableName), putRecord => new Delete(putRecord), 4) - assert(table.get(new Get(Bytes.toBytes("delete1"))). - getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")) == null) - assert(table.get(new Get(Bytes.toBytes("delete3"))). - getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")) == null) - assert(Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("delete2"))). - getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")))).equals("foo2")) + assert( + table + .get(new Get(Bytes.toBytes("delete1"))) + .getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")) == null) + assert( + table + .get(new Get(Bytes.toBytes("delete3"))) + .getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")) == null) + assert( + Bytes + .toString( + CellUtil.cloneValue(table + .get(new Get(Bytes.toBytes("delete2"))) + .getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")))) + .equals("foo2")) } finally { table.close() connection.close() @@ -171,11 +205,12 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { table.close() connection.close() } - val rdd = sc.parallelize(Array( - Bytes.toBytes("get1"), - Bytes.toBytes("get2"), - Bytes.toBytes("get3"), - Bytes.toBytes("get4"))) + val rdd = sc.parallelize( + Array( + Bytes.toBytes("get1"), + Bytes.toBytes("get2"), + Bytes.toBytes("get3"), + Bytes.toBytes("get4"))) val getRdd = hbaseContext.bulkGet[Array[Byte], String]( TableName.valueOf(tableName), @@ -217,11 +252,12 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { test("BulkGet failure test: bad table") { val config = TEST_UTIL.getConfiguration - val rdd = sc.parallelize(Array( - Bytes.toBytes("get1"), - Bytes.toBytes("get2"), - Bytes.toBytes("get3"), - Bytes.toBytes("get4"))) + val rdd = sc.parallelize( + Array( + Bytes.toBytes("get1"), + Bytes.toBytes("get2"), + Bytes.toBytes("get3"), + Bytes.toBytes("get4"))) intercept[SparkException] { try { @@ -269,11 +305,12 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { connection.close() } - val rdd = sc.parallelize(Array( - Bytes.toBytes("get1"), - Bytes.toBytes("get2"), - Bytes.toBytes("get3"), - Bytes.toBytes("get4"))) + val rdd = sc.parallelize( + Array( + Bytes.toBytes("get1"), + Bytes.toBytes("get2"), + Bytes.toBytes("get3"), + Bytes.toBytes("get4"))) val getRdd = hbaseContext.bulkGet[Array[Byte], String]( TableName.valueOf(tableName), @@ -284,17 +321,20 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { }, (result: Result) => { if (result.listCells() != null) { - val cellValue = result.getColumnLatestCell( - Bytes.toBytes("c"), Bytes.toBytes("bad_column")) + val cellValue = + result.getColumnLatestCell(Bytes.toBytes("c"), Bytes.toBytes("bad_column")) if (cellValue == null) "null" else "bad" } else "noValue" }) var nullCounter = 0 var noValueCounter = 0 - getRdd.collect().foreach(r => { - if ("null".equals(r)) nullCounter += 1 - else if ("noValue".equals(r)) noValueCounter += 1 - }) + getRdd + .collect() + .foreach( + r => { + if ("null".equals(r)) nullCounter += 1 + else if ("noValue".equals(r)) noValueCounter += 1 + }) assert(nullCounter == 3) assert(noValueCounter == 1) } @@ -338,12 +378,18 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { val scanRdd = hbaseContext.hbaseRDD(TableName.valueOf(tableName), scan) try { - val scanList = scanRdd.map(r => r._1.copyBytes()).collect() + val scanList = scanRdd + .map(r => r._1.copyBytes()) + .collect() assert(scanList.length == 3) var cnt = 0 - scanRdd.map(r => r._2.listCells().size()).collect().foreach(l => { - cnt += l - }) + scanRdd + .map(r => r._2.listCells().size()) + .collect() + .foreach( + l => { + cnt += l + }) // the number of cells returned would be 4 without the Filter assert(cnt == 3); } catch { diff --git a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseDStreamFunctionsSuite.scala b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseDStreamFunctionsSuite.scala index 75925258..0f76af18 100644 --- a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseDStreamFunctionsSuite.scala +++ b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseDStreamFunctionsSuite.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -16,19 +17,21 @@ */ package org.apache.hadoop.hbase.spark +import org.apache.hadoop.hbase.{CellUtil, HBaseTestingUtility, TableName} import org.apache.hadoop.hbase.client._ +import org.apache.hadoop.hbase.spark.HBaseDStreamFunctions._ import org.apache.hadoop.hbase.util.Bytes -import org.apache.hadoop.hbase.{CellUtil, TableName, HBaseTestingUtility} +import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Milliseconds, StreamingContext} -import org.apache.spark.SparkContext -import org.apache.hadoop.hbase.spark.HBaseDStreamFunctions._ import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} - import scala.collection.mutable -class HBaseDStreamFunctionsSuite extends FunSuite with -BeforeAndAfterEach with BeforeAndAfterAll with Logging { +class HBaseDStreamFunctionsSuite + extends FunSuite + with BeforeAndAfterEach + with BeforeAndAfterAll + with Logging { @transient var sc: SparkContext = null var TEST_UTIL: HBaseTestingUtility = new HBaseTestingUtility @@ -62,27 +65,33 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { test("bulkput to test HBase client") { val config = TEST_UTIL.getConfiguration - val rdd1 = sc.parallelize(Array( - (Bytes.toBytes("1"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1")))), - (Bytes.toBytes("2"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("b"), Bytes.toBytes("foo2")))), - (Bytes.toBytes("3"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("c"), Bytes.toBytes("foo3")))))) - - val rdd2 = sc.parallelize(Array( - (Bytes.toBytes("4"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("d"), Bytes.toBytes("foo")))), - (Bytes.toBytes("5"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("e"), Bytes.toBytes("bar")))))) + val rdd1 = sc.parallelize( + Array( + ( + Bytes.toBytes("1"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1")))), + ( + Bytes.toBytes("2"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("b"), Bytes.toBytes("foo2")))), + ( + Bytes.toBytes("3"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("c"), Bytes.toBytes("foo3")))))) + + val rdd2 = sc.parallelize( + Array( + ( + Bytes.toBytes("4"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("d"), Bytes.toBytes("foo")))), + ( + Bytes.toBytes("5"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("e"), Bytes.toBytes("bar")))))) var isFinished = false val hbaseContext = new HBaseContext(sc, config) val ssc = new StreamingContext(sc, Milliseconds(200)) - val queue = mutable.Queue[RDD[(Array[Byte], Array[(Array[Byte], - Array[Byte], Array[Byte])])]]() + val queue = mutable.Queue[RDD[(Array[Byte], Array[(Array[Byte], Array[Byte], Array[Byte])])]]() queue += rdd1 queue += rdd2 val dStream = ssc.queueStream(queue) @@ -96,11 +105,12 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { put }) - dStream.foreachRDD(rdd => { - if (rdd.count() == 0) { - isFinished = true - } - }) + dStream.foreachRDD( + rdd => { + if (rdd.count() == 0) { + isFinished = true + } + }) ssc.start() @@ -114,24 +124,39 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { val table = connection.getTable(TableName.valueOf("t1")) try { - val foo1 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("1"))). - getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")))) + val foo1 = Bytes.toString( + CellUtil.cloneValue( + table + .get(new Get(Bytes.toBytes("1"))) + .getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")))) assert(foo1 == "foo1") - val foo2 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("2"))). - getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("b")))) + val foo2 = Bytes.toString( + CellUtil.cloneValue( + table + .get(new Get(Bytes.toBytes("2"))) + .getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("b")))) assert(foo2 == "foo2") - val foo3 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("3"))). - getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("c")))) + val foo3 = Bytes.toString( + CellUtil.cloneValue( + table + .get(new Get(Bytes.toBytes("3"))) + .getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("c")))) assert(foo3 == "foo3") - val foo4 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("4"))). - getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("d")))) + val foo4 = Bytes.toString( + CellUtil.cloneValue( + table + .get(new Get(Bytes.toBytes("4"))) + .getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("d")))) assert(foo4 == "foo") - val foo5 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("5"))). - getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("e")))) + val foo5 = Bytes.toString( + CellUtil.cloneValue( + table + .get(new Get(Bytes.toBytes("5"))) + .getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("e")))) assert(foo5 == "bar") } finally { table.close() diff --git a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseRDDFunctionsSuite.scala b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseRDDFunctionsSuite.scala index 9ea2c7fa..841310f3 100644 --- a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseRDDFunctionsSuite.scala +++ b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseRDDFunctionsSuite.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -16,17 +17,19 @@ */ package org.apache.hadoop.hbase.spark +import org.apache.hadoop.hbase.{CellUtil, HBaseTestingUtility, TableName} import org.apache.hadoop.hbase.client._ -import org.apache.hadoop.hbase.util.Bytes -import org.apache.hadoop.hbase.{CellUtil, TableName, HBaseTestingUtility} import org.apache.hadoop.hbase.spark.HBaseRDDFunctions._ +import org.apache.hadoop.hbase.util.Bytes import org.apache.spark.SparkContext import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} - import scala.collection.mutable -class HBaseRDDFunctionsSuite extends FunSuite with -BeforeAndAfterEach with BeforeAndAfterAll with Logging { +class HBaseRDDFunctionsSuite + extends FunSuite + with BeforeAndAfterEach + with BeforeAndAfterAll + with Logging { @transient var sc: SparkContext = null var TEST_UTIL: HBaseTestingUtility = new HBaseTestingUtility @@ -61,22 +64,28 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { test("bulkput to test HBase client") { val config = TEST_UTIL.getConfiguration - val rdd = sc.parallelize(Array( - (Bytes.toBytes("1"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1")))), - (Bytes.toBytes("2"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("b"), Bytes.toBytes("foo2")))), - (Bytes.toBytes("3"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("c"), Bytes.toBytes("foo3")))), - (Bytes.toBytes("4"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("d"), Bytes.toBytes("foo")))), - (Bytes.toBytes("5"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("e"), Bytes.toBytes("bar")))))) + val rdd = sc.parallelize( + Array( + ( + Bytes.toBytes("1"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1")))), + ( + Bytes.toBytes("2"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("b"), Bytes.toBytes("foo2")))), + ( + Bytes.toBytes("3"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("c"), Bytes.toBytes("foo3")))), + ( + Bytes.toBytes("4"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("d"), Bytes.toBytes("foo")))), + ( + Bytes.toBytes("5"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("e"), Bytes.toBytes("bar")))))) val hbaseContext = new HBaseContext(sc, config) rdd.hbaseBulkPut( - hbaseContext, + hbaseContext, TableName.valueOf(tableName), (putRecord) => { val put = new Put(putRecord._1) @@ -88,24 +97,39 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { val table = connection.getTable(TableName.valueOf("t1")) try { - val foo1 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("1"))). - getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")))) + val foo1 = Bytes.toString( + CellUtil.cloneValue( + table + .get(new Get(Bytes.toBytes("1"))) + .getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")))) assert(foo1 == "foo1") - val foo2 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("2"))). - getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("b")))) + val foo2 = Bytes.toString( + CellUtil.cloneValue( + table + .get(new Get(Bytes.toBytes("2"))) + .getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("b")))) assert(foo2 == "foo2") - val foo3 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("3"))). - getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("c")))) + val foo3 = Bytes.toString( + CellUtil.cloneValue( + table + .get(new Get(Bytes.toBytes("3"))) + .getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("c")))) assert(foo3 == "foo3") - val foo4 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("4"))). - getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("d")))) + val foo4 = Bytes.toString( + CellUtil.cloneValue( + table + .get(new Get(Bytes.toBytes("4"))) + .getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("d")))) assert(foo4 == "foo") - val foo5 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("5"))). - getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("e")))) + val foo5 = Bytes.toString( + CellUtil.cloneValue( + table + .get(new Get(Bytes.toBytes("5"))) + .getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("e")))) assert(foo5 == "bar") } finally { table.close() @@ -129,23 +153,31 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo3")) table.put(put) - val rdd = sc.parallelize(Array( - Bytes.toBytes("delete1"), - Bytes.toBytes("delete3"))) + val rdd = sc.parallelize(Array(Bytes.toBytes("delete1"), Bytes.toBytes("delete3"))) val hbaseContext = new HBaseContext(sc, config) - rdd.hbaseBulkDelete(hbaseContext, + rdd.hbaseBulkDelete( + hbaseContext, TableName.valueOf(tableName), putRecord => new Delete(putRecord), 4) - assert(table.get(new Get(Bytes.toBytes("delete1"))). - getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")) == null) - assert(table.get(new Get(Bytes.toBytes("delete3"))). - getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")) == null) - assert(Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("delete2"))). - getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")))).equals("foo2")) + assert( + table + .get(new Get(Bytes.toBytes("delete1"))) + .getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")) == null) + assert( + table + .get(new Get(Bytes.toBytes("delete3"))) + .getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")) == null) + assert( + Bytes + .toString( + CellUtil.cloneValue(table + .get(new Get(Bytes.toBytes("delete2"))) + .getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")))) + .equals("foo2")) } finally { table.close() connection.close() @@ -173,15 +205,19 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { connection.close() } - val rdd = sc.parallelize(Array( - Bytes.toBytes("get1"), - Bytes.toBytes("get2"), - Bytes.toBytes("get3"), - Bytes.toBytes("get4"))) + val rdd = sc.parallelize( + Array( + Bytes.toBytes("get1"), + Bytes.toBytes("get2"), + Bytes.toBytes("get3"), + Bytes.toBytes("get4"))) val hbaseContext = new HBaseContext(sc, config) - //Get with custom convert logic - val getRdd = rdd.hbaseBulkGet[String](hbaseContext, TableName.valueOf(tableName), 2, + // Get with custom convert logic + val getRdd = rdd.hbaseBulkGet[String]( + hbaseContext, + TableName.valueOf(tableName), + 2, record => { new Get(record) }, @@ -235,36 +271,44 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { connection.close() } - val rdd = sc.parallelize(Array( - Bytes.toBytes("get1"), - Bytes.toBytes("get2"), - Bytes.toBytes("get3"), - Bytes.toBytes("get4"))) + val rdd = sc.parallelize( + Array( + Bytes.toBytes("get1"), + Bytes.toBytes("get2"), + Bytes.toBytes("get3"), + Bytes.toBytes("get4"))) val hbaseContext = new HBaseContext(sc, config) - val getRdd = rdd.hbaseBulkGet(hbaseContext, TableName.valueOf("t1"), 2, - record => { - new Get(record) - }).map((row) => { - if (row != null && row._2.listCells() != null) { - val it = row._2.listCells().iterator() - val B = new StringBuilder - - B.append(Bytes.toString(row._2.getRow) + ":") - - while (it.hasNext) { - val cell = it.next - val q = Bytes.toString(CellUtil.cloneQualifier(cell)) - if (q.equals("counter")) { - B.append("(" + q + "," + Bytes.toLong(CellUtil.cloneValue(cell)) + ")") + val getRdd = rdd + .hbaseBulkGet( + hbaseContext, + TableName.valueOf("t1"), + 2, + record => { + new Get(record) + }) + .map( + (row) => { + if (row != null && row._2.listCells() != null) { + val it = row._2.listCells().iterator() + val B = new StringBuilder + + B.append(Bytes.toString(row._2.getRow) + ":") + + while (it.hasNext) { + val cell = it.next + val q = Bytes.toString(CellUtil.cloneQualifier(cell)) + if (q.equals("counter")) { + B.append("(" + q + "," + Bytes.toLong(CellUtil.cloneValue(cell)) + ")") + } else { + B.append("(" + q + "," + Bytes.toString(CellUtil.cloneValue(cell)) + ")") + } + } + "" + B.toString } else { - B.append("(" + q + "," + Bytes.toString(CellUtil.cloneValue(cell)) + ")") + "" } - } - "" + B.toString - } else { - "" - }}) + }) val getArray = getRdd.collect() @@ -276,53 +320,77 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { test("foreachPartition with puts to test HBase client") { val config = TEST_UTIL.getConfiguration - val rdd = sc.parallelize(Array( - (Bytes.toBytes("1foreach"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1")))), - (Bytes.toBytes("2foreach"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("b"), Bytes.toBytes("foo2")))), - (Bytes.toBytes("3foreach"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("c"), Bytes.toBytes("foo3")))), - (Bytes.toBytes("4foreach"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("d"), Bytes.toBytes("foo")))), - (Bytes.toBytes("5foreach"), - Array((Bytes.toBytes(columnFamily), Bytes.toBytes("e"), Bytes.toBytes("bar")))))) + val rdd = sc.parallelize( + Array( + ( + Bytes.toBytes("1foreach"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1")))), + ( + Bytes.toBytes("2foreach"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("b"), Bytes.toBytes("foo2")))), + ( + Bytes.toBytes("3foreach"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("c"), Bytes.toBytes("foo3")))), + ( + Bytes.toBytes("4foreach"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("d"), Bytes.toBytes("foo")))), + ( + Bytes.toBytes("5foreach"), + Array((Bytes.toBytes(columnFamily), Bytes.toBytes("e"), Bytes.toBytes("bar")))))) val hbaseContext = new HBaseContext(sc, config) - rdd.hbaseForeachPartition(hbaseContext, (it, conn) => { - val bufferedMutator = conn.getBufferedMutator(TableName.valueOf("t1")) - it.foreach((putRecord) => { - val put = new Put(putRecord._1) - putRecord._2.foreach((putValue) => put.addColumn(putValue._1, putValue._2, putValue._3)) - bufferedMutator.mutate(put) + rdd.hbaseForeachPartition( + hbaseContext, + (it, conn) => { + val bufferedMutator = conn.getBufferedMutator(TableName.valueOf("t1")) + it.foreach( + (putRecord) => { + val put = new Put(putRecord._1) + putRecord._2.foreach((putValue) => put.addColumn(putValue._1, putValue._2, putValue._3)) + bufferedMutator.mutate(put) + }) + bufferedMutator.flush() + bufferedMutator.close() }) - bufferedMutator.flush() - bufferedMutator.close() - }) val connection = ConnectionFactory.createConnection(config) val table = connection.getTable(TableName.valueOf("t1")) try { - val foo1 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("1foreach"))). - getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")))) + val foo1 = Bytes.toString( + CellUtil.cloneValue( + table + .get(new Get(Bytes.toBytes("1foreach"))) + .getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")))) assert(foo1 == "foo1") - val foo2 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("2foreach"))). - getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("b")))) + val foo2 = Bytes.toString( + CellUtil.cloneValue( + table + .get(new Get(Bytes.toBytes("2foreach"))) + .getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("b")))) assert(foo2 == "foo2") - val foo3 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("3foreach"))). - getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("c")))) + val foo3 = Bytes.toString( + CellUtil.cloneValue( + table + .get(new Get(Bytes.toBytes("3foreach"))) + .getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("c")))) assert(foo3 == "foo3") - val foo4 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("4foreach"))). - getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("d")))) + val foo4 = Bytes.toString( + CellUtil.cloneValue( + table + .get(new Get(Bytes.toBytes("4foreach"))) + .getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("d")))) assert(foo4 == "foo") - val foo5 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("5"))). - getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("e")))) + val foo5 = Bytes.toString( + CellUtil.cloneValue( + table + .get(new Get(Bytes.toBytes("5"))) + .getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("e")))) assert(foo5 == "bar") } finally { table.close() @@ -350,43 +418,47 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { connection.close() } - val rdd = sc.parallelize(Array( - Bytes.toBytes("get1"), - Bytes.toBytes("get2"), - Bytes.toBytes("get3"), - Bytes.toBytes("get4"))) + val rdd = sc.parallelize( + Array( + Bytes.toBytes("get1"), + Bytes.toBytes("get2"), + Bytes.toBytes("get3"), + Bytes.toBytes("get4"))) val hbaseContext = new HBaseContext(sc, config) - //Get with custom convert logic - val getRdd = rdd.hbaseMapPartitions(hbaseContext, (it, conn) => { - val table = conn.getTable(TableName.valueOf("t1")) - var res = mutable.MutableList[String]() - - it.foreach(r => { - val get = new Get(r) - val result = table.get(get) - if (result.listCells != null) { - val it = result.listCells().iterator() - val B = new StringBuilder - - B.append(Bytes.toString(result.getRow) + ":") - - while (it.hasNext) { - val cell = it.next() - val q = Bytes.toString(CellUtil.cloneQualifier(cell)) - if (q.equals("counter")) { - B.append("(" + q + "," + Bytes.toLong(CellUtil.cloneValue(cell)) + ")") + // Get with custom convert logic + val getRdd = rdd.hbaseMapPartitions( + hbaseContext, + (it, conn) => { + val table = conn.getTable(TableName.valueOf("t1")) + var res = mutable.MutableList[String]() + + it.foreach( + r => { + val get = new Get(r) + val result = table.get(get) + if (result.listCells != null) { + val it = result.listCells().iterator() + val B = new StringBuilder + + B.append(Bytes.toString(result.getRow) + ":") + + while (it.hasNext) { + val cell = it.next() + val q = Bytes.toString(CellUtil.cloneQualifier(cell)) + if (q.equals("counter")) { + B.append("(" + q + "," + Bytes.toLong(CellUtil.cloneValue(cell)) + ")") + } else { + B.append("(" + q + "," + Bytes.toString(CellUtil.cloneValue(cell)) + ")") + } + } + res += "" + B.toString } else { - B.append("(" + q + "," + Bytes.toString(CellUtil.cloneValue(cell)) + ")") + res += "" } - } - res += "" + B.toString - } else { - res += "" - } + }) + res.iterator }) - res.iterator - }) val getArray = getRdd.collect() diff --git a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseTestSource.scala b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseTestSource.scala index ccb46256..a27876f1 100644 --- a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseTestSource.scala +++ b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseTestSource.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark import org.apache.hadoop.hbase.spark.datasources.HBaseSparkConf @@ -36,27 +36,24 @@ class HBaseTestSource extends RelationProvider { } } -case class DummyScan( - cacheSize: Int, - batchNum: Int, - blockCachingEnable: Boolean, - rowNum: Int)(@transient val sqlContext: SQLContext) - extends BaseRelation with TableScan { +case class DummyScan(cacheSize: Int, batchNum: Int, blockCachingEnable: Boolean, rowNum: Int)( + @transient val sqlContext: SQLContext) + extends BaseRelation + with TableScan { private def sparkConf = SparkEnv.get.conf override def schema: StructType = StructType(StructField("i", IntegerType, nullable = false) :: Nil) - override def buildScan(): RDD[Row] = sqlContext.sparkContext.parallelize(0 until rowNum) + override def buildScan(): RDD[Row] = sqlContext.sparkContext + .parallelize(0 until rowNum) .map(Row(_)) - .map{ x => - if (sparkConf.getInt(HBaseSparkConf.QUERY_BATCHSIZE, - -1) != batchNum || - sparkConf.getInt(HBaseSparkConf.QUERY_CACHEDROWS, - -1) != cacheSize || - sparkConf.getBoolean(HBaseSparkConf.QUERY_CACHEBLOCKS, - false) != blockCachingEnable) { - throw new Exception("HBase Spark configuration cannot be set properly") - } - x - } + .map { + x => + if (sparkConf.getInt(HBaseSparkConf.QUERY_BATCHSIZE, -1) != batchNum || + sparkConf.getInt(HBaseSparkConf.QUERY_CACHEDROWS, -1) != cacheSize || + sparkConf.getBoolean(HBaseSparkConf.QUERY_CACHEBLOCKS, false) != blockCachingEnable) { + throw new Exception("HBase Spark configuration cannot be set properly") + } + x + } } diff --git a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/PartitionFilterSuite.scala b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/PartitionFilterSuite.scala index 49600847..17a8fe56 100644 --- a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/PartitionFilterSuite.scala +++ b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/PartitionFilterSuite.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,29 +15,29 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark -import org.apache.hadoop.hbase.spark.datasources.{HBaseSparkConf, HBaseTableCatalog} import org.apache.hadoop.hbase.{HBaseTestingUtility, TableName} -import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.hadoop.hbase.spark.datasources.{HBaseSparkConf, HBaseTableCatalog} import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.{DataFrame, SQLContext} import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} case class FilterRangeRecord( - intCol0: Int, - boolCol1: Boolean, - doubleCol2: Double, - floatCol3: Float, - intCol4: Int, - longCol5: Long, - shortCol6: Short, - stringCol7: String, - byteCol8: Byte) + intCol0: Int, + boolCol1: Boolean, + doubleCol2: Double, + floatCol3: Float, + intCol4: Int, + longCol5: Long, + shortCol6: Short, + stringCol7: String, + byteCol8: Byte) object FilterRangeRecord { def apply(i: Int): FilterRangeRecord = { - FilterRangeRecord(if (i % 2 == 0) i else -i, + FilterRangeRecord( + if (i % 2 == 0) i else -i, i % 2 == 0, if (i % 2 == 0) i.toDouble else -i.toDouble, i.toFloat, @@ -48,8 +49,11 @@ object FilterRangeRecord { } } -class PartitionFilterSuite extends FunSuite with - BeforeAndAfterEach with BeforeAndAfterAll with Logging { +class PartitionFilterSuite + extends FunSuite + with BeforeAndAfterEach + with BeforeAndAfterAll + with Logging { @transient var sc: SparkContext = null var TEST_UTIL: HBaseTestingUtility = new HBaseTestingUtility @@ -57,8 +61,7 @@ class PartitionFilterSuite extends FunSuite with var df: DataFrame = null def withCatalog(cat: String): DataFrame = { - sqlContext - .read + sqlContext.read .options(Map(HBaseTableCatalog.tableCatalog -> cat)) .format("org.apache.hadoop.hbase.spark") .load() @@ -90,9 +93,7 @@ class PartitionFilterSuite extends FunSuite with // The original raw data used for construct result set without going through // data frame logic. It is used to verify the result set retrieved from data frame logic. - val rawResult = (0 until 32).map { i => - FilterRangeRecord(i) - } + val rawResult = (0 until 32).map { i => FilterRangeRecord(i) } def collectToSet[T](df: DataFrame): Set[T] = { df.collect().map(_.getAs[T](0)).toSet @@ -117,8 +118,10 @@ class PartitionFilterSuite extends FunSuite with val sql = sqlContext import sql.implicits._ - sc.parallelize(rawResult).toDF.write.options( - Map(HBaseTableCatalog.tableCatalog -> catalog, HBaseTableCatalog.newTable -> "5")) + sc.parallelize(rawResult) + .toDF + .write + .options(Map(HBaseTableCatalog.tableCatalog -> catalog, HBaseTableCatalog.newTable -> "5")) .format("org.apache.hadoop.hbase.spark") .save() } @@ -129,28 +132,28 @@ class PartitionFilterSuite extends FunSuite with } /** - *expected result: only showing top 20 rows - *+-------+ - *|intCol0| - *+-------+ - *| -31 | - *| -29 | - *| -27 | - *| -25 | - *| -23 | - *| -21 | - *| -19 | - *| -17 | - *| -15 | - *| -13 | - *| -11 | - *| -9 | - *| -7 | - *| -5 | - *| -3 | - *| -1 | - *+---- + - */ + * expected result: only showing top 20 rows + * +-------+ + * |intCol0| + * +-------+ + * | -31 | + * | -29 | + * | -27 | + * | -25 | + * | -23 | + * | -21 | + * | -19 | + * | -17 | + * | -15 | + * | -13 | + * | -11 | + * | -9 | + * | -7 | + * | -5 | + * | -3 | + * | -1 | + * +---- + + */ test("rangeTable rowkey less than 0") { val sql = sqlContext import sql.implicits._ @@ -165,28 +168,28 @@ class PartitionFilterSuite extends FunSuite with } /** - *expected result: only showing top 20 rows - *+-------+ - *|intCol4| - *+-------+ - *| -31 | - *| -29 | - *| -27 | - *| -25 | - *| -23 | - *| -21 | - *| -19 | - *| -17 | - *| -15 | - *| -13 | - *| -11 | - *| -9 | - *| -7 | - *| -5 | - *| -3 | - *| -1 | - *+-------+ - */ + * expected result: only showing top 20 rows + * +-------+ + * |intCol4| + * +-------+ + * | -31 | + * | -29 | + * | -27 | + * | -25 | + * | -23 | + * | -21 | + * | -19 | + * | -17 | + * | -15 | + * | -13 | + * | -11 | + * | -9 | + * | -7 | + * | -5 | + * | -3 | + * | -1 | + * +-------+ + */ test("rangeTable int col less than 0") { val sql = sqlContext import sql.implicits._ @@ -201,30 +204,30 @@ class PartitionFilterSuite extends FunSuite with } /** - *expected result: only showing top 20 rows - *+-----------+ - *| doubleCol2| - *+-----------+ - *| 0.0 | - *| 2.0 | - *|-31.0 | - *|-29.0 | - *|-27.0 | - *|-25.0 | - *|-23.0 | - *|-21.0 | - *|-19.0 | - *|-17.0 | - *|-15.0 | - *|-13.0 | - *|-11.0 | - *| -9.0 | - *| -7.0 | - *| -5.0 | - *| -3.0 | - *| -1.0 | - *+-----------+ - */ + * expected result: only showing top 20 rows + * +-----------+ + * | doubleCol2| + * +-----------+ + * | 0.0 | + * | 2.0 | + * |-31.0 | + * |-29.0 | + * |-27.0 | + * |-25.0 | + * |-23.0 | + * |-21.0 | + * |-19.0 | + * |-17.0 | + * |-15.0 | + * |-13.0 | + * |-11.0 | + * | -9.0 | + * | -7.0 | + * | -5.0 | + * | -3.0 | + * | -1.0 | + * +-----------+ + */ test("rangeTable double col less than 0") { val sql = sqlContext import sql.implicits._ @@ -239,24 +242,23 @@ class PartitionFilterSuite extends FunSuite with } /** - * expected result: only showing top 20 rows - *+-------+ - *|intCol0| - *+-------+ - *| -31 | - *| -29 | - *| -27 | - *| -25 | - *| -23 | - *| -21 | - *| -19 | - *| -17 | - *| -15 | - *| -13 | - *| -11 | - *+-------+ - * - */ + * expected result: only showing top 20 rows + * +-------+ + * |intCol0| + * +-------+ + * | -31 | + * | -29 | + * | -27 | + * | -25 | + * | -23 | + * | -21 | + * | -19 | + * | -17 | + * | -15 | + * | -13 | + * | -11 | + * +-------+ + */ test("rangeTable lessequal than -10") { val sql = sqlContext import sql.implicits._ @@ -271,24 +273,24 @@ class PartitionFilterSuite extends FunSuite with } /** - *expected result: only showing top 20 rows - *+-------+ - *|intCol0| - *+----+ - *| -31 | - *| -29 | - *| -27 | - *| -25 | - *| -23 | - *| -21 | - *| -19 | - *| -17 | - *| -15 | - *| -13 | - *| -11 | - *| -9 | - *+-------+ - */ + * expected result: only showing top 20 rows + * +-------+ + * |intCol0| + * +----+ + * | -31 | + * | -29 | + * | -27 | + * | -25 | + * | -23 | + * | -21 | + * | -19 | + * | -17 | + * | -15 | + * | -13 | + * | -11 | + * | -9 | + * +-------+ + */ test("rangeTable lessequal than -9") { val sql = sqlContext import sql.implicits._ @@ -303,32 +305,32 @@ class PartitionFilterSuite extends FunSuite with } /** - *expected result: only showing top 20 rows - *+-------+ - *|intCol0| - *+-------+ - *| 0 | - *| 2 | - *| 4 | - *| 6 | - *| 8 | - *| 10 | - *| 12 | - *| 14 | - *| 16 | - *| 18 | - *| 20 | - *| 22 | - *| 24 | - *| 26 | - *| 28 | - *| 30 | - *| -9 | - *| -7 | - *| -5 | - *| -3 | - *+-------+ - */ + * expected result: only showing top 20 rows + * +-------+ + * |intCol0| + * +-------+ + * | 0 | + * | 2 | + * | 4 | + * | 6 | + * | 8 | + * | 10 | + * | 12 | + * | 14 | + * | 16 | + * | 18 | + * | 20 | + * | 22 | + * | 24 | + * | 26 | + * | 28 | + * | 30 | + * | -9 | + * | -7 | + * | -5 | + * | -3 | + * +-------+ + */ test("rangeTable greaterequal than -9") { val sql = sqlContext import sql.implicits._ @@ -343,28 +345,28 @@ class PartitionFilterSuite extends FunSuite with } /** - *expected result: only showing top 20 rows - *+-------+ - *|intCol0| - *+-------+ - *| 0 | - *| 2 | - *| 4 | - *| 6 | - *| 8 | - *| 10 | - *| 12 | - *| 14 | - *| 16 | - *| 18 | - *| 20 | - *| 22 | - *| 24 | - *| 26 | - *| 28 | - *| 30 | - *+-------+ - */ + * expected result: only showing top 20 rows + * +-------+ + * |intCol0| + * +-------+ + * | 0 | + * | 2 | + * | 4 | + * | 6 | + * | 8 | + * | 10 | + * | 12 | + * | 14 | + * | 16 | + * | 18 | + * | 20 | + * | 22 | + * | 24 | + * | 26 | + * | 28 | + * | 30 | + * +-------+ + */ test("rangeTable greaterequal than 0") { val sql = sqlContext import sql.implicits._ @@ -379,22 +381,22 @@ class PartitionFilterSuite extends FunSuite with } /** - *expected result: only showing top 20 rows - *+-------+ - *|intCol0| - *+-------+ - *| 12 | - *| 14 | - *| 16 | - *| 18 | - *| 20 | - *| 22 | - *| 24 | - *| 26 | - *| 28 | - *| 30 | - *+-------+ - */ + * expected result: only showing top 20 rows + * +-------+ + * |intCol0| + * +-------+ + * | 12 | + * | 14 | + * | 16 | + * | 18 | + * | 20 | + * | 22 | + * | 24 | + * | 26 | + * | 28 | + * | 30 | + * +-------+ + */ test("rangeTable greater than 10") { val sql = sqlContext import sql.implicits._ @@ -409,23 +411,23 @@ class PartitionFilterSuite extends FunSuite with } /** - *expected result: only showing top 20 rows - *+-------+ - *|intCol0| - *+-------+ - *| 0 | - *| 2 | - *| 4 | - *| 6 | - *| 8 | - *| 10 | - *| -9 | - *| -7 | - *| -5 | - *| -3 | - *| -1 | - *+-------+ - */ + * expected result: only showing top 20 rows + * +-------+ + * |intCol0| + * +-------+ + * | 0 | + * | 2 | + * | 4 | + * | 6 | + * | 8 | + * | 10 | + * | -9 | + * | -7 | + * | -5 | + * | -3 | + * | -1 | + * +-------+ + */ test("rangeTable and") { val sql = sqlContext import sql.implicits._ @@ -433,39 +435,42 @@ class PartitionFilterSuite extends FunSuite with val s = df.filter($"intCol0" > -10 && $"intCol0" <= 10).select($"intCol0") s.show // filter results without going through dataframe - val expected = rawResult.filter(x => x.intCol0 > -10 && x.intCol0 <= 10 ).map(_.intCol0).toSet + val expected = rawResult + .filter(x => x.intCol0 > -10 && x.intCol0 <= 10) + .map(_.intCol0) + .toSet // filter results going through dataframe val result = collectToSet[Int](s) assert(expected === result) } /** - *expected result: only showing top 20 rows - *+-------+ - *|intCol0| - *+-------+ - *| 12 | - *| 14 | - *| 16 | - *| 18 | - *| 20 | - *| 22 | - *| 24 | - *| 26 | - *| 28 | - *| 30 | - *| -31 | - *| -29 | - *| -27 | - *| -25 | - *| -23 | - *| -21 | - *| -19 | - *| -17 | - *| -15 | - *| -13 | - *+-------+ - */ + * expected result: only showing top 20 rows + * +-------+ + * |intCol0| + * +-------+ + * | 12 | + * | 14 | + * | 16 | + * | 18 | + * | 20 | + * | 22 | + * | 24 | + * | 26 | + * | 28 | + * | 30 | + * | -31 | + * | -29 | + * | -27 | + * | -25 | + * | -23 | + * | -21 | + * | -19 | + * | -17 | + * | -15 | + * | -13 | + * +-------+ + */ test("or") { val sql = sqlContext @@ -474,39 +479,42 @@ class PartitionFilterSuite extends FunSuite with val s = df.filter($"intCol0" <= -10 || $"intCol0" > 10).select($"intCol0") s.show // filter results without going through dataframe - val expected = rawResult.filter(x => x.intCol0 <= -10 || x.intCol0 > 10).map(_.intCol0).toSet + val expected = rawResult + .filter(x => x.intCol0 <= -10 || x.intCol0 > 10) + .map(_.intCol0) + .toSet // filter results going through dataframe val result = collectToSet[Int](s) assert(expected === result) } /** - *expected result: only showing top 20 rows - *+-------+ - *|intCol0| - *+-------+ - *| 0 | - *| 2 | - *| 4 | - *| 6 | - *| 8 | - *| 10 | - *| 12 | - *| 14 | - *| 16 | - *| 18 | - *| 20 | - *| 22 | - *| 24 | - *| 26 | - *| 28 | - *| 30 | - *| -31 | - *| -29 | - *| -27 | - *| -25 | - *+-------+ - */ + * expected result: only showing top 20 rows + * +-------+ + * |intCol0| + * +-------+ + * | 0 | + * | 2 | + * | 4 | + * | 6 | + * | 8 | + * | 10 | + * | 12 | + * | 14 | + * | 16 | + * | 18 | + * | 20 | + * | 22 | + * | 24 | + * | 26 | + * | 28 | + * | 30 | + * | -31 | + * | -29 | + * | -27 | + * | -25 | + * +-------+ + */ test("rangeTable all") { val sql = sqlContext import sql.implicits._ diff --git a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/StartsWithSuite.scala b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/StartsWithSuite.scala index 2dde0aef..9f34b853 100644 --- a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/StartsWithSuite.scala +++ b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/StartsWithSuite.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,16 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark - import org.apache.hadoop.hbase.spark.datasources.Utils import org.apache.hadoop.hbase.util.Bytes import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} -class StartsWithSuite extends FunSuite with -BeforeAndAfterEach with BeforeAndAfterAll with Logging { +class StartsWithSuite extends FunSuite with BeforeAndAfterEach with BeforeAndAfterAll with Logging { test("simple1") { val t = new Array[Byte](2) diff --git a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/TableOutputFormatSuite.scala b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/TableOutputFormatSuite.scala index f64447d8..28d99418 100644 --- a/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/TableOutputFormatSuite.scala +++ b/spark/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/TableOutputFormatSuite.scala @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,31 +15,29 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.spark - import java.text.SimpleDateFormat import java.util.{Date, Locale} - -import org.apache.hadoop.hbase.mapreduce.TableOutputFormat import org.apache.hadoop.hbase.{HBaseTestingUtility, TableName, TableNotFoundException} +import org.apache.hadoop.hbase.mapreduce.TableOutputFormat import org.apache.hadoop.hbase.util.Bytes import org.apache.hadoop.mapreduce.{Job, TaskAttemptID, TaskType} import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl import org.apache.spark.{SparkConf, SparkContext} import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} - import scala.util.{Failure, Success, Try} - // Unit tests for HBASE-20521: change get configuration(TableOutputFormat.conf) object first sequence from jobContext to getConf // this suite contains two tests, one for normal case(getConf return null, use jobContext), create new TableOutputformat object without init TableOutputFormat.conf object, // configuration object inside checkOutputSpecs came from jobContext. // The other one(getConf return conf object) we manually call "setConf" to init TableOutputFormat.conf, for making it more straight forward, we specify a nonexistent table // name in conf object, checkOutputSpecs will then throw TableNotFoundException exception -class TableOutputFormatSuite extends FunSuite with - BeforeAndAfterEach with BeforeAndAfterAll with Logging{ +class TableOutputFormatSuite + extends FunSuite + with BeforeAndAfterEach + with BeforeAndAfterAll + with Logging { @transient var sc: SparkContext = null var TEST_UTIL = new HBaseTestingUtility @@ -52,15 +51,14 @@ class TableOutputFormatSuite extends FunSuite with logInfo(" - minicluster started") try { TEST_UTIL.deleteTable(TableName.valueOf(tableName)) - } - catch { + } catch { case e: Exception => logInfo(" - no table " + tableName + " found") } TEST_UTIL.createTable(TableName.valueOf(tableName), Bytes.toBytes(columnFamily)) logInfo(" - created table") - //set "validateOutputSpecs" true anyway, force to validate output spec + // set "validateOutputSpecs" true anyway, force to validate output spec val sparkConf = new SparkConf() .setMaster("local") .setAppName("test") @@ -92,7 +90,8 @@ class TableOutputFormatSuite extends FunSuite with // Mock up jobContext object and execute actions in "write" function // from "org.apache.spark.internal.io.SparkHadoopMapReduceWriter" // this case should run normally without any exceptions - test("TableOutputFormat.checkOutputSpecs test without setConf called, should return true and without exceptions") { + test( + "TableOutputFormat.checkOutputSpecs test without setConf called, should return true and without exceptions") { val jobContext = getJobContext() val format = jobContext.getOutputFormatClass val jobFormat = format.newInstance @@ -106,7 +105,8 @@ class TableOutputFormatSuite extends FunSuite with // Set configuration externally, checkOutputSpec should use configuration object set by "SetConf" method // rather than jobContext, this case should throw "TableNotFoundException" exception - test("TableOutputFormat.checkOutputSpecs test without setConf called, should throw TableNotFoundException") { + test( + "TableOutputFormat.checkOutputSpecs test without setConf called, should throw TableNotFoundException") { val jobContext = getJobContext() val format = jobContext.getOutputFormatClass val jobFormat = format.newInstance @@ -119,12 +119,12 @@ class TableOutputFormatSuite extends FunSuite with } match { case Success(_) => assert(false) case Failure(e: Exception) => { - if(e.isInstanceOf[TableNotFoundException]) + if (e.isInstanceOf[TableNotFoundException]) assert(true) else assert(false) } - case _ => None + case _ => None } } diff --git a/spark/pom.xml b/spark/pom.xml index 3127a1fb..3f1eb21c 100644 --- a/spark/pom.xml +++ b/spark/pom.xml @@ -1,6 +1,5 @@ - - + + 2.12.15 2.12 + 1.1.2 From e628718429c77358523614a3aa6ca3a55212cd2f Mon Sep 17 00:00:00 2001 From: Attila Zsolt Piros <2017933+attilapiros@users.noreply.github.com> Date: Mon, 16 Oct 2023 00:54:37 -0700 Subject: [PATCH 25/32] HBASE-28152 Replace scala.util.parsing.json with org.json4s.jackson (#126) In https://issues.apache.org/jira/browse/HBASE-28137 to support Spark 3.4 the scala-parser-combinators was added as direct dependency to HBase Spark Connector. This was needed as Spark 3.4 is not using scala-parser-combinators and it is not inherited as transitive dependency. But this solution has a disadvantage. As the HBase Spark Connector assembly jar does not include any 3rd party libraries the scala-parser-combinators must be added to the spark classpath for HBase Spark Connector to work. A much better solution is to replace scala.util.parsing.json with org.json4s.jackson which is used by Spark core, see https://github.com/apache/spark/blob/branch-3.4/core/pom.xml#L279-L280. Signed-off-by: Balazs Meszaros --- spark/hbase-spark/pom.xml | 6 ------ .../spark/datasources/HBaseTableCatalog.scala | 17 ++++++++++------- spark/pom.xml | 1 - 3 files changed, 10 insertions(+), 14 deletions(-) diff --git a/spark/hbase-spark/pom.xml b/spark/hbase-spark/pom.xml index 6e2a6241..10a9fd2b 100644 --- a/spark/hbase-spark/pom.xml +++ b/spark/hbase-spark/pom.xml @@ -55,12 +55,6 @@ ${scala.version} provided - - org.scala-lang.modules - scala-parser-combinators_${scala.binary.version} - ${scala-parser-combinators.version} - compile - org.apache.spark spark-core_${scala.binary.version} diff --git a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableCatalog.scala b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableCatalog.scala index d76a2453..d88d306c 100644 --- a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableCatalog.scala +++ b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableCatalog.scala @@ -22,8 +22,10 @@ import org.apache.hadoop.hbase.spark.{Logging, SchemaConverters} import org.apache.hadoop.hbase.util.Bytes import org.apache.spark.sql.types._ import org.apache.yetus.audience.InterfaceAudience +import org.json4s.DefaultFormats +import org.json4s.Formats +import org.json4s.jackson.JsonMethods import scala.collection.mutable -import scala.util.parsing.json.JSON // The definition of each column cell, which may be composite type // TODO: add avro support @@ -246,11 +248,12 @@ object HBaseTableCatalog { val parameters = convert(params) // println(jString) val jString = parameters(tableCatalog) - val map = JSON.parseFull(jString).get.asInstanceOf[Map[String, _]] - val tableMeta = map.get(table).get.asInstanceOf[Map[String, _]] - val nSpace = tableMeta.get(nameSpace).getOrElse("default").asInstanceOf[String] - val tName = tableMeta.get(tableName).get.asInstanceOf[String] - val cIter = map.get(columns).get.asInstanceOf[Map[String, Map[String, String]]].toIterator + implicit val formats: Formats = DefaultFormats + val map = JsonMethods.parse(jString) + val tableMeta = map \ table + val nSpace = (tableMeta \ nameSpace).extractOrElse("default") + val tName = (tableMeta \ tableName).extract[String] + val cIter = (map \ columns).extract[Map[String, Map[String, String]]] val schemaMap = mutable.HashMap.empty[String, Field] cIter.foreach { case (name, column) => @@ -272,7 +275,7 @@ object HBaseTableCatalog { len) schemaMap.+=((name, f)) } - val rKey = RowKey(map.get(rowKey).get.asInstanceOf[String]) + val rKey = RowKey((map \ rowKey).extract[String]) HBaseTableCatalog(nSpace, tName, rKey, SchemaMap(schemaMap), parameters) } diff --git a/spark/pom.xml b/spark/pom.xml index 27328c2f..3f1eb21c 100644 --- a/spark/pom.xml +++ b/spark/pom.xml @@ -49,7 +49,6 @@ Please take caution when this version is modified --> 2.12.15 2.12 - 1.1.2 From 012ea2a8e26fca92cd8ffb80f0e7ef7fa8d885de Mon Sep 17 00:00:00 2001 From: "Tak Lon (Stephen) Wu" Date: Tue, 17 Oct 2023 21:47:46 -0700 Subject: [PATCH 26/32] Preparing hbase-connectors release 1.0.1RC0; tagging and updates to CHANGESLOG.md and RELEASENOTES.md Signed-off-by: Tak Lon (Stephen) Wu --- CHANGELOG.md | 135 ++++++++++++++++++++++++++++++++++++++++++------ RELEASENOTES.md | 72 ++++++++++++++++++++------ pom.xml | 2 +- 3 files changed, 177 insertions(+), 32 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 2bd209c3..a52940ea 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,22 +1,125 @@ + +# HBASE Changelog + +## Release hbase-connectors-1.0.1 - Unreleased (as of 2023-10-18) + + + +### NEW FEATURES: + +| JIRA | Summary | Priority | Component | +|:---- |:---- | :--- |:---- | +| [HBASE-28152](https://issues.apache.org/jira/browse/HBASE-28152) | Replace scala.util.parsing.json with org.json4s.jackson which used in Spark too | Major | spark | +| [HBASE-28137](https://issues.apache.org/jira/browse/HBASE-28137) | Add scala-parser-combinators dependency to connectors for Spark 3.4 | Major | spark | +| [HBASE-27397](https://issues.apache.org/jira/browse/HBASE-27397) | Spark-hbase support for 'startWith' predicate. | Minor | hbase-connectors | + + +### IMPROVEMENTS: + +| JIRA | Summary | Priority | Component | +|:---- |:---- | :--- |:---- | +| [HBASE-27625](https://issues.apache.org/jira/browse/HBASE-27625) | Bump commons-lang3 to 3.12.0, surefire to 3.0.0 and dependency manage reload4j to 1.2.25 to support jdk11 | Major | hbase-connectors, spark | +| [HBASE-27705](https://issues.apache.org/jira/browse/HBASE-27705) | Respect SparkContext hadoop configuration | Major | spark | +| [HBASE-27639](https://issues.apache.org/jira/browse/HBASE-27639) | Support hbase-connectors compilation with HBase 2.5.3, Hadoop 3.2.4 and Spark 3.2.3 | Major | hbase-connectors, spark | +| [HBASE-26534](https://issues.apache.org/jira/browse/HBASE-26534) | Update dependencies in hbase-connectors: HBase version to 2.4.8, and make Hadoop 3 and Spark 3 defaults | Minor | hadoop3, hbase-connectors, spark | +| [HBASE-25684](https://issues.apache.org/jira/browse/HBASE-25684) | Dependency manage log4j in hbase-connectors | Minor | hbase-connectors | +| [HBASE-25326](https://issues.apache.org/jira/browse/HBASE-25326) | Allow hbase-connector to be used with Apache Spark 3.0 | Minor | . | +| [HBASE-23606](https://issues.apache.org/jira/browse/HBASE-23606) | Remove external deprecations in hbase-connectors | Minor | . | +| [HBASE-24230](https://issues.apache.org/jira/browse/HBASE-24230) | Support user-defined version timestamp when bulk load data | Minor | hbase-connectors | +| [HBASE-23608](https://issues.apache.org/jira/browse/HBASE-23608) | Remove redundant groupId from spark module in hbase-connectors | Trivial | . | +| [HBASE-23592](https://issues.apache.org/jira/browse/HBASE-23592) | Refactor tests in hbase-kafka-proxy in hbase-connectors | Trivial | . | +| [HBASE-24110](https://issues.apache.org/jira/browse/HBASE-24110) | Move to Apache parent POM version 23 for connectors | Minor | hbase-connectors | +| [HBASE-23603](https://issues.apache.org/jira/browse/HBASE-23603) | Update Apache POM to version 21 for hbase-connectors | Trivial | . | +| [HBASE-23607](https://issues.apache.org/jira/browse/HBASE-23607) | Update Maven plugins in hbase-connectors | Minor | . | +| [HBASE-23579](https://issues.apache.org/jira/browse/HBASE-23579) | Fix Checkstyle errors in hbase-connectors | Minor | . | +| [HBASE-23586](https://issues.apache.org/jira/browse/HBASE-23586) | Use StandardCharsets instead of String in TestQualifierMatching in hbase-connectors | Trivial | . | +| [HBASE-23580](https://issues.apache.org/jira/browse/HBASE-23580) | Refactor TestRouteRules in hbase-connectors | Trivial | . | +| [HBASE-23576](https://issues.apache.org/jira/browse/HBASE-23576) | Bump Checkstyle from 8.11 to 8.18 in hbase-connectors | Minor | . | +| [HBASE-22817](https://issues.apache.org/jira/browse/HBASE-22817) | Use hbase-shaded dependencies in hbase-spark | Major | hbase-connectors | +| [HBASE-23075](https://issues.apache.org/jira/browse/HBASE-23075) | Upgrade jackson to version 2.9.10 due to CVE-2019-16335 and CVE-2019-14540 | Major | dependencies, hbase-connectors, REST, security | + + +### BUG FIXES: + +| JIRA | Summary | Priority | Component | +|:---- |:---- | :--- |:---- | +| [HBASE-26863](https://issues.apache.org/jira/browse/HBASE-26863) | Rowkey pushdown does not work with complex conditions | Major | hbase-connectors | +| [HBASE-27488](https://issues.apache.org/jira/browse/HBASE-27488) | [hbase-connectors] Duplicate result when searching HBase by Spark | Major | hbase-connectors | +| [HBASE-27656](https://issues.apache.org/jira/browse/HBASE-27656) | Make sure the close method of the SmartSonnection is called | Major | spark | +| [HBASE-27176](https://issues.apache.org/jira/browse/HBASE-27176) | [hbase-connectors] Fail to build hbase-connectors because of checkstyle error | Major | build, hbase-connectors | +| [HBASE-27801](https://issues.apache.org/jira/browse/HBASE-27801) | Remove redundant avro.version property from Kafka connector | Minor | hbase-connectors, kafka | +| [HBASE-27630](https://issues.apache.org/jira/browse/HBASE-27630) | hbase-spark bulkload stage directory limited to hdfs only | Major | spark | +| [HBASE-27624](https://issues.apache.org/jira/browse/HBASE-27624) | Cannot Specify Namespace via the hbase.table Option in Spark Connector | Major | hbase-connectors, spark | +| [HBASE-22338](https://issues.apache.org/jira/browse/HBASE-22338) | LICENSE file only contains Apache 2.0 | Critical | hbase-connectors | +| [HBASE-26211](https://issues.apache.org/jira/browse/HBASE-26211) | [hbase-connectors] Pushdown filters in Spark do not work correctly with long types | Major | hbase-connectors | +| [HBASE-25236](https://issues.apache.org/jira/browse/HBASE-25236) | [hbase-connectors] Run package phase on spark modules | Major | hbase-connectors | +| [HBASE-24276](https://issues.apache.org/jira/browse/HBASE-24276) | hbase spark connector doesn't support writing to table not in default namespace | Major | hbase-connectors, spark | +| [HBASE-24088](https://issues.apache.org/jira/browse/HBASE-24088) | Solve the ambiguous reference for scala 2.12 | Minor | hbase-connectors | +| [HBASE-23295](https://issues.apache.org/jira/browse/HBASE-23295) | hbase-connectors HBaseContext should use most recent delegation token | Major | hbase-connectors | +| [HBASE-23351](https://issues.apache.org/jira/browse/HBASE-23351) | Updating hbase version to 2.2.2 | Major | hbase-connectors | +| [HBASE-23348](https://issues.apache.org/jira/browse/HBASE-23348) | Spark's createTable method throws an exception while the table is being split | Major | hbase-connectors | +| [HBASE-23346](https://issues.apache.org/jira/browse/HBASE-23346) | Import ReturnCode in SparkSQLPushDownFilter | Major | hbase-connectors | +| [HBASE-23327](https://issues.apache.org/jira/browse/HBASE-23327) | Add missing maven functions for hb\_maven | Critical | hbase-connectors | +| [HBASE-23059](https://issues.apache.org/jira/browse/HBASE-23059) | Run mvn install for root in precommit | Major | hbase-connectors | +| [HBASE-22711](https://issues.apache.org/jira/browse/HBASE-22711) | Spark connector doesn't use the given mapping when inserting data | Major | hbase-connectors | +| [HBASE-22674](https://issues.apache.org/jira/browse/HBASE-22674) | precommit docker image installs JRE over JDK (multiple repos) | Critical | build, hbase-connectors | +| [HBASE-22336](https://issues.apache.org/jira/browse/HBASE-22336) | Add CHANGELOG, README and RELEASENOTES to binary tarball | Critical | hbase-connectors | +| [HBASE-22320](https://issues.apache.org/jira/browse/HBASE-22320) | hbase-connectors personality skips non-scaladoc tests | Critical | . | +| [HBASE-22319](https://issues.apache.org/jira/browse/HBASE-22319) | Fix for warning The assembly descriptor contains a filesystem-root relative reference | Minor | hbase-connectors | + + +### SUB-TASKS: + +| JIRA | Summary | Priority | Component | +|:---- |:---- | :--- |:---- | +| [HBASE-28054](https://issues.apache.org/jira/browse/HBASE-28054) | [hbase-connectors] Add spotless in hbase-connectors pre commit check | Major | build, community, hbase-connectors, jenkins | +| [HBASE-28006](https://issues.apache.org/jira/browse/HBASE-28006) | [hbase-connectors] Run spotless:apply on code base | Major | build, hbase-connectors | +| [HBASE-27178](https://issues.apache.org/jira/browse/HBASE-27178) | [hbase-connectors] Add spotless plugin to format code (including scala code) | Major | build, hbase-connectors | +| [HBASE-25136](https://issues.apache.org/jira/browse/HBASE-25136) | Migrate HBase-Connectors-PreCommit jenkins job from Hadoop to hbase | Major | hbase-connectors, jenkins | + + +### OTHER: + +| JIRA | Summary | Priority | Component | +|:---- |:---- | :--- |:---- | +| [HBASE-27114](https://issues.apache.org/jira/browse/HBASE-27114) | Upgrade scalatest maven plugin for thread-safety | Major | build, spark | +| [HBASE-27883](https://issues.apache.org/jira/browse/HBASE-27883) | [hbase-connectors] Use log4j2 instead of log4j for logging | Blocker | hbase-connectors | +| [HBASE-27679](https://issues.apache.org/jira/browse/HBASE-27679) | Bump junit to 4.13.2 in hbase-connectors | Major | hbase-connectors | +| [HBASE-27680](https://issues.apache.org/jira/browse/HBASE-27680) | Bump hbase, hbase-thirdparty, hadoop and spark for hbase-connectors | Major | hbase-connectors | +| [HBASE-27678](https://issues.apache.org/jira/browse/HBASE-27678) | Update checkstyle in hbase-connectors | Major | hbase-connectors | +| [HBASE-27285](https://issues.apache.org/jira/browse/HBASE-27285) | Fix sonar report paths | Minor | hbase-connectors | +| [HBASE-27272](https://issues.apache.org/jira/browse/HBASE-27272) | Enable code coverage reporting to SonarQube in hbase-connectors | Minor | hbase-connectors | +| [HBASE-26664](https://issues.apache.org/jira/browse/HBASE-26664) | HBASE-26664 hbase-connector upgrades extra-enforcer-rules to 1.5.1 | Major | hbase-connectors | +| [HBASE-26334](https://issues.apache.org/jira/browse/HBASE-26334) | Upgrade commons-io to 2.11.0 in hbase-connectors | Major | hbase-connectors | +| [HBASE-26314](https://issues.apache.org/jira/browse/HBASE-26314) | Upgrade commons-io to 2.8.0 in hbase-connectors | Major | hbase-connectors | +| [HBASE-25579](https://issues.apache.org/jira/browse/HBASE-25579) | HBase Connectors pom should include nexus staging repo management | Major | community, hbase-connectors | +| [HBASE-25479](https://issues.apache.org/jira/browse/HBASE-25479) | [connectors] Purge use of VisibleForTesting | Major | hbase-connectors | +| [HBASE-25388](https://issues.apache.org/jira/browse/HBASE-25388) | Replacing Producer implementation with an extension of MockProducer on testing side in hbase-connectors | Major | hbase-connectors | +| [HBASE-24883](https://issues.apache.org/jira/browse/HBASE-24883) | Migrate hbase-connectors testing to ci-hadoop | Major | build, hbase-connectors | +| [HBASE-23565](https://issues.apache.org/jira/browse/HBASE-23565) | Execute tests in hbase-connectors precommit | Critical | hbase-connectors | +| [HBASE-23032](https://issues.apache.org/jira/browse/HBASE-23032) | Upgrade to Curator 4.2.0 | Major | . | +| [HBASE-22599](https://issues.apache.org/jira/browse/HBASE-22599) | Let hbase-connectors compile against HBase 2.2.0 | Major | hbase-connectors | +| [HBASE-22698](https://issues.apache.org/jira/browse/HBASE-22698) | [hbase-connectors] Add license header to README.md | Major | hbase-connectors | +| [HBASE-22636](https://issues.apache.org/jira/browse/HBASE-22636) | hbase spark module README is in txt format. | Trivial | hbase-connectors | +| [HBASE-22318](https://issues.apache.org/jira/browse/HBASE-22318) | Fix for warning The POM for org.glassfish:javax.el:jar is missing | Minor | hbase-connectors | -# HBase Changelog ## Release connector-1.0.0 - Unreleased (as of 2019-04-26) diff --git a/RELEASENOTES.md b/RELEASENOTES.md index dc8844d3..ebe91f4a 100644 --- a/RELEASENOTES.md +++ b/RELEASENOTES.md @@ -1,20 +1,62 @@ +# RELEASENOTES +# HBASE hbase-connectors-1.0.1 Release Notes + +These release notes cover new developer and user-facing incompatibilities, important issues, features, and major improvements. + + +--- + +* [HBASE-26534](https://issues.apache.org/jira/browse/HBASE-26534) | *Minor* | **Update dependencies in hbase-connectors: HBase version to 2.4.8, and make Hadoop 3 and Spark 3 defaults** + +HBASE-26534 upgrades hbase-thirdparty to 4.0.1, hbase to 2.4.9, spark to 3.1.2, and hadoop to 3.2.0. Also It builds with spark3 with scala-2.12 and hadoop3 profile as default option. + + +--- + +* [HBASE-26334](https://issues.apache.org/jira/browse/HBASE-26334) | *Major* | **Upgrade commons-io to 2.11.0 in hbase-connectors** + +Upgraded commons-io to 2.11.0. + + +--- + +* [HBASE-26314](https://issues.apache.org/jira/browse/HBASE-26314) | *Major* | **Upgrade commons-io to 2.8.0 in hbase-connectors** + +Upgraded commons-io to 2.8.0. + + +--- + +* [HBASE-23576](https://issues.apache.org/jira/browse/HBASE-23576) | *Minor* | **Bump Checkstyle from 8.11 to 8.18 in hbase-connectors** + +Bumped the Checkstyle version from 8.11 to 8.18 + + +--- + +* [HBASE-22817](https://issues.apache.org/jira/browse/HBASE-22817) | *Major* | **Use hbase-shaded dependencies in hbase-spark** + + +The HBase connector for working with Apache Spark now works with the shaded client artifacts provided by the Apache HBase project and avoids adding additional third party dependencies to the classpath. + + # HBase connector-1.0.0 Release Notes diff --git a/pom.xml b/pom.xml index 17f2bec1..e849cd16 100644 --- a/pom.xml +++ b/pom.xml @@ -120,7 +120,7 @@ - 1.0.1-SNAPSHOT + 1.0.1 true yyyy-MM-dd'T'HH:mm ${maven.build.timestamp} From ee95dc66a9cf2254b9ace05c6844e67ae52dbe87 Mon Sep 17 00:00:00 2001 From: "Tak Lon (Stephen) Wu" Date: Tue, 17 Oct 2023 21:47:49 -0700 Subject: [PATCH 27/32] Preparing development version 1.1.0-SNAPSHOT Signed-off-by: Tak Lon (Stephen) Wu --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index e849cd16..bdd41e9f 100644 --- a/pom.xml +++ b/pom.xml @@ -120,7 +120,7 @@ - 1.0.1 + 1.1.0-SNAPSHOT true yyyy-MM-dd'T'HH:mm ${maven.build.timestamp} From 3f15ae13745e065cdcbd4d6b1b44e5f2381b3610 Mon Sep 17 00:00:00 2001 From: "Tak Lon (Stephen) Wu" Date: Tue, 17 Oct 2023 23:07:02 -0700 Subject: [PATCH 28/32] Preparing hbase-connectors release 1.0.1RC1; tagging and updates to CHANGESLOG.md and RELEASENOTES.md Signed-off-by: Tak Lon (Stephen) Wu --- CHANGELOG.md | 2 + RELEASENOTES.md | 134 ------------------------------------------------ pom.xml | 2 +- 3 files changed, 3 insertions(+), 135 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index a52940ea..9ea614d1 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -81,6 +81,7 @@ | [HBASE-22711](https://issues.apache.org/jira/browse/HBASE-22711) | Spark connector doesn't use the given mapping when inserting data | Major | hbase-connectors | | [HBASE-22674](https://issues.apache.org/jira/browse/HBASE-22674) | precommit docker image installs JRE over JDK (multiple repos) | Critical | build, hbase-connectors | | [HBASE-22336](https://issues.apache.org/jira/browse/HBASE-22336) | Add CHANGELOG, README and RELEASENOTES to binary tarball | Critical | hbase-connectors | +| [HBASE-22329](https://issues.apache.org/jira/browse/HBASE-22329) | Fix for warning The parameter forkMode is deprecated since version in hbase-spark-it | Minor | hbase-connectors | | [HBASE-22320](https://issues.apache.org/jira/browse/HBASE-22320) | hbase-connectors personality skips non-scaladoc tests | Critical | . | | [HBASE-22319](https://issues.apache.org/jira/browse/HBASE-22319) | Fix for warning The assembly descriptor contains a filesystem-root relative reference | Minor | hbase-connectors | @@ -113,6 +114,7 @@ | [HBASE-25479](https://issues.apache.org/jira/browse/HBASE-25479) | [connectors] Purge use of VisibleForTesting | Major | hbase-connectors | | [HBASE-25388](https://issues.apache.org/jira/browse/HBASE-25388) | Replacing Producer implementation with an extension of MockProducer on testing side in hbase-connectors | Major | hbase-connectors | | [HBASE-24883](https://issues.apache.org/jira/browse/HBASE-24883) | Migrate hbase-connectors testing to ci-hadoop | Major | build, hbase-connectors | +| [HBASE-24261](https://issues.apache.org/jira/browse/HBASE-24261) | Redo all of our github notification integrations on new ASF infra feature | Major | community, hbase-connectors | | [HBASE-23565](https://issues.apache.org/jira/browse/HBASE-23565) | Execute tests in hbase-connectors precommit | Critical | hbase-connectors | | [HBASE-23032](https://issues.apache.org/jira/browse/HBASE-23032) | Upgrade to Curator 4.2.0 | Major | . | | [HBASE-22599](https://issues.apache.org/jira/browse/HBASE-22599) | Let hbase-connectors compile against HBase 2.2.0 | Major | hbase-connectors | diff --git a/RELEASENOTES.md b/RELEASENOTES.md index ebe91f4a..0530e273 100644 --- a/RELEASENOTES.md +++ b/RELEASENOTES.md @@ -58,137 +58,3 @@ The HBase connector for working with Apache Spark now works with the shaded clie -# HBase connector-1.0.0 Release Notes - -These release notes cover new developer and user-facing incompatibilities, important issues, features, and major improvements. - - ---- - -* [HBASE-13992](https://issues.apache.org/jira/browse/HBASE-13992) | *Major* | **Integrate SparkOnHBase into HBase** - -This release includes initial support for running Spark against HBase with a richer feature set than was previously possible with MapReduce bindings: - -\* Support for Spark and Spark Streaming against Spark 2.1.1 -\* RDD/DStream formation from scan operations -\* convenience methods for interacting with HBase from an HBase backed RDD / DStream instance -\* examples in both the Spark Java API and Spark Scala API -\* support for running against a secure HBase cluster - - ---- - -* [HBASE-14849](https://issues.apache.org/jira/browse/HBASE-14849) | *Major* | **Add option to set block cache to false on SparkSQL executions** - -For user configurable parameters for HBase datasources. Please refer to org.apache.hadoop.hbase.spark.datasources.HBaseSparkConf for details. - -User can either set them in SparkConf, which will take effect globally, or configure it per table, which will overwrite the value set in SparkConf. If not set, the default value will take effect. - -Currently three parameters are supported. -1. spark.hbase.blockcache.enable for blockcache enable/disable. Default is enable, but note that this potentially may slow down the system. -2. spark.hbase.cacheSize for cache size when performing HBase table scan. Default value is 1000 -3. spark.hbase.batchNum for the batch number when performing HBase table scan. Default value is 1000. - - ---- - -* [HBASE-15184](https://issues.apache.org/jira/browse/HBASE-15184) | *Critical* | **SparkSQL Scan operation doesn't work on kerberos cluster** - -Before this patch, users of the spark HBaseContext would fail due to lack of privilege exceptions. - -Note: -\* It is preferred to have spark in spark-on-yarn mode if Kerberos is used. -\* This is orthogonal to issues with a kerberized spark cluster via InputFormats. - - ---- - -* [HBASE-15572](https://issues.apache.org/jira/browse/HBASE-15572) | *Major* | **Adding optional timestamp semantics to HBase-Spark** - -Right now the timestamp is always latest. With this patch, users can select timestamps they want. -In this patch, 4 parameters, "timestamp", "minTimestamp", "maxiTimestamp" and "maxVersions" are added to HBaseSparkConf. Users can select a timestamp, they can also select a time range with minimum timestamp and maximum timestamp. - - ---- - -* [HBASE-17574](https://issues.apache.org/jira/browse/HBASE-17574) | *Major* | **Clean up how to run tests under hbase-spark module** - -Run test under root dir or hbase-spark dir -1. mvn test //run all small and medium java tests, and all scala tests -2. mvn test -P skipSparkTests //skip all scala and java tests in hbase-spark -3. mvn test -P runAllTests //run all tests, including scala and all java test even the large test - -Run specified test case, since we have two plugins, we need specify both java and scala. -When only test scala or jave test case, disable the other one use -Dxx=None as follow: -1. mvn test -Dtest=TestJavaHBaseContext -DwildcardSuites=None // java unit test -2. mvn test -Dtest=None -DwildcardSuites=org.apache.hadoop.hbase.spark.BulkLoadSuite //scala unit test, only support full name in scalatest plugin - - ---- - -* [HBASE-17933](https://issues.apache.org/jira/browse/HBASE-17933) | *Major* | **[hbase-spark] Support Java api for bulkload** - - -The integration module for Apache Spark now includes Java-friendly equivalents for the `bulkLoad` and `bulkLoadThinRows` methods in `JavaHBaseContext`. - - ---- - -* [HBASE-18175](https://issues.apache.org/jira/browse/HBASE-18175) | *Critical* | **Add hbase-spark integration test into hbase-spark-it** - - -HBase now ships with an integration test for our integration with Apache Spark. - -You can run this test on a cluster by using an equivalent to the below, e.g. if the version of HBase is 2.0.0-alpha-2 - -``` -spark-submit --class org.apache.hadoop.hbase.spark.IntegrationTestSparkBulkLoad HBASE_HOME/lib/hbase-spark-it-2.0.0-alpha-2-tests.jar -Dhbase.spark.bulkload.chainlength=500000 -m slowDeterministic -``` - - ---- - -* [HBASE-16179](https://issues.apache.org/jira/browse/HBASE-16179) | *Critical* | **Fix compilation errors when building hbase-spark against Spark 2.0** - -As of this JIRA, Spark version is upgraded from 1.6 to 2.1.1 - - ---- - -* [HBASE-21002](https://issues.apache.org/jira/browse/HBASE-21002) | *Minor* | **Create assembly and scripts to start Kafka Proxy** - -Adds a kafka proxy that appears to hbase as a replication peer. Use to tee table edits to kafka. Has mechanism for dropping/routing updates. See https://github.com/apache/hbase-connectors/tree/master/kafka for documentation. - - ---- - -* [HBASE-21434](https://issues.apache.org/jira/browse/HBASE-21434) | *Major* | **[hbase-connectors] Cleanup of kafka dependencies; clarify hadoop version** - -Cleaned up kafka submodule dependencies. Added used dependencies to pom and removed the unused. Depends explicitly on hadoop2. No messing w/ hadoop3 versions. - - ---- - -* [HBASE-21446](https://issues.apache.org/jira/browse/HBASE-21446) | *Major* | **[hbase-connectors] Update spark and scala versions; add some doc on how to generate artifacts with different versions** - -Updates our hbase-spark integration so defaults spark 2.4.0 (October 2018) from 2.1.1 and Scala 2.11.12 (from 2.11.8). - - ---- - -* [HBASE-15320](https://issues.apache.org/jira/browse/HBASE-15320) | *Major* | **HBase connector for Kafka Connect** - -This commit adds a kafka connector. The connectors acts as a replication peer and sends modifications in HBase to kafka. - -For further information, please refer to kafka/README.md. - - ---- - -* [HBASE-14789](https://issues.apache.org/jira/browse/HBASE-14789) | *Major* | **Enhance the current spark-hbase connector** - -New features in hbase-spark: -\* native type support (short, int, long, float, double), -\* support for Dataframe writes, -\* avro support, -\* catalog can be defined in json. diff --git a/pom.xml b/pom.xml index bdd41e9f..e849cd16 100644 --- a/pom.xml +++ b/pom.xml @@ -120,7 +120,7 @@ - 1.1.0-SNAPSHOT + 1.0.1 true yyyy-MM-dd'T'HH:mm ${maven.build.timestamp} From 8d8b4353a57f7590ebca880ee01ad50448c74844 Mon Sep 17 00:00:00 2001 From: "Tak Lon (Stephen) Wu" Date: Tue, 17 Oct 2023 23:07:05 -0700 Subject: [PATCH 29/32] Preparing development version 1.1.0-SNAPSHOT Signed-off-by: Tak Lon (Stephen) Wu --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index e849cd16..bdd41e9f 100644 --- a/pom.xml +++ b/pom.xml @@ -120,7 +120,7 @@ - 1.0.1 + 1.1.0-SNAPSHOT true yyyy-MM-dd'T'HH:mm ${maven.build.timestamp} From 3879144ad2e66cba043e4e14f2f0af6165052b20 Mon Sep 17 00:00:00 2001 From: "Tak Lon (Stephen) Wu" Date: Tue, 17 Oct 2023 23:16:54 -0700 Subject: [PATCH 30/32] Revert "Preparing hbase-connectors release 1.0.1RC1; tagging and updates to CHANGESLOG.md and RELEASENOTES.md" This reverts commit 3f15ae13745e065cdcbd4d6b1b44e5f2381b3610. --- CHANGELOG.md | 2 - RELEASENOTES.md | 134 ++++++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 134 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 9ea614d1..a52940ea 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -81,7 +81,6 @@ | [HBASE-22711](https://issues.apache.org/jira/browse/HBASE-22711) | Spark connector doesn't use the given mapping when inserting data | Major | hbase-connectors | | [HBASE-22674](https://issues.apache.org/jira/browse/HBASE-22674) | precommit docker image installs JRE over JDK (multiple repos) | Critical | build, hbase-connectors | | [HBASE-22336](https://issues.apache.org/jira/browse/HBASE-22336) | Add CHANGELOG, README and RELEASENOTES to binary tarball | Critical | hbase-connectors | -| [HBASE-22329](https://issues.apache.org/jira/browse/HBASE-22329) | Fix for warning The parameter forkMode is deprecated since version in hbase-spark-it | Minor | hbase-connectors | | [HBASE-22320](https://issues.apache.org/jira/browse/HBASE-22320) | hbase-connectors personality skips non-scaladoc tests | Critical | . | | [HBASE-22319](https://issues.apache.org/jira/browse/HBASE-22319) | Fix for warning The assembly descriptor contains a filesystem-root relative reference | Minor | hbase-connectors | @@ -114,7 +113,6 @@ | [HBASE-25479](https://issues.apache.org/jira/browse/HBASE-25479) | [connectors] Purge use of VisibleForTesting | Major | hbase-connectors | | [HBASE-25388](https://issues.apache.org/jira/browse/HBASE-25388) | Replacing Producer implementation with an extension of MockProducer on testing side in hbase-connectors | Major | hbase-connectors | | [HBASE-24883](https://issues.apache.org/jira/browse/HBASE-24883) | Migrate hbase-connectors testing to ci-hadoop | Major | build, hbase-connectors | -| [HBASE-24261](https://issues.apache.org/jira/browse/HBASE-24261) | Redo all of our github notification integrations on new ASF infra feature | Major | community, hbase-connectors | | [HBASE-23565](https://issues.apache.org/jira/browse/HBASE-23565) | Execute tests in hbase-connectors precommit | Critical | hbase-connectors | | [HBASE-23032](https://issues.apache.org/jira/browse/HBASE-23032) | Upgrade to Curator 4.2.0 | Major | . | | [HBASE-22599](https://issues.apache.org/jira/browse/HBASE-22599) | Let hbase-connectors compile against HBase 2.2.0 | Major | hbase-connectors | diff --git a/RELEASENOTES.md b/RELEASENOTES.md index 0530e273..ebe91f4a 100644 --- a/RELEASENOTES.md +++ b/RELEASENOTES.md @@ -58,3 +58,137 @@ The HBase connector for working with Apache Spark now works with the shaded clie +# HBase connector-1.0.0 Release Notes + +These release notes cover new developer and user-facing incompatibilities, important issues, features, and major improvements. + + +--- + +* [HBASE-13992](https://issues.apache.org/jira/browse/HBASE-13992) | *Major* | **Integrate SparkOnHBase into HBase** + +This release includes initial support for running Spark against HBase with a richer feature set than was previously possible with MapReduce bindings: + +\* Support for Spark and Spark Streaming against Spark 2.1.1 +\* RDD/DStream formation from scan operations +\* convenience methods for interacting with HBase from an HBase backed RDD / DStream instance +\* examples in both the Spark Java API and Spark Scala API +\* support for running against a secure HBase cluster + + +--- + +* [HBASE-14849](https://issues.apache.org/jira/browse/HBASE-14849) | *Major* | **Add option to set block cache to false on SparkSQL executions** + +For user configurable parameters for HBase datasources. Please refer to org.apache.hadoop.hbase.spark.datasources.HBaseSparkConf for details. + +User can either set them in SparkConf, which will take effect globally, or configure it per table, which will overwrite the value set in SparkConf. If not set, the default value will take effect. + +Currently three parameters are supported. +1. spark.hbase.blockcache.enable for blockcache enable/disable. Default is enable, but note that this potentially may slow down the system. +2. spark.hbase.cacheSize for cache size when performing HBase table scan. Default value is 1000 +3. spark.hbase.batchNum for the batch number when performing HBase table scan. Default value is 1000. + + +--- + +* [HBASE-15184](https://issues.apache.org/jira/browse/HBASE-15184) | *Critical* | **SparkSQL Scan operation doesn't work on kerberos cluster** + +Before this patch, users of the spark HBaseContext would fail due to lack of privilege exceptions. + +Note: +\* It is preferred to have spark in spark-on-yarn mode if Kerberos is used. +\* This is orthogonal to issues with a kerberized spark cluster via InputFormats. + + +--- + +* [HBASE-15572](https://issues.apache.org/jira/browse/HBASE-15572) | *Major* | **Adding optional timestamp semantics to HBase-Spark** + +Right now the timestamp is always latest. With this patch, users can select timestamps they want. +In this patch, 4 parameters, "timestamp", "minTimestamp", "maxiTimestamp" and "maxVersions" are added to HBaseSparkConf. Users can select a timestamp, they can also select a time range with minimum timestamp and maximum timestamp. + + +--- + +* [HBASE-17574](https://issues.apache.org/jira/browse/HBASE-17574) | *Major* | **Clean up how to run tests under hbase-spark module** + +Run test under root dir or hbase-spark dir +1. mvn test //run all small and medium java tests, and all scala tests +2. mvn test -P skipSparkTests //skip all scala and java tests in hbase-spark +3. mvn test -P runAllTests //run all tests, including scala and all java test even the large test + +Run specified test case, since we have two plugins, we need specify both java and scala. +When only test scala or jave test case, disable the other one use -Dxx=None as follow: +1. mvn test -Dtest=TestJavaHBaseContext -DwildcardSuites=None // java unit test +2. mvn test -Dtest=None -DwildcardSuites=org.apache.hadoop.hbase.spark.BulkLoadSuite //scala unit test, only support full name in scalatest plugin + + +--- + +* [HBASE-17933](https://issues.apache.org/jira/browse/HBASE-17933) | *Major* | **[hbase-spark] Support Java api for bulkload** + + +The integration module for Apache Spark now includes Java-friendly equivalents for the `bulkLoad` and `bulkLoadThinRows` methods in `JavaHBaseContext`. + + +--- + +* [HBASE-18175](https://issues.apache.org/jira/browse/HBASE-18175) | *Critical* | **Add hbase-spark integration test into hbase-spark-it** + + +HBase now ships with an integration test for our integration with Apache Spark. + +You can run this test on a cluster by using an equivalent to the below, e.g. if the version of HBase is 2.0.0-alpha-2 + +``` +spark-submit --class org.apache.hadoop.hbase.spark.IntegrationTestSparkBulkLoad HBASE_HOME/lib/hbase-spark-it-2.0.0-alpha-2-tests.jar -Dhbase.spark.bulkload.chainlength=500000 -m slowDeterministic +``` + + +--- + +* [HBASE-16179](https://issues.apache.org/jira/browse/HBASE-16179) | *Critical* | **Fix compilation errors when building hbase-spark against Spark 2.0** + +As of this JIRA, Spark version is upgraded from 1.6 to 2.1.1 + + +--- + +* [HBASE-21002](https://issues.apache.org/jira/browse/HBASE-21002) | *Minor* | **Create assembly and scripts to start Kafka Proxy** + +Adds a kafka proxy that appears to hbase as a replication peer. Use to tee table edits to kafka. Has mechanism for dropping/routing updates. See https://github.com/apache/hbase-connectors/tree/master/kafka for documentation. + + +--- + +* [HBASE-21434](https://issues.apache.org/jira/browse/HBASE-21434) | *Major* | **[hbase-connectors] Cleanup of kafka dependencies; clarify hadoop version** + +Cleaned up kafka submodule dependencies. Added used dependencies to pom and removed the unused. Depends explicitly on hadoop2. No messing w/ hadoop3 versions. + + +--- + +* [HBASE-21446](https://issues.apache.org/jira/browse/HBASE-21446) | *Major* | **[hbase-connectors] Update spark and scala versions; add some doc on how to generate artifacts with different versions** + +Updates our hbase-spark integration so defaults spark 2.4.0 (October 2018) from 2.1.1 and Scala 2.11.12 (from 2.11.8). + + +--- + +* [HBASE-15320](https://issues.apache.org/jira/browse/HBASE-15320) | *Major* | **HBase connector for Kafka Connect** + +This commit adds a kafka connector. The connectors acts as a replication peer and sends modifications in HBase to kafka. + +For further information, please refer to kafka/README.md. + + +--- + +* [HBASE-14789](https://issues.apache.org/jira/browse/HBASE-14789) | *Major* | **Enhance the current spark-hbase connector** + +New features in hbase-spark: +\* native type support (short, int, long, float, double), +\* support for Dataframe writes, +\* avro support, +\* catalog can be defined in json. From e5217d13ed729703580ff2d1b02378ada2d94f4d Mon Sep 17 00:00:00 2001 From: "Tak Lon (Stephen) Wu" Date: Wed, 18 Oct 2023 00:41:15 -0700 Subject: [PATCH 31/32] Preparing hbase-connectors release 1.0.1RC1; tagging and updates to CHANGESLOG.md and RELEASENOTES.md Signed-off-by: Tak Lon (Stephen) Wu --- CHANGELOG.md | 2 ++ pom.xml | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index a52940ea..9ea614d1 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -81,6 +81,7 @@ | [HBASE-22711](https://issues.apache.org/jira/browse/HBASE-22711) | Spark connector doesn't use the given mapping when inserting data | Major | hbase-connectors | | [HBASE-22674](https://issues.apache.org/jira/browse/HBASE-22674) | precommit docker image installs JRE over JDK (multiple repos) | Critical | build, hbase-connectors | | [HBASE-22336](https://issues.apache.org/jira/browse/HBASE-22336) | Add CHANGELOG, README and RELEASENOTES to binary tarball | Critical | hbase-connectors | +| [HBASE-22329](https://issues.apache.org/jira/browse/HBASE-22329) | Fix for warning The parameter forkMode is deprecated since version in hbase-spark-it | Minor | hbase-connectors | | [HBASE-22320](https://issues.apache.org/jira/browse/HBASE-22320) | hbase-connectors personality skips non-scaladoc tests | Critical | . | | [HBASE-22319](https://issues.apache.org/jira/browse/HBASE-22319) | Fix for warning The assembly descriptor contains a filesystem-root relative reference | Minor | hbase-connectors | @@ -113,6 +114,7 @@ | [HBASE-25479](https://issues.apache.org/jira/browse/HBASE-25479) | [connectors] Purge use of VisibleForTesting | Major | hbase-connectors | | [HBASE-25388](https://issues.apache.org/jira/browse/HBASE-25388) | Replacing Producer implementation with an extension of MockProducer on testing side in hbase-connectors | Major | hbase-connectors | | [HBASE-24883](https://issues.apache.org/jira/browse/HBASE-24883) | Migrate hbase-connectors testing to ci-hadoop | Major | build, hbase-connectors | +| [HBASE-24261](https://issues.apache.org/jira/browse/HBASE-24261) | Redo all of our github notification integrations on new ASF infra feature | Major | community, hbase-connectors | | [HBASE-23565](https://issues.apache.org/jira/browse/HBASE-23565) | Execute tests in hbase-connectors precommit | Critical | hbase-connectors | | [HBASE-23032](https://issues.apache.org/jira/browse/HBASE-23032) | Upgrade to Curator 4.2.0 | Major | . | | [HBASE-22599](https://issues.apache.org/jira/browse/HBASE-22599) | Let hbase-connectors compile against HBase 2.2.0 | Major | hbase-connectors | diff --git a/pom.xml b/pom.xml index bdd41e9f..e849cd16 100644 --- a/pom.xml +++ b/pom.xml @@ -120,7 +120,7 @@ - 1.1.0-SNAPSHOT + 1.0.1 true yyyy-MM-dd'T'HH:mm ${maven.build.timestamp} From 307607cf7287084b3ce49cdd96d094e2ede9363a Mon Sep 17 00:00:00 2001 From: "Tak Lon (Stephen) Wu" Date: Wed, 18 Oct 2023 00:41:18 -0700 Subject: [PATCH 32/32] Preparing development version 1.1.0-SNAPSHOT Signed-off-by: Tak Lon (Stephen) Wu --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index e849cd16..bdd41e9f 100644 --- a/pom.xml +++ b/pom.xml @@ -120,7 +120,7 @@ - 1.0.1 + 1.1.0-SNAPSHOT true yyyy-MM-dd'T'HH:mm ${maven.build.timestamp}