HBASE-21273 Move classes out of o.a.spark packages
Signed-off-by: Ted Yu <tyu@apache.org>
This commit is contained in:
parent
e8df847d4e
commit
ce29e9ea5c
|
@ -32,11 +32,11 @@ 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.spark.datasources.BytesEncoder;
|
||||
import org.apache.hadoop.hbase.spark.datasources.Field;
|
||||
import org.apache.hadoop.hbase.spark.datasources.JavaBytesEncoder;
|
||||
import org.apache.hadoop.hbase.spark.protobuf.generated.SparkFilterProtos;
|
||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.spark.sql.datasources.hbase.Field;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
|
|
@ -34,7 +34,6 @@ 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.datasources.hbase.{Utils, Field, HBaseTableCatalog}
|
||||
import org.apache.spark.sql.{DataFrame, SaveMode, Row, SQLContext}
|
||||
import org.apache.spark.sql.sources._
|
||||
import org.apache.spark.sql.types._
|
||||
|
|
|
@ -23,8 +23,7 @@ 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.spark.sql.datasources.hbase.{Field, Utils}
|
||||
import org.apache.spark.sql.types._
|
||||
|
||||
/**
|
||||
* Dynamic logic for SQL push down logic there is an instance for most
|
||||
* common operations and a pass through for other operations not covered here
|
||||
|
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.spark.sql.datasources.hbase
|
||||
package org.apache.hadoop.hbase.spark.datasources
|
||||
|
||||
import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
|
||||
import org.apache.spark.sql.types.DataType
|
|
@ -15,21 +15,17 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.spark.sql.datasources.hbase
|
||||
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
|
||||
import org.apache.hadoop.hbase.spark.SchemaConverters
|
||||
import org.apache.hadoop.hbase.spark.datasources._
|
||||
import org.apache.hadoop.hbase.spark.hbase._
|
||||
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.json4s.jackson.JsonMethods._
|
||||
|
||||
import scala.collection.mutable
|
||||
|
||||
// Due the access issue defined in spark, we have to locate the file in this package.
|
||||
// The definition of each column cell, which may be composite type
|
||||
// TODO: add avro support
|
||||
@InterfaceAudience.Private
|
|
@ -25,7 +25,6 @@ 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.util.ShutdownHookManager
|
||||
import org.apache.spark.sql.datasources.hbase.Field
|
||||
import org.apache.spark.{SparkEnv, TaskContext, Partition}
|
||||
import org.apache.spark.rdd.RDD
|
||||
|
||||
|
|
|
@ -16,7 +16,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.spark.sql.datasources.hbase
|
||||
package org.apache.hadoop.hbase.spark.datasources
|
||||
|
||||
import org.apache.hadoop.hbase.spark.AvroSerdes
|
||||
import org.apache.hadoop.hbase.util.Bytes
|
|
@ -20,7 +20,7 @@ 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.spark.sql.datasources.hbase.HBaseTableCatalog
|
||||
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
|
||||
|
|
|
@ -17,7 +17,7 @@
|
|||
|
||||
package org.apache.hadoop.hbase.spark.example.datasources
|
||||
|
||||
import org.apache.spark.sql.datasources.hbase.HBaseTableCatalog
|
||||
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
|
||||
|
|
|
@ -17,7 +17,7 @@
|
|||
|
||||
package org.apache.hadoop.hbase.spark.example.datasources
|
||||
|
||||
import org.apache.spark.sql.datasources.hbase.HBaseTableCatalog
|
||||
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
|
||||
|
|
|
@ -20,10 +20,9 @@ package org.apache.hadoop.hbase.spark
|
|||
import org.apache.avro.Schema
|
||||
import org.apache.avro.generic.GenericData
|
||||
import org.apache.hadoop.hbase.client.{ConnectionFactory, Put}
|
||||
import org.apache.hadoop.hbase.spark.datasources.HBaseSparkConf
|
||||
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.datasources.hbase.HBaseTableCatalog
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.{DataFrame, SQLContext}
|
||||
import org.apache.spark.{SparkConf, SparkContext}
|
||||
|
|
|
@ -17,9 +17,8 @@
|
|||
|
||||
package org.apache.hadoop.hbase.spark
|
||||
|
||||
import org.apache.hadoop.hbase.spark.datasources.{DoubleSerDes, SerDes}
|
||||
import org.apache.hadoop.hbase.spark.datasources.{DataTypeParserWrapper, DoubleSerDes, HBaseTableCatalog}
|
||||
import org.apache.hadoop.hbase.util.Bytes
|
||||
import org.apache.spark.sql.datasources.hbase.{DataTypeParserWrapper, HBaseTableCatalog}
|
||||
import org.apache.spark.sql.types._
|
||||
import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
|
||||
|
||||
|
|
|
@ -17,9 +17,8 @@
|
|||
|
||||
package org.apache.hadoop.hbase.spark
|
||||
|
||||
import org.apache.hadoop.hbase.spark.datasources.HBaseSparkConf
|
||||
import org.apache.hadoop.hbase.{TableName, HBaseTestingUtility}
|
||||
import org.apache.spark.sql.datasources.hbase.HBaseTableCatalog
|
||||
import org.apache.hadoop.hbase.spark.datasources.{HBaseSparkConf, HBaseTableCatalog}
|
||||
import org.apache.hadoop.hbase.{HBaseTestingUtility, TableName}
|
||||
import org.apache.spark.sql.{DataFrame, SQLContext}
|
||||
import org.apache.spark.{SparkConf, SparkContext}
|
||||
import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
|
||||
|
|
Loading…
Reference in New Issue