Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Custom sink provider for structured streaming #238

Open
wants to merge 5 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 6 additions & 0 deletions core/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -184,6 +184,12 @@
</configuration>
</plugin>
</plugins>
<resources>
<resource>
<directory>src/main/resources/META-INF/</directory>
<targetPath>META-INF</targetPath>
</resource>
</resources>
</build>

</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
org.apache.spark.sql.execution.streaming.HBaseStreamSinkProvider
Original file line number Diff line number Diff line change
@@ -0,0 +1,73 @@
package org.apache.spark.sql.execution.streaming

import java.util.Locale

import org.apache.spark.sql.catalyst.CatalystTypeConverters
import org.apache.spark.sql.execution.datasources.hbase.Logging
import org.apache.spark.sql.sources.{DataSourceRegister, StreamSinkProvider}
import org.apache.spark.sql.streaming.OutputMode
import org.apache.spark.sql.{DataFrame, Row, SQLContext}

class HBaseStreamSink(sqlContext: SQLContext,
parameters: Map[String, String],
partitionColumns: Seq[String],
outputMode: OutputMode)
extends Sink
with Logging {

@volatile private var latestBatchId = -1L

private val defaultFormat = "org.apache.spark.sql.execution.datasources.hbase"
private val prefix = "hbase."

private val specifiedHBaseParams = parameters
.keySet
.filter(_.toLowerCase(Locale.ROOT).startsWith(prefix))
.map { k => k.drop(prefix.length).toString -> parameters(k) }
.toMap

override def addBatch(batchId: Long, data: DataFrame): Unit = synchronized {
if (batchId <= latestBatchId) {
logInfo(s"Skipping already committed batch $batchId")
} else {
// use a local variable to make sure the map closure doesn't capture the whole DataFrame
val schema = data.schema
val res = data.queryExecution.toRdd.mapPartitions { rows =>
val converter = CatalystTypeConverters.createToScalaConverter(schema)
rows.map(converter(_).asInstanceOf[Row])
}

val df = sqlContext.sparkSession.createDataFrame(res, schema)
df.write
.options(specifiedHBaseParams)
.format(defaultFormat)
.save()
}
}
}

/**
* In option must be specified string with HBaseTableCatalog.tableCatalog
* {{{
* inputDF.
* writeStream.
* format("hbase").
* option("checkpointLocation", checkPointProdPath).
* options(Map("hbase.schema_array"->schema_array,"hbase.schema_record"->schema_record, hbase.catalog->catalog)).
* outputMode(OutputMode.Update()).
* trigger(Trigger.ProcessingTime(30.seconds)).
* start
* }}}
*/
class HBaseStreamSinkProvider
extends StreamSinkProvider
with DataSourceRegister {
def createSink(sqlContext: SQLContext,
parameters: Map[String, String],
partitionColumns: Seq[String],
outputMode: OutputMode): Sink = {
new HBaseStreamSink(sqlContext, parameters, partitionColumns, outputMode)
}

def shortName(): String = "hbase"
}