HBASE-21450 [documentation] Point spark doc at hbase-connectors spark

Signed-off-by: Guanghao Zhang <zghao@apache.org>
This commit is contained in:
Michael Stack 2018-11-07 14:52:34 -08:00 committed by stack
parent 9370347efe
commit e65744a813
1 changed files with 33 additions and 24 deletions

View File

@ -58,7 +58,7 @@ takes in HBase configurations and pushes them to the Spark executors. This allow
us to have an HBase Connection per Spark Executor in a static location.
For reference, Spark Executors can be on the same nodes as the Region Servers or
on different nodes there is no dependence of co-location. Think of every Spark
on different nodes, there is no dependence on co-location. Think of every Spark
Executor as a multi-threaded client application. This allows any Spark Tasks
running on the executors to access the shared Connection object.
@ -134,7 +134,7 @@ try {
All functionality between Spark and HBase will be supported both in Scala and in
Java, with the exception of SparkSQL which will support any language that is
supported by Spark. For the remaining of this documentation we will focus on
Scala examples for now.
Scala examples.
The examples above illustrate how to do a foreachPartition with a connection. A
number of other Spark base functions are supported out of the box:
@ -148,7 +148,11 @@ access to HBase
`hBaseRDD`:: To simplify a distributed scan to create a RDD
// end::spark_base_functions[]
For examples of all these functionalities, see the HBase-Spark Module.
For examples of all these functionalities, see the
link:https://github.com/apache/hbase-connectors/tree/master/spark[hbase-spark integration]
in the link:https://github.com/apache/hbase-connectors[hbase-connectors] repository
(the hbase-spark connectors live outside hbase core in a related,
Apache HBase project maintained, associated repo).
== Spark Streaming
https://spark.apache.org/streaming/[Spark Streaming] is a micro batching stream
@ -157,12 +161,12 @@ companions in that HBase can help serve the following benefits alongside Spark
Streaming.
* A place to grab reference data or profile data on the fly
* A place to store counts or aggregates in a way that supports Spark Streaming
* A place to store counts or aggregates in a way that supports Spark Streaming's
promise of _only once processing_.
The HBase-Spark modules integration points with Spark Streaming are similar to
its normal Spark integration points, in that the following commands are possible
straight off a Spark Streaming DStream.
The link:https://github.com/apache/hbase-connectors/tree/master/spark[hbase-spark integration]
with Spark Streaming is similar to its normal Spark integration points, in that the following
commands are possible straight off a Spark Streaming DStream.
include::spark.adoc[tags=spark_base_functions]
@ -202,10 +206,10 @@ dStream.hbaseBulkPut(
----
There are three inputs to the `hbaseBulkPut` function.
. The hbaseContext that carries the configuration boardcast information link us
to the HBase Connections in the executors
. The table name of the table we are putting data into
. A function that will convert a record in the DStream into an HBase Put object.
The hbaseContext that carries the configuration broadcast information link
to the HBase Connections in the executor, the table name of the table we are
putting data into, and a function that will convert a record in the DStream
into an HBase Put object.
====
== Bulk Load
@ -213,11 +217,11 @@ to the HBase Connections in the executors
There are two options for bulk loading data into HBase with Spark. There is the
basic bulk load functionality that will work for cases where your rows have
millions of columns and cases where your columns are not consolidated and
partitions before the on the map side of the Spark bulk load process.
partitioned before the map side of the Spark bulk load process.
There is also a thin record bulk load option with Spark, this second option is
There is also a thin record bulk load option with Spark. This second option is
designed for tables that have less then 10k columns per row. The advantage
of this second option is higher throughput and less over all load on the Spark
of this second option is higher throughput and less over-all load on the Spark
shuffle operation.
Both implementations work more or less like the MapReduce bulk load process in
@ -225,7 +229,7 @@ that a partitioner partitions the rowkeys based on region splits and
the row keys are sent to the reducers in order, so that HFiles can be written
out directly from the reduce phase.
In Spark terms, the bulk load will be implemented around a the Spark
In Spark terms, the bulk load will be implemented around a Spark
`repartitionAndSortWithinPartitions` followed by a Spark `foreachPartition`.
First lets look at an example of using the basic bulk load functionality
@ -386,20 +390,24 @@ values for this row for all column families.
== SparkSQL/DataFrames
HBase-Spark Connector (in HBase-Spark Module) leverages
The link:https://github.com/apache/hbase-connectors/tree/master/spark[hbase-spark integration]
leverages
link:https://databricks.com/blog/2015/01/09/spark-sql-data-sources-api-unified-data-access-for-the-spark-platform.html[DataSource API]
(link:https://issues.apache.org/jira/browse/SPARK-3247[SPARK-3247])
introduced in Spark-1.2.0, bridges the gap between simple HBase KV store and complex
introduced in Spark-1.2.0, which bridges the gap between simple HBase KV store and complex
relational SQL queries and enables users to perform complex data analytical work
on top of HBase using Spark. HBase Dataframe is a standard Spark Dataframe, and is able to
interact with any other data sources such as Hive, Orc, Parquet, JSON, etc.
HBase-Spark Connector applies critical techniques such as partition pruning, column pruning,
The link:https://github.com/apache/hbase-connectors/tree/master/spark[hbase-spark integration]
applies critical techniques such as partition pruning, column pruning,
predicate pushdown and data locality.
To use HBase-Spark connector, users need to define the Catalog for the schema mapping
To use the
link:https://github.com/apache/hbase-connectors/tree/master/spark[hbase-spark integration]
connector, users need to define the Catalog for the schema mapping
between HBase and Spark tables, prepare the data and populate the HBase table,
then load HBase DataFrame. After that, users can do integrated query and access records
in HBase table with SQL query. Following illustrates the basic procedure.
then load the HBase DataFrame. After that, users can do integrated query and access records
in HBase tables with SQL query. The following illustrates the basic procedure.
=== Define catalog
@ -564,8 +572,9 @@ sqlContext.sql("select count(col1) from table").show
.Native Avro support
====
HBase-Spark Connector support different data formats like Avro, Jason, etc. The use case below
shows how spark supports Avro. User can persist the Avro record into HBase directly. Internally,
The link:https://github.com/apache/hbase-connectors/tree/master/spark[hbase-spark integration]
connector supports different data formats like Avro, JSON, etc. The use case below
shows how spark supports Avro. Users can persist the Avro record into HBase directly. Internally,
the Avro schema is converted to a native Spark Catalyst data type automatically.
Note that both key-value parts in an HBase table can be defined in Avro format.
@ -687,4 +696,4 @@ The date frame `df` returned by `withCatalog` function could be used to access t
After loading df DataFrame, users can query data. registerTempTable registers df DataFrame
as a temporary table using the table name avrotable. `sqlContext.sql` function allows the
user to execute SQL queries.
====
====