A library for reading and writing Tensorflow TFRecord data from Apache Spark. The implementation is based on Spark Tensorflow Connector, but it is rewritten in Spark FileFormat trait to provide the partitioning function.
The artifacts are published to bintray and maven central repositories.
- Version 0.1.x targets Spark 2.3 and Scala 2.11
- Version 0.2.x targets Spark 2.4 and both Scala 2.11 and 2.12
- Version 0.3.x targets Spark 3.0 and Scala 2.12
- Version 0.4.x targets Spark 3.2 and Scala 2.12
- Version 0.5.x targets Spark 3.2 and Scala 2.13
- Version 0.6.x targets Spark 3.4 and both Scala 2.12 and 2.13
- Version 0.7.x targets Spark 3.5 and both Scala 2.12 and 2.13
To use the package, please include the dependency as follows
<dependency>
<groupId>com.linkedin.sparktfrecord</groupId>
<artifactId>spark-tfrecord_2.12</artifactId>
<version>your.version</version>
</dependency>
The library can be built with Maven 3.3.9 or newer as shown below:
# Build Spark-TFRecord
git clone https://github.com/linkedin/spark-tfrecord.git
cd spark-tfrecord
mvn -Pscala-2.12 clean install
# One can specify the spark version and tensorflow hadoop version, for example
mvn -Pscala-2.12 clean install -Dspark.version=3.0.0 -Dtensorflow.hadoop.version=1.15.0
Run this library in Spark using the --jars
command line option in spark-shell
, pyspark
or spark-submit
. For example:
$SPARK_HOME/bin/spark-shell --jars target/spark-tfrecord_2.12-0.3.0.jar
This library allows reading TensorFlow records in local or distributed filesystem as Spark DataFrames. When reading TensorFlow records into Spark DataFrame, the API accepts several options:
load
: input path to TensorFlow records. Similar to Spark can accept standard Hadoop globbing expressions.schema
: schema of TensorFlow records. Optional schema defined using Spark StructType. If not provided, the schema is inferred from TensorFlow records.recordType
: input format of TensorFlow records. By default it is Example. Possible values are:Example
: TensorFlow Example recordsSequenceExample
: TensorFlow SequenceExample recordsByteArray
:Array[Byte]
type in scala.
When writing Spark DataFrame to TensorFlow records, the API accepts several options:
save
: output path to TensorFlow records. Output path to TensorFlow records on local or distributed filesystem. compression. While reading compressed TensorFlow records,codec
can be inferred automatically, so this option is not required for reading.recordType
: output format of TensorFlow records. By default it is Example. Possible values are:Example
: TensorFlow Example recordsSequenceExample
: TensorFlow SequenceExample recordsByteArray
:Array[Byte]
type in scala. For use cases when writing objects other than tensorflow Example or SequenceExample. For example, protos can be transformed to byte arrays using.toByteArray
.
The writer support partitionBy operation. So the following command will partition the output by "partitionColumn".
df.write.mode(SaveMode.Overwrite).partitionBy("partitionColumn").format("tfrecord").option("recordType", "Example").save(output_dir)
Note we use format("tfrecord")
instead format("tfrecords")
. So if you migrate from Spark-Tensorflow-Connector, make sure this is changed accordingly.
This library supports automatic schema inference when reading TensorFlow records into Spark DataFrames. Schema inference is expensive since it requires an extra pass through the data.
The schema inference rules are described in the table below:
TFRecordType | Feature Type | Inferred Spark Data Type |
---|---|---|
Example, SequenceExample | Int64List | LongType if all lists have length=1, else ArrayType(LongType) |
Example, SequenceExample | FloatList | FloatType if all lists have length=1, else ArrayType(FloatType) |
Example, SequenceExample | BytesList | StringType if all lists have length=1, else ArrayType(StringType) |
SequenceExample | FeatureList of Int64List | ArrayType(ArrayType(LongType)) |
SequenceExample | FeatureList of FloatList | ArrayType(ArrayType(FloatType)) |
SequenceExample | FeatureList of BytesList | ArrayType(ArrayType(StringType)) |
The supported Spark data types are listed in the table below:
Type | Spark DataTypes |
---|---|
Scalar | IntegerType, LongType, FloatType, DoubleType, DecimalType, StringType, BinaryType |
Array | VectorType, ArrayType of IntegerType, LongType, FloatType, DoubleType, DecimalType, BinaryType, or StringType |
Array of Arrays | ArrayType of ArrayType of IntegerType, LongType, FloatType, DoubleType, DecimalType, BinaryType, or StringType |
Run PySpark with the spark_connector in the jars argument as shown below:
$SPARK_HOME/bin/pyspark --jars target/spark-tfrecord_2.12-0.3.0.jar
The following Python code snippet demonstrates usage on test data.
from pyspark.sql.types import *
path = "test-output.tfrecord"
fields = [StructField("id", IntegerType()), StructField("IntegerCol", IntegerType()),
StructField("LongCol", LongType()), StructField("FloatCol", FloatType()),
StructField("DoubleCol", DoubleType()), StructField("VectorCol", ArrayType(DoubleType(), True)),
StructField("StringCol", StringType())]
schema = StructType(fields)
test_rows = [[11, 1, 23, 10.0, 14.0, [1.0, 2.0], "r1"], [21, 2, 24, 12.0, 15.0, [2.0, 2.0], "r2"]]
rdd = spark.sparkContext.parallelize(test_rows)
df = spark.createDataFrame(rdd, schema)
df.write.mode("overwrite").format("tfrecord").option("recordType", "Example").save(path)
df = spark.read.format("tfrecord").option("recordType", "Example").load(path)
df.show()
Run Spark shell with the spark_connector in the jars argument as shown below:
$SPARK_HOME/bin/spark-shell --jars target/spark-tfrecord_2.12-0.3.0.jar
The following Scala code snippet demonstrates usage on test data.
import org.apache.commons.io.FileUtils
import org.apache.spark.sql.{ DataFrame, Row }
import org.apache.spark.sql.catalyst.expressions.GenericRow
import org.apache.spark.sql.types._
val path = "test-output.tfrecord"
val testRows: Array[Row] = Array(
new GenericRow(Array[Any](11, 1, 23L, 10.0F, 14.0, List(1.0, 2.0), "r1")),
new GenericRow(Array[Any](21, 2, 24L, 12.0F, 15.0, List(2.0, 2.0), "r2")))
val schema = StructType(List(StructField("id", IntegerType),
StructField("IntegerCol", IntegerType),
StructField("LongCol", LongType),
StructField("FloatCol", FloatType),
StructField("DoubleCol", DoubleType),
StructField("VectorCol", ArrayType(DoubleType, true)),
StructField("StringCol", StringType)))
val rdd = spark.sparkContext.parallelize(testRows)
//Save DataFrame as TFRecords
val df: DataFrame = spark.createDataFrame(rdd, schema)
df.write.format("tfrecord").option("recordType", "Example").save(path)
//Read TFRecords into DataFrame.
//The DataFrame schema is inferred from the TFRecords if no custom schema is provided.
val importedDf1: DataFrame = spark.read.format("tfrecord").option("recordType", "Example").load(path)
importedDf1.show()
//Read TFRecords into DataFrame using custom schema
val importedDf2: DataFrame = spark.read.format("tfrecord").schema(schema).load(path)
importedDf2.show()
The following example shows to how to use partitionBy, which is not supported by Spark Tensorflow Connector
// launch spark-shell with the following command:
// SPARK_HOME/bin/spark-shell --jar target/spark-tfrecord_2.12-0.3.0.jar
import org.apache.spark.sql.SaveMode
val df = Seq((8, "bat"),(8, "abc"), (1, "xyz"), (2, "aaa")).toDF("number", "word")
df.show
// scala> df.show
// +------+----+
// |number|word|
// +------+----+
// | 8| bat|
// | 8| abc|
// | 1| xyz|
// | 2| aaa|
// +------+----+
val tf_output_dir = "/tmp/tfrecord-test"
// dump the tfrecords to files.
df.repartition(3, col("number")).write.mode(SaveMode.Overwrite).partitionBy("number").format("tfrecord").option("recordType", "Example").save(tf_output_dir)
// ls /tmp/tfrecord-test
// _SUCCESS number=1 number=2 number=8
// read back the tfrecords from files.
val new_df = spark.read.format("tfrecord").option("recordType", "Example").load(tf_output_dir)
new_df.show
// scala> new_df.show
// +----+------+
// |word|number|
// +----+------+
// | bat| 8|
// | abc| 8|
// | xyz| 1|
// | aaa| 2|
Please read CONTRIBUTING.md for details on our code of conduct, and the process for submitting pull requests to us.
This project is licensed under the BSD 2-CLAUSE LICENSE - see the LICENSE.md file for details