-
Notifications
You must be signed in to change notification settings - Fork 28
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Merge pull request #40 from samelamin/python
Add PySpark and DML query support
- Loading branch information
Showing
9 changed files
with
288 additions
and
220 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
86 changes: 86 additions & 0 deletions
86
src/main/scala/com/samelamin/spark/bigquery/BigQueryDataFrame.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,86 @@ | ||
package com.samelamin.spark.bigquery | ||
|
||
import com.google.api.services.bigquery.model.TableReference | ||
import com.google.cloud.hadoop.io.bigquery._ | ||
import com.google.gson._ | ||
import com.samelamin.spark.bigquery.converters.{BigQueryAdapter, SchemaConverters} | ||
import org.apache.hadoop.fs.{FileSystem, Path} | ||
import org.apache.hadoop.io.{LongWritable, NullWritable} | ||
import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat | ||
import org.apache.spark.sql.DataFrame | ||
import org.slf4j.LoggerFactory | ||
|
||
import scala.util.Random | ||
/** | ||
* Created by samelamin on 12/08/2017. | ||
*/ | ||
class BigQueryDataFrame(self: DataFrame) extends Serializable { | ||
val adaptedDf = BigQueryAdapter(self) | ||
private val logger = LoggerFactory.getLogger(classOf[BigQueryClient]) | ||
|
||
@transient | ||
lazy val hadoopConf = self.sparkSession.sparkContext.hadoopConfiguration | ||
lazy val bq = BigQueryClient.getInstance(self.sqlContext) | ||
|
||
@transient | ||
lazy val jsonParser = new JsonParser() | ||
|
||
/** | ||
* Save DataFrame data into BigQuery table using Hadoop writer API | ||
* | ||
* @param fullyQualifiedOutputTableId output-table id of the form | ||
* [optional projectId]:[datasetId].[tableId] | ||
* @param isPartitionedByDay partion the table by day | ||
*/ | ||
def saveAsBigQueryTable(fullyQualifiedOutputTableId: String, | ||
isPartitionedByDay: Boolean = false, | ||
timePartitionExpiration: Long = 0, | ||
writeDisposition: WriteDisposition.Value = null, | ||
createDisposition: CreateDisposition.Value = null): Unit = { | ||
val destinationTable = BigQueryStrings.parseTableReference(fullyQualifiedOutputTableId) | ||
val bigQuerySchema = SchemaConverters.SqlToBQSchema(adaptedDf) | ||
val gcsPath = writeDFToGoogleStorage(adaptedDf,destinationTable,bigQuerySchema) | ||
bq.load(destinationTable, | ||
bigQuerySchema, | ||
gcsPath, | ||
isPartitionedByDay, | ||
timePartitionExpiration, | ||
writeDisposition, | ||
createDisposition) | ||
delete(new Path(gcsPath)) | ||
} | ||
|
||
def writeDFToGoogleStorage(adaptedDf: DataFrame, | ||
destinationTable: TableReference, | ||
bigQuerySchema: String): String = { | ||
val tableName = BigQueryStrings.toString(destinationTable) | ||
|
||
BigQueryConfiguration.configureBigQueryOutput(hadoopConf, tableName, bigQuerySchema) | ||
hadoopConf.set("mapreduce.job.outputformat.class", classOf[BigQueryOutputFormat[_, _]].getName) | ||
val bucket = self.sparkSession.conf.get(BigQueryConfiguration.GCS_BUCKET_KEY) | ||
val temp = s"spark-bigquery-${System.currentTimeMillis()}=${Random.nextInt(Int.MaxValue)}" | ||
val gcsPath = s"gs://$bucket/hadoop/tmp/spark-bigquery/$temp" | ||
if(hadoopConf.get(BigQueryConfiguration.TEMP_GCS_PATH_KEY) == null) { | ||
hadoopConf.set(BigQueryConfiguration.TEMP_GCS_PATH_KEY, gcsPath) | ||
} | ||
|
||
logger.info(s"Loading $gcsPath into $tableName") | ||
adaptedDf | ||
.toJSON | ||
.rdd | ||
.map(json => (null, jsonParser.parse(json))) | ||
.saveAsNewAPIHadoopFile(gcsPath, | ||
classOf[GsonBigQueryInputFormat], | ||
classOf[LongWritable], | ||
classOf[TextOutputFormat[NullWritable, JsonObject]], | ||
hadoopConf) | ||
gcsPath | ||
} | ||
|
||
|
||
|
||
private def delete(path: Path): Unit = { | ||
val fs = FileSystem.get(path.toUri, hadoopConf) | ||
fs.delete(path, true) | ||
} | ||
} |
132 changes: 132 additions & 0 deletions
132
src/main/scala/com/samelamin/spark/bigquery/BigQuerySQLContext.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,132 @@ | ||
package com.samelamin.spark.bigquery | ||
|
||
|
||
import java.math.BigInteger | ||
|
||
import com.google.cloud.hadoop.io.bigquery.{AvroBigQueryInputFormat, _} | ||
import com.samelamin.spark.bigquery.converters.SchemaConverters | ||
import org.apache.avro.Schema | ||
import org.apache.avro.generic.GenericData | ||
import org.apache.hadoop.io.LongWritable | ||
import org.apache.spark.sql.types.StructType | ||
import org.apache.spark.sql.{DataFrame, Row, SQLContext} | ||
import org.slf4j.LoggerFactory | ||
|
||
/** | ||
* Created by samelamin on 12/08/2017. | ||
*/ | ||
class BigQuerySQLContext(sqlContext: SQLContext) extends Serializable { | ||
lazy val bq = BigQueryClient.getInstance(sqlContext) | ||
@transient | ||
lazy val hadoopConf = sqlContext.sparkSession.sparkContext.hadoopConfiguration | ||
private val logger = LoggerFactory.getLogger(classOf[BigQueryClient]) | ||
|
||
/** | ||
* Set whether to allow schema updates | ||
*/ | ||
def setAllowSchemaUpdates(value: Boolean = true): Unit = { | ||
hadoopConf.set(bq.ALLOW_SCHEMA_UPDATES, value.toString) | ||
} | ||
|
||
/** | ||
* Set whether to use the Standard SQL Dialect | ||
*/ | ||
def useStandardSQLDialect(value: Boolean = true): Unit = { | ||
hadoopConf.set(bq.USE_STANDARD_SQL_DIALECT, value.toString) | ||
} | ||
/** | ||
* Set GCP project ID for BigQuery. | ||
*/ | ||
def setBigQueryProjectId(projectId: String): Unit = { | ||
hadoopConf.set(BigQueryConfiguration.PROJECT_ID_KEY, projectId) | ||
} | ||
|
||
def setGSProjectId(projectId: String): Unit = { | ||
// Also set project ID for GCS connector | ||
hadoopConf.set("fs.gs.project.id", projectId) | ||
} | ||
|
||
def setBQTableTimestampColumn(timestampColumn: String): Unit = { | ||
hadoopConf.set("timestamp_column", timestampColumn) | ||
} | ||
|
||
/** | ||
* Set GCS bucket for temporary BigQuery files. | ||
*/ | ||
def setBigQueryGcsBucket(gcsBucket: String): Unit = { | ||
hadoopConf.set(BigQueryConfiguration.GCS_BUCKET_KEY, gcsBucket) | ||
sqlContext.sparkSession.conf.set(BigQueryConfiguration.GCS_BUCKET_KEY, gcsBucket) | ||
} | ||
|
||
/** | ||
* Set BigQuery dataset location, e.g. US, EU. | ||
*/ | ||
def setBigQueryDatasetLocation(location: String): Unit = { | ||
hadoopConf.set(bq.STAGING_DATASET_LOCATION, location) | ||
} | ||
|
||
/** | ||
* Set GCP JSON key file. | ||
*/ | ||
def setGcpJsonKeyFile(jsonKeyFile: String): Unit = { | ||
hadoopConf.set("mapred.bq.auth.service.account.json.keyfile", jsonKeyFile) | ||
hadoopConf.set("fs.gs.auth.service.account.json.keyfile", jsonKeyFile) | ||
} | ||
|
||
/** | ||
* Set GCP pk12 key file. | ||
*/ | ||
def setGcpPk12KeyFile(pk12KeyFile: String): Unit = { | ||
hadoopConf.set("google.cloud.auth.service.account.keyfile", pk12KeyFile) | ||
hadoopConf.set("mapred.bq.auth.service.account.keyfile", pk12KeyFile) | ||
hadoopConf.set("fs.gs.auth.service.account.keyfile", pk12KeyFile) | ||
} | ||
|
||
def bigQuerySelect(sqlQuery: String): DataFrame = { | ||
bq.selectQuery(sqlQuery) | ||
val tableData = sqlContext.sparkSession.sparkContext.newAPIHadoopRDD( | ||
hadoopConf, | ||
classOf[AvroBigQueryInputFormat], | ||
classOf[LongWritable], | ||
classOf[GenericData.Record]).map(x=>x._2) | ||
val schemaString = tableData.map(_.getSchema.toString).first() | ||
val schema = new Schema.Parser().parse(schemaString) | ||
val structType = SchemaConverters.avroToSqlType(schema).dataType.asInstanceOf[StructType] | ||
val converter = SchemaConverters.createConverterToSQL(schema) | ||
.asInstanceOf[GenericData.Record => Row] | ||
sqlContext.createDataFrame(tableData.map(converter), structType) | ||
} | ||
|
||
def runDMLQuery(runDMLQuery:String):Unit = { | ||
bq.runDMLQuery(runDMLQuery) | ||
} | ||
|
||
def getLatestBQModifiedTime(tableReference: String): Option[BigInteger] = { | ||
bq.getLatestModifiedTime(BigQueryStrings.parseTableReference(tableReference)) | ||
} | ||
|
||
def getBigQuerySchema(tableReference: String): StructType = { | ||
SchemaConverters.BQToSQLSchema(bq.getTableSchema(BigQueryStrings.parseTableReference(tableReference))) | ||
} | ||
|
||
|
||
/** | ||
* Load a BigQuery table as a [[DataFrame]]. | ||
*/ | ||
def bigQueryTable(tableSpec: String): DataFrame = { | ||
val tableRef = BigQueryStrings.parseTableReference(tableSpec) | ||
BigQueryConfiguration.configureBigQueryInput( | ||
hadoopConf, tableRef.getProjectId, tableRef.getDatasetId, tableRef.getTableId) | ||
val tableData = sqlContext.sparkContext.newAPIHadoopRDD( | ||
hadoopConf, | ||
classOf[AvroBigQueryInputFormat], | ||
classOf[LongWritable], | ||
classOf[GenericData.Record]).map(x=>x._2) | ||
val schemaString = tableData.map(_.getSchema.toString).first() | ||
val schema = new Schema.Parser().parse(schemaString) | ||
val structType = SchemaConverters.avroToSqlType(schema).dataType.asInstanceOf[StructType] | ||
val converter = SchemaConverters.createConverterToSQL(schema) | ||
.asInstanceOf[GenericData.Record => Row] | ||
sqlContext.createDataFrame(tableData.map(converter), structType) | ||
} | ||
} |
Oops, something went wrong.