[SPARK-25347][ML][DOC] Spark datasource for image/libsvm user guide

## What changes were proposed in this pull request?

Spark datasource for image/libsvm user guide

## How was this patch tested?
Scala:
<img width="1022" alt="1" src="https://user-images.githubusercontent.com/19235986/47330111-a4f2e900-d6a9-11e8-9a6f-609fb8cd0f8a.png">

Java:
<img width="1019" alt="2" src="https://user-images.githubusercontent.com/19235986/47330114-a9b79d00-d6a9-11e8-97fe-c7e4b8dd5086.png">

Python:
<img width="1022" alt="3" src="https://user-images.githubusercontent.com/19235986/47330120-afad7e00-d6a9-11e8-8a0c-4340c2af727b.png">

R:
<img width="1024" alt="4" src="https://user-images.githubusercontent.com/19235986/47330126-b3410500-d6a9-11e8-9329-5e6217718edd.png">

Closes #22675 from WeichenXu123/add_image_source_doc.

Authored-by: WeichenXu <weichen.xu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
This commit is contained in:
WeichenXu 2018-10-25 23:03:16 +08:00 committed by Wenchen Fan
parent 002f9c169e
commit 6540c2f8f3
3 changed files with 120 additions and 7 deletions

View file

@ -1,5 +1,7 @@
- text: Basic statistics
url: ml-statistics.html
- text: Data sources
url: ml-datasource
- text: Pipelines
url: ml-pipeline.html
- text: Extracting, transforming and selecting features

108
docs/ml-datasource.md Normal file
View file

@ -0,0 +1,108 @@
---
layout: global
title: Data sources
displayTitle: Data sources
---
In this section, we introduce how to use data source in ML to load data.
Beside some general data sources such as Parquet, CSV, JSON and JDBC, we also provide some specific data sources for ML.
**Table of Contents**
* This will become a table of contents (this text will be scraped).
{:toc}
## Image data source
This image data source is used to load image files from a directory, it can load compressed image (jpeg, png, etc.) into raw image representation via `ImageIO` in Java library.
The loaded DataFrame has one `StructType` column: "image", containing image data stored as image schema.
The schema of the `image` column is:
- origin: `StringType` (represents the file path of the image)
- height: `IntegerType` (height of the image)
- width: `IntegerType` (width of the image)
- nChannels: `IntegerType` (number of image channels)
- mode: `IntegerType` (OpenCV-compatible type)
- data: `BinaryType` (Image bytes in OpenCV-compatible order: row-wise BGR in most cases)
<div class="codetabs">
<div data-lang="scala" markdown="1">
[`ImageDataSource`](api/scala/index.html#org.apache.spark.ml.source.image.ImageDataSource)
implements a Spark SQL data source API for loading image data as a DataFrame.
{% highlight scala %}
scala> val df = spark.read.format("image").option("dropInvalid", true).load("data/mllib/images/origin/kittens")
df: org.apache.spark.sql.DataFrame = [image: struct<origin: string, height: int ... 4 more fields>]
scala> df.select("image.origin", "image.width", "image.height").show(truncate=false)
+-----------------------------------------------------------------------+-----+------+
|origin |width|height|
+-----------------------------------------------------------------------+-----+------+
|file:///spark/data/mllib/images/origin/kittens/54893.jpg |300 |311 |
|file:///spark/data/mllib/images/origin/kittens/DP802813.jpg |199 |313 |
|file:///spark/data/mllib/images/origin/kittens/29.5.a_b_EGDP022204.jpg |300 |200 |
|file:///spark/data/mllib/images/origin/kittens/DP153539.jpg |300 |296 |
+-----------------------------------------------------------------------+-----+------+
{% endhighlight %}
</div>
<div data-lang="java" markdown="1">
[`ImageDataSource`](api/java/org/apache/spark/ml/source/image/ImageDataSource.html)
implements Spark SQL data source API for loading image data as DataFrame.
{% highlight java %}
Dataset<Row> imagesDF = spark.read().format("image").option("dropInvalid", true).load("data/mllib/images/origin/kittens");
imageDF.select("image.origin", "image.width", "image.height").show(false);
/*
Will output:
+-----------------------------------------------------------------------+-----+------+
|origin |width|height|
+-----------------------------------------------------------------------+-----+------+
|file:///spark/data/mllib/images/origin/kittens/54893.jpg |300 |311 |
|file:///spark/data/mllib/images/origin/kittens/DP802813.jpg |199 |313 |
|file:///spark/data/mllib/images/origin/kittens/29.5.a_b_EGDP022204.jpg |300 |200 |
|file:///spark/data/mllib/images/origin/kittens/DP153539.jpg |300 |296 |
+-----------------------------------------------------------------------+-----+------+
*/
{% endhighlight %}
</div>
<div data-lang="python" markdown="1">
In PySpark we provide Spark SQL data source API for loading image data as DataFrame.
{% highlight python %}
>>> df = spark.read.format("image").option("dropInvalid", true).load("data/mllib/images/origin/kittens")
>>> df.select("image.origin", "image.width", "image.height").show(truncate=False)
+-----------------------------------------------------------------------+-----+------+
|origin |width|height|
+-----------------------------------------------------------------------+-----+------+
|file:///spark/data/mllib/images/origin/kittens/54893.jpg |300 |311 |
|file:///spark/data/mllib/images/origin/kittens/DP802813.jpg |199 |313 |
|file:///spark/data/mllib/images/origin/kittens/29.5.a_b_EGDP022204.jpg |300 |200 |
|file:///spark/data/mllib/images/origin/kittens/DP153539.jpg |300 |296 |
+-----------------------------------------------------------------------+-----+------+
{% endhighlight %}
</div>
<div data-lang="r" markdown="1">
In SparkR we provide Spark SQL data source API for loading image data as DataFrame.
{% highlight r %}
> df = read.df("data/mllib/images/origin/kittens", "image")
> head(select(df, df$image.origin, df$image.width, df$image.height))
1 file:///spark/data/mllib/images/origin/kittens/54893.jpg
2 file:///spark/data/mllib/images/origin/kittens/DP802813.jpg
3 file:///spark/data/mllib/images/origin/kittens/29.5.a_b_EGDP022204.jpg
4 file:///spark/data/mllib/images/origin/kittens/DP153539.jpg
width height
1 300 311
2 199 313
3 300 200
4 300 296
{% endhighlight %}
</div>
</div>

View file

@ -19,14 +19,17 @@ package org.apache.spark.ml.source.image
/**
* `image` package implements Spark SQL data source API for loading image data as `DataFrame`.
* The loaded `DataFrame` has one `StructType` column: `image`.
* It can load compressed image (jpeg, png, etc.) into raw image representation via `ImageIO`
* in Java library.
* The loaded `DataFrame` has one `StructType` column: `image`, containing image data stored
* as image schema.
* The schema of the `image` column is:
* - origin: String (represents the file path of the image)
* - height: Int (height of the image)
* - width: Int (width of the image)
* - nChannels: Int (number of the image channels)
* - mode: Int (OpenCV-compatible type)
* - data: BinaryType (Image bytes in OpenCV-compatible order: row-wise BGR in most cases)
* - origin: `StringType` (represents the file path of the image)
* - height: `IntegerType` (height of the image)
* - width: `IntegerType` (width of the image)
* - nChannels: `IntegerType` (number of image channels)
* - mode: `IntegerType` (OpenCV-compatible type)
* - data: `BinaryType` (Image bytes in OpenCV-compatible order: row-wise BGR in most cases)
*
* To use image data source, you need to set "image" as the format in `DataFrameReader` and
* optionally specify the data source options, for example: