HBASE-16179 Fix compilation errors when building hbase-spark against Spark 2.0

Signed-off-by: Mike Drob <mdrob@apache.org>
This commit is contained in:
tedyu 2018-03-09 07:58:53 -08:00
parent 06550bc93b
commit ee3a469641
25 changed files with 189 additions and 60 deletions

View File

@ -33,9 +33,11 @@
<properties> <properties>
<spark.version>1.6.0</spark.version> <spark.version>2.1.1</spark.version>
<scala.version>2.10.4</scala.version> <!-- The following version is in sync with Spark's choice
<scala.binary.version>2.10</scala.binary.version> Please take caution when this version is modified -->
<scala.version>2.11.8</scala.version>
<scala.binary.version>2.11</scala.binary.version>
<!-- Test inclusion patterns used by failsafe configuration --> <!-- Test inclusion patterns used by failsafe configuration -->
<unittest.include>**/Test*.java</unittest.include> <unittest.include>**/Test*.java</unittest.include>
<integrationtest.include>**/IntegrationTest*.java</integrationtest.include> <integrationtest.include>**/IntegrationTest*.java</integrationtest.include>
@ -263,6 +265,12 @@
</exclusion> </exclusion>
</exclusions> </exclusions>
</dependency> </dependency>
<dependency>
<groupId>org.scala-lang.modules</groupId>
<artifactId>scala-xml_2.11</artifactId>
<version>1.0.4</version>
<scope>provided</scope>
</dependency>
<dependency> <dependency>
<groupId>org.apache.spark</groupId> <groupId>org.apache.spark</groupId>
<artifactId>spark-sql_${scala.binary.version}</artifactId> <artifactId>spark-sql_${scala.binary.version}</artifactId>

View File

@ -319,7 +319,7 @@ public class IntegrationTestSparkBulkLoad extends IntegrationTestBase {
PairFlatMapFunction<Tuple2<ImmutableBytesWritable, Result>, SparkLinkKey, SparkLinkChain> { PairFlatMapFunction<Tuple2<ImmutableBytesWritable, Result>, SparkLinkKey, SparkLinkChain> {
@Override @Override
public Iterable<Tuple2<SparkLinkKey, SparkLinkChain>> call(Tuple2<ImmutableBytesWritable, public Iterator<Tuple2<SparkLinkKey, SparkLinkChain>> call(Tuple2<ImmutableBytesWritable,
Result> v) throws Exception { Result> v) throws Exception {
Result value = v._2(); Result value = v._2();
long longRk = Bytes.toLong(value.getRow()); long longRk = Bytes.toLong(value.getRow());
@ -334,7 +334,7 @@ public class IntegrationTestSparkBulkLoad extends IntegrationTestBase {
new Tuple2<>(new SparkLinkKey(chainId, order), new SparkLinkChain(longRk, next)); new Tuple2<>(new SparkLinkKey(chainId, order), new SparkLinkChain(longRk, next));
list.add(tuple2); list.add(tuple2);
} }
return list; return list.iterator();
} }
} }

View File

@ -30,9 +30,11 @@
<artifactId>hbase-spark</artifactId> <artifactId>hbase-spark</artifactId>
<name>Apache HBase - Spark</name> <name>Apache HBase - Spark</name>
<properties> <properties>
<spark.version>1.6.0</spark.version> <spark.version>2.1.1</spark.version>
<scala.version>2.10.4</scala.version> <!-- The following version is in sync with Spark's choice
<scala.binary.version>2.10</scala.binary.version> Please take caution when this version is modified -->
<scala.version>2.11.8</scala.version>
<scala.binary.version>2.11</scala.binary.version>
<top.dir>${project.basedir}/..</top.dir> <top.dir>${project.basedir}/..</top.dir>
</properties> </properties>
<dependencies> <dependencies>

View File

@ -27,6 +27,7 @@ import java.util.Map;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
import org.apache.hadoop.hbase.filter.FilterBase; import org.apache.hadoop.hbase.filter.FilterBase;
import org.apache.hadoop.hbase.spark.datasources.BytesEncoder; import org.apache.hadoop.hbase.spark.datasources.BytesEncoder;
import org.apache.hadoop.hbase.spark.datasources.JavaBytesEncoder; import org.apache.hadoop.hbase.spark.datasources.JavaBytesEncoder;

View File

@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.HColumnDescriptor
import org.apache.hadoop.hbase.TableName import org.apache.hadoop.hbase.TableName
import org.apache.hadoop.hbase.CellUtil import org.apache.hadoop.hbase.CellUtil
import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapred.JobConf
import org.apache.spark.Logging
import org.apache.spark.rdd.RDD import org.apache.spark.rdd.RDD
import org.apache.spark.sql.datasources.hbase.{Utils, Field, HBaseTableCatalog} import org.apache.spark.sql.datasources.hbase.{Utils, Field, HBaseTableCatalog}
import org.apache.spark.sql.{DataFrame, SaveMode, Row, SQLContext} import org.apache.spark.sql.{DataFrame, SaveMode, Row, SQLContext}

View File

@ -25,7 +25,6 @@ import org.apache.hadoop.hbase.ipc.RpcControllerFactory
import org.apache.hadoop.hbase.security.{User, UserProvider} import org.apache.hadoop.hbase.security.{User, UserProvider}
import org.apache.hadoop.hbase.spark.datasources.HBaseSparkConf import org.apache.hadoop.hbase.spark.datasources.HBaseSparkConf
import org.apache.hadoop.hbase.{HConstants, TableName} import org.apache.hadoop.hbase.{HConstants, TableName}
import org.apache.spark.Logging
import scala.collection.mutable import scala.collection.mutable

View File

@ -39,7 +39,7 @@ import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.hbase.spark.HBaseRDDFunctions._ import org.apache.hadoop.hbase.spark.HBaseRDDFunctions._
import org.apache.hadoop.hbase.client._ import org.apache.hadoop.hbase.client._
import scala.reflect.ClassTag import scala.reflect.ClassTag
import org.apache.spark.{Logging, SerializableWritable, SparkContext} import org.apache.spark.{SerializableWritable, SparkContext}
import org.apache.hadoop.hbase.mapreduce.{TableMapReduceUtil, import org.apache.hadoop.hbase.mapreduce.{TableMapReduceUtil,
TableInputFormat, IdentityTableMapper} TableInputFormat, IdentityTableMapper}
import org.apache.hadoop.hbase.io.ImmutableBytesWritable import org.apache.hadoop.hbase.io.ImmutableBytesWritable
@ -60,7 +60,7 @@ import scala.collection.mutable
* to the working and managing the life cycle of Connections. * to the working and managing the life cycle of Connections.
*/ */
@InterfaceAudience.Public @InterfaceAudience.Public
class HBaseContext(@transient sc: SparkContext, class HBaseContext(@transient val sc: SparkContext,
@transient val config: Configuration, @transient val config: Configuration,
val tmpHdfsConfgFile: String = null) val tmpHdfsConfgFile: String = null)
extends Serializable with Logging { extends Serializable with Logging {

View File

@ -29,6 +29,8 @@ import org.apache.spark.api.java.{JavaRDD, JavaSparkContext}
import org.apache.spark.api.java.function.{FlatMapFunction, Function, VoidFunction} import org.apache.spark.api.java.function.{FlatMapFunction, Function, VoidFunction}
import org.apache.spark.streaming.api.java.JavaDStream import org.apache.spark.streaming.api.java.JavaDStream
import java.lang.Iterable
import scala.collection.JavaConversions._ import scala.collection.JavaConversions._
import scala.reflect.ClassTag import scala.reflect.ClassTag
@ -41,8 +43,8 @@ import scala.reflect.ClassTag
* @param config This is the config information to out HBase cluster * @param config This is the config information to out HBase cluster
*/ */
@InterfaceAudience.Public @InterfaceAudience.Public
class JavaHBaseContext(@transient jsc: JavaSparkContext, class JavaHBaseContext(@transient val jsc: JavaSparkContext,
@transient config: Configuration) extends Serializable { @transient val config: Configuration) extends Serializable {
val hbaseContext = new HBaseContext(jsc.sc, config) val hbaseContext = new HBaseContext(jsc.sc, config)
/** /**
@ -107,15 +109,9 @@ class JavaHBaseContext(@transient jsc: JavaSparkContext,
def mapPartitions[T, R](javaRdd: JavaRDD[T], def mapPartitions[T, R](javaRdd: JavaRDD[T],
f: FlatMapFunction[(java.util.Iterator[T], f: FlatMapFunction[(java.util.Iterator[T],
Connection), R]): JavaRDD[R] = { Connection), R]): JavaRDD[R] = {
def fn = (it: Iterator[T], conn: Connection) =>
asScalaIterator(
f.call((asJavaIterator(it), conn)).iterator()
)
JavaRDD.fromRDD(hbaseContext.mapPartitions(javaRdd.rdd, JavaRDD.fromRDD(hbaseContext.mapPartitions(javaRdd.rdd,
(iterator: Iterator[T], connection: Connection) => (it: Iterator[T], conn: Connection) =>
fn(iterator, connection))(fakeClassTag[R]))(fakeClassTag[R]) f.call(it, conn))(fakeClassTag[R]))(fakeClassTag[R])
} }
/** /**

View File

@ -0,0 +1,119 @@
/*
* 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.log4j.LogManager
import org.slf4j.{Logger, LoggerFactory}
import org.slf4j.impl.StaticLoggerBinder
/**
* Utility trait for classes that want to log data. Creates a SLF4J logger for the class and allows
* logging messages at different levels using methods that only evaluate parameters lazily if the
* log level is enabled.
* Logging is private in Spark 2.0
* This is to isolate incompatibilties across Spark releases.
*/
trait Logging {
// Make the log field transient so that objects with Logging can
// be serialized and used on another machine
@transient private var log_ : Logger = null
// Method to get the logger name for this object
protected def logName = {
// Ignore trailing $'s in the class names for Scala objects
this.getClass.getName.stripSuffix("$")
}
// Method to get or create the logger for this object
protected def log: Logger = {
if (log_ == null) {
initializeLogIfNecessary(false)
log_ = LoggerFactory.getLogger(logName)
}
log_
}
// Log methods that take only a String
protected def logInfo(msg: => String) {
if (log.isInfoEnabled) log.info(msg)
}
protected def logDebug(msg: => String) {
if (log.isDebugEnabled) log.debug(msg)
}
protected def logTrace(msg: => String) {
if (log.isTraceEnabled) log.trace(msg)
}
protected def logWarning(msg: => String) {
if (log.isWarnEnabled) log.warn(msg)
}
protected def logError(msg: => String) {
if (log.isErrorEnabled) log.error(msg)
}
// Log methods that take Throwables (Exceptions/Errors) too
protected def logInfo(msg: => String, throwable: Throwable) {
if (log.isInfoEnabled) log.info(msg, throwable)
}
protected def logDebug(msg: => String, throwable: Throwable) {
if (log.isDebugEnabled) log.debug(msg, throwable)
}
protected def logTrace(msg: => String, throwable: Throwable) {
if (log.isTraceEnabled) log.trace(msg, throwable)
}
protected def logWarning(msg: => String, throwable: Throwable) {
if (log.isWarnEnabled) log.warn(msg, throwable)
}
protected def logError(msg: => String, throwable: Throwable) {
if (log.isErrorEnabled) log.error(msg, throwable)
}
protected def initializeLogIfNecessary(isInterpreter: Boolean): Unit = {
if (!Logging.initialized) {
Logging.initLock.synchronized {
if (!Logging.initialized) {
initializeLogging(isInterpreter)
}
}
}
}
private def initializeLogging(isInterpreter: Boolean): Unit = {
// Don't use a logger in here, as this is itself occurring during initialization of a logger
// If Log4j 1.2 is being used, but is not initialized, load a default properties file
val binderClass = StaticLoggerBinder.getSingleton.getLoggerFactoryClassStr
Logging.initialized = true
// Force a call into slf4j to initialize it. Avoids this happening from multiple threads
// and triggering this: http://mailman.qos.ch/pipermail/slf4j-dev/2010-April/002956.html
log
}
}
private object Logging {
@volatile private var initialized = false
val initLock = new Object()
}

View File

@ -24,12 +24,12 @@ import org.apache.spark.rdd.NewHadoopRDD
import org.apache.spark.{InterruptibleIterator, Partition, SparkContext, TaskContext} import org.apache.spark.{InterruptibleIterator, Partition, SparkContext, TaskContext}
@InterfaceAudience.Public @InterfaceAudience.Public
class NewHBaseRDD[K,V](@transient sc : SparkContext, class NewHBaseRDD[K,V](@transient val sc : SparkContext,
@transient inputFormatClass: Class[_ <: InputFormat[K, V]], @transient val inputFormatClass: Class[_ <: InputFormat[K, V]],
@transient keyClass: Class[K], @transient val keyClass: Class[K],
@transient valueClass: Class[V], @transient val valueClass: Class[V],
@transient conf: Configuration, @transient private val __conf: Configuration,
val hBaseContext: HBaseContext) extends NewHadoopRDD(sc,inputFormatClass, keyClass, valueClass, conf) { val hBaseContext: HBaseContext) extends NewHadoopRDD(sc, inputFormatClass, keyClass, valueClass, __conf) {
override def compute(theSplit: Partition, context: TaskContext): InterruptibleIterator[(K, V)] = { override def compute(theSplit: Partition, context: TaskContext): InterruptibleIterator[(K, V)] = {
hBaseContext.applyCreds() hBaseContext.applyCreds()

View File

@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.spark.hbase._
import org.apache.hadoop.hbase.spark.datasources.HBaseResources._ import org.apache.hadoop.hbase.spark.datasources.HBaseResources._
import org.apache.hadoop.hbase.util.ShutdownHookManager import org.apache.hadoop.hbase.util.ShutdownHookManager
import org.apache.spark.sql.datasources.hbase.Field import org.apache.spark.sql.datasources.hbase.Field
import org.apache.spark.{SparkEnv, TaskContext, Logging, Partition} import org.apache.spark.{SparkEnv, TaskContext, Partition}
import org.apache.spark.rdd.RDD import org.apache.spark.rdd.RDD
import scala.collection.mutable import scala.collection.mutable
@ -36,7 +36,8 @@ class HBaseTableScanRDD(relation: HBaseRelation,
val hbaseContext: HBaseContext, val hbaseContext: HBaseContext,
@transient val filter: Option[SparkSQLPushDownFilter] = None, @transient val filter: Option[SparkSQLPushDownFilter] = None,
val columns: Seq[Field] = Seq.empty val columns: Seq[Field] = Seq.empty
)extends RDD[Result](relation.sqlContext.sparkContext, Nil) with Logging { ) extends RDD[Result](relation.sqlContext.sparkContext, Nil)
{
private def sparkConf = SparkEnv.get.conf private def sparkConf = SparkEnv.get.conf
@transient var ranges = Seq.empty[Range] @transient var ranges = Seq.empty[Range]
@transient var points = Seq.empty[Array[Byte]] @transient var points = Seq.empty[Array[Byte]]

View File

@ -18,11 +18,11 @@
package org.apache.hadoop.hbase.spark.datasources package org.apache.hadoop.hbase.spark.datasources
import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.spark.Logging
import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability; import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.spark.datasources.JavaBytesEncoder.JavaBytesEncoder import org.apache.hadoop.hbase.spark.datasources.JavaBytesEncoder.JavaBytesEncoder
import org.apache.hadoop.hbase.util.Bytes import org.apache.hadoop.hbase.util.Bytes
import org.apache.spark.Logging
import org.apache.spark.sql.types._ import org.apache.spark.sql.types._
/** /**

View File

@ -18,9 +18,9 @@ package org.apache.hadoop.hbase.spark.datasources
import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.spark.datasources.JavaBytesEncoder.JavaBytesEncoder import org.apache.hadoop.hbase.spark.datasources.JavaBytesEncoder.JavaBytesEncoder
import org.apache.hadoop.hbase.spark.Logging
import org.apache.hadoop.hbase.spark.hbase._ import org.apache.hadoop.hbase.spark.hbase._
import org.apache.hadoop.hbase.util.Bytes import org.apache.hadoop.hbase.util.Bytes
import org.apache.spark.Logging
import org.apache.spark.sql.types._ import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.unsafe.types.UTF8String

View File

@ -17,15 +17,13 @@
package org.apache.spark.sql.datasources.hbase package org.apache.spark.sql.datasources.hbase
import org.apache.spark.sql.catalyst.SqlLexical import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
import org.apache.spark.sql.catalyst.util.DataTypeParser
import org.apache.spark.sql.types.DataType import org.apache.spark.sql.types.DataType
// TODO: Only used in test suite. trait DataTypeParser {
object DataTypeParserWrapper { def parse(dataTypeString: String): DataType
lazy val dataTypeParser = new DataTypeParser { }
override val lexical = new SqlLexical
} object DataTypeParserWrapper extends DataTypeParser{
def parse(dataTypeString: String): DataType = CatalystSqlParser.parseDataType(dataTypeString)
def parse(dataTypeString: String): DataType = dataTypeParser.toDataType(dataTypeString)
} }

View File

@ -19,12 +19,11 @@ package org.apache.spark.sql.datasources.hbase
import org.apache.avro.Schema import org.apache.avro.Schema
import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.spark.Logging
import org.apache.hadoop.hbase.spark.SchemaConverters import org.apache.hadoop.hbase.spark.SchemaConverters
import org.apache.hadoop.hbase.spark.datasources._ import org.apache.hadoop.hbase.spark.datasources._
import org.apache.hadoop.hbase.spark.hbase._ import org.apache.hadoop.hbase.spark.hbase._
import org.apache.hadoop.hbase.util.Bytes import org.apache.hadoop.hbase.util.Bytes
import org.apache.spark.Logging
import org.apache.spark.sql.catalyst.util.DataTypeParser
import org.apache.spark.sql.types._ import org.apache.spark.sql.types._
import org.json4s.jackson.JsonMethods._ import org.json4s.jackson.JsonMethods._
@ -79,7 +78,7 @@ case class Field(
} }
val dt = { val dt = {
sType.map(DataTypeParser.parse(_)).getOrElse{ sType.map(DataTypeParserWrapper.parse(_)).getOrElse{
schema.map{ x=> schema.map{ x=>
SchemaConverters.toSqlType(x).dataType SchemaConverters.toSqlType(x).dataType
}.get }.get

View File

@ -20,7 +20,7 @@ package org.apache.spark.sql.datasources.hbase
import org.apache.hadoop.hbase.spark.AvroSerdes import org.apache.hadoop.hbase.spark.AvroSerdes
import org.apache.hadoop.hbase.util.Bytes import org.apache.hadoop.hbase.util.Bytes
import org.apache.spark.sql.execution.SparkSqlSerializer //import org.apache.spark.sql.execution.SparkSqlSerializer
import org.apache.spark.sql.types._ import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.unsafe.types.UTF8String
@ -60,8 +60,8 @@ object Utils {
val newArray = new Array[Byte](length) val newArray = new Array[Byte](length)
System.arraycopy(src, offset, newArray, 0, length) System.arraycopy(src, offset, newArray, 0, length)
newArray newArray
// TODO: add more data type support // TODO: SparkSqlSerializer.deserialize[Any](src)
case _ => SparkSqlSerializer.deserialize[Any](src) case _ => throw new Exception(s"unsupported data type ${f.dt}")
} }
} }
} }

View File

@ -20,11 +20,12 @@ package org.apache.hadoop.hbase.spark
import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.hadoop.hbase.client.{Get, ConnectionFactory} import org.apache.hadoop.hbase.client.{Get, ConnectionFactory}
import org.apache.hadoop.hbase.io.hfile.{CacheConfig, HFile} import org.apache.hadoop.hbase.io.hfile.{CacheConfig, HFile}
import org.apache.hadoop.hbase.spark.Logging
import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles
import org.apache.hadoop.hbase.{HConstants, CellUtil, HBaseTestingUtility, TableName} import org.apache.hadoop.hbase.{HConstants, CellUtil, HBaseTestingUtility, TableName}
import org.apache.hadoop.hbase.util.Bytes import org.apache.hadoop.hbase.util.Bytes
import org.apache.hadoop.hbase.spark.HBaseRDDFunctions._ import org.apache.hadoop.hbase.spark.HBaseRDDFunctions._
import org.apache.spark.{SparkContext, Logging} import org.apache.spark.SparkContext
import org.junit.rules.TemporaryFolder import org.junit.rules.TemporaryFolder
import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}

View File

@ -20,13 +20,14 @@ package org.apache.hadoop.hbase.spark
import org.apache.avro.Schema import org.apache.avro.Schema
import org.apache.avro.generic.GenericData import org.apache.avro.generic.GenericData
import org.apache.hadoop.hbase.client.{ConnectionFactory, Put} import org.apache.hadoop.hbase.client.{ConnectionFactory, Put}
import org.apache.hadoop.hbase.spark.Logging
import org.apache.hadoop.hbase.spark.datasources.HBaseSparkConf import org.apache.hadoop.hbase.spark.datasources.HBaseSparkConf
import org.apache.hadoop.hbase.util.Bytes import org.apache.hadoop.hbase.util.Bytes
import org.apache.hadoop.hbase.{HBaseTestingUtility, TableName} import org.apache.hadoop.hbase.{HBaseTestingUtility, TableName}
import org.apache.spark.sql.datasources.hbase.HBaseTableCatalog import org.apache.spark.sql.datasources.hbase.HBaseTableCatalog
import org.apache.spark.sql.functions._ import org.apache.spark.sql.functions._
import org.apache.spark.sql.{DataFrame, SQLContext} import org.apache.spark.sql.{DataFrame, SQLContext}
import org.apache.spark.{Logging, SparkConf, SparkContext} import org.apache.spark.{SparkConf, SparkContext}
import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
case class HBaseRecord( case class HBaseRecord(
@ -377,8 +378,8 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging {
assert(results.length == 2) assert(results.length == 2)
assert(executionRules.dynamicLogicExpression.toExpressionString. val expr = executionRules.dynamicLogicExpression.toExpressionString
equals("( KEY_FIELD <= 0 AND KEY_FIELD >= 1 )")) assert(expr.equals("( ( KEY_FIELD isNotNull AND KEY_FIELD <= 0 ) AND KEY_FIELD >= 1 )"), expr)
assert(executionRules.rowKeyFilter.points.size == 0) assert(executionRules.rowKeyFilter.points.size == 0)
assert(executionRules.rowKeyFilter.ranges.size == 1) assert(executionRules.rowKeyFilter.ranges.size == 1)
@ -653,8 +654,9 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging {
assert(localResult(0).getInt(2) == 8) assert(localResult(0).getInt(2) == 8)
val executionRules = DefaultSourceStaticUtils.lastFiveExecutionRules.poll() val executionRules = DefaultSourceStaticUtils.lastFiveExecutionRules.poll()
assert(executionRules.dynamicLogicExpression.toExpressionString. val expr = executionRules.dynamicLogicExpression.toExpressionString
equals("( I_FIELD > 0 AND I_FIELD < 1 )")) logInfo(expr)
assert(expr.equals("( ( I_FIELD isNotNull AND I_FIELD > 0 ) AND I_FIELD < 1 )"), expr)
} }

View File

@ -19,9 +19,9 @@ package org.apache.hadoop.hbase.spark
import java.util import java.util
import org.apache.hadoop.hbase.spark.Logging
import org.apache.hadoop.hbase.spark.datasources.{HBaseSparkConf, JavaBytesEncoder} import org.apache.hadoop.hbase.spark.datasources.{HBaseSparkConf, JavaBytesEncoder}
import org.apache.hadoop.hbase.util.Bytes import org.apache.hadoop.hbase.util.Bytes
import org.apache.spark.Logging
import org.apache.spark.sql.types._ import org.apache.spark.sql.types._
import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}

View File

@ -17,9 +17,9 @@
package org.apache.hadoop.hbase.spark package org.apache.hadoop.hbase.spark
import org.apache.hadoop.hbase.spark.Logging
import org.apache.hadoop.hbase.spark.datasources.{DoubleSerDes, SerDes} import org.apache.hadoop.hbase.spark.datasources.{DoubleSerDes, SerDes}
import org.apache.hadoop.hbase.util.Bytes import org.apache.hadoop.hbase.util.Bytes
import org.apache.spark.Logging
import org.apache.spark.sql.datasources.hbase.{DataTypeParserWrapper, HBaseTableCatalog} import org.apache.spark.sql.datasources.hbase.{DataTypeParserWrapper, HBaseTableCatalog}
import org.apache.spark.sql.types._ import org.apache.spark.sql.types._
import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}

View File

@ -22,9 +22,9 @@ import scala.util.Random
import org.apache.hadoop.hbase.client.{BufferedMutator, Table, RegionLocator, import org.apache.hadoop.hbase.client.{BufferedMutator, Table, RegionLocator,
Connection, BufferedMutatorParams, Admin, TableBuilder} Connection, BufferedMutatorParams, Admin, TableBuilder}
import org.apache.hadoop.hbase.spark.Logging
import org.apache.hadoop.conf.Configuration import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.hbase.TableName import org.apache.hadoop.hbase.TableName
import org.apache.spark.Logging
import org.scalatest.FunSuite import org.scalatest.FunSuite
case class HBaseConnectionKeyMocker (confId: Int) extends HBaseConnectionKey (null) { case class HBaseConnectionKeyMocker (confId: Int) extends HBaseConnectionKey (null) {

View File

@ -20,7 +20,8 @@ import org.apache.hadoop.hbase.client._
import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter
import org.apache.hadoop.hbase.util.Bytes import org.apache.hadoop.hbase.util.Bytes
import org.apache.hadoop.hbase.{ CellUtil, TableName, HBaseTestingUtility} import org.apache.hadoop.hbase.{ CellUtil, TableName, HBaseTestingUtility}
import org.apache.spark.{SparkException, Logging, SparkContext} import org.apache.hadoop.hbase.spark.Logging
import org.apache.spark.{SparkException, SparkContext}
import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
class HBaseContextSuite extends FunSuite with class HBaseContextSuite extends FunSuite with

View File

@ -17,11 +17,12 @@
package org.apache.hadoop.hbase.spark package org.apache.hadoop.hbase.spark
import org.apache.hadoop.hbase.client._ import org.apache.hadoop.hbase.client._
import org.apache.hadoop.hbase.spark.Logging
import org.apache.hadoop.hbase.util.Bytes import org.apache.hadoop.hbase.util.Bytes
import org.apache.hadoop.hbase.{CellUtil, TableName, HBaseTestingUtility} import org.apache.hadoop.hbase.{CellUtil, TableName, HBaseTestingUtility}
import org.apache.spark.rdd.RDD import org.apache.spark.rdd.RDD
import org.apache.spark.streaming.{Milliseconds, StreamingContext} import org.apache.spark.streaming.{Milliseconds, StreamingContext}
import org.apache.spark.{SparkContext, Logging} import org.apache.spark.SparkContext
import org.apache.hadoop.hbase.spark.HBaseDStreamFunctions._ import org.apache.hadoop.hbase.spark.HBaseDStreamFunctions._
import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}

View File

@ -17,10 +17,11 @@
package org.apache.hadoop.hbase.spark package org.apache.hadoop.hbase.spark
import org.apache.hadoop.hbase.client._ import org.apache.hadoop.hbase.client._
import org.apache.hadoop.hbase.spark.Logging
import org.apache.hadoop.hbase.util.Bytes import org.apache.hadoop.hbase.util.Bytes
import org.apache.hadoop.hbase.{CellUtil, TableName, HBaseTestingUtility} import org.apache.hadoop.hbase.{CellUtil, TableName, HBaseTestingUtility}
import org.apache.hadoop.hbase.spark.HBaseRDDFunctions._ import org.apache.hadoop.hbase.spark.HBaseRDDFunctions._
import org.apache.spark.{Logging, SparkContext} import org.apache.spark.SparkContext
import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
import scala.collection.mutable import scala.collection.mutable

View File

@ -18,10 +18,11 @@
package org.apache.hadoop.hbase.spark package org.apache.hadoop.hbase.spark
import org.apache.hadoop.hbase.spark.datasources.HBaseSparkConf import org.apache.hadoop.hbase.spark.datasources.HBaseSparkConf
import org.apache.hadoop.hbase.spark.Logging
import org.apache.hadoop.hbase.{TableName, HBaseTestingUtility} import org.apache.hadoop.hbase.{TableName, HBaseTestingUtility}
import org.apache.spark.sql.datasources.hbase.HBaseTableCatalog import org.apache.spark.sql.datasources.hbase.HBaseTableCatalog
import org.apache.spark.sql.{DataFrame, SQLContext} import org.apache.spark.sql.{DataFrame, SQLContext}
import org.apache.spark.{SparkConf, SparkContext, Logging} import org.apache.spark.{SparkConf, SparkContext}
import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
case class FilterRangeRecord( case class FilterRangeRecord(