You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by imatiach-msft <gi...@git.apache.org> on 2017/10/05 17:13:24 UTC

[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

GitHub user imatiach-msft opened a pull request:

    https://github.com/apache/spark/pull/19439

    [SPARK-21866][ML][PySpark] Adding spark image reader

    ## What changes were proposed in this pull request?
    Adding spark image reader, an implementation of schema for representing images in spark DataFrames
    
    The code is taken from the spark package located here:
    (https://github.com/Microsoft/spark-images)
    
    Please see the JIRA for more information (https://issues.apache.org/jira/browse/SPARK-21866)
    
    Please see mailing list for SPIP vote and approval information:
    (http://apache-spark-developers-list.1001551.n3.nabble.com/VOTE-SPIP-SPARK-21866-Image-support-in-Apache-Spark-td22510.html)
    
    # Background and motivation
    As Apache Spark is being used more and more in the industry, some new use cases are emerging for different data formats beyond the traditional SQL types or the numerical types (vectors and matrices). Deep Learning applications commonly deal with image processing. A number of projects add some Deep Learning capabilities to Spark (see list below), but they struggle to communicate with each other or with MLlib pipelines because there is no standard way to represent an image in Spark DataFrames. We propose to federate efforts for representing images in Spark by defining a representation that caters to the most common needs of users and library developers.
    This SPIP proposes a specification to represent images in Spark DataFrames and Datasets (based on existing industrial standards), and an interface for loading sources of images. It is not meant to be a full-fledged image processing library, but rather the core description that other libraries and users can rely on. Several packages already offer various processing facilities for transforming images or doing more complex operations, and each has various design tradeoffs that make them better as standalone solutions.
    This project is a joint collaboration between Microsoft and Databricks, which have been testing this design in two open source packages: MMLSpark and Deep Learning Pipelines.
    The proposed image format is an in-memory, decompressed representation that targets low-level applications. It is significantly more liberal in memory usage than compressed image representations such as JPEG, PNG, etc., but it allows easy communication with popular image processing libraries and has no decoding overhead.
    
    ## How was this patch tested?
    
    Unit tests in scala ImageSchemaSuite, unit tests in python

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/imatiach-msft/spark ilmat/spark-images

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/19439.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #19439
    
----
commit 22baf022b2f109bb1f5eba0b13ea34de894cd14c
Author: Ilya Matiach <il...@microsoft.com>
Date:   2017-10-04T21:10:26Z

    [SPARK-21866][ML][PySpark] Adding spark image reader

----


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r148028651
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,139 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +from pyspark.ml.util import *
    +from pyspark.ml.param.shared import *
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame, SparkSession, SQLContext
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +imageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +
    +def getOcvTypes(spark=None):
    +    """
    +    Returns the OpenCV type mapping supported
    +
    +    :param sparkSession (SparkSession): The current spark session
    +    :rtype dict: The OpenCV type mapping supported
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    --- End diff --
    
    the tests fail otherwise - if there are multiple spark contexts during tests (when tests are run in parallel), it seems one of the other ones gets picked up, and when it is stopped the code fails with error stating that it cannot use closed spark context


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r144857345
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,256 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC2" -> 8, "CV_8UC3" -> 16, "CV_8UC4" -> 24,
    +    "CV_8S" -> 1, "CV_8SC1" -> 1, "CV_8SC2" -> 9, "CV_8SC3" -> 17, "CV_8SC4" -> 25,
    +    "CV_16U" -> 2, "CV_16UC1" -> 2, "CV_16UC2" -> 10, "CV_16UC3" -> 18, "CV_16UC4" -> 26,
    +    "CV_16S" -> 3, "CV_16SC1" -> 3, "CV_16SC2" -> 11, "CV_16SC3" -> 19, "CV_16SC4" -> 27,
    +    "CV_32S" -> 4, "CV_32SC1" -> 4, "CV_32SC2" -> 12, "CV_32SC3" -> 20, "CV_32SC4" -> 28,
    +    "CV_32F" -> 5, "CV_32FC1" -> 5, "CV_32FC2" -> 13, "CV_32FC3" -> 21, "CV_32FC4" -> 29,
    +    "CV_64F" -> 6, "CV_64FC1" -> 6, "CV_64FC2" -> 14, "CV_64FC3" -> 22, "CV_64FC4" -> 30
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField("origin", StringType, true) ::
    +    StructField("height", IntegerType, false) ::
    +    StructField("width", IntegerType, false) ::
    +    StructField("nChannels", IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField("mode", StringType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField("data", BinaryType, false) :: Nil)
    +
    +  // DataFrame with a single column of images named "image" (nullable)
    +  private val imageDFSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the height of the image
    +   *
    +   * @return The height of the image
    +   */
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the width of the image
    +   *
    +   * @return The width of the image
    +   */
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the number of channels in the image
    +   *
    +   * @return The number of channels in the image
    +   */
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the OpenCV representation
    +   *
    +   * @return The OpenCV representation
    +   */
    +  def getMode(row: Row): String = row.getString(4)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the image data
    +   *
    +   * @return The image data
    +   */
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * :: Experimental ::
    +   * Check if the DataFrame column contains images (i.e. has ImageSchema)
    +   *
    +   * @param df       DataFrame
    +   * @param column   Column name
    +   * @return         True if the given column matches the image schema
    +   */
    +  def isImageColumn(df: DataFrame, column: String): Boolean =
    +    df.schema(column).dataType == columnSchema
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return       Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row = Row(Row(origin, -1, -1, -1, undefinedImageType,
    +    Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in DataFrame Row
    +   *
    +   * @param origin   Arbitrary string that identifies the image
    +   * @param bytes    Image bytes (for example, jpeg)
    +   * @return         DataFrame Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val isGray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val hasAlpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) = if (isGray) {
    +        (1, "CV_8UC1")
    +      } else if (hasAlpha) {
    +        (4, "CV_8UC4")
    +      } else {
    +        (3, "CV_8UC3")
    +      }
    +
    +      val imageSize = height * width * nChannels
    +      assert(imageSize < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](imageSize)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (isGray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      } else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    +            if (nChannels == 4) {
    +              decoded(offset + 3) = color.getAlpha.toByte
    +            }
    +            offset += nChannels
    +          }
    +        }
    +      }
    +
    +      // the internal "Row" is needed, because the image is a single DataFrame column
    +      Some(Row(Row(origin, height, width, nChannels, mode, decoded)))
    +    }
    +  }
    +
    +  /**
    +   * :: Experimental ::
    +   * Read the directory of images from the local or remote source
    +   *
    +   * @param path               Path to the image directory
    +   * @param sparkSession       Spark Session
    +   * @param recursive          Recursive path search flag
    +   * @param numPartitions      Number of the DataFrame partitions
    +   * @param dropImageFailures  Drop the files that are not valid images from the result
    +   * @param sampleRatio        Fraction of the files loaded
    +   * @return                   DataFrame with a single column "image" of images;
    +   *                           see ImageSchema for the details
    +   */
    +  def readImages(path: String,
    +                 sparkSession: SparkSession = null,
    +                 recursive: Boolean = false,
    +                 numPartitions: Int = 0,
    +                 dropImageFailures: Boolean = false,
    +                 sampleRatio: Double = 1.0): DataFrame = {
    +    require(sampleRatio <= 1.0 && sampleRatio >= 0, "sampleRatio should be between 0 and 1")
    +
    +    val session = if (sparkSession != null) sparkSession else SparkSession.builder().getOrCreate
    +    val partitions =
    +      if (numPartitions > 0) {
    +        numPartitions
    +      } else {
    +        session.sparkContext.defaultParallelism
    +      }
    +
    +    val oldRecursiveFlag = RecursiveFlag.setRecursiveFlag(Some(recursive.toString), session)
    +    val oldPathFilter: Option[Class[_]] =
    +      if (sampleRatio < 1) {
    +        SamplePathFilter.setPathFilter(Some(classOf[SamplePathFilter]), sampleRatio, session)
    +      } else {
    +        None
    +      }
    +
    +    val result: DataFrame =
    --- End diff --
    
    removed return type


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r148909027
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,192 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +"""
    +.. attribute:: ImageSchema
    +
    +    A singleton-like attribute of :class:`_ImageSchema` in this module.
    --- End diff --
    
    Thanks for the thoughts!  Keeping it as is sounds good to me.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #84100 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84100/testReport)** for PR 19439 at commit [`aa3c704`](https://github.com/apache/spark/commit/aa3c704918d4b5585ac00771741fd7e86548a215).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r143689721
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,217 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.Since
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC2" -> 8, "CV_8UC3" -> 16, "CV_8UC4" -> 24,
    +    "CV_8S" -> 1, "CV_8SC1" -> 1, "CV_8SC2" -> 9, "CV_8SC3" -> 17, "CV_8SC4" -> 25,
    +    "CV_16U" -> 2, "CV_16UC1" -> 2, "CV_16UC2" -> 10, "CV_16UC3" -> 18, "CV_16UC4" -> 26,
    +    "CV_16S" -> 3, "CV_16SC1" -> 3, "CV_16SC2" -> 11, "CV_16SC3" -> 19, "CV_16SC4" -> 27,
    +    "CV_32S" -> 4, "CV_32SC1" -> 4, "CV_32SC2" -> 12, "CV_32SC3" -> 20, "CV_32SC4" -> 28,
    +    "CV_32F" -> 5, "CV_32FC1" -> 5, "CV_32FC2" -> 13, "CV_32FC3" -> 21, "CV_32FC4" -> 29,
    +    "CV_64F" -> 6, "CV_64FC1" -> 6, "CV_64FC2" -> 14, "CV_64FC3" -> 22, "CV_64FC4" -> 30
    +  )
    +
    +  /** Schema for the image column: Row(String, Int, Int, Int, Array[Byte]) */
    +  val columnSchema = StructType(
    +    StructField("origin", StringType, true) ::
    +      StructField("height", IntegerType, false) ::
    +      StructField("width", IntegerType, false) ::
    +      StructField("nChannels", IntegerType, false) ::
    +      // OpenCV-compatible type: CV_8UC3 in most cases
    +      StructField("mode", StringType, false) ::
    +      // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +      StructField("data", BinaryType, false) :: Nil)
    +
    +  // Dataframe with a single column of images named "image" (nullable)
    +  private val imageDFSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  @Since("2.3.0")
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  @Since("2.3.0")
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  @Since("2.3.0")
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  @Since("2.3.0")
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  @Since("2.3.0")
    +  def getMode(row: Row): String = row.getString(4)
    +
    +  @Since("2.3.0")
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /** Check if the dataframe column contains images (i.e. has ImageSchema)
    +   *
    +   * @param df       Dataframe
    +   * @param column   Column name
    +   * @return         True if the given column matches the image schema
    +   */
    +  @Since("2.3.0")
    +  def isImageColumn(df: DataFrame, column: String): Boolean =
    +    df.schema(column).dataType == columnSchema
    +
    +  /** Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return       Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row = Row(Row(origin, -1, -1, -1, undefinedImageType,
    +    Array.ofDim[Byte](0)))
    +
    +  /** Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in dataframe Row
    +   *
    +   * @param origin   Arbitrary string that identifies the image
    +   * @param bytes    Image bytes (for example, jpeg)
    +   * @return         Dataframe Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val is_gray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val has_alpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) =
    +        if (is_gray) (1, "CV_8UC1")
    +        else if (has_alpha) (4, "CV_8UC4")
    +        else (3, "CV_8UC3")
    +
    +      assert(height*width*nChannels < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](height*width*nChannels)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (is_gray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for(h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      }
    +      else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    +            if (nChannels == 4) {
    +              decoded(offset + 3) = color.getAlpha.toByte
    +            }
    +            offset += nChannels
    +          }
    +        }
    +      }
    +
    +      // the internal "Row" is needed, because the image is a single dataframe column
    +      Some(Row(Row(origin, height, width, nChannels, mode, decoded)))
    +    }
    +  }
    +
    +  /** Read the directory of images from the local or remote source
    +   *
    +   * @param path               Path to the image directory
    +   * @param sparkSession       Spark Session
    +   * @param recursive          Recursive path search flag
    +   * @param numPartitions      Number of the dataframe partitions
    +   * @param dropImageFailures  Drop the files that are not valid images from the result
    +   * @param sampleRatio        Fraction of the files loaded
    +   * @return                   Dataframe with a single column "image" of images;
    +   *                           see ImageSchema for the details
    +   */
    +  @Since("2.3.0")
    +  def readImages(path: String,
    +                 sparkSession: SparkSession = null,
    +                 recursive: Boolean = false,
    +                 numPartitions: Int = 0,
    +                 dropImageFailures: Boolean = false,
    +                 sampleRatio: Double = 1.0): DataFrame = {
    +    require(sampleRatio <= 1.0 && sampleRatio >= 0, "sampleRatio should be between 0 and 1")
    +
    +    val session = if (sparkSession != null) sparkSession else SparkSession.builder().getOrCreate
    +    val partitions =
    +      if (numPartitions > 0) numPartitions
    +      else session.sparkContext.defaultParallelism
    +
    +    val oldRecursiveFlag = RecursiveFlag.setRecursiveFlag(Some(recursive.toString), session)
    +    val oldPathFilter: Option[Class[_]] =
    +      if (sampleRatio < 1) {
    +        SamplePathFilter.setPathFilter(Some(classOf[SamplePathFilter]), Some(sampleRatio), session)
    +      }
    +      else {
    +        None
    +      }
    +
    +    var result: DataFrame = null
    +    try {
    +      val streams = session.sparkContext.binaryFiles(path, partitions)
    +        .repartition(partitions)
    +
    +      val images = if (dropImageFailures) {
    +        streams.flatMap{
    +          case (origin, stream) => decode(origin, stream.toArray)
    +        }
    +      }
    +      else {
    --- End diff --
    
    ```scala
    }
    else {
    ```
    to
    ```scala
    } else {
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by MrBago <gi...@git.apache.org>.
Github user MrBago commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r147298609
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,122 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +import pyspark
    +from pyspark import SparkContext
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame
    +from pyspark.ml.param.shared import *
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +imageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +ocvTypes = {
    +    undefinedImageType: -1,
    +    "CV_8U": 0, "CV_8UC1": 0, "CV_8UC2": 8, "CV_8UC3": 16, "CV_8UC4": 24,
    +    "CV_8S": 1, "CV_8SC1": 1, "CV_8SC2": 9, "CV_8SC3": 17, "CV_8SC4": 25,
    +    "CV_16U": 2, "CV_16UC1": 2, "CV_16UC2": 10, "CV_16UC3": 18, "CV_16UC4": 26,
    +    "CV_16S": 3, "CV_16SC1": 3, "CV_16SC2": 11, "CV_16SC3": 19, "CV_16SC4": 27,
    +    "CV_32S": 4, "CV_32SC1": 4, "CV_32SC2": 12, "CV_32SC3": 20, "CV_32SC4": 28,
    +    "CV_32F": 5, "CV_32FC1": 5, "CV_32FC2": 13, "CV_32FC3": 21, "CV_32FC4": 29,
    +    "CV_64F": 6, "CV_64FC1": 6, "CV_64FC2": 14, "CV_64FC3": 22, "CV_64FC4": 30
    +}
    +
    +# DataFrame with a single column of images named "image" (nullable)
    +imageSchema = StructType(StructField("image", StructType([
    +    StructField(imageFields[0], StringType(),  True),
    +    StructField(imageFields[1], IntegerType(), False),
    +    StructField(imageFields[2], IntegerType(), False),
    +    StructField(imageFields[3], IntegerType(), False),
    +    # OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(imageFields[4], StringType(), False),
    +    # bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(imageFields[5], BinaryType(), False)]), True))
    +
    +
    +def toNDArray(image):
    +    """
    +    Converts an image to a one-dimensional array.
    +
    +    :param image (object): The image to be converted
    +    :rtype array: The image as a one-dimensional array
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    height = image.height
    +    width = image.width
    +    nChannels = image.nChannels
    +    return np.ndarray(
    +        shape=(height, width, nChannels),
    +        dtype=np.uint8,
    --- End diff --
    
    Can we add a check in this function, something like `assert image.mode in [0, 8, 16, 24]` (I believe that maps to "CV_8UC[1-4]").


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r145490296
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/HadoopUtils.scala ---
    @@ -0,0 +1,119 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import scala.language.existentials
    +import scala.util.Random
    +
    +import org.apache.commons.io.FilenameUtils
    +import org.apache.hadoop.conf.{Configuration, Configured}
    +import org.apache.hadoop.fs.{Path, PathFilter}
    +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
    +
    +import org.apache.spark.sql.SparkSession
    +
    +private object RecursiveFlag {
    +
    +  /**
    +   * Sets a value of spark recursive flag.
    +   * If value is a None, it unsets the flag.
    +   *
    +   * @param value value to set
    +   * @param spark existing spark session
    +   * @return previous value of this flag
    +   */
    +  def setRecursiveFlag(value: Option[String], spark: SparkSession): Option[String] = {
    +    val flagName = FileInputFormat.INPUT_DIR_RECURSIVE
    +    val hadoopConf = spark.sparkContext.hadoopConfiguration
    +    val old = Option(hadoopConf.get(flagName))
    +
    +    value match {
    +      case Some(v) => hadoopConf.set(flagName, v)
    +      case None => hadoopConf.unset(flagName)
    +    }
    +
    +    old
    +  }
    +}
    +
    +/**
    + * Filter that allows loading a fraction of HDFS files.
    + */
    +private class SamplePathFilter extends Configured with PathFilter {
    +  val random = new Random()
    +
    +  // Ratio of files to be read from disk
    +  var sampleRatio: Double = 1
    +
    +  override def setConf(conf: Configuration): Unit = {
    +    if (conf != null) {
    +      sampleRatio = conf.getDouble(SamplePathFilter.ratioParam, 1)
    +    }
    +  }
    +
    +  override def accept(path: Path): Boolean = {
    +    // Note: checking fileSystem.isDirectory is very slow here, so we use basic rules instead
    +    !SamplePathFilter.isFile(path) || random.nextDouble() < sampleRatio
    +  }
    +}
    +
    +private object SamplePathFilter {
    +  val ratioParam = "sampleRatio"
    +
    +  def isFile(path: Path): Boolean = FilenameUtils.getExtension(path.toString) != ""
    +
    +  /**
    +   * Sets HDFS PathFilter
    +   *
    +   * @param value Filter class that is passed to HDFS
    +   * @param sampleRatio Fraction of the files that the filter picks
    +   * @param spark Existing Spark session
    +   * @return Returns the previous HDFS path filter
    +   */
    +  def setPathFilter(value: Option[Class[_]],
    +                    sampleRatio: Double,
    +                    spark: SparkSession): Option[Class[_]] = {
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    I just noticed: Where is data/mllib/images/kittens/DP153539.jpg from?  (It's missing in the license list.)


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r144190368
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,133 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +import pyspark
    +from pyspark import SparkContext
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame
    +from pyspark.ml.param.shared import *
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +ImageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +ocvTypes = {
    +    undefinedImageType: -1,
    +    "CV_8U": 0, "CV_8UC1": 0, "CV_8UC2": 8, "CV_8UC3": 16, "CV_8UC4": 24,
    +    "CV_8S": 1, "CV_8SC1": 1, "CV_8SC2": 9, "CV_8SC3": 17, "CV_8SC4": 25,
    +    "CV_16U": 2, "CV_16UC1": 2, "CV_16UC2": 10, "CV_16UC3": 18, "CV_16UC4": 26,
    +    "CV_16S": 3, "CV_16SC1": 3, "CV_16SC2": 11, "CV_16SC3": 19, "CV_16SC4": 27,
    +    "CV_32S": 4, "CV_32SC1": 4, "CV_32SC2": 12, "CV_32SC3": 20, "CV_32SC4": 28,
    +    "CV_32F": 5, "CV_32FC1": 5, "CV_32FC2": 13, "CV_32FC3": 21, "CV_32FC4": 29,
    +    "CV_64F": 6, "CV_64FC1": 6, "CV_64FC2": 14, "CV_64FC3": 22, "CV_64FC4": 30
    +}
    +
    +ImageSchema = StructType([
    +    StructField(ImageFields[0], StringType(),  True),
    +    StructField(ImageFields[1], IntegerType(), False),
    +    StructField(ImageFields[2], IntegerType(), False),
    +    StructField(ImageFields[3], IntegerType(), False),
    +    # OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(ImageFields[4], StringType(), False),
    +    # bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(ImageFields[5], BinaryType(), False)])
    +
    +
    +# TODO: generalize to other datatypes and number of channels
    +def toNDArray(image):
    +    """
    +    Converts an image to a 1-dimensional array
    +
    +    Args:
    +        image (object): The image to be converted
    +
    +    Returns:
    +        array: The image as a 1-dimensional array
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    height = image.height
    +    width = image.width
    +    return np.asarray(image.data, dtype=np.uint8) \
    +             .reshape((height, width, 3))[:, :, (2, 1, 0)]
    +
    +
    +# TODO: generalize to other datatypes and number of channels
    +def toImage(array, origin="", mode="CV_8UC3"):
    +    """
    +
    +    Converts a one-dimensional array to a 2 dimensional image
    +
    +    Args:
    +        array (array):
    +        origin (str):
    +        mode (int):
    +
    +    Returns:
    +        object: 2 dimensional image
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    length = np.prod(array.shape)
    +
    +    data = bytearray(array.astype(dtype=np.int8)[:, :, (2, 1, 0)]
    +                          .reshape(length))
    +    height = array.shape[0]
    +    width = array.shape[1]
    +    nChannels = array.shape[2]
    +    # Creating new Row with _create_row(), because Row(name = value, ... )
    --- End diff --
    
    @holdenk is @MrBago 's resolution reasonable?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #83218 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83218/testReport)** for PR 19439 at commit [`030fe38`](https://github.com/apache/spark/commit/030fe38a22ebb3052b5e7762cd7227ffbe2b7c1d).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/83672/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Jenkins retest this please


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r144298500
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/HadoopUtils.scala ---
    @@ -0,0 +1,119 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import scala.language.existentials
    +import scala.util.Random
    +
    +import org.apache.commons.io.FilenameUtils
    +import org.apache.hadoop.conf.{Configuration, Configured}
    +import org.apache.hadoop.fs.{Path, PathFilter}
    +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
    +
    +import org.apache.spark.sql.SparkSession
    +
    +private object RecursiveFlag {
    +
    +  /**
    +   * Sets a value of spark recursive flag.
    +   * If value is a None, it unsets the flag.
    +   *
    +   * @param value value to set
    +   * @param spark existing spark session
    +   * @return previous value of this flag
    +   */
    +  def setRecursiveFlag(value: Option[String], spark: SparkSession): Option[String] = {
    +    val flagName = FileInputFormat.INPUT_DIR_RECURSIVE
    +    val hadoopConf = spark.sparkContext.hadoopConfiguration
    +    val old = Option(hadoopConf.get(flagName))
    +
    +    value match {
    +      case Some(v) => hadoopConf.set(flagName, v)
    +      case None => hadoopConf.unset(flagName)
    +    }
    +
    +    old
    +  }
    +}
    +
    +/**
    + * Filter that allows loading a fraction of HDFS files.
    + */
    +private class SamplePathFilter extends Configured with PathFilter {
    +  val random = new Random()
    +
    +  // Ratio of files to be read from disk
    +  var sampleRatio: Double = 1
    +
    +  override def setConf(conf: Configuration): Unit = {
    +    if (conf != null) {
    +      sampleRatio = conf.getDouble(SamplePathFilter.ratioParam, 1)
    +    }
    +  }
    +
    +  override def accept(path: Path): Boolean = {
    +    // Note: checking fileSystem.isDirectory is very slow here, so we use basic rules instead
    +    !SamplePathFilter.isFile(path) ||
    +      random.nextDouble() < sampleRatio
    +  }
    +}
    +
    +private object SamplePathFilter {
    +  val ratioParam = "sampleRatio"
    +
    +  def isFile(path: Path): Boolean = FilenameUtils.getExtension(path.toString) != ""
    +
    +  /**
    +   * Sets HDFS PathFilter
    +   *
    +   * @param value       Filter class that is passed to HDFS
    +   * @param sampleRatio Fraction of the files that the filter picks
    +   * @param spark       Existing Spark session
    +   * @return            Returns the previous HDFS path filter
    +   */
    +  def setPathFilter(value: Option[Class[_]], sampleRatio: Double,
    +                    spark: SparkSession) : Option[Class[_]] = {
    +    val flagName = FileInputFormat.PATHFILTER_CLASS
    +    val hadoopConf = spark.sparkContext.hadoopConfiguration
    +    val old = Option(hadoopConf.getClass(flagName, null))
    +    hadoopConf.setDouble(SamplePathFilter.ratioParam, sampleRatio)
    +
    +    value match {
    +      case Some(v) => hadoopConf.setClass(flagName, v, classOf[PathFilter])
    +      case None => hadoopConf.unset(flagName)
    +    }
    +    old
    +  }
    +
    +  /**
    +   * Unsets HDFS PathFilter
    +   *
    +   * @param value       Filter class to restore to HDFS
    +   * @param spark       Existing Spark session
    --- End diff --
    
    nit:
    
    ```scala
    @param value Filter class to restore to HDFS
    @param spark Existing Spark session
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/82889/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r147756618
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,140 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +from pyspark.ml.util import *
    +from pyspark.ml.param.shared import *
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame, SparkSession, SQLContext
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +imageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +
    +def getOcvTypes(spark=None):
    +    """
    +    Returns the OpenCV type mapping supported
    +
    +    :param sparkSession (SparkSession): The current spark session
    +    :rtype dict: The OpenCV type mapping supported
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.ocvTypes
    +
    +
    +# DataFrame with a single column of images named "image" (nullable)
    +def getImageSchema(spark=None):
    +    """
    +    Returns the image schema
    +
    +    :param spark (SparkSession): The current spark session
    +    :rtype StructType: The image schema
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.imageSchema
    +
    +
    +def toNDArray(image):
    +    """
    +    Converts an image to a one-dimensional array.
    +
    +    :param image (object): The image to be converted
    +    :rtype array: The image as a one-dimensional array
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    height = image.height
    +    width = image.width
    +    nChannels = image.nChannels
    +    return np.ndarray(
    +        shape=(height, width, nChannels),
    +        dtype=np.uint8,
    +        buffer=image.data,
    +        strides=(width * nChannels, nChannels, 1))
    +
    +
    +def toImage(array, origin="", spark=None):
    +    """
    +    Converts a one-dimensional array to a two-dimensional image.
    +
    +    :param array (array): The array to convert to image
    +    :param origin (str): Path to the image
    +    :param mode (str): OpenCV compatible type
    +    :param spark (SparkSession): The current spark session
    +    :rtype object: Two dimensional image
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    if array.ndim != 3:
    +        raise
    +    height, width, nChannels = array.shape
    +    ocvTypes = getOcvTypes(spark)
    +    if nChannels == 1:
    +        mode = ocvTypes["CV_8UC1"]
    +    elif nChannels == 3:
    +        mode = ocvTypes["CV_8UC3"]
    +    elif nChannels == 4:
    +        mode = ocvTypes["CV_8UC4"]
    +    else:
    +        raise
    +    data = bytearray(array.astype(dtype=np.uint8).ravel())
    +    # Creating new Row with _create_row(), because Row(name = value, ... )
    +    # orders fields by name, which conflicts with expected schema order
    +    # when the new DataFrame is created by UDF
    +    return _create_row(imageFields,
    +                       [origin, height, width, nChannels, mode, data])
    +
    +
    +def readImages(path, recursive=False, numPartitions=0,
    +               dropImageFailures=False, sampleRatio=1.0, spark=None):
    +    """
    +    Reads the directory of images from the local or remote source.
    +
    +    :param path (str): Path to the image directory
    +    :param recursive (bool): Recursive search flag
    +    :param numPartitions (int): Number of DataFrame partitions
    +    :param dropImageFailures (bool): Drop the files that are not valid images
    +    :param sampleRatio (double): Fraction of the images loaded
    +    :param spark (SparkSession): The current spark session
    +    :rtype DataFrame: DataFrame with a single column of "images",
    +           see ImageSchema for details
    +
    +    Examples:
    +
    +    >>> df = readImages('python/test_support/image/kittens', recursive=True)
    +    >>> df.count
    +    4
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    schema = ctx._jvm.org.apache.spark.ml.image.ImageSchema
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r144190588
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/HadoopUtils.scala ---
    @@ -0,0 +1,122 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import scala.language.existentials
    +import scala.util.Random
    +
    +import org.apache.commons.io.FilenameUtils
    +import org.apache.hadoop.conf.{Configuration, Configured}
    +import org.apache.hadoop.fs.{Path, PathFilter}
    +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
    +
    +import org.apache.spark.sql.SparkSession
    +
    +private object RecursiveFlag {
    +
    +  /**
    +   * Sets a value of spark recursive flag
    +   *
    +   * @param value value to set
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r148429895
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,236 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields: Array[String] = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes: Map[String, Int] = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC3" -> 16, "CV_8UC4" -> 24
    +  )
    +
    +  /**
    +   * Used for conversion to python
    +   */
    +  val _ocvTypes: java.util.Map[String, Int] = ocvTypes.asJava
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField(imageFields(0), StringType, true) ::
    +    StructField(imageFields(1), IntegerType, false) ::
    +    StructField(imageFields(2), IntegerType, false) ::
    +    StructField(imageFields(3), IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(imageFields(4), IntegerType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(imageFields(5), BinaryType, false) :: Nil)
    +
    +  /**
    +   * DataFrame with a single column of images named "image" (nullable)
    +   */
    +  val imageSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the height of the image
    +   *
    +   * @return The height of the image
    +   */
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the width of the image
    +   *
    +   * @return The width of the image
    +   */
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the number of channels in the image
    +   *
    +   * @return The number of channels in the image
    +   */
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the OpenCV representation as an int
    +   *
    +   * @return The OpenCV representation as an int
    +   */
    +  def getMode(row: Row): Int = row.getInt(4)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the image data
    +   *
    +   * @return The image data
    +   */
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row =
    +    Row(Row(origin, -1, -1, -1, ocvTypes(undefinedImageType), Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in DataFrame Row
    +   *
    +   * @param origin Arbitrary string that identifies the image
    +   * @param bytes Image bytes (for example, jpeg)
    +   * @return DataFrame Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val isGray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val hasAlpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) = if (isGray) {
    +        (1, ocvTypes("CV_8UC1"))
    +      } else if (hasAlpha) {
    +        (4, ocvTypes("CV_8UC4"))
    +      } else {
    +        (3, ocvTypes("CV_8UC3"))
    +      }
    +
    +      val imageSize = height * width * nChannels
    +      assert(imageSize < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](imageSize)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (isGray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      } else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    +            if (nChannels == 4) {
    +              decoded(offset + 3) = color.getAlpha.toByte
    +            }
    +            offset += nChannels
    +          }
    +        }
    +      }
    +
    +      // the internal "Row" is needed, because the image is a single DataFrame column
    +      Some(Row(Row(origin, height, width, nChannels, mode, decoded)))
    +    }
    +  }
    +
    +  /**
    +   * :: Experimental ::
    +   * Read the directory of images from the local or remote source
    +   *
    +   * @param path Path to the image directory
    +   * @param sparkSession Spark Session, if omitted gets or creates the session
    +   * @param recursive Recursive path search flag
    +   * @param numPartitions Number of the DataFrame partitions,
    +   *                      if omitted uses defaultParallelism instead
    +   * @param dropImageFailures Drop the files that are not valid images from the result
    +   * @param sampleRatio Fraction of the files loaded
    +   * @return DataFrame with a single column "image" of images;
    +   *         see ImageSchema for the details
    +   */
    +  def readImages(
    +      path: String,
    +      sparkSession: SparkSession = null,
    --- End diff --
    
    BTW, default values do not work in Java API IIRC - [scala-style-guide#default-parameter-values](https://github.com/databricks/scala-style-guide#default-parameter-values).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/82591/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r150375473
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,196 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +"""
    +.. attribute:: ImageSchema
    +
    +    An attribute of :class:`_ImageSchema` in this module.
    +
    +.. autoclass:: _ImageSchema
    +   :members:
    +"""
    +
    +import numpy as np
    +from pyspark import SparkContext
    +from pyspark.sql.types import Row, _create_row, _parse_datatype_json_string
    +from pyspark.sql import DataFrame, SparkSession
    +
    +
    +class _ImageSchema(object):
    +    """
    +    Internal class for `pyspark.ml.image.ImageSchema` attribute. Meant to be private and
    +    not to be instantized. Use `pyspark.ml.image.ImageSchema` attribute to access the
    +    APIs of this class.
    +    """
    +
    +    def __init__(self):
    +        self._imageSchema = None
    +        self._ocvTypes = None
    +        self._imageFields = None
    +        self._undefinedImageType = None
    +
    +    @property
    +    def imageSchema(self):
    +        """
    +        Returns the image schema.
    +
    +        :rtype StructType: a DataFrame with a single column of images
    +               named "image" (nullable)
    +
    +        .. versionadded:: 2.3.0
    +        """
    +
    +        if self._imageSchema is None:
    +            ctx = SparkContext._active_spark_context
    +            jschema = ctx._jvm.org.apache.spark.ml.image.ImageSchema.imageSchema()
    +            self._imageSchema = _parse_datatype_json_string(jschema.json())
    +        return self._imageSchema
    +
    +    @property
    +    def ocvTypes(self):
    +        """
    +        Returns the OpenCV type mapping supported
    +
    +        :rtype dict: The OpenCV type mapping supported
    +
    +        .. versionadded:: 2.3.0
    +        """
    +
    +        if self._ocvTypes is None:
    +            ctx = SparkContext._active_spark_context
    +            self._ocvTypes = dict(ctx._jvm.org.apache.spark.ml.image.ImageSchema.javaOcvTypes())
    +        return self._ocvTypes
    +
    +    @property
    +    def imageFields(self):
    +        """
    +        Returns field names of image columns.
    +
    +        :rtype list: a list of field names.
    +
    +        .. versionadded:: 2.3.0
    +        """
    +
    +        if self._imageFields is None:
    +            ctx = SparkContext._active_spark_context
    +            self._imageFields = list(ctx._jvm.org.apache.spark.ml.image.ImageSchema.imageFields())
    +        return self._imageFields
    +
    +    @property
    +    def undefinedImageType(self):
    +        """
    +        Returns the name of undefined image type for the invalid image.
    +
    +        .. versionadded:: 2.3.0
    +        """
    +
    +        if self._undefinedImageType is None:
    +            ctx = SparkContext._active_spark_context
    +            self._undefinedImageType = \
    +                ctx._jvm.org.apache.spark.ml.image.ImageSchema.undefinedImageType()
    +        return self._undefinedImageType
    +
    +    def toNDArray(self, image):
    +        """
    +        Converts an image to a one-dimensional array.
    +
    +        :param image: The image to be converted
    +        :rtype array: The image as a one-dimensional array
    +
    +        .. versionadded:: 2.3.0
    +        """
    +
    +        height = image.height
    +        width = image.width
    +        nChannels = image.nChannels
    +        return np.ndarray(
    +            shape=(height, width, nChannels),
    +            dtype=np.uint8,
    +            buffer=image.data,
    +            strides=(width * nChannels, nChannels, 1))
    +
    +    def toImage(self, array, origin=""):
    +        """
    +        Converts an array with metadata to a two-dimensional image.
    +
    +        :param array array: The array to convert to image
    +        :param str origin: Path to the image, optional
    +        :rtype object: Two dimensional image
    +
    +        .. versionadded:: 2.3.0
    +        """
    +
    +        if array.ndim != 3:
    +            raise ValueError("Invalid array shape")
    +        height, width, nChannels = array.shape
    +        ocvTypes = ImageSchema.ocvTypes
    +        if nChannels == 1:
    +            mode = ocvTypes["CV_8UC1"]
    +        elif nChannels == 3:
    +            mode = ocvTypes["CV_8UC3"]
    +        elif nChannels == 4:
    +            mode = ocvTypes["CV_8UC4"]
    +        else:
    +            raise ValueError("Invalid number of channels")
    +        data = bytearray(array.astype(dtype=np.uint8).ravel())
    +        # Creating new Row with _create_row(), because Row(name = value, ... )
    +        # orders fields by name, which conflicts with expected schema order
    +        # when the new DataFrame is created by UDF
    +        return _create_row(self.imageFields,
    +                           [origin, height, width, nChannels, mode, data])
    +
    +    def readImages(self, path, recursive=False, numPartitions=-1,
    +                   dropImageFailures=False, sampleRatio=1.0, seed=0):
    +        """
    +        Reads the directory of images from the local or remote source.
    +
    +        WARNINGS:
    +          - If multiple jobs are run in parallel with different sampleRatio or recursive flag,
    +            there may be a race condition where one job overwrites the hadoop configs of another.
    --- End diff --
    
    Let's use `.. note::`:
    
    ```
    .. note:: If multiple jobs are run in parallel with different sampleRatio or recursive flag,
        there may be a race condition where one job overwrites the hadoop configs of another.
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r143924621
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC2" -> 8, "CV_8UC3" -> 16, "CV_8UC4" -> 24,
    +    "CV_8S" -> 1, "CV_8SC1" -> 1, "CV_8SC2" -> 9, "CV_8SC3" -> 17, "CV_8SC4" -> 25,
    +    "CV_16U" -> 2, "CV_16UC1" -> 2, "CV_16UC2" -> 10, "CV_16UC3" -> 18, "CV_16UC4" -> 26,
    +    "CV_16S" -> 3, "CV_16SC1" -> 3, "CV_16SC2" -> 11, "CV_16SC3" -> 19, "CV_16SC4" -> 27,
    +    "CV_32S" -> 4, "CV_32SC1" -> 4, "CV_32SC2" -> 12, "CV_32SC3" -> 20, "CV_32SC4" -> 28,
    +    "CV_32F" -> 5, "CV_32FC1" -> 5, "CV_32FC2" -> 13, "CV_32FC3" -> 21, "CV_32FC4" -> 29,
    +    "CV_64F" -> 6, "CV_64FC1" -> 6, "CV_64FC2" -> 14, "CV_64FC3" -> 22, "CV_64FC4" -> 30
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField("origin", StringType, true) ::
    +      StructField("height", IntegerType, false) ::
    +      StructField("width", IntegerType, false) ::
    +      StructField("nChannels", IntegerType, false) ::
    +      // OpenCV-compatible type: CV_8UC3 in most cases
    +      StructField("mode", StringType, false) ::
    +      // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +      StructField("data", BinaryType, false) :: Nil)
    +
    +  // Dataframe with a single column of images named "image" (nullable)
    +  private val imageDFSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  @Since("2.3.0")
    --- End diff --
    
    Nit: if the whole class is since 2.3.0, then implicitly everything in it is from at least 2.3.0 or later. You don't need these.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r143925260
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC2" -> 8, "CV_8UC3" -> 16, "CV_8UC4" -> 24,
    +    "CV_8S" -> 1, "CV_8SC1" -> 1, "CV_8SC2" -> 9, "CV_8SC3" -> 17, "CV_8SC4" -> 25,
    +    "CV_16U" -> 2, "CV_16UC1" -> 2, "CV_16UC2" -> 10, "CV_16UC3" -> 18, "CV_16UC4" -> 26,
    +    "CV_16S" -> 3, "CV_16SC1" -> 3, "CV_16SC2" -> 11, "CV_16SC3" -> 19, "CV_16SC4" -> 27,
    +    "CV_32S" -> 4, "CV_32SC1" -> 4, "CV_32SC2" -> 12, "CV_32SC3" -> 20, "CV_32SC4" -> 28,
    +    "CV_32F" -> 5, "CV_32FC1" -> 5, "CV_32FC2" -> 13, "CV_32FC3" -> 21, "CV_32FC4" -> 29,
    +    "CV_64F" -> 6, "CV_64FC1" -> 6, "CV_64FC2" -> 14, "CV_64FC3" -> 22, "CV_64FC4" -> 30
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField("origin", StringType, true) ::
    +      StructField("height", IntegerType, false) ::
    +      StructField("width", IntegerType, false) ::
    +      StructField("nChannels", IntegerType, false) ::
    +      // OpenCV-compatible type: CV_8UC3 in most cases
    +      StructField("mode", StringType, false) ::
    +      // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +      StructField("data", BinaryType, false) :: Nil)
    +
    +  // Dataframe with a single column of images named "image" (nullable)
    +  private val imageDFSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  @Since("2.3.0")
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  @Since("2.3.0")
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  @Since("2.3.0")
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  @Since("2.3.0")
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  @Since("2.3.0")
    +  def getMode(row: Row): String = row.getString(4)
    +
    +  @Since("2.3.0")
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * Check if the dataframe column contains images (i.e. has ImageSchema)
    +   *
    +   * @param df       Dataframe
    +   * @param column   Column name
    +   * @return         True if the given column matches the image schema
    +   */
    +  @Since("2.3.0")
    +  def isImageColumn(df: DataFrame, column: String): Boolean =
    +    df.schema(column).dataType == columnSchema
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return       Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row = Row(Row(origin, -1, -1, -1, undefinedImageType,
    +    Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in dataframe Row
    +   *
    +   * @param origin   Arbitrary string that identifies the image
    +   * @param bytes    Image bytes (for example, jpeg)
    +   * @return         Dataframe Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val is_gray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val has_alpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) = if (is_gray) {
    +        (1, "CV_8UC1")
    +      } else if (has_alpha) {
    +        (4, "CV_8UC4")
    +      } else {
    +        (3, "CV_8UC3")
    +      }
    +
    +      assert(height * width * nChannels < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](height*width*nChannels)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (is_gray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for(h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      }
    +      else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    +            if (nChannels == 4) {
    +              decoded(offset + 3) = color.getAlpha.toByte
    +            }
    +            offset += nChannels
    +          }
    +        }
    +      }
    +
    +      // the internal "Row" is needed, because the image is a single dataframe column
    +      Some(Row(Row(origin, height, width, nChannels, mode, decoded)))
    +    }
    +  }
    +
    +  /**
    +   * :: Experimental ::
    +   * Read the directory of images from the local or remote source
    +   *
    +   * @param path               Path to the image directory
    +   * @param sparkSession       Spark Session
    +   * @param recursive          Recursive path search flag
    +   * @param numPartitions      Number of the dataframe partitions
    +   * @param dropImageFailures  Drop the files that are not valid images from the result
    +   * @param sampleRatio        Fraction of the files loaded
    +   * @return                   Dataframe with a single column "image" of images;
    +   *                           see ImageSchema for the details
    +   */
    +  @Since("2.3.0")
    +  def readImages(path: String,
    +                 sparkSession: SparkSession = null,
    +                 recursive: Boolean = false,
    +                 numPartitions: Int = 0,
    +                 dropImageFailures: Boolean = false,
    +                 sampleRatio: Double = 1.0): DataFrame = {
    +    require(sampleRatio <= 1.0 && sampleRatio >= 0, "sampleRatio should be between 0 and 1")
    +
    +    val session = if (sparkSession != null) sparkSession else SparkSession.builder().getOrCreate
    +    val partitions =
    +      if (numPartitions > 0) {
    +        numPartitions
    +      } else {
    +        session.sparkContext.defaultParallelism
    +      }
    +
    +    val oldRecursiveFlag = RecursiveFlag.setRecursiveFlag(Some(recursive.toString), session)
    +    val oldPathFilter: Option[Class[_]] =
    +      if (sampleRatio < 1) {
    +        SamplePathFilter.setPathFilter(Some(classOf[SamplePathFilter]), sampleRatio, session)
    +      } else {
    +        None
    +      }
    +
    +    var result: DataFrame = null
    --- End diff --
    
    I think this pattern might be clearer as `val result = try {...` rather than this Java style


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by MrBago <gi...@git.apache.org>.
Github user MrBago commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r143853274
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,133 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +import pyspark
    +from pyspark import SparkContext
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame
    +from pyspark.ml.param.shared import *
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +ImageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +ocvTypes = {
    +    undefinedImageType: -1,
    +    "CV_8U": 0, "CV_8UC1": 0, "CV_8UC2": 8, "CV_8UC3": 16, "CV_8UC4": 24,
    +    "CV_8S": 1, "CV_8SC1": 1, "CV_8SC2": 9, "CV_8SC3": 17, "CV_8SC4": 25,
    +    "CV_16U": 2, "CV_16UC1": 2, "CV_16UC2": 10, "CV_16UC3": 18, "CV_16UC4": 26,
    +    "CV_16S": 3, "CV_16SC1": 3, "CV_16SC2": 11, "CV_16SC3": 19, "CV_16SC4": 27,
    +    "CV_32S": 4, "CV_32SC1": 4, "CV_32SC2": 12, "CV_32SC3": 20, "CV_32SC4": 28,
    +    "CV_32F": 5, "CV_32FC1": 5, "CV_32FC2": 13, "CV_32FC3": 21, "CV_32FC4": 29,
    +    "CV_64F": 6, "CV_64FC1": 6, "CV_64FC2": 14, "CV_64FC3": 22, "CV_64FC4": 30
    +}
    +
    +ImageSchema = StructType([
    +    StructField(ImageFields[0], StringType(),  True),
    +    StructField(ImageFields[1], IntegerType(), False),
    +    StructField(ImageFields[2], IntegerType(), False),
    +    StructField(ImageFields[3], IntegerType(), False),
    +    # OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(ImageFields[4], StringType(), False),
    +    # bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(ImageFields[5], BinaryType(), False)])
    +
    +
    +# TODO: generalize to other datatypes and number of channels
    +def toNDArray(image):
    +    """
    +    Converts an image to a 1-dimensional array
    +
    +    Args:
    +        image (object): The image to be converted
    +
    +    Returns:
    +        array: The image as a 1-dimensional array
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    height = image.height
    +    width = image.width
    +    return np.asarray(image.data, dtype=np.uint8) \
    +             .reshape((height, width, 3))[:, :, (2, 1, 0)]
    +
    +
    +# TODO: generalize to other datatypes and number of channels
    +def toImage(array, origin="", mode="CV_8UC3"):
    +    """
    +
    +    Converts a one-dimensional array to a 2 dimensional image
    +
    +    Args:
    +        array (array):
    +        origin (str):
    +        mode (int):
    +
    +    Returns:
    +        object: 2 dimensional image
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    length = np.prod(array.shape)
    +
    +    data = bytearray(array.astype(dtype=np.int8)[:, :, (2, 1, 0)]
    +                          .reshape(length))
    +    height = array.shape[0]
    +    width = array.shape[1]
    +    nChannels = array.shape[2]
    +    # Creating new Row with _create_row(), because Row(name = value, ... )
    --- End diff --
    
    @holdenk I believe the ordered by name schema works in general, there is a serialization bug that I'm aware of, I filed it here, https://issues.apache.org/jira/browse/SPARK-22232 but I hope we can fix that for 2.3 (I can help with that).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r148695760
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    --- End diff --
    
    This is why we need first-class support for UDTs!  :)  If this will stay public, can you please add some Scaladoc to it and also review the fields to tighten the privacy (probably to ```private[ml]```) where reasonable?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/83344/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #82480 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82480/testReport)** for PR 19439 at commit [`22baf02`](https://github.com/apache/spark/commit/22baf022b2f109bb1f5eba0b13ea34de894cd14c).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r144189855
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,133 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +import pyspark
    +from pyspark import SparkContext
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame
    +from pyspark.ml.param.shared import *
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +ImageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +ocvTypes = {
    +    undefinedImageType: -1,
    +    "CV_8U": 0, "CV_8UC1": 0, "CV_8UC2": 8, "CV_8UC3": 16, "CV_8UC4": 24,
    +    "CV_8S": 1, "CV_8SC1": 1, "CV_8SC2": 9, "CV_8SC3": 17, "CV_8SC4": 25,
    +    "CV_16U": 2, "CV_16UC1": 2, "CV_16UC2": 10, "CV_16UC3": 18, "CV_16UC4": 26,
    +    "CV_16S": 3, "CV_16SC1": 3, "CV_16SC2": 11, "CV_16SC3": 19, "CV_16SC4": 27,
    +    "CV_32S": 4, "CV_32SC1": 4, "CV_32SC2": 12, "CV_32SC3": 20, "CV_32SC4": 28,
    +    "CV_32F": 5, "CV_32FC1": 5, "CV_32FC2": 13, "CV_32FC3": 21, "CV_32FC4": 29,
    +    "CV_64F": 6, "CV_64FC1": 6, "CV_64FC2": 14, "CV_64FC3": 22, "CV_64FC4": 30
    +}
    +
    +ImageSchema = StructType([
    +    StructField(ImageFields[0], StringType(),  True),
    +    StructField(ImageFields[1], IntegerType(), False),
    +    StructField(ImageFields[2], IntegerType(), False),
    +    StructField(ImageFields[3], IntegerType(), False),
    +    # OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(ImageFields[4], StringType(), False),
    +    # bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(ImageFields[5], BinaryType(), False)])
    +
    +
    +# TODO: generalize to other datatypes and number of channels
    +def toNDArray(image):
    +    """
    +    Converts an image to a 1-dimensional array
    +
    +    Args:
    +        image (object): The image to be converted
    +
    +    Returns:
    +        array: The image as a 1-dimensional array
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    height = image.height
    +    width = image.width
    +    return np.asarray(image.data, dtype=np.uint8) \
    +             .reshape((height, width, 3))[:, :, (2, 1, 0)]
    +
    +
    +# TODO: generalize to other datatypes and number of channels
    +def toImage(array, origin="", mode="CV_8UC3"):
    +    """
    +
    +    Converts a one-dimensional array to a 2 dimensional image
    +
    +    Args:
    +        array (array):
    +        origin (str):
    +        mode (int):
    +
    +    Returns:
    +        object: 2 dimensional image
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    length = np.prod(array.shape)
    +
    +    data = bytearray(array.astype(dtype=np.int8)[:, :, (2, 1, 0)]
    +                          .reshape(length))
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r147693080
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,252 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC3" -> 16, "CV_8UC4" -> 24
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField(imageFields(0), StringType, true) ::
    +    StructField(imageFields(1), IntegerType, false) ::
    +    StructField(imageFields(2), IntegerType, false) ::
    +    StructField(imageFields(3), IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(imageFields(4), IntegerType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(imageFields(5), BinaryType, false) :: Nil)
    +
    +  /**
    +   * DataFrame with a single column of images named "image" (nullable)
    +   */
    +  val imageSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the height of the image
    +   *
    +   * @return The height of the image
    +   */
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the width of the image
    +   *
    +   * @return The width of the image
    +   */
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the number of channels in the image
    +   *
    +   * @return The number of channels in the image
    +   */
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the OpenCV representation as an int
    +   *
    +   * @return The OpenCV representation as an int
    +   */
    +  def getMode(row: Row): Int = row.getInt(4)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the image data
    +   *
    +   * @return The image data
    +   */
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * :: Experimental ::
    +   * Check if the DataFrame column contains images (i.e. has ImageSchema)
    +   *
    +   * @param df DataFrame
    +   * @param column Column name
    +   * @return True if the given column matches the image schema
    +   */
    +  def isImageColumn(df: DataFrame, column: String): Boolean =
    +    df.schema(column).dataType == columnSchema
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row =
    +    Row(Row(origin, -1, -1, -1, ocvTypes(undefinedImageType), Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in DataFrame Row
    +   *
    +   * @param origin Arbitrary string that identifies the image
    +   * @param bytes Image bytes (for example, jpeg)
    +   * @return DataFrame Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val isGray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val hasAlpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) = if (isGray) {
    +        (1, ocvTypes("CV_8UC1"))
    +      } else if (hasAlpha) {
    +        (4, ocvTypes("CV_8UC4"))
    +      } else {
    +        (3, ocvTypes("CV_8UC3"))
    +      }
    +
    +      val imageSize = height * width * nChannels
    +      assert(imageSize < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](imageSize)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (isGray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      } else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    +            if (nChannels == 4) {
    +              decoded(offset + 3) = color.getAlpha.toByte
    +            }
    +            offset += nChannels
    +          }
    +        }
    +      }
    +
    +      // the internal "Row" is needed, because the image is a single DataFrame column
    +      Some(Row(Row(origin, height, width, nChannels, mode, decoded)))
    +    }
    +  }
    +
    +  /**
    +   * :: Experimental ::
    +   * Read the directory of images from the local or remote source
    +   *
    +   * @param path Path to the image directory
    +   * @param sparkSession Spark Session
    +   * @param recursive Recursive path search flag
    +   * @param numPartitions Number of the DataFrame partitions
    +   * @param dropImageFailures Drop the files that are not valid images from the result
    +   * @param sampleRatio Fraction of the files loaded
    +   * @return DataFrame with a single column "image" of images;
    +   *         see ImageSchema for the details
    +   */
    +  def readImages(
    +      path: String,
    +      sparkSession: SparkSession = null,
    +      recursive: Boolean = false,
    +      numPartitions: Int = 0,
    +      dropImageFailures: Boolean = false,
    +      sampleRatio: Double = 1.0): DataFrame = {
    +    require(sampleRatio <= 1.0 && sampleRatio >= 0, "sampleRatio should be between 0 and 1")
    +
    +    val session = if (sparkSession != null) sparkSession else SparkSession.builder().getOrCreate
    +    val partitions =
    +      if (numPartitions > 0) {
    +        numPartitions
    +      } else {
    +        session.sparkContext.defaultParallelism
    +      }
    +
    +    val oldRecursiveFlag = RecursiveFlag.setRecursiveFlag(Some(recursive.toString), session)
    +    val sampleImages = sampleRatio < 1
    +    val oldPathFilter: Option[Class[_]] =
    +      if (sampleImages) {
    +        SamplePathFilter.setPathFilter(Some(classOf[SamplePathFilter]), sampleRatio, session)
    +      } else {
    +        None
    +      }
    +
    +    try {
    +      val streams = session.sparkContext.binaryFiles(path, partitions)
    +        .repartition(partitions)
    --- End diff --
    
    Looks we can make this inlined.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by thunterdb <gi...@git.apache.org>.
Github user thunterdb commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r150247999
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/HadoopUtils.scala ---
    @@ -0,0 +1,109 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import scala.language.existentials
    +import scala.util.Random
    +
    +import org.apache.commons.io.FilenameUtils
    +import org.apache.hadoop.conf.{Configuration, Configured}
    +import org.apache.hadoop.fs.{Path, PathFilter}
    +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
    +
    +import org.apache.spark.sql.SparkSession
    +
    +private object RecursiveFlag {
    +  /**
    +   * Sets the spark recursive flag and then restores it.
    +   *
    +   * @param value Value to set
    +   * @param spark Existing spark session
    +   * @param f The function to evaluate after setting the flag
    +   * @return Returns the evaluation result T of the function
    +   */
    +  def withRecursiveFlag[T](value: Boolean, spark: SparkSession)(f: => T): T = {
    +    val flagName = FileInputFormat.INPUT_DIR_RECURSIVE
    +    val hadoopConf = spark.sparkContext.hadoopConfiguration
    +    val old = Option(hadoopConf.get(flagName))
    +    hadoopConf.set(flagName, value.toString)
    +    try f finally {
    +      old match {
    +        case Some(v) => hadoopConf.set(flagName, v)
    +        case None => hadoopConf.unset(flagName)
    +      }
    +    }
    +  }
    +}
    +
    +/**
    + * Filter that allows loading a fraction of HDFS files.
    + */
    +private class SamplePathFilter extends Configured with PathFilter {
    --- End diff --
    
    Oh, it would be pretty simple to work it out this way (pseudocode). Note that a hash is a random variable between -2^31 and 2^31-1, so:
    ```
    val fraction: Double = ??? // Something between 0 and 1
    val pathname: String = ???
    val hash = pathname.hashcode() // Could use some other other, more robust methods that return longs. 
    val shouldKeep: Boolean = math.abs(hash) < (math.pow(2, 31) * fraction)
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r147691860
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,252 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC3" -> 16, "CV_8UC4" -> 24
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField(imageFields(0), StringType, true) ::
    +    StructField(imageFields(1), IntegerType, false) ::
    +    StructField(imageFields(2), IntegerType, false) ::
    +    StructField(imageFields(3), IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(imageFields(4), IntegerType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(imageFields(5), BinaryType, false) :: Nil)
    +
    +  /**
    +   * DataFrame with a single column of images named "image" (nullable)
    +   */
    +  val imageSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the height of the image
    +   *
    +   * @return The height of the image
    +   */
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the width of the image
    +   *
    +   * @return The width of the image
    +   */
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the number of channels in the image
    +   *
    +   * @return The number of channels in the image
    +   */
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the OpenCV representation as an int
    +   *
    +   * @return The OpenCV representation as an int
    +   */
    +  def getMode(row: Row): Int = row.getInt(4)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the image data
    +   *
    +   * @return The image data
    +   */
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * :: Experimental ::
    +   * Check if the DataFrame column contains images (i.e. has ImageSchema)
    +   *
    +   * @param df DataFrame
    +   * @param column Column name
    +   * @return True if the given column matches the image schema
    +   */
    +  def isImageColumn(df: DataFrame, column: String): Boolean =
    +    df.schema(column).dataType == columnSchema
    --- End diff --
    
    I think this is too much if `columnSchema` is going to be exposed ...


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by holdenk <gi...@git.apache.org>.
Github user holdenk commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r143836725
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,133 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +import pyspark
    +from pyspark import SparkContext
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame
    +from pyspark.ml.param.shared import *
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +ImageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +ocvTypes = {
    +    undefinedImageType: -1,
    +    "CV_8U": 0, "CV_8UC1": 0, "CV_8UC2": 8, "CV_8UC3": 16, "CV_8UC4": 24,
    +    "CV_8S": 1, "CV_8SC1": 1, "CV_8SC2": 9, "CV_8SC3": 17, "CV_8SC4": 25,
    +    "CV_16U": 2, "CV_16UC1": 2, "CV_16UC2": 10, "CV_16UC3": 18, "CV_16UC4": 26,
    +    "CV_16S": 3, "CV_16SC1": 3, "CV_16SC2": 11, "CV_16SC3": 19, "CV_16SC4": 27,
    +    "CV_32S": 4, "CV_32SC1": 4, "CV_32SC2": 12, "CV_32SC3": 20, "CV_32SC4": 28,
    +    "CV_32F": 5, "CV_32FC1": 5, "CV_32FC2": 13, "CV_32FC3": 21, "CV_32FC4": 29,
    +    "CV_64F": 6, "CV_64FC1": 6, "CV_64FC2": 14, "CV_64FC3": 22, "CV_64FC4": 30
    +}
    +
    +ImageSchema = StructType([
    +    StructField(ImageFields[0], StringType(),  True),
    +    StructField(ImageFields[1], IntegerType(), False),
    +    StructField(ImageFields[2], IntegerType(), False),
    +    StructField(ImageFields[3], IntegerType(), False),
    +    # OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(ImageFields[4], StringType(), False),
    +    # bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(ImageFields[5], BinaryType(), False)])
    +
    +
    +# TODO: generalize to other datatypes and number of channels
    +def toNDArray(image):
    +    """
    +    Converts an image to a 1-dimensional array
    +
    +    Args:
    +        image (object): The image to be converted
    +
    +    Returns:
    +        array: The image as a 1-dimensional array
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    height = image.height
    +    width = image.width
    +    return np.asarray(image.data, dtype=np.uint8) \
    +             .reshape((height, width, 3))[:, :, (2, 1, 0)]
    +
    +
    +# TODO: generalize to other datatypes and number of channels
    +def toImage(array, origin="", mode="CV_8UC3"):
    +    """
    +
    +    Converts a one-dimensional array to a 2 dimensional image
    +
    +    Args:
    +        array (array):
    +        origin (str):
    +        mode (int):
    --- End diff --
    
    The argument description here is incorrect and not very useful (or the default parameters for the function are incorrect).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r143924856
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    --- End diff --
    
    Does this and/or its contents need to be public? or can they be `private[spark]`?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r147736427
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,252 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields = Array("origin", "height", "width", "nChannels", "mode", "data")
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r144299496
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,256 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC2" -> 8, "CV_8UC3" -> 16, "CV_8UC4" -> 24,
    +    "CV_8S" -> 1, "CV_8SC1" -> 1, "CV_8SC2" -> 9, "CV_8SC3" -> 17, "CV_8SC4" -> 25,
    +    "CV_16U" -> 2, "CV_16UC1" -> 2, "CV_16UC2" -> 10, "CV_16UC3" -> 18, "CV_16UC4" -> 26,
    +    "CV_16S" -> 3, "CV_16SC1" -> 3, "CV_16SC2" -> 11, "CV_16SC3" -> 19, "CV_16SC4" -> 27,
    +    "CV_32S" -> 4, "CV_32SC1" -> 4, "CV_32SC2" -> 12, "CV_32SC3" -> 20, "CV_32SC4" -> 28,
    +    "CV_32F" -> 5, "CV_32FC1" -> 5, "CV_32FC2" -> 13, "CV_32FC3" -> 21, "CV_32FC4" -> 29,
    +    "CV_64F" -> 6, "CV_64FC1" -> 6, "CV_64FC2" -> 14, "CV_64FC3" -> 22, "CV_64FC4" -> 30
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField("origin", StringType, true) ::
    +    StructField("height", IntegerType, false) ::
    +    StructField("width", IntegerType, false) ::
    +    StructField("nChannels", IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField("mode", StringType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField("data", BinaryType, false) :: Nil)
    +
    +  // DataFrame with a single column of images named "image" (nullable)
    +  private val imageDFSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the height of the image
    +   *
    +   * @return The height of the image
    +   */
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the width of the image
    +   *
    +   * @return The width of the image
    +   */
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the number of channels in the image
    +   *
    +   * @return The number of channels in the image
    +   */
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the OpenCV representation
    +   *
    +   * @return The OpenCV representation
    +   */
    +  def getMode(row: Row): String = row.getString(4)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the image data
    +   *
    +   * @return The image data
    +   */
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * :: Experimental ::
    +   * Check if the DataFrame column contains images (i.e. has ImageSchema)
    +   *
    +   * @param df       DataFrame
    +   * @param column   Column name
    +   * @return         True if the given column matches the image schema
    +   */
    +  def isImageColumn(df: DataFrame, column: String): Boolean =
    +    df.schema(column).dataType == columnSchema
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return       Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row = Row(Row(origin, -1, -1, -1, undefinedImageType,
    +    Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in DataFrame Row
    +   *
    +   * @param origin   Arbitrary string that identifies the image
    +   * @param bytes    Image bytes (for example, jpeg)
    +   * @return         DataFrame Row or None (if the decompression fails)
    --- End diff --
    
    nit:
    
    ```scala
    @param origin Arbitrary string that identifies the image
    @param bytes Image bytes (for example, jpeg)
    @return DataFrame Row or None (if the decompression fails)
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r150736890
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -80,7 +80,7 @@ def imageFields(self):
             """
             Returns field names of image columns.
     
    -        :rtype list: a list of field names.
    +        :return: list: a list of field names.
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #82773 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82773/testReport)** for PR 19439 at commit [`52632cf`](https://github.com/apache/spark/commit/52632cf600303d6a3512efe3e7a1c159ee10f05f).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r147935560
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields: Array[String] = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes: Map[String, Int] = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC3" -> 16, "CV_8UC4" -> 24
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField(imageFields(0), StringType, true) ::
    +    StructField(imageFields(1), IntegerType, false) ::
    +    StructField(imageFields(2), IntegerType, false) ::
    +    StructField(imageFields(3), IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(imageFields(4), IntegerType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(imageFields(5), BinaryType, false) :: Nil)
    +
    +  /**
    +   * DataFrame with a single column of images named "image" (nullable)
    +   */
    +  val imageSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the height of the image
    +   *
    +   * @return The height of the image
    +   */
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the width of the image
    +   *
    +   * @return The width of the image
    +   */
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the number of channels in the image
    +   *
    +   * @return The number of channels in the image
    +   */
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the OpenCV representation as an int
    +   *
    +   * @return The OpenCV representation as an int
    +   */
    +  def getMode(row: Row): Int = row.getInt(4)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the image data
    +   *
    +   * @return The image data
    +   */
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row =
    +    Row(Row(origin, -1, -1, -1, ocvTypes(undefinedImageType), Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in DataFrame Row
    +   *
    +   * @param origin Arbitrary string that identifies the image
    +   * @param bytes Image bytes (for example, jpeg)
    +   * @return DataFrame Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val isGray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val hasAlpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) = if (isGray) {
    +        (1, ocvTypes("CV_8UC1"))
    +      } else if (hasAlpha) {
    +        (4, ocvTypes("CV_8UC4"))
    +      } else {
    +        (3, ocvTypes("CV_8UC3"))
    +      }
    +
    +      val imageSize = height * width * nChannels
    +      assert(imageSize < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](imageSize)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (isGray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      } else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    +            if (nChannels == 4) {
    +              decoded(offset + 3) = color.getAlpha.toByte
    +            }
    +            offset += nChannels
    +          }
    +        }
    +      }
    +
    +      // the internal "Row" is needed, because the image is a single DataFrame column
    +      Some(Row(Row(origin, height, width, nChannels, mode, decoded)))
    +    }
    +  }
    +
    +  /**
    +   * :: Experimental ::
    +   * Read the directory of images from the local or remote source
    +   *
    +   * @param path Path to the image directory
    +   * @param sparkSession Spark Session, if omitted gets or creates the session
    +   * @param recursive Recursive path search flag
    +   * @param numPartitions Number of the DataFrame partitions,
    +   *                      if omitted uses defaultParallelism instead
    +   * @param dropImageFailures Drop the files that are not valid images from the result
    +   * @param sampleRatio Fraction of the files loaded
    +   * @return DataFrame with a single column "image" of images;
    +   *         see ImageSchema for the details
    +   */
    +  def readImages(
    +      path: String,
    +      sparkSession: SparkSession = null,
    +      recursive: Boolean = false,
    +      numPartitions: Int = 0,
    +      dropImageFailures: Boolean = false,
    +      sampleRatio: Double = 1.0): DataFrame = {
    +    require(sampleRatio <= 1.0 && sampleRatio >= 0, "sampleRatio should be between 0 and 1")
    +
    +    val session = if (sparkSession != null) sparkSession else SparkSession.builder().getOrCreate
    +    val partitions =
    +      if (numPartitions > 0) {
    +        numPartitions
    +      } else {
    +        session.sparkContext.defaultParallelism
    +      }
    +
    +    RecursiveFlag.withRecursiveFlag(recursive, session)(
    +      SamplePathFilter.withPathFilter(sampleRatio, session)({
    --- End diff --
    
    ditto:
    
    ```
    withPathFilter(sampleRatio, session) {
      ...
    }
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #83273 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83273/testReport)** for PR 19439 at commit [`38deaee`](https://github.com/apache/spark/commit/38deaeee17ea37814015c65b85a450ef3c35e03c).
     * This patch **fails Python style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class _ImageSchema(object):`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r147986087
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,139 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +from pyspark.ml.util import *
    +from pyspark.ml.param.shared import *
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame, SparkSession, SQLContext
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +imageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +
    +def getOcvTypes(spark=None):
    +    """
    +    Returns the OpenCV type mapping supported
    +
    +    :param sparkSession (SparkSession): The current spark session
    +    :rtype dict: The OpenCV type mapping supported
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.ocvTypes
    +
    +
    +# DataFrame with a single column of images named "image" (nullable)
    +def getImageSchema(spark=None):
    +    """
    +    Returns the image schema
    +
    +    :param spark (SparkSession): The current spark session
    +    :rtype StructType: The image schema
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.imageSchema
    --- End diff --
    
    Hm, isn't this a JVM object? I think we should do something like:
    
    ```python
    from pyspark.sql.types import _parse_datatype_json_string
    ...
    jschema = ctx._jvm.org.apache.spark.ml.image.ImageSchema.imageSchema()
    return _parse_datatype_json_string(jschema.json())
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r147999393
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,139 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +from pyspark.ml.util import *
    +from pyspark.ml.param.shared import *
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame, SparkSession, SQLContext
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +imageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +
    +def getOcvTypes(spark=None):
    +    """
    +    Returns the OpenCV type mapping supported
    +
    +    :param sparkSession (SparkSession): The current spark session
    +    :rtype dict: The OpenCV type mapping supported
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.ocvTypes
    --- End diff --
    
    Or.. just simply define a dict in Python side. .... 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by thunterdb <gi...@git.apache.org>.
Github user thunterdb commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r147661078
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/HadoopUtils.scala ---
    @@ -0,0 +1,120 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import scala.language.existentials
    +import scala.util.Random
    +
    +import org.apache.commons.io.FilenameUtils
    +import org.apache.hadoop.conf.{Configuration, Configured}
    +import org.apache.hadoop.fs.{Path, PathFilter}
    +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
    +
    +import org.apache.spark.sql.SparkSession
    +
    +private object RecursiveFlag {
    +
    +  /**
    +   * Sets a value of spark recursive flag.
    +   * If value is a None, it unsets the flag.
    +   *
    +   * @param value value to set
    +   * @param spark existing spark session
    +   * @return previous value of this flag
    +   */
    +  def setRecursiveFlag(value: Option[String], spark: SparkSession): Option[String] = {
    +    val flagName = FileInputFormat.INPUT_DIR_RECURSIVE
    +    val hadoopConf = spark.sparkContext.hadoopConfiguration
    +    val old = Option(hadoopConf.get(flagName))
    +
    +    value match {
    +      case Some(v) => hadoopConf.set(flagName, v)
    +      case None => hadoopConf.unset(flagName)
    +    }
    +
    +    old
    +  }
    +}
    +
    +/**
    + * Filter that allows loading a fraction of HDFS files.
    + */
    +private class SamplePathFilter extends Configured with PathFilter {
    +  val random = new Random()
    +
    +  // Ratio of files to be read from disk
    +  var sampleRatio: Double = 1
    +
    +  override def setConf(conf: Configuration): Unit = {
    +    if (conf != null) {
    +      sampleRatio = conf.getDouble(SamplePathFilter.ratioParam, 1)
    +    }
    +  }
    +
    +  override def accept(path: Path): Boolean = {
    +    // Note: checking fileSystem.isDirectory is very slow here, so we use basic rules instead
    +    !SamplePathFilter.isFile(path) || random.nextDouble() < sampleRatio
    --- End diff --
    
    I would prefer that we do not use a seed and that the result is deterministic, based for example on some hash of the file name, to make it more robust to future code changes. That being said, there is no fundamental issues with the current implementation and other developers may have differing opinions, so the current implementation is fine as far as I am concerned.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r145007975
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,129 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +import pyspark
    +from pyspark import SparkContext
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame
    +from pyspark.ml.param.shared import *
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +ImageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +ocvTypes = {
    +    undefinedImageType: -1,
    +    "CV_8U": 0, "CV_8UC1": 0, "CV_8UC2": 8, "CV_8UC3": 16, "CV_8UC4": 24,
    +    "CV_8S": 1, "CV_8SC1": 1, "CV_8SC2": 9, "CV_8SC3": 17, "CV_8SC4": 25,
    +    "CV_16U": 2, "CV_16UC1": 2, "CV_16UC2": 10, "CV_16UC3": 18, "CV_16UC4": 26,
    +    "CV_16S": 3, "CV_16SC1": 3, "CV_16SC2": 11, "CV_16SC3": 19, "CV_16SC4": 27,
    +    "CV_32S": 4, "CV_32SC1": 4, "CV_32SC2": 12, "CV_32SC3": 20, "CV_32SC4": 28,
    +    "CV_32F": 5, "CV_32FC1": 5, "CV_32FC2": 13, "CV_32FC3": 21, "CV_32FC4": 29,
    +    "CV_64F": 6, "CV_64FC1": 6, "CV_64FC2": 14, "CV_64FC3": 22, "CV_64FC4": 30
    +}
    +
    +ImageSchema = StructType([
    --- End diff --
    
    I mean, I see a separate `object` for `ImageSchema` whereas `ImageSchema` here is an `StructType`, and I think I am not seeing `ImageFields` in Scala side. 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r143908055
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC2" -> 8, "CV_8UC3" -> 16, "CV_8UC4" -> 24,
    +    "CV_8S" -> 1, "CV_8SC1" -> 1, "CV_8SC2" -> 9, "CV_8SC3" -> 17, "CV_8SC4" -> 25,
    +    "CV_16U" -> 2, "CV_16UC1" -> 2, "CV_16UC2" -> 10, "CV_16UC3" -> 18, "CV_16UC4" -> 26,
    +    "CV_16S" -> 3, "CV_16SC1" -> 3, "CV_16SC2" -> 11, "CV_16SC3" -> 19, "CV_16SC4" -> 27,
    +    "CV_32S" -> 4, "CV_32SC1" -> 4, "CV_32SC2" -> 12, "CV_32SC3" -> 20, "CV_32SC4" -> 28,
    +    "CV_32F" -> 5, "CV_32FC1" -> 5, "CV_32FC2" -> 13, "CV_32FC3" -> 21, "CV_32FC4" -> 29,
    +    "CV_64F" -> 6, "CV_64FC1" -> 6, "CV_64FC2" -> 14, "CV_64FC3" -> 22, "CV_64FC4" -> 30
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField("origin", StringType, true) ::
    +      StructField("height", IntegerType, false) ::
    +      StructField("width", IntegerType, false) ::
    +      StructField("nChannels", IntegerType, false) ::
    +      // OpenCV-compatible type: CV_8UC3 in most cases
    +      StructField("mode", StringType, false) ::
    +      // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +      StructField("data", BinaryType, false) :: Nil)
    +
    +  // Dataframe with a single column of images named "image" (nullable)
    +  private val imageDFSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  @Since("2.3.0")
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  @Since("2.3.0")
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  @Since("2.3.0")
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  @Since("2.3.0")
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  @Since("2.3.0")
    +  def getMode(row: Row): String = row.getString(4)
    +
    +  @Since("2.3.0")
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * Check if the dataframe column contains images (i.e. has ImageSchema)
    +   *
    +   * @param df       Dataframe
    +   * @param column   Column name
    +   * @return         True if the given column matches the image schema
    +   */
    +  @Since("2.3.0")
    +  def isImageColumn(df: DataFrame, column: String): Boolean =
    +    df.schema(column).dataType == columnSchema
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return       Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row = Row(Row(origin, -1, -1, -1, undefinedImageType,
    +    Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in dataframe Row
    +   *
    +   * @param origin   Arbitrary string that identifies the image
    +   * @param bytes    Image bytes (for example, jpeg)
    +   * @return         Dataframe Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val is_gray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val has_alpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) = if (is_gray) {
    +        (1, "CV_8UC1")
    +      } else if (has_alpha) {
    +        (4, "CV_8UC4")
    +      } else {
    +        (3, "CV_8UC3")
    +      }
    +
    +      assert(height * width * nChannels < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](height*width*nChannels)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (is_gray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for(h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      }
    +      else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    +            if (nChannels == 4) {
    +              decoded(offset + 3) = color.getAlpha.toByte
    +            }
    +            offset += nChannels
    +          }
    +        }
    +      }
    +
    +      // the internal "Row" is needed, because the image is a single dataframe column
    +      Some(Row(Row(origin, height, width, nChannels, mode, decoded)))
    +    }
    +  }
    +
    +  /**
    +   * :: Experimental ::
    +   * Read the directory of images from the local or remote source
    +   *
    +   * @param path               Path to the image directory
    +   * @param sparkSession       Spark Session
    +   * @param recursive          Recursive path search flag
    +   * @param numPartitions      Number of the dataframe partitions
    +   * @param dropImageFailures  Drop the files that are not valid images from the result
    +   * @param sampleRatio        Fraction of the files loaded
    +   * @return                   Dataframe with a single column "image" of images;
    +   *                           see ImageSchema for the details
    +   */
    +  @Since("2.3.0")
    +  def readImages(path: String,
    +                 sparkSession: SparkSession = null,
    +                 recursive: Boolean = false,
    +                 numPartitions: Int = 0,
    +                 dropImageFailures: Boolean = false,
    +                 sampleRatio: Double = 1.0): DataFrame = {
    +    require(sampleRatio <= 1.0 && sampleRatio >= 0, "sampleRatio should be between 0 and 1")
    +
    +    val session = if (sparkSession != null) sparkSession else SparkSession.builder().getOrCreate
    +    val partitions =
    +      if (numPartitions > 0) {
    +        numPartitions
    +      } else {
    +        session.sparkContext.defaultParallelism
    +      }
    +
    +    val oldRecursiveFlag = RecursiveFlag.setRecursiveFlag(Some(recursive.toString), session)
    +    val oldPathFilter: Option[Class[_]] =
    +      if (sampleRatio < 1) {
    +        SamplePathFilter.setPathFilter(Some(classOf[SamplePathFilter]), sampleRatio, session)
    +      } else {
    +        None
    +      }
    +
    +    var result: DataFrame = null
    +    try {
    +      val streams = session.sparkContext.binaryFiles(path, partitions)
    +        .repartition(partitions)
    +
    +      val images = if (dropImageFailures) {
    +        streams.flatMap {
    +          case (origin, stream) => decode(origin, stream.toArray)
    +        }
    +      } else {
    +        streams.map {
    +          case (origin, stream) => decode(origin, stream.toArray).getOrElse(invalidImageRow(origin))
    +        }
    +      }
    +
    +      result = session.createDataFrame(images, imageDFSchema)
    +    }
    +    finally {
    +      // return Hadoop flags to the original values
    +      RecursiveFlag.setRecursiveFlag(oldRecursiveFlag, session)
    +      SamplePathFilter.unsetPathFilter(oldPathFilter, session)
    --- End diff --
    
    Seems this unconditionally changes default path filter if any when `sampleRatio` is less than 1. I guess we should avoid to unset it?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    @jkbradley agreed, we can add a warning for now.  The user can always ignore sampling by setting the sampling rate to 1 as a workaround.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r147690566
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/image/ImageSchemaSuite.scala ---
    @@ -0,0 +1,109 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.nio.file.Paths
    +import java.util.Arrays
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.ml.image.ImageSchema._
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.types._
    +
    +class ImageSchemaSuite extends SparkFunSuite with MLlibTestSparkContext {
    +  // Single column of images named "image"
    +  private lazy val imagePath =
    +    Thread.currentThread().getContextClassLoader.getResource("test-data/images").getPath
    +
    +  test("Smoke test: create basic ImageSchema dataframe") {
    +    val origin = "path"
    +    val width = 1
    +    val height = 1
    +    val nChannels = 3
    +    val data = Array[Byte](0, 0, 0)
    +    val mode = ocvTypes("CV_8UC3")
    +
    +    // Internal Row corresponds to image StructType
    +    val rows = Seq(Row(Row(origin, height, width, nChannels, mode, data)),
    +      Row(Row(null, height, width, nChannels, mode, data)))
    +    val rdd = sc.makeRDD(rows)
    +    val df = spark.createDataFrame(rdd, ImageSchema.imageSchema)
    +
    +    assert(df.count === 2, "incorrect image count")
    +    assert(ImageSchema.isImageColumn(df, "image"), "data do not fit ImageSchema")
    +  }
    +
    +  test("readImages count test") {
    +    var df = readImages(imagePath, recursive = false)
    +    assert(df.count === 1)
    +
    +    df = readImages(imagePath, recursive = true, dropImageFailures = false)
    +    assert(df.count === 9)
    +
    +    df = readImages(imagePath, recursive = true, dropImageFailures = true)
    +    val count100 = df.count
    +    assert(count100 === 7)
    +
    +    df = readImages(imagePath, recursive = true, sampleRatio = 0.5, dropImageFailures = true)
    +    // Random number about half of the size of the original dataset
    +    val count50 = df.count
    +    assert(count50 > 0.2 * count100 && count50 < 0.8 * count100)
    --- End diff --
    
    Without a close look, this test seems flaky in my local:
    
    ```
    6 was greater than 1.4000000000000001, but 6 was not less than 5.6000000000000005
    ScalaTestFailureLocation: org.apache.spark.ml.image.ImageSchemaSuite$$anonfun$3 at (ImageSchemaSuite.scala:66)
    org.scalatest.exceptions.TestFailedException: 6 was greater than 1.4000000000000001, but 6 was not less than 5.6000000000000005
    	at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:528)
    	at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1560)
    ```
    
    or
    
    ```
    1 was not greater than 1.4000000000000001
    ScalaTestFailureLocation: org.apache.spark.ml.image.ImageSchemaSuite$$anonfun$3 at (ImageSchemaSuite.scala:66)
    org.scalatest.exceptions.TestFailedException: 1 was not greater than 1.4000000000000001
    	at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:528)
    	at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1560)
    	at org.scalatest.Assertions$AssertionsHelper.macroAssert(Assertions.scala:501)
    ```
    
    It roughly fails at 1 out of 7 tries.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by MrBago <gi...@git.apache.org>.
Github user MrBago commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r147298335
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,133 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +from pyspark.ml.util import *
    +from pyspark.ml.param.shared import *
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame, SparkSession, SQLContext
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +imageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +
    +def getOcvTypes(spark=None):
    +    """
    +    Returns the OpenCV type mapping supported
    +
    +    :param sparkSession (SparkSession): The current spark session
    +    :rtype dict: The OpenCV type mapping supported
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.ocvTypes
    +
    +
    +# DataFrame with a single column of images named "image" (nullable)
    +def getImageSchema(spark=None):
    +    """
    +    Returns the image schema
    +
    +    :param spark (SparkSession): The current spark session
    +    :rtype StructType: The image schema
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.imageSchema
    +
    +
    +def toNDArray(image):
    +    """
    +    Converts an image to a one-dimensional array.
    +
    +    :param image (object): The image to be converted
    +    :rtype array: The image as a one-dimensional array
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    height = image.height
    +    width = image.width
    +    nChannels = image.nChannels
    +    return np.ndarray(
    +        shape=(height, width, nChannels),
    +        dtype=np.uint8,
    +        buffer=image.data,
    +        strides=(width * nChannels, nChannels, 1))
    +
    +
    +def toImage(array, origin="", mode=None, spark=None):
    +    """
    +    Converts a one-dimensional array to a two-dimensional image.
    +
    +    :param array (array): The array to convert to image
    +    :param origin (str): Path to the image
    +    :param mode (str): OpenCV compatible type
    +    :param spark (SparkSession): The current spark session
    +    :rtype object: Two dimensional image
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ocvTypes = getOcvTypes(spark)
    +    mode = mode or ocvTypes["CV_8UC3"]
    +    data = bytearray(array.astype(dtype=np.uint8).ravel())
    +    height = array.shape[0]
    +    width = array.shape[1]
    +    nChannels = array.shape[2]
    --- End diff --
    
    I think this should be something like:
    ```
    def toImage(array, origin="", spark=None):
        spark = spark or SparkSession.builder.getOrCreate()
    
        if array.ndim != 3:
            raise
        height, width, nChannels = array.shape
    
        ocvTypes = getOcvTypes(spark)
        if nChannels == 1:
            mode = ocvTypes["CV_8UC1"]
        elif nChannels == 3:
            mode = ocvTypes["CV_8UC3"]
        elif nChannels == 4:
            mode =ocvTypes["CV_8UC4"]
        else:
            raise
    
        data = bytearray(array.astype(dtype=np.uint8, copy=False).ravel())
        return _create_row(...)
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r147004514
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,258 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC2" -> 8, "CV_8UC3" -> 16, "CV_8UC4" -> 24,
    +    "CV_8S" -> 1, "CV_8SC1" -> 1, "CV_8SC2" -> 9, "CV_8SC3" -> 17, "CV_8SC4" -> 25,
    +    "CV_16U" -> 2, "CV_16UC1" -> 2, "CV_16UC2" -> 10, "CV_16UC3" -> 18, "CV_16UC4" -> 26,
    +    "CV_16S" -> 3, "CV_16SC1" -> 3, "CV_16SC2" -> 11, "CV_16SC3" -> 19, "CV_16SC4" -> 27,
    +    "CV_32S" -> 4, "CV_32SC1" -> 4, "CV_32SC2" -> 12, "CV_32SC3" -> 20, "CV_32SC4" -> 28,
    +    "CV_32F" -> 5, "CV_32FC1" -> 5, "CV_32FC2" -> 13, "CV_32FC3" -> 21, "CV_32FC4" -> 29,
    +    "CV_64F" -> 6, "CV_64FC1" -> 6, "CV_64FC2" -> 14, "CV_64FC3" -> 22, "CV_64FC4" -> 30
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField(imageFields(0), StringType, true) ::
    +    StructField(imageFields(1), IntegerType, false) ::
    +    StructField(imageFields(2), IntegerType, false) ::
    +    StructField(imageFields(3), IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(imageFields(4), IntegerType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(imageFields(5), BinaryType, false) :: Nil)
    +
    +  /**
    +   * DataFrame with a single column of images named "image" (nullable)
    +   */
    +  val imageSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the height of the image
    +   *
    +   * @return The height of the image
    +   */
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the width of the image
    +   *
    +   * @return The width of the image
    +   */
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the number of channels in the image
    +   *
    +   * @return The number of channels in the image
    +   */
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the OpenCV representation as an int
    +   *
    +   * @return The OpenCV representation as an int
    +   */
    +  def getMode(row: Row): Int = row.getInt(4)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the image data
    +   *
    +   * @return The image data
    +   */
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * :: Experimental ::
    +   * Check if the DataFrame column contains images (i.e. has ImageSchema)
    +   *
    +   * @param df DataFrame
    +   * @param column Column name
    +   * @return True if the given column matches the image schema
    +   */
    +  def isImageColumn(df: DataFrame, column: String): Boolean =
    +    df.schema(column).dataType == columnSchema
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row =
    +    Row(Row(origin, -1, -1, -1, ocvTypes(undefinedImageType), Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in DataFrame Row
    +   *
    +   * @param origin Arbitrary string that identifies the image
    +   * @param bytes Image bytes (for example, jpeg)
    +   * @return DataFrame Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val isGray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val hasAlpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) = if (isGray) {
    +        (1, ocvTypes("CV_8UC1"))
    +      } else if (hasAlpha) {
    +        (4, ocvTypes("CV_8UC4"))
    +      } else {
    +        (3, ocvTypes("CV_8UC3"))
    +      }
    +
    +      val imageSize = height * width * nChannels
    +      assert(imageSize < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](imageSize)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (isGray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      } else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    +            if (nChannels == 4) {
    +              decoded(offset + 3) = color.getAlpha.toByte
    +            }
    +            offset += nChannels
    +          }
    +        }
    +      }
    +
    +      // the internal "Row" is needed, because the image is a single DataFrame column
    +      Some(Row(Row(origin, height, width, nChannels, mode, decoded)))
    +    }
    +  }
    +
    +  /**
    +   * :: Experimental ::
    +   * Read the directory of images from the local or remote source
    +   *
    +   * @param path Path to the image directory
    +   * @param sparkSession Spark Session
    +   * @param recursive Recursive path search flag
    +   * @param numPartitions Number of the DataFrame partitions
    +   * @param dropImageFailures Drop the files that are not valid images from the result
    +   * @param sampleRatio Fraction of the files loaded
    +   * @return DataFrame with a single column "image" of images;
    +   *         see ImageSchema for the details
    +   */
    +  def readImages(
    +      path: String,
    +      sparkSession: SparkSession = null,
    +      recursive: Boolean = false,
    +      numPartitions: Int = 0,
    +      dropImageFailures: Boolean = false,
    +      sampleRatio: Double = 1.0): DataFrame = {
    +    require(sampleRatio <= 1.0 && sampleRatio >= 0, "sampleRatio should be between 0 and 1")
    +
    +    val session = if (sparkSession != null) sparkSession else SparkSession.builder().getOrCreate
    +    val partitions =
    +      if (numPartitions > 0) {
    +        numPartitions
    +      } else {
    +        session.sparkContext.defaultParallelism
    +      }
    +
    +    val oldRecursiveFlag = RecursiveFlag.setRecursiveFlag(Some(recursive.toString), session)
    +    val sampleImages = sampleRatio < 1
    +    val oldPathFilter: Option[Class[_]] =
    +      if (sampleImages) {
    +        SamplePathFilter.setPathFilter(Some(classOf[SamplePathFilter]), sampleRatio, session)
    +      } else {
    +        None
    +      }
    +
    +    try {
    +      val streams = session.sparkContext.binaryFiles(path, partitions)
    +        .repartition(partitions)
    --- End diff --
    
    this is a bug in binaryFiles - even though we give it the partitions, binaryFiles ignores it.  
    This is a bug introduced in spark 2.1 from spark 2.0, in file PortableDataStream.scala the argument “minPartitions” is no longer used (with the push to master on 11/7/6):
    
      /**
       * Allow minPartitions set by end-user in order to keep compatibility with old Hadoop API
       * which is set through setMaxSplitSize
       */
      def setMinPartitions(sc: SparkContext, context: JobContext, minPartitions: Int) {
        val defaultMaxSplitBytes = sc.getConf.get(config.FILES_MAX_PARTITION_BYTES)
        val openCostInBytes = sc.getConf.get(config.FILES_OPEN_COST_IN_BYTES)
        val defaultParallelism = sc.defaultParallelism
        val files = listStatus(context).asScala
        val totalBytes = files.filterNot(_.isDirectory).map(_.getLen + openCostInBytes).sum
        val bytesPerCore = totalBytes / defaultParallelism
        val maxSplitSize = Math.min(defaultMaxSplitBytes, Math.max(openCostInBytes, bytesPerCore))
        super.setMaxSplitSize(maxSplitSize)
      } 
    
    The code previously, in version 2.0, was:
    
      def setMinPartitions(context: JobContext, minPartitions: Int) {
        val totalLen = listStatus(context).asScala.filterNot(_.isDirectory).map(_.getLen).sum
        val maxSplitSize = math.ceil(totalLen / math.max(minPartitions, 1.0)).toLong
        super.setMaxSplitSize(maxSplitSize)
      } 
    
    The new code is very smart, but it ignores what the user passes in and uses the data size, which is kind of a breaking change in some sense 
    In our specific case this was a problem, because we initially read in just the files names and only after that the dataframe becomes very large, when reading in the images themselves – and in this case the new code does not handle the partitioning very well.
    I’m not sure if it can be easily fixed because I don’t understand the full context of the change in spark (but at the very least the unused parameter should be removed to avoid confusion).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #83218 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83218/testReport)** for PR 19439 at commit [`030fe38`](https://github.com/apache/spark/commit/030fe38a22ebb3052b5e7762cd7227ffbe2b7c1d).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r147284697
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,122 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +import pyspark
    +from pyspark import SparkContext
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame
    +from pyspark.ml.param.shared import *
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +imageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +ocvTypes = {
    +    undefinedImageType: -1,
    +    "CV_8U": 0, "CV_8UC1": 0, "CV_8UC2": 8, "CV_8UC3": 16, "CV_8UC4": 24,
    +    "CV_8S": 1, "CV_8SC1": 1, "CV_8SC2": 9, "CV_8SC3": 17, "CV_8SC4": 25,
    +    "CV_16U": 2, "CV_16UC1": 2, "CV_16UC2": 10, "CV_16UC3": 18, "CV_16UC4": 26,
    +    "CV_16S": 3, "CV_16SC1": 3, "CV_16SC2": 11, "CV_16SC3": 19, "CV_16SC4": 27,
    +    "CV_32S": 4, "CV_32SC1": 4, "CV_32SC2": 12, "CV_32SC3": 20, "CV_32SC4": 28,
    +    "CV_32F": 5, "CV_32FC1": 5, "CV_32FC2": 13, "CV_32FC3": 21, "CV_32FC4": 29,
    +    "CV_64F": 6, "CV_64FC1": 6, "CV_64FC2": 14, "CV_64FC3": 22, "CV_64FC4": 30
    +}
    --- End diff --
    
    good point, I have created an ocvTypes method which just gets the dictionary from the scala side


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r144299121
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,256 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC2" -> 8, "CV_8UC3" -> 16, "CV_8UC4" -> 24,
    +    "CV_8S" -> 1, "CV_8SC1" -> 1, "CV_8SC2" -> 9, "CV_8SC3" -> 17, "CV_8SC4" -> 25,
    +    "CV_16U" -> 2, "CV_16UC1" -> 2, "CV_16UC2" -> 10, "CV_16UC3" -> 18, "CV_16UC4" -> 26,
    +    "CV_16S" -> 3, "CV_16SC1" -> 3, "CV_16SC2" -> 11, "CV_16SC3" -> 19, "CV_16SC4" -> 27,
    +    "CV_32S" -> 4, "CV_32SC1" -> 4, "CV_32SC2" -> 12, "CV_32SC3" -> 20, "CV_32SC4" -> 28,
    +    "CV_32F" -> 5, "CV_32FC1" -> 5, "CV_32FC2" -> 13, "CV_32FC3" -> 21, "CV_32FC4" -> 29,
    +    "CV_64F" -> 6, "CV_64FC1" -> 6, "CV_64FC2" -> 14, "CV_64FC3" -> 22, "CV_64FC4" -> 30
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField("origin", StringType, true) ::
    +    StructField("height", IntegerType, false) ::
    +    StructField("width", IntegerType, false) ::
    +    StructField("nChannels", IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField("mode", StringType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField("data", BinaryType, false) :: Nil)
    +
    +  // DataFrame with a single column of images named "image" (nullable)
    +  private val imageDFSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the height of the image
    +   *
    +   * @return The height of the image
    +   */
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the width of the image
    +   *
    +   * @return The width of the image
    +   */
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the number of channels in the image
    +   *
    +   * @return The number of channels in the image
    +   */
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the OpenCV representation
    +   *
    +   * @return The OpenCV representation
    +   */
    +  def getMode(row: Row): String = row.getString(4)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the image data
    +   *
    +   * @return The image data
    +   */
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * :: Experimental ::
    +   * Check if the DataFrame column contains images (i.e. has ImageSchema)
    +   *
    +   * @param df       DataFrame
    +   * @param column   Column name
    +   * @return         True if the given column matches the image schema
    --- End diff --
    
    nit:
    
    ```scala
    @param df DataFrame
    @param column Column name
    @return True if the given column matches the image schema
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r150447228
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,239 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +/**
    + * :: Experimental ::
    + * Defines the image schema and methods to read and manipulate images.
    + */
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val ocvTypes: Map[String, Int] = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC3" -> 16, "CV_8UC4" -> 24
    +  )
    +
    +  /**
    +   * Used for conversion to python
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/83768/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by dakirsa <gi...@git.apache.org>.
Github user dakirsa commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r145679632
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,133 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +import pyspark
    +from pyspark import SparkContext
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame
    +from pyspark.ml.param.shared import *
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +ImageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +ocvTypes = {
    +    undefinedImageType: -1,
    +    "CV_8U": 0, "CV_8UC1": 0, "CV_8UC2": 8, "CV_8UC3": 16, "CV_8UC4": 24,
    +    "CV_8S": 1, "CV_8SC1": 1, "CV_8SC2": 9, "CV_8SC3": 17, "CV_8SC4": 25,
    +    "CV_16U": 2, "CV_16UC1": 2, "CV_16UC2": 10, "CV_16UC3": 18, "CV_16UC4": 26,
    +    "CV_16S": 3, "CV_16SC1": 3, "CV_16SC2": 11, "CV_16SC3": 19, "CV_16SC4": 27,
    +    "CV_32S": 4, "CV_32SC1": 4, "CV_32SC2": 12, "CV_32SC3": 20, "CV_32SC4": 28,
    +    "CV_32F": 5, "CV_32FC1": 5, "CV_32FC2": 13, "CV_32FC3": 21, "CV_32FC4": 29,
    +    "CV_64F": 6, "CV_64FC1": 6, "CV_64FC2": 14, "CV_64FC3": 22, "CV_64FC4": 30
    +}
    +
    +ImageSchema = StructType([
    +    StructField(ImageFields[0], StringType(),  True),
    +    StructField(ImageFields[1], IntegerType(), False),
    +    StructField(ImageFields[2], IntegerType(), False),
    +    StructField(ImageFields[3], IntegerType(), False),
    +    # OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(ImageFields[4], StringType(), False),
    +    # bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(ImageFields[5], BinaryType(), False)])
    +
    +
    +# TODO: generalize to other datatypes and number of channels
    +def toNDArray(image):
    +    """
    +    Converts an image to a 1-dimensional array
    +
    +    Args:
    +        image (object): The image to be converted
    +
    +    Returns:
    +        array: The image as a 1-dimensional array
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    height = image.height
    +    width = image.width
    +    return np.asarray(image.data, dtype=np.uint8) \
    +             .reshape((height, width, 3))[:, :, (2, 1, 0)]
    --- End diff --
    
    Sorry for the late reply -- yes, you are totally right. OpenCV returns BGR, and we should do the same.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    @liancheng  I see you've worked with PathFilters in Spark SQL, so I'll ask here: We're uncertain about how PathFilters are used in Hadoop, and it would be helpful to understand (and use) them in order to ensure deterministic behavior for sampling in this image reader.
    
    Background: We use the PathFilter abstraction for the new "SamplePathFilter" class introduced by this PR.  That filter is for sampling a subset of rows.
    
    Our question: Will this be deterministic?
    
    My thoughts:
    * If we set a random seed, then this *may* be deterministic depending on the usage of PathFilters by the filesystem.
      * If a new PathFilter is instantiated for each partition read, then we get determinism.  (Since a new instance is created using the same seed, it will behave the same way each time, assuming files in the partition are read in the same order.)
      * If a PathFilter may be reused across partitions, then we cannot guarantee determinism.
    * Is this explanation reasonable, and do you know what we should expect?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #82827 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82827/testReport)** for PR 19439 at commit [`697585f`](https://github.com/apache/spark/commit/697585fd2544d5a97534c7895a92e5677fb70a8d).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #83275 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83275/testReport)** for PR 19439 at commit [`61290cc`](https://github.com/apache/spark/commit/61290ccfcb56ea6787e033330351b27df4219e48).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r147697713
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,140 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +from pyspark.ml.util import *
    +from pyspark.ml.param.shared import *
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame, SparkSession, SQLContext
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +imageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +
    +def getOcvTypes(spark=None):
    +    """
    +    Returns the OpenCV type mapping supported
    +
    +    :param sparkSession (SparkSession): The current spark session
    +    :rtype dict: The OpenCV type mapping supported
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.ocvTypes
    +
    +
    +# DataFrame with a single column of images named "image" (nullable)
    +def getImageSchema(spark=None):
    +    """
    +    Returns the image schema
    +
    +    :param spark (SparkSession): The current spark session
    +    :rtype StructType: The image schema
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.imageSchema
    +
    +
    +def toNDArray(image):
    +    """
    +    Converts an image to a one-dimensional array.
    +
    +    :param image (object): The image to be converted
    +    :rtype array: The image as a one-dimensional array
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    height = image.height
    +    width = image.width
    +    nChannels = image.nChannels
    +    return np.ndarray(
    +        shape=(height, width, nChannels),
    +        dtype=np.uint8,
    +        buffer=image.data,
    +        strides=(width * nChannels, nChannels, 1))
    +
    +
    +def toImage(array, origin="", spark=None):
    +    """
    +    Converts a one-dimensional array to a two-dimensional image.
    +
    +    :param array (array): The array to convert to image
    +    :param origin (str): Path to the image
    +    :param mode (str): OpenCV compatible type
    +    :param spark (SparkSession): The current spark session
    +    :rtype object: Two dimensional image
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    if array.ndim != 3:
    +        raise
    +    height, width, nChannels = array.shape
    +    ocvTypes = getOcvTypes(spark)
    +    if nChannels == 1:
    +        mode = ocvTypes["CV_8UC1"]
    +    elif nChannels == 3:
    +        mode = ocvTypes["CV_8UC3"]
    +    elif nChannels == 4:
    +        mode = ocvTypes["CV_8UC4"]
    +    else:
    +        raise
    +    data = bytearray(array.astype(dtype=np.uint8).ravel())
    +    # Creating new Row with _create_row(), because Row(name = value, ... )
    +    # orders fields by name, which conflicts with expected schema order
    +    # when the new DataFrame is created by UDF
    +    return _create_row(imageFields,
    +                       [origin, height, width, nChannels, mode, data])
    +
    +
    +def readImages(path, recursive=False, numPartitions=0,
    +               dropImageFailures=False, sampleRatio=1.0, spark=None):
    +    """
    +    Reads the directory of images from the local or remote source.
    +
    +    :param path (str): Path to the image directory
    +    :param recursive (bool): Recursive search flag
    +    :param numPartitions (int): Number of DataFrame partitions
    +    :param dropImageFailures (bool): Drop the files that are not valid images
    +    :param sampleRatio (double): Fraction of the images loaded
    +    :param spark (SparkSession): The current spark session
    +    :rtype DataFrame: DataFrame with a single column of "images",
    +           see ImageSchema for details
    +
    +    Examples:
    +
    +    >>> df = readImages('python/test_support/image/kittens', recursive=True)
    +    >>> df.count
    +    4
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    schema = ctx._jvm.org.apache.spark.ml.image.ImageSchema
    --- End diff --
    
    tiny nit: `schema` -> something like ... `image_schema` to prevent confusion.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r144855697
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/HadoopUtils.scala ---
    @@ -0,0 +1,119 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import scala.language.existentials
    +import scala.util.Random
    +
    +import org.apache.commons.io.FilenameUtils
    +import org.apache.hadoop.conf.{Configuration, Configured}
    +import org.apache.hadoop.fs.{Path, PathFilter}
    +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
    +
    +import org.apache.spark.sql.SparkSession
    +
    +private object RecursiveFlag {
    +
    +  /**
    +   * Sets a value of spark recursive flag.
    +   * If value is a None, it unsets the flag.
    +   *
    +   * @param value value to set
    +   * @param spark existing spark session
    +   * @return previous value of this flag
    +   */
    +  def setRecursiveFlag(value: Option[String], spark: SparkSession): Option[String] = {
    +    val flagName = FileInputFormat.INPUT_DIR_RECURSIVE
    +    val hadoopConf = spark.sparkContext.hadoopConfiguration
    +    val old = Option(hadoopConf.get(flagName))
    +
    +    value match {
    +      case Some(v) => hadoopConf.set(flagName, v)
    +      case None => hadoopConf.unset(flagName)
    +    }
    +
    +    old
    +  }
    +}
    +
    +/**
    + * Filter that allows loading a fraction of HDFS files.
    + */
    +private class SamplePathFilter extends Configured with PathFilter {
    +  val random = new Random()
    +
    +  // Ratio of files to be read from disk
    +  var sampleRatio: Double = 1
    +
    +  override def setConf(conf: Configuration): Unit = {
    +    if (conf != null) {
    +      sampleRatio = conf.getDouble(SamplePathFilter.ratioParam, 1)
    +    }
    +  }
    +
    +  override def accept(path: Path): Boolean = {
    +    // Note: checking fileSystem.isDirectory is very slow here, so we use basic rules instead
    +    !SamplePathFilter.isFile(path) ||
    +      random.nextDouble() < sampleRatio
    +  }
    +}
    +
    +private object SamplePathFilter {
    +  val ratioParam = "sampleRatio"
    +
    +  def isFile(path: Path): Boolean = FilenameUtils.getExtension(path.toString) != ""
    +
    +  /**
    +   * Sets HDFS PathFilter
    +   *
    +   * @param value       Filter class that is passed to HDFS
    +   * @param sampleRatio Fraction of the files that the filter picks
    +   * @param spark       Existing Spark session
    +   * @return            Returns the previous HDFS path filter
    +   */
    +  def setPathFilter(value: Option[Class[_]], sampleRatio: Double,
    +                    spark: SparkSession) : Option[Class[_]] = {
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    retest this please


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by holdenk <gi...@git.apache.org>.
Github user holdenk commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r149486897
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,192 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +"""
    +.. attribute:: ImageSchema
    +
    +    A singleton-like attribute of :class:`_ImageSchema` in this module.
    +
    +.. autoclass:: _ImageSchema
    +   :members:
    +"""
    +
    +from pyspark import SparkContext
    +from pyspark.sql.types import Row, _create_row, _parse_datatype_json_string
    +from pyspark.sql import DataFrame, SparkSession
    +import numpy as np
    +
    +
    +class _ImageSchema(object):
    +    """
    +    Internal class for `pyspark.ml.image.ImageSchema` attribute. Meant to be private and
    +    not to be instantized. Use `pyspark.ml.image.ImageSchema` attribute to access the
    +    APIs of this class.
    +    """
    +
    +    def __init__(self):
    +        self._imageSchema = None
    +        self._ocvTypes = None
    +        self._imageFields = None
    +        self._undefinedImageType = None
    +
    +    @property
    +    def imageSchema(self):
    +        """
    +        Returns the image schema.
    +
    +        :rtype StructType: a DataFrame with a single column of images
    +               named "image" (nullable)
    +
    +        .. versionadded:: 2.3.0
    +        """
    +
    +        if self._imageSchema is None:
    +            ctx = SparkContext._active_spark_context
    +            jschema = ctx._jvm.org.apache.spark.ml.image.ImageSchema.imageSchema()
    +            self._imageSchema = _parse_datatype_json_string(jschema.json())
    +        return self._imageSchema
    +
    +    @property
    +    def ocvTypes(self):
    +        """
    +        Returns the OpenCV type mapping supported
    +
    +        :rtype dict: The OpenCV type mapping supported
    +
    +        .. versionadded:: 2.3.0
    +        """
    +
    +        if self._ocvTypes is None:
    +            ctx = SparkContext._active_spark_context
    +            self._ocvTypes = dict(ctx._jvm.org.apache.spark.ml.image.ImageSchema._ocvTypes())
    +        return self._ocvTypes
    +
    +    @property
    +    def imageFields(self):
    +        """
    +        Returns field names of image columns.
    +
    +        :rtype list: a list of field names.
    +
    +        .. versionadded:: 2.3.0
    +        """
    +
    +        if self._imageFields is None:
    +            ctx = SparkContext._active_spark_context
    +            self._imageFields = list(ctx._jvm.org.apache.spark.ml.image.ImageSchema.imageFields())
    +        return self._imageFields
    +
    +    @property
    +    def undefinedImageType(self):
    +        """
    +        Returns the name of undefined image type for the invalid image.
    +
    +        .. versionadded:: 2.3.0
    +        """
    +
    +        if self._undefinedImageType is None:
    +            ctx = SparkContext._active_spark_context
    +            self._undefinedImageType = \
    +                ctx._jvm.org.apache.spark.ml.image.ImageSchema.undefinedImageType()
    +        return self._undefinedImageType
    +
    +    def toNDArray(self, image):
    +        """
    +        Converts an image to a one-dimensional array.
    +
    +        :param image: The image to be converted
    +        :rtype array: The image as a one-dimensional array
    +
    +        .. versionadded:: 2.3.0
    +        """
    +
    +        height = image.height
    +        width = image.width
    +        nChannels = image.nChannels
    +        return np.ndarray(
    +            shape=(height, width, nChannels),
    +            dtype=np.uint8,
    +            buffer=image.data,
    +            strides=(width * nChannels, nChannels, 1))
    +
    +    def toImage(self, array, origin=""):
    +        """
    +        Converts a one-dimensional array to a two-dimensional image.
    --- End diff --
    
    I think calling the input a 1-d array is a little confusing perhaps? Maybe 1-d array w/meta data?
    
    e.g. The check of `array.ndim != 3` later seems at odds with this statement & the toNDArray function above all would not be what I think of as 1-d arrays but I could be off-base.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r144742634
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/image/ImageSchemaSuite.scala ---
    @@ -0,0 +1,124 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.nio.file.Paths
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.ml.image.ImageSchema._
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.types._
    +
    +class ImageSchemaSuite extends SparkFunSuite with MLlibTestSparkContext {
    +  // Single column of images named "image"
    +  private val imageDFSchema =
    +    StructType(StructField("image", ImageSchema.columnSchema, true) :: Nil)
    +  private lazy val imagePath =
    +    Thread.currentThread().getContextClassLoader.getResource("test-data/images").getPath
    +
    +  test("Smoke test: create basic ImageSchema dataframe") {
    +    val origin = "path"
    +    val width = 1
    +    val height = 1
    +    val nChannels = 3
    +    val data = Array[Byte](0, 0, 0)
    +    val mode = "CV_8UC3"
    +
    +    // Internal Row corresponds to image StructType
    +    val rows = Seq(Row(Row(origin, height, width, nChannels, mode, data)),
    +      Row(Row(null, height, width, nChannels, mode, data)))
    +    val rdd = sc.makeRDD(rows)
    +    val df = spark.createDataFrame(rdd, imageDFSchema)
    +
    +    assert(df.count == 2, "incorrect image count")
    +    assert(ImageSchema.isImageColumn(df, "image"), "data do not fit ImageSchema")
    +  }
    +
    +  test("readImages count test") {
    +    var df = readImages(imagePath, recursive = false)
    +    assert(df.count == 0)
    +
    +    df = readImages(imagePath, recursive = true, dropImageFailures = false)
    +    assert(df.count == 8)
    +
    +    df = readImages(imagePath, recursive = true, dropImageFailures = true)
    +    val count100 = df.count
    +    assert(count100 == 7)
    +
    +    df = readImages(imagePath, recursive = true, sampleRatio = 0.5, dropImageFailures = true)
    +    // Random number about half of the size of the original dataset
    +    val count50 = df.count
    +    assert(count50 > 0.2 * count100 && count50 < 0.8 * count100)
    +  }
    +
    +  test("readImages partition test") {
    +    val df = readImages(imagePath, recursive = true, dropImageFailures = true, numPartitions = 3)
    +    assert(df.rdd.getNumPartitions == 3)
    +  }
    +
    +  // Images with the different number of channels
    +  test("readImages pixel values test") {
    +
    +    val images = readImages(imagePath + "/multi-channel/", recursive = false).collect
    +
    +    images.foreach{
    +      rrow => {
    +        val row = rrow.getAs[Row](0)
    +        val filename = Paths.get(getOrigin(row)).getFileName().toString()
    +        if(firstBytes20.contains(filename)) {
    +          val mode = getMode(row)
    +          val bytes20 = getData(row).slice(0, 20)
    +
    +          val expectedMode = firstBytes20(filename)._1
    +          val expectedBytes = firstBytes20(filename)._2
    +
    +          assert(expectedMode == mode, "mode of the image is not read correctly")
    +
    +          if (!compareBytes(expectedBytes, bytes20)) {
    +            throw new Exception("incorrect numeric value for flattened image")
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  // number of channels and first 20 bytes of OpenCV representation
    +  // - default representation for 3-channel RGB images is BGR row-wise:
    +  //   (B00, G00, R00,      B10, G10, R10,      ...)
    +  // - default representation for 4-channel RGB images is BGRA row-wise:
    +  //   (B00, G00, R00, A00, B10, G10, R10, A00, ...)
    +  private val firstBytes20 = Map(
    +    "grayscale.png" ->
    +      (("CV_8UC1", Array[Byte](0, 0, 0, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 1, 0, 0, 3, 5, 2, 1))),
    +    "RGB.png" -> (("CV_8UC3",
    +      Array[Byte](-34, -66, -98, -38, -69, -98, -62, -90, -117,
    +        -70, -98, -124, -34, -63, -90, -20, -48, -74, -18, -45))),
    +    "RGBA.png" -> (("CV_8UC4",
    +      Array[Byte](-128, -128, -8, -1, -128, -128, -8, -1, -128,
    +        -128, -8, -1, 127, 127, -9, -1, 127, 127, -9, -1)))
    +  )
    +
    +  private def compareBytes(x: Array[Byte], y: Array[Byte]): Boolean = {
    +    val length = Math.min(x.length, y.length)
    +    for (i <- 0 to length-1) {
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r148698295
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,236 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields: Array[String] = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes: Map[String, Int] = Map(
    --- End diff --
    
    There isn't a great option for Scala- and Java-friendly maps.  So far, we tend to prefix with "java" as in "javaOcvTypes"


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r144193549
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC2" -> 8, "CV_8UC3" -> 16, "CV_8UC4" -> 24,
    +    "CV_8S" -> 1, "CV_8SC1" -> 1, "CV_8SC2" -> 9, "CV_8SC3" -> 17, "CV_8SC4" -> 25,
    +    "CV_16U" -> 2, "CV_16UC1" -> 2, "CV_16UC2" -> 10, "CV_16UC3" -> 18, "CV_16UC4" -> 26,
    +    "CV_16S" -> 3, "CV_16SC1" -> 3, "CV_16SC2" -> 11, "CV_16SC3" -> 19, "CV_16SC4" -> 27,
    +    "CV_32S" -> 4, "CV_32SC1" -> 4, "CV_32SC2" -> 12, "CV_32SC3" -> 20, "CV_32SC4" -> 28,
    +    "CV_32F" -> 5, "CV_32FC1" -> 5, "CV_32FC2" -> 13, "CV_32FC3" -> 21, "CV_32FC4" -> 29,
    +    "CV_64F" -> 6, "CV_64FC1" -> 6, "CV_64FC2" -> 14, "CV_64FC3" -> 22, "CV_64FC4" -> 30
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField("origin", StringType, true) ::
    +      StructField("height", IntegerType, false) ::
    +      StructField("width", IntegerType, false) ::
    +      StructField("nChannels", IntegerType, false) ::
    +      // OpenCV-compatible type: CV_8UC3 in most cases
    +      StructField("mode", StringType, false) ::
    +      // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +      StructField("data", BinaryType, false) :: Nil)
    +
    +  // Dataframe with a single column of images named "image" (nullable)
    +  private val imageDFSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  @Since("2.3.0")
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  @Since("2.3.0")
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  @Since("2.3.0")
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  @Since("2.3.0")
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  @Since("2.3.0")
    +  def getMode(row: Row): String = row.getString(4)
    +
    +  @Since("2.3.0")
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * Check if the dataframe column contains images (i.e. has ImageSchema)
    +   *
    +   * @param df       Dataframe
    +   * @param column   Column name
    +   * @return         True if the given column matches the image schema
    +   */
    +  @Since("2.3.0")
    +  def isImageColumn(df: DataFrame, column: String): Boolean =
    +    df.schema(column).dataType == columnSchema
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return       Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row = Row(Row(origin, -1, -1, -1, undefinedImageType,
    +    Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in dataframe Row
    +   *
    +   * @param origin   Arbitrary string that identifies the image
    +   * @param bytes    Image bytes (for example, jpeg)
    +   * @return         Dataframe Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val is_gray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r148054075
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,139 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +from pyspark.ml.util import *
    +from pyspark.ml.param.shared import *
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame, SparkSession, SQLContext
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +imageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +
    +def getOcvTypes(spark=None):
    +    """
    +    Returns the OpenCV type mapping supported
    +
    +    :param sparkSession (SparkSession): The current spark session
    +    :rtype dict: The OpenCV type mapping supported
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.ocvTypes
    +
    +
    +# DataFrame with a single column of images named "image" (nullable)
    +def getImageSchema(spark=None):
    +    """
    +    Returns the image schema
    +
    +    :param spark (SparkSession): The current spark session
    +    :rtype StructType: The image schema
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    --- End diff --
    
    nice suggestion, I tried this out and it worked!


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r147760700
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,252 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC3" -> 16, "CV_8UC4" -> 24
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField(imageFields(0), StringType, true) ::
    +    StructField(imageFields(1), IntegerType, false) ::
    +    StructField(imageFields(2), IntegerType, false) ::
    +    StructField(imageFields(3), IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(imageFields(4), IntegerType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(imageFields(5), BinaryType, false) :: Nil)
    +
    +  /**
    +   * DataFrame with a single column of images named "image" (nullable)
    +   */
    +  val imageSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the height of the image
    +   *
    +   * @return The height of the image
    +   */
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the width of the image
    +   *
    +   * @return The width of the image
    +   */
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the number of channels in the image
    +   *
    +   * @return The number of channels in the image
    +   */
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the OpenCV representation as an int
    +   *
    +   * @return The OpenCV representation as an int
    +   */
    +  def getMode(row: Row): Int = row.getInt(4)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the image data
    +   *
    +   * @return The image data
    +   */
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * :: Experimental ::
    +   * Check if the DataFrame column contains images (i.e. has ImageSchema)
    +   *
    +   * @param df DataFrame
    +   * @param column Column name
    +   * @return True if the given column matches the image schema
    +   */
    +  def isImageColumn(df: DataFrame, column: String): Boolean =
    +    df.schema(column).dataType == columnSchema
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row =
    +    Row(Row(origin, -1, -1, -1, ocvTypes(undefinedImageType), Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in DataFrame Row
    +   *
    +   * @param origin Arbitrary string that identifies the image
    +   * @param bytes Image bytes (for example, jpeg)
    +   * @return DataFrame Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val isGray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val hasAlpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) = if (isGray) {
    +        (1, ocvTypes("CV_8UC1"))
    +      } else if (hasAlpha) {
    +        (4, ocvTypes("CV_8UC4"))
    +      } else {
    +        (3, ocvTypes("CV_8UC3"))
    +      }
    +
    +      val imageSize = height * width * nChannels
    +      assert(imageSize < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](imageSize)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (isGray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      } else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    +            if (nChannels == 4) {
    +              decoded(offset + 3) = color.getAlpha.toByte
    +            }
    +            offset += nChannels
    +          }
    +        }
    +      }
    +
    +      // the internal "Row" is needed, because the image is a single DataFrame column
    +      Some(Row(Row(origin, height, width, nChannels, mode, decoded)))
    +    }
    +  }
    +
    +  /**
    +   * :: Experimental ::
    +   * Read the directory of images from the local or remote source
    +   *
    +   * @param path Path to the image directory
    +   * @param sparkSession Spark Session
    +   * @param recursive Recursive path search flag
    +   * @param numPartitions Number of the DataFrame partitions
    +   * @param dropImageFailures Drop the files that are not valid images from the result
    +   * @param sampleRatio Fraction of the files loaded
    +   * @return DataFrame with a single column "image" of images;
    +   *         see ImageSchema for the details
    +   */
    +  def readImages(
    +      path: String,
    +      sparkSession: SparkSession = null,
    +      recursive: Boolean = false,
    +      numPartitions: Int = 0,
    +      dropImageFailures: Boolean = false,
    +      sampleRatio: Double = 1.0): DataFrame = {
    +    require(sampleRatio <= 1.0 && sampleRatio >= 0, "sampleRatio should be between 0 and 1")
    +
    +    val session = if (sparkSession != null) sparkSession else SparkSession.builder().getOrCreate
    +    val partitions =
    +      if (numPartitions > 0) {
    +        numPartitions
    +      } else {
    +        session.sparkContext.defaultParallelism
    +      }
    +
    +    val oldRecursiveFlag = RecursiveFlag.setRecursiveFlag(Some(recursive.toString), session)
    --- End diff --
    
    interesting, I will try to rewrite the code using the "with" style.  The other alternative, explicitly writing out the code instead of calling a method, seems more messy since it puts the code inline instead of refactoring it into a method


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #83221 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83221/testReport)** for PR 19439 at commit [`9d434ad`](https://github.com/apache/spark/commit/9d434ad3f24810a8ff4cd6499a7b6cf3e6923b5d).
     * This patch **fails Python style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by dakirsa <gi...@git.apache.org>.
Github user dakirsa commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r143894101
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC2" -> 8, "CV_8UC3" -> 16, "CV_8UC4" -> 24,
    +    "CV_8S" -> 1, "CV_8SC1" -> 1, "CV_8SC2" -> 9, "CV_8SC3" -> 17, "CV_8SC4" -> 25,
    +    "CV_16U" -> 2, "CV_16UC1" -> 2, "CV_16UC2" -> 10, "CV_16UC3" -> 18, "CV_16UC4" -> 26,
    +    "CV_16S" -> 3, "CV_16SC1" -> 3, "CV_16SC2" -> 11, "CV_16SC3" -> 19, "CV_16SC4" -> 27,
    +    "CV_32S" -> 4, "CV_32SC1" -> 4, "CV_32SC2" -> 12, "CV_32SC3" -> 20, "CV_32SC4" -> 28,
    +    "CV_32F" -> 5, "CV_32FC1" -> 5, "CV_32FC2" -> 13, "CV_32FC3" -> 21, "CV_32FC4" -> 29,
    +    "CV_64F" -> 6, "CV_64FC1" -> 6, "CV_64FC2" -> 14, "CV_64FC3" -> 22, "CV_64FC4" -> 30
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField("origin", StringType, true) ::
    +      StructField("height", IntegerType, false) ::
    +      StructField("width", IntegerType, false) ::
    +      StructField("nChannels", IntegerType, false) ::
    +      // OpenCV-compatible type: CV_8UC3 in most cases
    +      StructField("mode", StringType, false) ::
    --- End diff --
    
    Technically, we could use either int or string (in fact, we are using Int in mmlspark implementation) -- but after discussion with Databricks we agreed to use strings for readability. The map is used to emphasize OpenCV compatibility, list all the potential use cases we are planning to cover, and to make a correct translation when actual OpenCV libraries are used to read the binaries. 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84100/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by thunterdb <gi...@git.apache.org>.
Github user thunterdb commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r147661505
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,252 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes = Map(
    --- End diff --
    
    Same thing here (and even more important): `Map[String, Int]` if I am not mistaken.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #82806 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82806/testReport)** for PR 19439 at commit [`9283eec`](https://github.com/apache/spark/commit/9283eecee9d97f38fdd1e55fb145cdb6a5ca6538).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r144857102
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,256 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC2" -> 8, "CV_8UC3" -> 16, "CV_8UC4" -> 24,
    +    "CV_8S" -> 1, "CV_8SC1" -> 1, "CV_8SC2" -> 9, "CV_8SC3" -> 17, "CV_8SC4" -> 25,
    +    "CV_16U" -> 2, "CV_16UC1" -> 2, "CV_16UC2" -> 10, "CV_16UC3" -> 18, "CV_16UC4" -> 26,
    +    "CV_16S" -> 3, "CV_16SC1" -> 3, "CV_16SC2" -> 11, "CV_16SC3" -> 19, "CV_16SC4" -> 27,
    +    "CV_32S" -> 4, "CV_32SC1" -> 4, "CV_32SC2" -> 12, "CV_32SC3" -> 20, "CV_32SC4" -> 28,
    +    "CV_32F" -> 5, "CV_32FC1" -> 5, "CV_32FC2" -> 13, "CV_32FC3" -> 21, "CV_32FC4" -> 29,
    +    "CV_64F" -> 6, "CV_64FC1" -> 6, "CV_64FC2" -> 14, "CV_64FC3" -> 22, "CV_64FC4" -> 30
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField("origin", StringType, true) ::
    +    StructField("height", IntegerType, false) ::
    +    StructField("width", IntegerType, false) ::
    +    StructField("nChannels", IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField("mode", StringType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField("data", BinaryType, false) :: Nil)
    +
    +  // DataFrame with a single column of images named "image" (nullable)
    +  private val imageDFSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the height of the image
    +   *
    +   * @return The height of the image
    +   */
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the width of the image
    +   *
    +   * @return The width of the image
    +   */
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the number of channels in the image
    +   *
    +   * @return The number of channels in the image
    +   */
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the OpenCV representation
    +   *
    +   * @return The OpenCV representation
    +   */
    +  def getMode(row: Row): String = row.getString(4)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the image data
    +   *
    +   * @return The image data
    +   */
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * :: Experimental ::
    +   * Check if the DataFrame column contains images (i.e. has ImageSchema)
    +   *
    +   * @param df       DataFrame
    +   * @param column   Column name
    +   * @return         True if the given column matches the image schema
    +   */
    +  def isImageColumn(df: DataFrame, column: String): Boolean =
    +    df.schema(column).dataType == columnSchema
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return       Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row = Row(Row(origin, -1, -1, -1, undefinedImageType,
    +    Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in DataFrame Row
    +   *
    +   * @param origin   Arbitrary string that identifies the image
    +   * @param bytes    Image bytes (for example, jpeg)
    +   * @return         DataFrame Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val isGray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val hasAlpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) = if (isGray) {
    +        (1, "CV_8UC1")
    +      } else if (hasAlpha) {
    +        (4, "CV_8UC4")
    +      } else {
    +        (3, "CV_8UC3")
    +      }
    +
    +      val imageSize = height * width * nChannels
    +      assert(imageSize < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](imageSize)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (isGray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      } else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    +            if (nChannels == 4) {
    +              decoded(offset + 3) = color.getAlpha.toByte
    +            }
    +            offset += nChannels
    +          }
    +        }
    +      }
    +
    +      // the internal "Row" is needed, because the image is a single DataFrame column
    +      Some(Row(Row(origin, height, width, nChannels, mode, decoded)))
    +    }
    +  }
    +
    +  /**
    +   * :: Experimental ::
    +   * Read the directory of images from the local or remote source
    +   *
    +   * @param path               Path to the image directory
    +   * @param sparkSession       Spark Session
    +   * @param recursive          Recursive path search flag
    +   * @param numPartitions      Number of the DataFrame partitions
    +   * @param dropImageFailures  Drop the files that are not valid images from the result
    +   * @param sampleRatio        Fraction of the files loaded
    +   * @return                   DataFrame with a single column "image" of images;
    +   *                           see ImageSchema for the details
    +   */
    +  def readImages(path: String,
    +                 sparkSession: SparkSession = null,
    +                 recursive: Boolean = false,
    +                 numPartitions: Int = 0,
    +                 dropImageFailures: Boolean = false,
    +                 sampleRatio: Double = 1.0): DataFrame = {
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84042/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r148030234
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,139 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +from pyspark.ml.util import *
    +from pyspark.ml.param.shared import *
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame, SparkSession, SQLContext
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +imageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +
    +def getOcvTypes(spark=None):
    +    """
    +    Returns the OpenCV type mapping supported
    +
    +    :param sparkSession (SparkSession): The current spark session
    +    :rtype dict: The OpenCV type mapping supported
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.ocvTypes
    +
    +
    +# DataFrame with a single column of images named "image" (nullable)
    +def getImageSchema(spark=None):
    +    """
    +    Returns the image schema
    +
    +    :param spark (SparkSession): The current spark session
    +    :rtype StructType: The image schema
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.imageSchema
    +
    +
    +def toNDArray(image):
    +    """
    +    Converts an image to a one-dimensional array.
    +
    +    :param image (object): The image to be converted
    +    :rtype array: The image as a one-dimensional array
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    height = image.height
    +    width = image.width
    +    nChannels = image.nChannels
    +    return np.ndarray(
    +        shape=(height, width, nChannels),
    +        dtype=np.uint8,
    +        buffer=image.data,
    +        strides=(width * nChannels, nChannels, 1))
    +
    +
    +def toImage(array, origin="", spark=None):
    +    """
    +    Converts a one-dimensional array to a two-dimensional image.
    +
    +    :param array (array): The array to convert to image
    +    :param origin (str): Path to the image
    +    :param spark (SparkSession): The current spark session
    +    :rtype object: Two dimensional image
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    if array.ndim != 3:
    +        raise
    --- End diff --
    
    how about ValueError - I'll update with that


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by WeichenXu123 <gi...@git.apache.org>.
Github user WeichenXu123 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r148700390
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,236 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields: Array[String] = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes: Map[String, Int] = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC3" -> 16, "CV_8UC4" -> 24
    +  )
    +
    +  /**
    +   * Used for conversion to python
    +   */
    +  val _ocvTypes: java.util.Map[String, Int] = ocvTypes.asJava
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField(imageFields(0), StringType, true) ::
    +    StructField(imageFields(1), IntegerType, false) ::
    +    StructField(imageFields(2), IntegerType, false) ::
    +    StructField(imageFields(3), IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(imageFields(4), IntegerType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(imageFields(5), BinaryType, false) :: Nil)
    +
    +  /**
    +   * DataFrame with a single column of images named "image" (nullable)
    +   */
    +  val imageSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the height of the image
    +   *
    +   * @return The height of the image
    +   */
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the width of the image
    +   *
    +   * @return The width of the image
    +   */
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the number of channels in the image
    +   *
    +   * @return The number of channels in the image
    +   */
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the OpenCV representation as an int
    +   *
    +   * @return The OpenCV representation as an int
    +   */
    +  def getMode(row: Row): Int = row.getInt(4)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the image data
    +   *
    +   * @return The image data
    +   */
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row =
    +    Row(Row(origin, -1, -1, -1, ocvTypes(undefinedImageType), Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in DataFrame Row
    +   *
    +   * @param origin Arbitrary string that identifies the image
    +   * @param bytes Image bytes (for example, jpeg)
    +   * @return DataFrame Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val isGray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val hasAlpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) = if (isGray) {
    +        (1, ocvTypes("CV_8UC1"))
    +      } else if (hasAlpha) {
    +        (4, ocvTypes("CV_8UC4"))
    +      } else {
    +        (3, ocvTypes("CV_8UC3"))
    +      }
    +
    +      val imageSize = height * width * nChannels
    +      assert(imageSize < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](imageSize)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (isGray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      } else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    +            if (nChannels == 4) {
    +              decoded(offset + 3) = color.getAlpha.toByte
    +            }
    +            offset += nChannels
    +          }
    +        }
    +      }
    +
    +      // the internal "Row" is needed, because the image is a single DataFrame column
    +      Some(Row(Row(origin, height, width, nChannels, mode, decoded)))
    +    }
    +  }
    +
    +  /**
    +   * :: Experimental ::
    +   * Read the directory of images from the local or remote source
    +   *
    +   * @param path Path to the image directory
    +   * @param sparkSession Spark Session, if omitted gets or creates the session
    +   * @param recursive Recursive path search flag
    +   * @param numPartitions Number of the DataFrame partitions,
    +   *                      if omitted uses defaultParallelism instead
    +   * @param dropImageFailures Drop the files that are not valid images from the result
    +   * @param sampleRatio Fraction of the files loaded
    +   * @return DataFrame with a single column "image" of images;
    +   *         see ImageSchema for the details
    +   */
    +  def readImages(
    +      path: String,
    +      sparkSession: SparkSession = null,
    +      recursive: Boolean = false,
    +      numPartitions: Int = 0,
    +      dropImageFailures: Boolean = false,
    +      sampleRatio: Double = 1.0): DataFrame = {
    +    require(sampleRatio <= 1.0 && sampleRatio >= 0, "sampleRatio should be between 0 and 1")
    +
    +    val session = if (sparkSession != null) sparkSession else SparkSession.builder().getOrCreate
    +    val partitions =
    +      if (numPartitions > 0) {
    +        numPartitions
    +      } else {
    +        session.sparkContext.defaultParallelism
    +      }
    +
    +    RecursiveFlag.withRecursiveFlag(recursive, session) {
    +      SamplePathFilter.withPathFilter(sampleRatio, session) {
    +        val streams = session.sparkContext.binaryFiles(path, partitions).repartition(partitions)
    --- End diff --
    
    But, we should add comment that this is workround for the param ignored bug 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r145490630
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,124 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +import pyspark
    +from pyspark import SparkContext
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame
    +from pyspark.ml.param.shared import *
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +ImageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r147698388
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,140 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +from pyspark.ml.util import *
    +from pyspark.ml.param.shared import *
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame, SparkSession, SQLContext
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +imageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +
    +def getOcvTypes(spark=None):
    +    """
    +    Returns the OpenCV type mapping supported
    +
    +    :param sparkSession (SparkSession): The current spark session
    +    :rtype dict: The OpenCV type mapping supported
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.ocvTypes
    +
    +
    +# DataFrame with a single column of images named "image" (nullable)
    +def getImageSchema(spark=None):
    +    """
    +    Returns the image schema
    +
    +    :param spark (SparkSession): The current spark session
    +    :rtype StructType: The image schema
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.imageSchema
    +
    +
    +def toNDArray(image):
    +    """
    +    Converts an image to a one-dimensional array.
    +
    +    :param image (object): The image to be converted
    +    :rtype array: The image as a one-dimensional array
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    height = image.height
    +    width = image.width
    +    nChannels = image.nChannels
    +    return np.ndarray(
    +        shape=(height, width, nChannels),
    +        dtype=np.uint8,
    +        buffer=image.data,
    +        strides=(width * nChannels, nChannels, 1))
    +
    +
    +def toImage(array, origin="", spark=None):
    +    """
    +    Converts a one-dimensional array to a two-dimensional image.
    +
    +    :param array (array): The array to convert to image
    +    :param origin (str): Path to the image
    +    :param mode (str): OpenCV compatible type
    --- End diff --
    
    Looks we can remove `mode`.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #82827 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82827/testReport)** for PR 19439 at commit [`697585f`](https://github.com/apache/spark/commit/697585fd2544d5a97534c7895a92e5677fb70a8d).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r147744313
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,140 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +from pyspark.ml.util import *
    +from pyspark.ml.param.shared import *
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame, SparkSession, SQLContext
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +imageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +
    +def getOcvTypes(spark=None):
    +    """
    +    Returns the OpenCV type mapping supported
    +
    +    :param sparkSession (SparkSession): The current spark session
    +    :rtype dict: The OpenCV type mapping supported
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.ocvTypes
    +
    +
    +# DataFrame with a single column of images named "image" (nullable)
    +def getImageSchema(spark=None):
    +    """
    +    Returns the image schema
    +
    +    :param spark (SparkSession): The current spark session
    +    :rtype StructType: The image schema
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.imageSchema
    +
    +
    +def toNDArray(image):
    +    """
    +    Converts an image to a one-dimensional array.
    +
    +    :param image (object): The image to be converted
    +    :rtype array: The image as a one-dimensional array
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    height = image.height
    +    width = image.width
    +    nChannels = image.nChannels
    +    return np.ndarray(
    +        shape=(height, width, nChannels),
    +        dtype=np.uint8,
    +        buffer=image.data,
    +        strides=(width * nChannels, nChannels, 1))
    +
    +
    +def toImage(array, origin="", spark=None):
    +    """
    +    Converts a one-dimensional array to a two-dimensional image.
    +
    +    :param array (array): The array to convert to image
    +    :param origin (str): Path to the image
    +    :param mode (str): OpenCV compatible type
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r147739022
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,252 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC3" -> 16, "CV_8UC4" -> 24
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField(imageFields(0), StringType, true) ::
    +    StructField(imageFields(1), IntegerType, false) ::
    +    StructField(imageFields(2), IntegerType, false) ::
    +    StructField(imageFields(3), IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(imageFields(4), IntegerType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(imageFields(5), BinaryType, false) :: Nil)
    +
    +  /**
    +   * DataFrame with a single column of images named "image" (nullable)
    +   */
    +  val imageSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the height of the image
    +   *
    +   * @return The height of the image
    +   */
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the width of the image
    +   *
    +   * @return The width of the image
    +   */
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the number of channels in the image
    +   *
    +   * @return The number of channels in the image
    +   */
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the OpenCV representation as an int
    +   *
    +   * @return The OpenCV representation as an int
    +   */
    +  def getMode(row: Row): Int = row.getInt(4)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the image data
    +   *
    +   * @return The image data
    +   */
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * :: Experimental ::
    +   * Check if the DataFrame column contains images (i.e. has ImageSchema)
    +   *
    +   * @param df DataFrame
    +   * @param column Column name
    +   * @return True if the given column matches the image schema
    +   */
    +  def isImageColumn(df: DataFrame, column: String): Boolean =
    +    df.schema(column).dataType == columnSchema
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row =
    +    Row(Row(origin, -1, -1, -1, ocvTypes(undefinedImageType), Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in DataFrame Row
    +   *
    +   * @param origin Arbitrary string that identifies the image
    +   * @param bytes Image bytes (for example, jpeg)
    +   * @return DataFrame Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val isGray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val hasAlpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) = if (isGray) {
    +        (1, ocvTypes("CV_8UC1"))
    +      } else if (hasAlpha) {
    +        (4, ocvTypes("CV_8UC4"))
    +      } else {
    +        (3, ocvTypes("CV_8UC3"))
    +      }
    +
    +      val imageSize = height * width * nChannels
    +      assert(imageSize < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](imageSize)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (isGray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      } else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    +            if (nChannels == 4) {
    +              decoded(offset + 3) = color.getAlpha.toByte
    +            }
    +            offset += nChannels
    +          }
    +        }
    +      }
    +
    +      // the internal "Row" is needed, because the image is a single DataFrame column
    +      Some(Row(Row(origin, height, width, nChannels, mode, decoded)))
    +    }
    +  }
    +
    +  /**
    +   * :: Experimental ::
    +   * Read the directory of images from the local or remote source
    +   *
    +   * @param path Path to the image directory
    +   * @param sparkSession Spark Session
    +   * @param recursive Recursive path search flag
    +   * @param numPartitions Number of the DataFrame partitions
    +   * @param dropImageFailures Drop the files that are not valid images from the result
    +   * @param sampleRatio Fraction of the files loaded
    +   * @return DataFrame with a single column "image" of images;
    +   *         see ImageSchema for the details
    +   */
    +  def readImages(
    +      path: String,
    +      sparkSession: SparkSession = null,
    +      recursive: Boolean = false,
    +      numPartitions: Int = 0,
    +      dropImageFailures: Boolean = false,
    +      sampleRatio: Double = 1.0): DataFrame = {
    +    require(sampleRatio <= 1.0 && sampleRatio >= 0, "sampleRatio should be between 0 and 1")
    +
    +    val session = if (sparkSession != null) sparkSession else SparkSession.builder().getOrCreate
    +    val partitions =
    +      if (numPartitions > 0) {
    +        numPartitions
    +      } else {
    +        session.sparkContext.defaultParallelism
    +      }
    +
    +    val oldRecursiveFlag = RecursiveFlag.setRecursiveFlag(Some(recursive.toString), session)
    +    val sampleImages = sampleRatio < 1
    +    val oldPathFilter: Option[Class[_]] =
    +      if (sampleImages) {
    +        SamplePathFilter.setPathFilter(Some(classOf[SamplePathFilter]), sampleRatio, session)
    +      } else {
    +        None
    +      }
    +
    +    try {
    +      val streams = session.sparkContext.binaryFiles(path, partitions)
    +        .repartition(partitions)
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r148027168
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields: Array[String] = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes: Map[String, Int] = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC3" -> 16, "CV_8UC4" -> 24
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField(imageFields(0), StringType, true) ::
    +    StructField(imageFields(1), IntegerType, false) ::
    +    StructField(imageFields(2), IntegerType, false) ::
    +    StructField(imageFields(3), IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(imageFields(4), IntegerType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(imageFields(5), BinaryType, false) :: Nil)
    +
    +  /**
    +   * DataFrame with a single column of images named "image" (nullable)
    +   */
    +  val imageSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the height of the image
    +   *
    +   * @return The height of the image
    +   */
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the width of the image
    +   *
    +   * @return The width of the image
    +   */
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the number of channels in the image
    +   *
    +   * @return The number of channels in the image
    +   */
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the OpenCV representation as an int
    +   *
    +   * @return The OpenCV representation as an int
    +   */
    +  def getMode(row: Row): Int = row.getInt(4)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the image data
    +   *
    +   * @return The image data
    +   */
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row =
    +    Row(Row(origin, -1, -1, -1, ocvTypes(undefinedImageType), Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in DataFrame Row
    +   *
    +   * @param origin Arbitrary string that identifies the image
    +   * @param bytes Image bytes (for example, jpeg)
    +   * @return DataFrame Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val isGray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val hasAlpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) = if (isGray) {
    +        (1, ocvTypes("CV_8UC1"))
    +      } else if (hasAlpha) {
    +        (4, ocvTypes("CV_8UC4"))
    +      } else {
    +        (3, ocvTypes("CV_8UC3"))
    +      }
    +
    +      val imageSize = height * width * nChannels
    +      assert(imageSize < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](imageSize)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (isGray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      } else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    +            if (nChannels == 4) {
    +              decoded(offset + 3) = color.getAlpha.toByte
    +            }
    +            offset += nChannels
    +          }
    +        }
    +      }
    +
    +      // the internal "Row" is needed, because the image is a single DataFrame column
    +      Some(Row(Row(origin, height, width, nChannels, mode, decoded)))
    +    }
    +  }
    +
    +  /**
    +   * :: Experimental ::
    +   * Read the directory of images from the local or remote source
    +   *
    +   * @param path Path to the image directory
    +   * @param sparkSession Spark Session, if omitted gets or creates the session
    +   * @param recursive Recursive path search flag
    +   * @param numPartitions Number of the DataFrame partitions,
    +   *                      if omitted uses defaultParallelism instead
    +   * @param dropImageFailures Drop the files that are not valid images from the result
    +   * @param sampleRatio Fraction of the files loaded
    +   * @return DataFrame with a single column "image" of images;
    +   *         see ImageSchema for the details
    +   */
    +  def readImages(
    +      path: String,
    +      sparkSession: SparkSession = null,
    +      recursive: Boolean = false,
    +      numPartitions: Int = 0,
    +      dropImageFailures: Boolean = false,
    +      sampleRatio: Double = 1.0): DataFrame = {
    +    require(sampleRatio <= 1.0 && sampleRatio >= 0, "sampleRatio should be between 0 and 1")
    +
    +    val session = if (sparkSession != null) sparkSession else SparkSession.builder().getOrCreate
    +    val partitions =
    +      if (numPartitions > 0) {
    +        numPartitions
    +      } else {
    +        session.sparkContext.defaultParallelism
    +      }
    +
    +    RecursiveFlag.withRecursiveFlag(recursive, session)(
    --- End diff --
    
    done but note those are parentheses not curly braces


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #83344 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83344/testReport)** for PR 19439 at commit [`387d90c`](https://github.com/apache/spark/commit/387d90ca79ae840adb4c799a156f782dc9472207).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #83335 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83335/testReport)** for PR 19439 at commit [`d8af945`](https://github.com/apache/spark/commit/d8af945eff4fab6a742666331ab122c78400c9b4).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r147741895
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,252 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC3" -> 16, "CV_8UC4" -> 24
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField(imageFields(0), StringType, true) ::
    +    StructField(imageFields(1), IntegerType, false) ::
    +    StructField(imageFields(2), IntegerType, false) ::
    +    StructField(imageFields(3), IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(imageFields(4), IntegerType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(imageFields(5), BinaryType, false) :: Nil)
    +
    +  /**
    +   * DataFrame with a single column of images named "image" (nullable)
    +   */
    +  val imageSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the height of the image
    +   *
    +   * @return The height of the image
    +   */
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the width of the image
    +   *
    +   * @return The width of the image
    +   */
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the number of channels in the image
    +   *
    +   * @return The number of channels in the image
    +   */
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the OpenCV representation as an int
    +   *
    +   * @return The OpenCV representation as an int
    +   */
    +  def getMode(row: Row): Int = row.getInt(4)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the image data
    +   *
    +   * @return The image data
    +   */
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * :: Experimental ::
    +   * Check if the DataFrame column contains images (i.e. has ImageSchema)
    +   *
    +   * @param df DataFrame
    +   * @param column Column name
    +   * @return True if the given column matches the image schema
    +   */
    +  def isImageColumn(df: DataFrame, column: String): Boolean =
    +    df.schema(column).dataType == columnSchema
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row =
    +    Row(Row(origin, -1, -1, -1, ocvTypes(undefinedImageType), Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in DataFrame Row
    +   *
    +   * @param origin Arbitrary string that identifies the image
    +   * @param bytes Image bytes (for example, jpeg)
    +   * @return DataFrame Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val isGray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val hasAlpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) = if (isGray) {
    +        (1, ocvTypes("CV_8UC1"))
    +      } else if (hasAlpha) {
    +        (4, ocvTypes("CV_8UC4"))
    +      } else {
    +        (3, ocvTypes("CV_8UC3"))
    +      }
    +
    +      val imageSize = height * width * nChannels
    +      assert(imageSize < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](imageSize)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (isGray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      } else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    +            if (nChannels == 4) {
    +              decoded(offset + 3) = color.getAlpha.toByte
    +            }
    +            offset += nChannels
    +          }
    +        }
    +      }
    +
    +      // the internal "Row" is needed, because the image is a single DataFrame column
    +      Some(Row(Row(origin, height, width, nChannels, mode, decoded)))
    +    }
    +  }
    +
    +  /**
    +   * :: Experimental ::
    +   * Read the directory of images from the local or remote source
    +   *
    +   * @param path Path to the image directory
    +   * @param sparkSession Spark Session
    +   * @param recursive Recursive path search flag
    +   * @param numPartitions Number of the DataFrame partitions
    +   * @param dropImageFailures Drop the files that are not valid images from the result
    +   * @param sampleRatio Fraction of the files loaded
    +   * @return DataFrame with a single column "image" of images;
    +   *         see ImageSchema for the details
    +   */
    +  def readImages(
    +      path: String,
    +      sparkSession: SparkSession = null,
    +      recursive: Boolean = false,
    +      numPartitions: Int = 0,
    +      dropImageFailures: Boolean = false,
    +      sampleRatio: Double = 1.0): DataFrame = {
    +    require(sampleRatio <= 1.0 && sampleRatio >= 0, "sampleRatio should be between 0 and 1")
    +
    +    val session = if (sparkSession != null) sparkSession else SparkSession.builder().getOrCreate
    +    val partitions =
    +      if (numPartitions > 0) {
    +        numPartitions
    +      } else {
    +        session.sparkContext.defaultParallelism
    +      }
    +
    +    val oldRecursiveFlag = RecursiveFlag.setRecursiveFlag(Some(recursive.toString), session)
    +    val sampleImages = sampleRatio < 1
    +    val oldPathFilter: Option[Class[_]] =
    +      if (sampleImages) {
    +        SamplePathFilter.setPathFilter(Some(classOf[SamplePathFilter]), sampleRatio, session)
    +      } else {
    +        None
    +      }
    +
    +    try {
    +      val streams = session.sparkContext.binaryFiles(path, partitions)
    +        .repartition(partitions)
    +
    +      val convert = (stream: (String, PortableDataStream)) =>
    --- End diff --
    
    @HyukjinKwon please see comment from @srowen for this code:
    "I think you can write this to avoid repeating decode(origin, stream.toArray)"
    to resolve I still kept convert method but added the "case (origin, bytes)" to make the code look cleaner


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r148698771
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,236 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields: Array[String] = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes: Map[String, Int] = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC3" -> 16, "CV_8UC4" -> 24
    +  )
    +
    +  /**
    +   * Used for conversion to python
    +   */
    +  val _ocvTypes: java.util.Map[String, Int] = ocvTypes.asJava
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField(imageFields(0), StringType, true) ::
    +    StructField(imageFields(1), IntegerType, false) ::
    +    StructField(imageFields(2), IntegerType, false) ::
    +    StructField(imageFields(3), IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(imageFields(4), IntegerType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(imageFields(5), BinaryType, false) :: Nil)
    +
    +  /**
    +   * DataFrame with a single column of images named "image" (nullable)
    +   */
    +  val imageSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the height of the image
    +   *
    +   * @return The height of the image
    +   */
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the width of the image
    +   *
    +   * @return The width of the image
    +   */
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the number of channels in the image
    +   *
    +   * @return The number of channels in the image
    +   */
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the OpenCV representation as an int
    +   *
    +   * @return The OpenCV representation as an int
    +   */
    +  def getMode(row: Row): Int = row.getInt(4)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the image data
    +   *
    +   * @return The image data
    +   */
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row =
    +    Row(Row(origin, -1, -1, -1, ocvTypes(undefinedImageType), Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in DataFrame Row
    +   *
    +   * @param origin Arbitrary string that identifies the image
    +   * @param bytes Image bytes (for example, jpeg)
    +   * @return DataFrame Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val isGray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val hasAlpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) = if (isGray) {
    +        (1, ocvTypes("CV_8UC1"))
    +      } else if (hasAlpha) {
    +        (4, ocvTypes("CV_8UC4"))
    +      } else {
    +        (3, ocvTypes("CV_8UC3"))
    +      }
    +
    +      val imageSize = height * width * nChannels
    +      assert(imageSize < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](imageSize)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (isGray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      } else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    +            if (nChannels == 4) {
    +              decoded(offset + 3) = color.getAlpha.toByte
    +            }
    +            offset += nChannels
    +          }
    +        }
    +      }
    +
    +      // the internal "Row" is needed, because the image is a single DataFrame column
    +      Some(Row(Row(origin, height, width, nChannels, mode, decoded)))
    +    }
    +  }
    +
    +  /**
    +   * :: Experimental ::
    +   * Read the directory of images from the local or remote source
    +   *
    +   * @param path Path to the image directory
    +   * @param sparkSession Spark Session, if omitted gets or creates the session
    +   * @param recursive Recursive path search flag
    +   * @param numPartitions Number of the DataFrame partitions,
    +   *                      if omitted uses defaultParallelism instead
    +   * @param dropImageFailures Drop the files that are not valid images from the result
    +   * @param sampleRatio Fraction of the files loaded
    +   * @return DataFrame with a single column "image" of images;
    +   *         see ImageSchema for the details
    +   */
    +  def readImages(
    +      path: String,
    +      sparkSession: SparkSession = null,
    +      recursive: Boolean = false,
    +      numPartitions: Int = 0,
    +      dropImageFailures: Boolean = false,
    +      sampleRatio: Double = 1.0): DataFrame = {
    +    require(sampleRatio <= 1.0 && sampleRatio >= 0, "sampleRatio should be between 0 and 1")
    +
    +    val session = if (sparkSession != null) sparkSession else SparkSession.builder().getOrCreate
    +    val partitions =
    +      if (numPartitions > 0) {
    +        numPartitions
    +      } else {
    +        session.sparkContext.defaultParallelism
    +      }
    +
    +    RecursiveFlag.withRecursiveFlag(recursive, session) {
    --- End diff --
    
    I synced with @liancheng about this, and it's not that safe in terms of the HadoopConfiguration but is the best option until we turn this into a SQL datasource (which will be blocked by having a binary files datasource).
    
    Here's the issue: These wrapping helpers RecursiveFlag and SamplePathFilter alter the global HadoopConfiguration. If multiple jobs are run in parallel, this job could affect other jobs.
    
    However, it will be hard to fix since you can't pass a config to the binaryFiles method.  And it probably won't affect too many use cases.
    
    Let's document this issue with a warning in the docstring for this method for now.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by thunterdb <gi...@git.apache.org>.
Github user thunterdb commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    @hhbyyh thank you for bringing up these questions. In response to your questions:
    
    > Does the current schema support or plan to support image feature data in Floats[] or Doubles[]?
    
    It does, indirectly: this is what the field types `CV_32FXX` do. You need to do some low-level casting to convert the byte array to array of numbers, but that should not be a limitation for most applications.
    
    > Correct me if I'm wrong, I don't see ocvTypes plays any role in the code. If the field is for future extension. maybe It's better to keep only the supported types. But not all the OpenCV types.
    
    Indeed. These fields are added so that users know what values are allowed for these fields. A scala-friendly choice would have been sealed traits or enumerations, but the consensus in Spark has been for a low-level representation. Nothing precludes adding a case class to represent this dataset in the future, with more type safety information.
    
    > In most scenarios, deep learning applications use rescaled/cropped images (typically 256, 224 or smaller). Maybe add an extra parameter "smallSideSize" to the readImages method, which is more convenient for the users and we can avoid to cache the image of original size (which could be 100 times larger than the scaled image). This can be done in follow up PR.
    
    This is a good point, that we all hit. The issue here is that there is no unique definition of rescaling (what interpolation? crop and scale? scale and crop?) and each library has made different choices. This is certainly something that would be good for a future PR.
    
    > Not sure about the reason to include "origin" info into the image data. Based on my experience, path info serves better as a separate column in the DataFrame. (E.g. prediction)
    
    Yes, this feature has been debated. Some developers have had a compelling need for directly accessing some information about the origin of the image directly inside the image.
    
    > IMO the parameter "recursive" may not be necessary. Existing wild card matching can provides more functions.
    
    Indeed, this feature may not seem that useful at first glance. For some hadoop file systems though, in which images are accessed in a batched manner, it is useful to traverse these batches. This is important for performance reasons. This is why it is marked as experimental for the time being.
    
    > For scala API, ImageSchema should be in a separate file but not to be mixed with image reading.
    
    I do not have a strong opinion about this point, I will let other developers decide.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r150735461
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -48,7 +48,7 @@ def imageSchema(self):
             """
             Returns the image schema.
     
    -        :rtype StructType: a DataFrame with a single column of images
    +        :return: StructType: a DataFrame with a single column of images
    --- End diff --
    
    done, and thank you for the useful tip on the documentation!


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84117/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r150375535
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,239 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +/**
    + * :: Experimental ::
    + * Defines the image schema and methods to read and manipulate images.
    + */
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val ocvTypes: Map[String, Int] = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC3" -> 16, "CV_8UC4" -> 24
    +  )
    +
    +  /**
    +   * Used for conversion to python
    --- End diff --
    
    Let's add `(Java-specific)`, for example, 
    
    ```
    (Java-specific) OpenCV type mapping supported
    ``` 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #83097 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83097/testReport)** for PR 19439 at commit [`ec64d7f`](https://github.com/apache/spark/commit/ec64d7f7a389ee75fecba3fbfa4585c6a0eae389).
     * This patch **fails Python style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r144856282
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,256 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC2" -> 8, "CV_8UC3" -> 16, "CV_8UC4" -> 24,
    +    "CV_8S" -> 1, "CV_8SC1" -> 1, "CV_8SC2" -> 9, "CV_8SC3" -> 17, "CV_8SC4" -> 25,
    +    "CV_16U" -> 2, "CV_16UC1" -> 2, "CV_16UC2" -> 10, "CV_16UC3" -> 18, "CV_16UC4" -> 26,
    +    "CV_16S" -> 3, "CV_16SC1" -> 3, "CV_16SC2" -> 11, "CV_16SC3" -> 19, "CV_16SC4" -> 27,
    +    "CV_32S" -> 4, "CV_32SC1" -> 4, "CV_32SC2" -> 12, "CV_32SC3" -> 20, "CV_32SC4" -> 28,
    +    "CV_32F" -> 5, "CV_32FC1" -> 5, "CV_32FC2" -> 13, "CV_32FC3" -> 21, "CV_32FC4" -> 29,
    +    "CV_64F" -> 6, "CV_64FC1" -> 6, "CV_64FC2" -> 14, "CV_64FC3" -> 22, "CV_64FC4" -> 30
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField("origin", StringType, true) ::
    +    StructField("height", IntegerType, false) ::
    +    StructField("width", IntegerType, false) ::
    +    StructField("nChannels", IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField("mode", StringType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField("data", BinaryType, false) :: Nil)
    +
    +  // DataFrame with a single column of images named "image" (nullable)
    +  private val imageDFSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the height of the image
    +   *
    +   * @return The height of the image
    +   */
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the width of the image
    +   *
    +   * @return The width of the image
    +   */
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the number of channels in the image
    +   *
    +   * @return The number of channels in the image
    +   */
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the OpenCV representation
    +   *
    +   * @return The OpenCV representation
    +   */
    +  def getMode(row: Row): String = row.getString(4)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the image data
    +   *
    +   * @return The image data
    +   */
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * :: Experimental ::
    +   * Check if the DataFrame column contains images (i.e. has ImageSchema)
    +   *
    +   * @param df       DataFrame
    +   * @param column   Column name
    +   * @return         True if the given column matches the image schema
    +   */
    +  def isImageColumn(df: DataFrame, column: String): Boolean =
    +    df.schema(column).dataType == columnSchema
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return       Row with the default values
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r144299736
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,256 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC2" -> 8, "CV_8UC3" -> 16, "CV_8UC4" -> 24,
    +    "CV_8S" -> 1, "CV_8SC1" -> 1, "CV_8SC2" -> 9, "CV_8SC3" -> 17, "CV_8SC4" -> 25,
    +    "CV_16U" -> 2, "CV_16UC1" -> 2, "CV_16UC2" -> 10, "CV_16UC3" -> 18, "CV_16UC4" -> 26,
    +    "CV_16S" -> 3, "CV_16SC1" -> 3, "CV_16SC2" -> 11, "CV_16SC3" -> 19, "CV_16SC4" -> 27,
    +    "CV_32S" -> 4, "CV_32SC1" -> 4, "CV_32SC2" -> 12, "CV_32SC3" -> 20, "CV_32SC4" -> 28,
    +    "CV_32F" -> 5, "CV_32FC1" -> 5, "CV_32FC2" -> 13, "CV_32FC3" -> 21, "CV_32FC4" -> 29,
    +    "CV_64F" -> 6, "CV_64FC1" -> 6, "CV_64FC2" -> 14, "CV_64FC3" -> 22, "CV_64FC4" -> 30
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField("origin", StringType, true) ::
    +    StructField("height", IntegerType, false) ::
    +    StructField("width", IntegerType, false) ::
    +    StructField("nChannels", IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField("mode", StringType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField("data", BinaryType, false) :: Nil)
    +
    +  // DataFrame with a single column of images named "image" (nullable)
    +  private val imageDFSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the height of the image
    +   *
    +   * @return The height of the image
    +   */
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the width of the image
    +   *
    +   * @return The width of the image
    +   */
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the number of channels in the image
    +   *
    +   * @return The number of channels in the image
    +   */
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the OpenCV representation
    +   *
    +   * @return The OpenCV representation
    +   */
    +  def getMode(row: Row): String = row.getString(4)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the image data
    +   *
    +   * @return The image data
    +   */
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * :: Experimental ::
    +   * Check if the DataFrame column contains images (i.e. has ImageSchema)
    +   *
    +   * @param df       DataFrame
    +   * @param column   Column name
    +   * @return         True if the given column matches the image schema
    +   */
    +  def isImageColumn(df: DataFrame, column: String): Boolean =
    +    df.schema(column).dataType == columnSchema
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return       Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row = Row(Row(origin, -1, -1, -1, undefinedImageType,
    +    Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in DataFrame Row
    +   *
    +   * @param origin   Arbitrary string that identifies the image
    +   * @param bytes    Image bytes (for example, jpeg)
    +   * @return         DataFrame Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val isGray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val hasAlpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) = if (isGray) {
    +        (1, "CV_8UC1")
    +      } else if (hasAlpha) {
    +        (4, "CV_8UC4")
    +      } else {
    +        (3, "CV_8UC3")
    +      }
    +
    +      val imageSize = height * width * nChannels
    +      assert(imageSize < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](imageSize)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (isGray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      } else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    +            if (nChannels == 4) {
    +              decoded(offset + 3) = color.getAlpha.toByte
    +            }
    +            offset += nChannels
    +          }
    +        }
    +      }
    +
    +      // the internal "Row" is needed, because the image is a single DataFrame column
    +      Some(Row(Row(origin, height, width, nChannels, mode, decoded)))
    +    }
    +  }
    +
    +  /**
    +   * :: Experimental ::
    +   * Read the directory of images from the local or remote source
    +   *
    +   * @param path               Path to the image directory
    +   * @param sparkSession       Spark Session
    +   * @param recursive          Recursive path search flag
    +   * @param numPartitions      Number of the DataFrame partitions
    +   * @param dropImageFailures  Drop the files that are not valid images from the result
    +   * @param sampleRatio        Fraction of the files loaded
    +   * @return                   DataFrame with a single column "image" of images;
    +   *                           see ImageSchema for the details
    --- End diff --
    
    ditto for indentation.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by WeichenXu123 <gi...@git.apache.org>.
Github user WeichenXu123 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r146163650
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,122 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +import pyspark
    +from pyspark import SparkContext
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame
    +from pyspark.ml.param.shared import *
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +imageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +ocvTypes = {
    +    undefinedImageType: -1,
    +    "CV_8U": 0, "CV_8UC1": 0, "CV_8UC2": 8, "CV_8UC3": 16, "CV_8UC4": 24,
    +    "CV_8S": 1, "CV_8SC1": 1, "CV_8SC2": 9, "CV_8SC3": 17, "CV_8SC4": 25,
    +    "CV_16U": 2, "CV_16UC1": 2, "CV_16UC2": 10, "CV_16UC3": 18, "CV_16UC4": 26,
    +    "CV_16S": 3, "CV_16SC1": 3, "CV_16SC2": 11, "CV_16SC3": 19, "CV_16SC4": 27,
    +    "CV_32S": 4, "CV_32SC1": 4, "CV_32SC2": 12, "CV_32SC3": 20, "CV_32SC4": 28,
    +    "CV_32F": 5, "CV_32FC1": 5, "CV_32FC2": 13, "CV_32FC3": 21, "CV_32FC4": 29,
    +    "CV_64F": 6, "CV_64FC1": 6, "CV_64FC2": 14, "CV_64FC3": 22, "CV_64FC4": 30
    +}
    --- End diff --
    
    The hardcoding `ocvTypes` exists both in scala and python side. I think it will bring troubles in future code maintenance, because we need always make sure they are the same. Maybe the better way is, load the `ocvTypes` from a file OR we write a extra script, to generate the `ocvTypes` code block for scala/python ?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r148291118
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,236 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields: Array[String] = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes: Map[String, Int] = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC3" -> 16, "CV_8UC4" -> 24
    +  )
    +
    +  /**
    +   * Used for conversion to python
    +   */
    +  val _ocvTypes: java.util.Map[String, Int] = ocvTypes.asJava
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField(imageFields(0), StringType, true) ::
    +    StructField(imageFields(1), IntegerType, false) ::
    +    StructField(imageFields(2), IntegerType, false) ::
    +    StructField(imageFields(3), IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(imageFields(4), IntegerType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(imageFields(5), BinaryType, false) :: Nil)
    +
    +  /**
    +   * DataFrame with a single column of images named "image" (nullable)
    +   */
    +  val imageSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    --- End diff --
    
    I saw the review comment and discussion in https://github.com/apache/spark/pull/19439#discussion_r144193350. In particular, these `get...` look too much to me for now actually. I don't strongly feel we should remove this out but if any other reviewers think in the same way, probably we should take out.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r147998308
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,139 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +from pyspark.ml.util import *
    +from pyspark.ml.param.shared import *
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame, SparkSession, SQLContext
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +imageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +
    +def getOcvTypes(spark=None):
    +    """
    +    Returns the OpenCV type mapping supported
    +
    +    :param sparkSession (SparkSession): The current spark session
    +    :rtype dict: The OpenCV type mapping supported
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.ocvTypes
    +
    +
    +# DataFrame with a single column of images named "image" (nullable)
    +def getImageSchema(spark=None):
    +    """
    +    Returns the image schema
    +
    +    :param spark (SparkSession): The current spark session
    +    :rtype StructType: The image schema
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.imageSchema
    --- End diff --
    
    I think we should have simple tests if these are supposed to be public APIs in Python side in order to to check if they can be called properly at least.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r147689647
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,252 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC3" -> 16, "CV_8UC4" -> 24
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField(imageFields(0), StringType, true) ::
    +    StructField(imageFields(1), IntegerType, false) ::
    +    StructField(imageFields(2), IntegerType, false) ::
    +    StructField(imageFields(3), IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(imageFields(4), IntegerType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(imageFields(5), BinaryType, false) :: Nil)
    +
    +  /**
    +   * DataFrame with a single column of images named "image" (nullable)
    +   */
    +  val imageSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the height of the image
    +   *
    +   * @return The height of the image
    +   */
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the width of the image
    +   *
    +   * @return The width of the image
    +   */
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the number of channels in the image
    +   *
    +   * @return The number of channels in the image
    +   */
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the OpenCV representation as an int
    +   *
    +   * @return The OpenCV representation as an int
    +   */
    +  def getMode(row: Row): Int = row.getInt(4)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the image data
    +   *
    +   * @return The image data
    +   */
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * :: Experimental ::
    +   * Check if the DataFrame column contains images (i.e. has ImageSchema)
    +   *
    +   * @param df DataFrame
    +   * @param column Column name
    +   * @return True if the given column matches the image schema
    +   */
    +  def isImageColumn(df: DataFrame, column: String): Boolean =
    +    df.schema(column).dataType == columnSchema
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row =
    +    Row(Row(origin, -1, -1, -1, ocvTypes(undefinedImageType), Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in DataFrame Row
    +   *
    +   * @param origin Arbitrary string that identifies the image
    +   * @param bytes Image bytes (for example, jpeg)
    +   * @return DataFrame Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val isGray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val hasAlpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) = if (isGray) {
    +        (1, ocvTypes("CV_8UC1"))
    +      } else if (hasAlpha) {
    +        (4, ocvTypes("CV_8UC4"))
    +      } else {
    +        (3, ocvTypes("CV_8UC3"))
    +      }
    +
    +      val imageSize = height * width * nChannels
    +      assert(imageSize < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](imageSize)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (isGray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      } else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    +            if (nChannels == 4) {
    +              decoded(offset + 3) = color.getAlpha.toByte
    +            }
    +            offset += nChannels
    +          }
    +        }
    +      }
    +
    +      // the internal "Row" is needed, because the image is a single DataFrame column
    +      Some(Row(Row(origin, height, width, nChannels, mode, decoded)))
    +    }
    +  }
    +
    +  /**
    +   * :: Experimental ::
    +   * Read the directory of images from the local or remote source
    +   *
    +   * @param path Path to the image directory
    +   * @param sparkSession Spark Session
    +   * @param recursive Recursive path search flag
    +   * @param numPartitions Number of the DataFrame partitions
    --- End diff --
    
    I think we should say it uses `defaultParallelism` as well when it's omitted.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r144856437
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,256 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC2" -> 8, "CV_8UC3" -> 16, "CV_8UC4" -> 24,
    +    "CV_8S" -> 1, "CV_8SC1" -> 1, "CV_8SC2" -> 9, "CV_8SC3" -> 17, "CV_8SC4" -> 25,
    +    "CV_16U" -> 2, "CV_16UC1" -> 2, "CV_16UC2" -> 10, "CV_16UC3" -> 18, "CV_16UC4" -> 26,
    +    "CV_16S" -> 3, "CV_16SC1" -> 3, "CV_16SC2" -> 11, "CV_16SC3" -> 19, "CV_16SC4" -> 27,
    +    "CV_32S" -> 4, "CV_32SC1" -> 4, "CV_32SC2" -> 12, "CV_32SC3" -> 20, "CV_32SC4" -> 28,
    +    "CV_32F" -> 5, "CV_32FC1" -> 5, "CV_32FC2" -> 13, "CV_32FC3" -> 21, "CV_32FC4" -> 29,
    +    "CV_64F" -> 6, "CV_64FC1" -> 6, "CV_64FC2" -> 14, "CV_64FC3" -> 22, "CV_64FC4" -> 30
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField("origin", StringType, true) ::
    +    StructField("height", IntegerType, false) ::
    +    StructField("width", IntegerType, false) ::
    +    StructField("nChannels", IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField("mode", StringType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField("data", BinaryType, false) :: Nil)
    +
    +  // DataFrame with a single column of images named "image" (nullable)
    +  private val imageDFSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the height of the image
    +   *
    +   * @return The height of the image
    +   */
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the width of the image
    +   *
    +   * @return The width of the image
    +   */
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the number of channels in the image
    +   *
    +   * @return The number of channels in the image
    +   */
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the OpenCV representation
    +   *
    +   * @return The OpenCV representation
    +   */
    +  def getMode(row: Row): String = row.getString(4)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the image data
    +   *
    +   * @return The image data
    +   */
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * :: Experimental ::
    +   * Check if the DataFrame column contains images (i.e. has ImageSchema)
    +   *
    +   * @param df       DataFrame
    +   * @param column   Column name
    +   * @return         True if the given column matches the image schema
    +   */
    +  def isImageColumn(df: DataFrame, column: String): Boolean =
    +    df.schema(column).dataType == columnSchema
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return       Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row = Row(Row(origin, -1, -1, -1, undefinedImageType,
    +    Array.ofDim[Byte](0)))
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r144191439
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/HadoopUtils.scala ---
    @@ -0,0 +1,122 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import scala.language.existentials
    +import scala.util.Random
    +
    +import org.apache.commons.io.FilenameUtils
    +import org.apache.hadoop.conf.{Configuration, Configured}
    +import org.apache.hadoop.fs.{Path, PathFilter}
    +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
    +
    +import org.apache.spark.sql.SparkSession
    +
    +private object RecursiveFlag {
    +
    +  /**
    +   * Sets a value of spark recursive flag
    +   *
    +   * @param value value to set
    +   * @param spark existing spark session
    +   * @return previous value of this flag
    +   */
    +  def setRecursiveFlag(value: Option[String], spark: SparkSession): Option[String] = {
    +    val flagName = FileInputFormat.INPUT_DIR_RECURSIVE
    +    val hadoopConf = spark.sparkContext.hadoopConfiguration
    +    val old = Option(hadoopConf.get(flagName))
    +
    +    value match {
    +      case Some(v) => hadoopConf.set(flagName, v)
    +      case None => hadoopConf.unset(flagName)
    +    }
    +
    +    old
    +  }
    +}
    +
    +/**
    + * Filter that allows loading a fraction of HDFS files.
    + */
    +private class SamplePathFilter extends Configured with PathFilter {
    +  val random = {
    +    val rd = new Random()
    +    rd.setSeed(0)
    +    rd
    +  }
    +
    +  // Ratio of files to be read from disk
    +  var sampleRatio: Double = 1
    +
    +  override def setConf(conf: Configuration): Unit = {
    +    if (conf != null) {
    +      sampleRatio = conf.getDouble(SamplePathFilter.ratioParam, 1)
    +    }
    +  }
    +
    +  override def accept(path: Path): Boolean = {
    +    // Note: checking fileSystem.isDirectory is very slow here, so we use basic rules instead
    +    !SamplePathFilter.isFile(path) ||
    +      random.nextDouble() < sampleRatio
    +  }
    +}
    +
    +private object SamplePathFilter {
    +  val ratioParam = "sampleRatio"
    +
    +  def isFile(path: Path): Boolean = FilenameUtils.getExtension(path.toString) != ""
    +
    +  /**
    +   * Sets hdfs PathFilter
    +   *
    +   * @param value       Filter class that is passed to HDFS
    +   * @param sampleRatio Fraction of the files that the filter picks
    +   * @param spark       Existing Spark session
    +   * @return            Returns the previous hdfs path filter
    +   */
    +  def setPathFilter(value: Option[Class[_]], sampleRatio: Double,
    +                    spark: SparkSession) : Option[Class[_]] = {
    +    val flagName = FileInputFormat.PATHFILTER_CLASS
    +    val hadoopConf = spark.sparkContext.hadoopConfiguration
    +    val old = Option(hadoopConf.getClass(flagName, null))
    +    hadoopConf.setDouble(SamplePathFilter.ratioParam, sampleRatio)
    +
    +    value match {
    +      case Some(v) => hadoopConf.setClass(flagName, v, classOf[PathFilter])
    +      case None => hadoopConf.unset(flagName)
    +    }
    +    old
    +  }
    +
    +  /**
    +   * Unsets hdfs PathFilter
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r144742600
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/image/ImageSchemaSuite.scala ---
    @@ -0,0 +1,124 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.nio.file.Paths
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.ml.image.ImageSchema._
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.types._
    +
    +class ImageSchemaSuite extends SparkFunSuite with MLlibTestSparkContext {
    +  // Single column of images named "image"
    +  private val imageDFSchema =
    +    StructType(StructField("image", ImageSchema.columnSchema, true) :: Nil)
    +  private lazy val imagePath =
    +    Thread.currentThread().getContextClassLoader.getResource("test-data/images").getPath
    +
    +  test("Smoke test: create basic ImageSchema dataframe") {
    +    val origin = "path"
    +    val width = 1
    +    val height = 1
    +    val nChannels = 3
    +    val data = Array[Byte](0, 0, 0)
    +    val mode = "CV_8UC3"
    +
    +    // Internal Row corresponds to image StructType
    +    val rows = Seq(Row(Row(origin, height, width, nChannels, mode, data)),
    +      Row(Row(null, height, width, nChannels, mode, data)))
    +    val rdd = sc.makeRDD(rows)
    +    val df = spark.createDataFrame(rdd, imageDFSchema)
    +
    +    assert(df.count == 2, "incorrect image count")
    +    assert(ImageSchema.isImageColumn(df, "image"), "data do not fit ImageSchema")
    +  }
    +
    +  test("readImages count test") {
    +    var df = readImages(imagePath, recursive = false)
    +    assert(df.count == 0)
    +
    +    df = readImages(imagePath, recursive = true, dropImageFailures = false)
    +    assert(df.count == 8)
    +
    +    df = readImages(imagePath, recursive = true, dropImageFailures = true)
    +    val count100 = df.count
    +    assert(count100 == 7)
    +
    +    df = readImages(imagePath, recursive = true, sampleRatio = 0.5, dropImageFailures = true)
    +    // Random number about half of the size of the original dataset
    +    val count50 = df.count
    +    assert(count50 > 0.2 * count100 && count50 < 0.8 * count100)
    +  }
    +
    +  test("readImages partition test") {
    +    val df = readImages(imagePath, recursive = true, dropImageFailures = true, numPartitions = 3)
    +    assert(df.rdd.getNumPartitions == 3)
    +  }
    +
    +  // Images with the different number of channels
    +  test("readImages pixel values test") {
    +
    +    val images = readImages(imagePath + "/multi-channel/", recursive = false).collect
    +
    +    images.foreach{
    +      rrow => {
    +        val row = rrow.getAs[Row](0)
    +        val filename = Paths.get(getOrigin(row)).getFileName().toString()
    +        if(firstBytes20.contains(filename)) {
    +          val mode = getMode(row)
    +          val bytes20 = getData(row).slice(0, 20)
    +
    +          val expectedMode = firstBytes20(filename)._1
    +          val expectedBytes = firstBytes20(filename)._2
    +
    +          assert(expectedMode == mode, "mode of the image is not read correctly")
    +
    +          if (!compareBytes(expectedBytes, bytes20)) {
    +            throw new Exception("incorrect numeric value for flattened image")
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r150164008
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,192 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +"""
    +.. attribute:: ImageSchema
    +
    +    A singleton-like attribute of :class:`_ImageSchema` in this module.
    +
    +.. autoclass:: _ImageSchema
    +   :members:
    +"""
    +
    +from pyspark import SparkContext
    +from pyspark.sql.types import Row, _create_row, _parse_datatype_json_string
    +from pyspark.sql import DataFrame, SparkSession
    +import numpy as np
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #82849 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82849/testReport)** for PR 19439 at commit [`1b3e6b7`](https://github.com/apache/spark/commit/1b3e6b70d449141cc41ab7ece216b06e7197e430).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r145174101
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,129 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +import pyspark
    +from pyspark import SparkContext
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame
    +from pyspark.ml.param.shared import *
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +ImageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +ocvTypes = {
    +    undefinedImageType: -1,
    +    "CV_8U": 0, "CV_8UC1": 0, "CV_8UC2": 8, "CV_8UC3": 16, "CV_8UC4": 24,
    +    "CV_8S": 1, "CV_8SC1": 1, "CV_8SC2": 9, "CV_8SC3": 17, "CV_8SC4": 25,
    +    "CV_16U": 2, "CV_16UC1": 2, "CV_16UC2": 10, "CV_16UC3": 18, "CV_16UC4": 26,
    +    "CV_16S": 3, "CV_16SC1": 3, "CV_16SC2": 11, "CV_16SC3": 19, "CV_16SC4": 27,
    +    "CV_32S": 4, "CV_32SC1": 4, "CV_32SC2": 12, "CV_32SC3": 20, "CV_32SC4": 28,
    +    "CV_32F": 5, "CV_32FC1": 5, "CV_32FC2": 13, "CV_32FC3": 21, "CV_32FC4": 29,
    +    "CV_64F": 6, "CV_64FC1": 6, "CV_64FC2": 14, "CV_64FC3": 22, "CV_64FC4": 30
    +}
    +
    +ImageSchema = StructType([
    --- End diff --
    
    good point, I added an ImageFields in scala side as well.  I also fixed imageSchema in python side and exposed it in scala side as public.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by MrBago <gi...@git.apache.org>.
Github user MrBago commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r145499084
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,255 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC2" -> 8, "CV_8UC3" -> 16, "CV_8UC4" -> 24,
    +    "CV_8S" -> 1, "CV_8SC1" -> 1, "CV_8SC2" -> 9, "CV_8SC3" -> 17, "CV_8SC4" -> 25,
    +    "CV_16U" -> 2, "CV_16UC1" -> 2, "CV_16UC2" -> 10, "CV_16UC3" -> 18, "CV_16UC4" -> 26,
    +    "CV_16S" -> 3, "CV_16SC1" -> 3, "CV_16SC2" -> 11, "CV_16SC3" -> 19, "CV_16SC4" -> 27,
    +    "CV_32S" -> 4, "CV_32SC1" -> 4, "CV_32SC2" -> 12, "CV_32SC3" -> 20, "CV_32SC4" -> 28,
    +    "CV_32F" -> 5, "CV_32FC1" -> 5, "CV_32FC2" -> 13, "CV_32FC3" -> 21, "CV_32FC4" -> 29,
    +    "CV_64F" -> 6, "CV_64FC1" -> 6, "CV_64FC2" -> 14, "CV_64FC3" -> 22, "CV_64FC4" -> 30
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Array[Byte])
    --- End diff --
    
    I believe you're missing one `Int` above.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r148692696
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,236 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields: Array[String] = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes: Map[String, Int] = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC3" -> 16, "CV_8UC4" -> 24
    +  )
    +
    +  /**
    +   * Used for conversion to python
    +   */
    +  val _ocvTypes: java.util.Map[String, Int] = ocvTypes.asJava
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField(imageFields(0), StringType, true) ::
    +    StructField(imageFields(1), IntegerType, false) ::
    +    StructField(imageFields(2), IntegerType, false) ::
    +    StructField(imageFields(3), IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(imageFields(4), IntegerType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(imageFields(5), BinaryType, false) :: Nil)
    +
    +  /**
    +   * DataFrame with a single column of images named "image" (nullable)
    +   */
    +  val imageSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the height of the image
    +   *
    +   * @return The height of the image
    +   */
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the width of the image
    +   *
    +   * @return The width of the image
    +   */
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the number of channels in the image
    +   *
    +   * @return The number of channels in the image
    +   */
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the OpenCV representation as an int
    +   *
    +   * @return The OpenCV representation as an int
    +   */
    +  def getMode(row: Row): Int = row.getInt(4)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the image data
    +   *
    +   * @return The image data
    +   */
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row =
    +    Row(Row(origin, -1, -1, -1, ocvTypes(undefinedImageType), Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in DataFrame Row
    +   *
    +   * @param origin Arbitrary string that identifies the image
    +   * @param bytes Image bytes (for example, jpeg)
    +   * @return DataFrame Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val isGray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val hasAlpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) = if (isGray) {
    +        (1, ocvTypes("CV_8UC1"))
    +      } else if (hasAlpha) {
    +        (4, ocvTypes("CV_8UC4"))
    +      } else {
    +        (3, ocvTypes("CV_8UC3"))
    +      }
    +
    +      val imageSize = height * width * nChannels
    +      assert(imageSize < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](imageSize)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (isGray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      } else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    +            if (nChannels == 4) {
    +              decoded(offset + 3) = color.getAlpha.toByte
    +            }
    +            offset += nChannels
    +          }
    +        }
    +      }
    +
    +      // the internal "Row" is needed, because the image is a single DataFrame column
    +      Some(Row(Row(origin, height, width, nChannels, mode, decoded)))
    +    }
    +  }
    +
    +  /**
    +   * :: Experimental ::
    --- End diff --
    
    To mark something Experimental, please use both this line and also the ```@Experimental``` annotation for the method.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r148683226
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,236 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields: Array[String] = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes: Map[String, Int] = Map(
    --- End diff --
    
    Quick workaround I could think is, something like naming _ocvTypes to ocvTypesAsJava and documenting that's Java specific but.. want to know what others think .. 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #83845 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83845/testReport)** for PR 19439 at commit [`c4a6dfe`](https://github.com/apache/spark/commit/c4a6dfe736b49384b8744241b40219192e400051).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r148029475
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,139 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +from pyspark.ml.util import *
    +from pyspark.ml.param.shared import *
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame, SparkSession, SQLContext
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +imageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +
    +def getOcvTypes(spark=None):
    +    """
    +    Returns the OpenCV type mapping supported
    +
    +    :param sparkSession (SparkSession): The current spark session
    +    :rtype dict: The OpenCV type mapping supported
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.ocvTypes
    +
    +
    +# DataFrame with a single column of images named "image" (nullable)
    +def getImageSchema(spark=None):
    +    """
    +    Returns the image schema
    +
    +    :param spark (SparkSession): The current spark session
    +    :rtype StructType: The image schema
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.imageSchema
    +
    +
    +def toNDArray(image):
    +    """
    +    Converts an image to a one-dimensional array.
    +
    +    :param image (object): The image to be converted
    +    :rtype array: The image as a one-dimensional array
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    height = image.height
    +    width = image.width
    +    nChannels = image.nChannels
    +    return np.ndarray(
    +        shape=(height, width, nChannels),
    +        dtype=np.uint8,
    +        buffer=image.data,
    +        strides=(width * nChannels, nChannels, 1))
    +
    +
    +def toImage(array, origin="", spark=None):
    +    """
    +    Converts a one-dimensional array to a two-dimensional image.
    +
    +    :param array (array): The array to convert to image
    +    :param origin (str): Path to the image
    +    :param spark (SparkSession): The current spark session
    +    :rtype object: Two dimensional image
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    if array.ndim != 3:
    +        raise
    --- End diff --
    
    @MrBago recommended this code change above, you are right I did not realize the syntax is invalid, what specific exception should I raise?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r148031857
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,139 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +from pyspark.ml.util import *
    +from pyspark.ml.param.shared import *
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame, SparkSession, SQLContext
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +imageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +
    +def getOcvTypes(spark=None):
    +    """
    +    Returns the OpenCV type mapping supported
    +
    +    :param sparkSession (SparkSession): The current spark session
    +    :rtype dict: The OpenCV type mapping supported
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.ocvTypes
    +
    +
    +# DataFrame with a single column of images named "image" (nullable)
    +def getImageSchema(spark=None):
    +    """
    +    Returns the image schema
    +
    +    :param spark (SparkSession): The current spark session
    +    :rtype StructType: The image schema
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.imageSchema
    +
    +
    +def toNDArray(image):
    +    """
    +    Converts an image to a one-dimensional array.
    +
    +    :param image (object): The image to be converted
    +    :rtype array: The image as a one-dimensional array
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    height = image.height
    +    width = image.width
    +    nChannels = image.nChannels
    +    return np.ndarray(
    +        shape=(height, width, nChannels),
    +        dtype=np.uint8,
    +        buffer=image.data,
    +        strides=(width * nChannels, nChannels, 1))
    +
    +
    +def toImage(array, origin="", spark=None):
    +    """
    +    Converts a one-dimensional array to a two-dimensional image.
    +
    +    :param array (array): The array to convert to image
    +    :param origin (str): Path to the image
    +    :param spark (SparkSession): The current spark session
    +    :rtype object: Two dimensional image
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    if array.ndim != 3:
    +        raise
    +    height, width, nChannels = array.shape
    +    ocvTypes = getOcvTypes(spark)
    +    if nChannels == 1:
    +        mode = ocvTypes["CV_8UC1"]
    +    elif nChannels == 3:
    +        mode = ocvTypes["CV_8UC3"]
    +    elif nChannels == 4:
    +        mode = ocvTypes["CV_8UC4"]
    +    else:
    +        raise
    +    data = bytearray(array.astype(dtype=np.uint8).ravel())
    +    # Creating new Row with _create_row(), because Row(name = value, ... )
    +    # orders fields by name, which conflicts with expected schema order
    +    # when the new DataFrame is created by UDF
    +    return _create_row(imageFields,
    +                       [origin, height, width, nChannels, mode, data])
    +
    +
    +def readImages(path, spark=None, recursive=False, numPartitions=0,
    +               dropImageFailures=False, sampleRatio=1.0):
    +    """
    +    Reads the directory of images from the local or remote source.
    +
    +    :param path (str): Path to the image directory
    +    :param spark (SparkSession): The current spark session
    +    :param recursive (bool): Recursive search flag
    +    :param numPartitions (int): Number of DataFrame partitions
    +    :param dropImageFailures (bool): Drop the files that are not valid images
    +    :param sampleRatio (double): Fraction of the images loaded
    +    :rtype DataFrame: DataFrame with a single column of "images",
    +           see ImageSchema for details
    +
    +    Examples:
    +
    +    >>> df = readImages('python/test_support/image/kittens', recursive=True)
    +    >>> df.count
    +    4
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    image_schema = ctx._jvm.org.apache.spark.ml.image.ImageSchema
    +    sql_ctx = SQLContext(ctx)
    +    jsession = spark._jsparkSession
    +    jresult = image_schema.readImages(path, jsession, recursive, numPartitions,
    +                                      dropImageFailures, float(sampleRatio))
    +    return DataFrame(jresult, sql_ctx)
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #83768 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83768/testReport)** for PR 19439 at commit [`f4acfd4`](https://github.com/apache/spark/commit/f4acfd4410b99a098cf259991ef41a592083341c).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #83303 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83303/testReport)** for PR 19439 at commit [`84d9177`](https://github.com/apache/spark/commit/84d9177b16267fad5564d71981e84685cc08cf6f).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r148695505
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,236 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields: Array[String] = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes: Map[String, Int] = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC3" -> 16, "CV_8UC4" -> 24
    +  )
    +
    +  /**
    +   * Used for conversion to python
    +   */
    +  val _ocvTypes: java.util.Map[String, Int] = ocvTypes.asJava
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    --- End diff --
    
    For legibility, it'd be nice to define the imageFields values here (inline).  You could then define imageFields by extracting those values from columnSchema.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by WeichenXu123 <gi...@git.apache.org>.
Github user WeichenXu123 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r148706148
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,236 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields: Array[String] = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes: Map[String, Int] = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC3" -> 16, "CV_8UC4" -> 24
    +  )
    +
    +  /**
    +   * Used for conversion to python
    +   */
    +  val _ocvTypes: java.util.Map[String, Int] = ocvTypes.asJava
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField(imageFields(0), StringType, true) ::
    +    StructField(imageFields(1), IntegerType, false) ::
    +    StructField(imageFields(2), IntegerType, false) ::
    +    StructField(imageFields(3), IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(imageFields(4), IntegerType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(imageFields(5), BinaryType, false) :: Nil)
    +
    +  /**
    +   * DataFrame with a single column of images named "image" (nullable)
    +   */
    +  val imageSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the height of the image
    +   *
    +   * @return The height of the image
    +   */
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the width of the image
    +   *
    +   * @return The width of the image
    +   */
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the number of channels in the image
    +   *
    +   * @return The number of channels in the image
    +   */
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the OpenCV representation as an int
    +   *
    +   * @return The OpenCV representation as an int
    +   */
    +  def getMode(row: Row): Int = row.getInt(4)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the image data
    +   *
    +   * @return The image data
    +   */
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row =
    +    Row(Row(origin, -1, -1, -1, ocvTypes(undefinedImageType), Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in DataFrame Row
    +   *
    +   * @param origin Arbitrary string that identifies the image
    +   * @param bytes Image bytes (for example, jpeg)
    +   * @return DataFrame Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val isGray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val hasAlpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) = if (isGray) {
    +        (1, ocvTypes("CV_8UC1"))
    +      } else if (hasAlpha) {
    +        (4, ocvTypes("CV_8UC4"))
    +      } else {
    +        (3, ocvTypes("CV_8UC3"))
    +      }
    +
    +      val imageSize = height * width * nChannels
    +      assert(imageSize < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](imageSize)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (isGray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      } else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    +            if (nChannels == 4) {
    +              decoded(offset + 3) = color.getAlpha.toByte
    +            }
    +            offset += nChannels
    +          }
    +        }
    +      }
    +
    +      // the internal "Row" is needed, because the image is a single DataFrame column
    +      Some(Row(Row(origin, height, width, nChannels, mode, decoded)))
    +    }
    +  }
    +
    +  /**
    +   * :: Experimental ::
    +   * Read the directory of images from the local or remote source
    +   *
    +   * @param path Path to the image directory
    +   * @param sparkSession Spark Session, if omitted gets or creates the session
    +   * @param recursive Recursive path search flag
    +   * @param numPartitions Number of the DataFrame partitions,
    +   *                      if omitted uses defaultParallelism instead
    +   * @param dropImageFailures Drop the files that are not valid images from the result
    +   * @param sampleRatio Fraction of the files loaded
    +   * @return DataFrame with a single column "image" of images;
    +   *         see ImageSchema for the details
    +   */
    +  def readImages(
    +      path: String,
    +      sparkSession: SparkSession = null,
    +      recursive: Boolean = false,
    +      numPartitions: Int = 0,
    +      dropImageFailures: Boolean = false,
    +      sampleRatio: Double = 1.0): DataFrame = {
    +    require(sampleRatio <= 1.0 && sampleRatio >= 0, "sampleRatio should be between 0 and 1")
    +
    +    val session = if (sparkSession != null) sparkSession else SparkSession.builder().getOrCreate
    +    val partitions =
    +      if (numPartitions > 0) {
    +        numPartitions
    +      } else {
    +        session.sparkContext.defaultParallelism
    +      }
    +
    +    RecursiveFlag.withRecursiveFlag(recursive, session) {
    +      SamplePathFilter.withPathFilter(sampleRatio, session) {
    +        val streams = session.sparkContext.binaryFiles(path, partitions).repartition(partitions)
    --- End diff --
    
    Yes, if there are large numbers of files it will still be expensive. But if `repartition` is postponed to be after image data loaded, then the cost will be much more expensive.
    How about setting a default value of numPartitions = -1 , representing no `repartition` handling, other case keep current code.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r150449349
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -80,7 +80,7 @@ def imageFields(self):
             """
             Returns field names of image columns.
     
    -        :rtype list: a list of field names.
    +        :return: list: a list of field names.
    --- End diff --
    
    `:return: a list of field names.`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r150166092
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,192 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +"""
    +.. attribute:: ImageSchema
    +
    +    A singleton-like attribute of :class:`_ImageSchema` in this module.
    +
    +.. autoclass:: _ImageSchema
    +   :members:
    +"""
    +
    +from pyspark import SparkContext
    +from pyspark.sql.types import Row, _create_row, _parse_datatype_json_string
    +from pyspark.sql import DataFrame, SparkSession
    +import numpy as np
    +
    +
    +class _ImageSchema(object):
    +    """
    +    Internal class for `pyspark.ml.image.ImageSchema` attribute. Meant to be private and
    +    not to be instantized. Use `pyspark.ml.image.ImageSchema` attribute to access the
    +    APIs of this class.
    +    """
    +
    +    def __init__(self):
    +        self._imageSchema = None
    +        self._ocvTypes = None
    +        self._imageFields = None
    +        self._undefinedImageType = None
    +
    +    @property
    +    def imageSchema(self):
    +        """
    +        Returns the image schema.
    +
    +        :rtype StructType: a DataFrame with a single column of images
    +               named "image" (nullable)
    +
    +        .. versionadded:: 2.3.0
    +        """
    +
    +        if self._imageSchema is None:
    +            ctx = SparkContext._active_spark_context
    +            jschema = ctx._jvm.org.apache.spark.ml.image.ImageSchema.imageSchema()
    +            self._imageSchema = _parse_datatype_json_string(jschema.json())
    +        return self._imageSchema
    +
    +    @property
    +    def ocvTypes(self):
    +        """
    +        Returns the OpenCV type mapping supported
    +
    +        :rtype dict: The OpenCV type mapping supported
    +
    +        .. versionadded:: 2.3.0
    +        """
    +
    +        if self._ocvTypes is None:
    +            ctx = SparkContext._active_spark_context
    +            self._ocvTypes = dict(ctx._jvm.org.apache.spark.ml.image.ImageSchema._ocvTypes())
    +        return self._ocvTypes
    +
    +    @property
    +    def imageFields(self):
    +        """
    +        Returns field names of image columns.
    +
    +        :rtype list: a list of field names.
    +
    +        .. versionadded:: 2.3.0
    +        """
    +
    +        if self._imageFields is None:
    +            ctx = SparkContext._active_spark_context
    +            self._imageFields = list(ctx._jvm.org.apache.spark.ml.image.ImageSchema.imageFields())
    +        return self._imageFields
    +
    +    @property
    +    def undefinedImageType(self):
    +        """
    +        Returns the name of undefined image type for the invalid image.
    +
    +        .. versionadded:: 2.3.0
    +        """
    +
    +        if self._undefinedImageType is None:
    +            ctx = SparkContext._active_spark_context
    +            self._undefinedImageType = \
    +                ctx._jvm.org.apache.spark.ml.image.ImageSchema.undefinedImageType()
    +        return self._undefinedImageType
    +
    +    def toNDArray(self, image):
    +        """
    +        Converts an image to a one-dimensional array.
    +
    +        :param image: The image to be converted
    +        :rtype array: The image as a one-dimensional array
    +
    +        .. versionadded:: 2.3.0
    +        """
    +
    +        height = image.height
    +        width = image.width
    +        nChannels = image.nChannels
    +        return np.ndarray(
    +            shape=(height, width, nChannels),
    +            dtype=np.uint8,
    +            buffer=image.data,
    +            strides=(width * nChannels, nChannels, 1))
    +
    +    def toImage(self, array, origin=""):
    +        """
    +        Converts a one-dimensional array to a two-dimensional image.
    +
    +        :param array array: The array to convert to image
    +        :param str origin: Path to the image
    --- End diff --
    
    yes, do I need to change anything?  I added optional to the param doc summary.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r150164867
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,192 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +"""
    +.. attribute:: ImageSchema
    +
    +    A singleton-like attribute of :class:`_ImageSchema` in this module.
    --- End diff --
    
    removed the "singleton-like" wording in the doc - please let me know if any other changes are needed here



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r144856202
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,256 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC2" -> 8, "CV_8UC3" -> 16, "CV_8UC4" -> 24,
    +    "CV_8S" -> 1, "CV_8SC1" -> 1, "CV_8SC2" -> 9, "CV_8SC3" -> 17, "CV_8SC4" -> 25,
    +    "CV_16U" -> 2, "CV_16UC1" -> 2, "CV_16UC2" -> 10, "CV_16UC3" -> 18, "CV_16UC4" -> 26,
    +    "CV_16S" -> 3, "CV_16SC1" -> 3, "CV_16SC2" -> 11, "CV_16SC3" -> 19, "CV_16SC4" -> 27,
    +    "CV_32S" -> 4, "CV_32SC1" -> 4, "CV_32SC2" -> 12, "CV_32SC3" -> 20, "CV_32SC4" -> 28,
    +    "CV_32F" -> 5, "CV_32FC1" -> 5, "CV_32FC2" -> 13, "CV_32FC3" -> 21, "CV_32FC4" -> 29,
    +    "CV_64F" -> 6, "CV_64FC1" -> 6, "CV_64FC2" -> 14, "CV_64FC3" -> 22, "CV_64FC4" -> 30
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField("origin", StringType, true) ::
    +    StructField("height", IntegerType, false) ::
    +    StructField("width", IntegerType, false) ::
    +    StructField("nChannels", IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField("mode", StringType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField("data", BinaryType, false) :: Nil)
    +
    +  // DataFrame with a single column of images named "image" (nullable)
    +  private val imageDFSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the height of the image
    +   *
    +   * @return The height of the image
    +   */
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the width of the image
    +   *
    +   * @return The width of the image
    +   */
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the number of channels in the image
    +   *
    +   * @return The number of channels in the image
    +   */
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the OpenCV representation
    +   *
    +   * @return The OpenCV representation
    +   */
    +  def getMode(row: Row): String = row.getString(4)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the image data
    +   *
    +   * @return The image data
    +   */
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * :: Experimental ::
    +   * Check if the DataFrame column contains images (i.e. has ImageSchema)
    +   *
    +   * @param df       DataFrame
    +   * @param column   Column name
    +   * @return         True if the given column matches the image schema
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    good catch, it's from here:
    https://ccsearch.creativecommons.org/image/detail/B2CVP_j5KjwZm7UAVJ3Hvw==
    let me add it to the list


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r144742476
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/image/ImageSchemaSuite.scala ---
    @@ -0,0 +1,124 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.nio.file.Paths
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.ml.image.ImageSchema._
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.types._
    +
    +class ImageSchemaSuite extends SparkFunSuite with MLlibTestSparkContext {
    +  // Single column of images named "image"
    +  private val imageDFSchema =
    +    StructType(StructField("image", ImageSchema.columnSchema, true) :: Nil)
    +  private lazy val imagePath =
    +    Thread.currentThread().getContextClassLoader.getResource("test-data/images").getPath
    +
    +  test("Smoke test: create basic ImageSchema dataframe") {
    +    val origin = "path"
    +    val width = 1
    +    val height = 1
    +    val nChannels = 3
    +    val data = Array[Byte](0, 0, 0)
    +    val mode = "CV_8UC3"
    +
    +    // Internal Row corresponds to image StructType
    +    val rows = Seq(Row(Row(origin, height, width, nChannels, mode, data)),
    +      Row(Row(null, height, width, nChannels, mode, data)))
    +    val rdd = sc.makeRDD(rows)
    +    val df = spark.createDataFrame(rdd, imageDFSchema)
    +
    +    assert(df.count == 2, "incorrect image count")
    +    assert(ImageSchema.isImageColumn(df, "image"), "data do not fit ImageSchema")
    +  }
    +
    +  test("readImages count test") {
    +    var df = readImages(imagePath, recursive = false)
    +    assert(df.count == 0)
    +
    +    df = readImages(imagePath, recursive = true, dropImageFailures = false)
    +    assert(df.count == 8)
    +
    +    df = readImages(imagePath, recursive = true, dropImageFailures = true)
    +    val count100 = df.count
    +    assert(count100 == 7)
    +
    +    df = readImages(imagePath, recursive = true, sampleRatio = 0.5, dropImageFailures = true)
    +    // Random number about half of the size of the original dataset
    +    val count50 = df.count
    +    assert(count50 > 0.2 * count100 && count50 < 0.8 * count100)
    +  }
    +
    +  test("readImages partition test") {
    +    val df = readImages(imagePath, recursive = true, dropImageFailures = true, numPartitions = 3)
    +    assert(df.rdd.getNumPartitions == 3)
    +  }
    +
    +  // Images with the different number of channels
    +  test("readImages pixel values test") {
    +
    +    val images = readImages(imagePath + "/multi-channel/", recursive = false).collect
    +
    +    images.foreach{
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #83222 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83222/testReport)** for PR 19439 at commit [`6ed56e1`](https://github.com/apache/spark/commit/6ed56e10514108e0c6cd0091ee7351b59d923936).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r150165810
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,192 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +"""
    +.. attribute:: ImageSchema
    +
    +    A singleton-like attribute of :class:`_ImageSchema` in this module.
    +
    +.. autoclass:: _ImageSchema
    +   :members:
    +"""
    +
    +from pyspark import SparkContext
    +from pyspark.sql.types import Row, _create_row, _parse_datatype_json_string
    +from pyspark.sql import DataFrame, SparkSession
    +import numpy as np
    +
    +
    +class _ImageSchema(object):
    +    """
    +    Internal class for `pyspark.ml.image.ImageSchema` attribute. Meant to be private and
    +    not to be instantized. Use `pyspark.ml.image.ImageSchema` attribute to access the
    +    APIs of this class.
    +    """
    +
    +    def __init__(self):
    +        self._imageSchema = None
    +        self._ocvTypes = None
    +        self._imageFields = None
    +        self._undefinedImageType = None
    +
    +    @property
    +    def imageSchema(self):
    +        """
    +        Returns the image schema.
    +
    +        :rtype StructType: a DataFrame with a single column of images
    +               named "image" (nullable)
    +
    +        .. versionadded:: 2.3.0
    +        """
    +
    +        if self._imageSchema is None:
    +            ctx = SparkContext._active_spark_context
    +            jschema = ctx._jvm.org.apache.spark.ml.image.ImageSchema.imageSchema()
    +            self._imageSchema = _parse_datatype_json_string(jschema.json())
    +        return self._imageSchema
    +
    +    @property
    +    def ocvTypes(self):
    +        """
    +        Returns the OpenCV type mapping supported
    +
    +        :rtype dict: The OpenCV type mapping supported
    +
    +        .. versionadded:: 2.3.0
    +        """
    +
    +        if self._ocvTypes is None:
    +            ctx = SparkContext._active_spark_context
    +            self._ocvTypes = dict(ctx._jvm.org.apache.spark.ml.image.ImageSchema._ocvTypes())
    +        return self._ocvTypes
    +
    +    @property
    +    def imageFields(self):
    +        """
    +        Returns field names of image columns.
    +
    +        :rtype list: a list of field names.
    +
    +        .. versionadded:: 2.3.0
    +        """
    +
    +        if self._imageFields is None:
    +            ctx = SparkContext._active_spark_context
    +            self._imageFields = list(ctx._jvm.org.apache.spark.ml.image.ImageSchema.imageFields())
    +        return self._imageFields
    +
    +    @property
    +    def undefinedImageType(self):
    +        """
    +        Returns the name of undefined image type for the invalid image.
    +
    +        .. versionadded:: 2.3.0
    +        """
    +
    +        if self._undefinedImageType is None:
    +            ctx = SparkContext._active_spark_context
    +            self._undefinedImageType = \
    +                ctx._jvm.org.apache.spark.ml.image.ImageSchema.undefinedImageType()
    +        return self._undefinedImageType
    +
    +    def toNDArray(self, image):
    +        """
    +        Converts an image to a one-dimensional array.
    +
    +        :param image: The image to be converted
    +        :rtype array: The image as a one-dimensional array
    +
    +        .. versionadded:: 2.3.0
    +        """
    +
    +        height = image.height
    +        width = image.width
    +        nChannels = image.nChannels
    +        return np.ndarray(
    +            shape=(height, width, nChannels),
    +            dtype=np.uint8,
    +            buffer=image.data,
    +            strides=(width * nChannels, nChannels, 1))
    +
    +    def toImage(self, array, origin=""):
    +        """
    +        Converts a one-dimensional array to a two-dimensional image.
    --- End diff --
    
    @holdenk done, good catch, changed wording to "Converts an array with metadata to a two-dimensional image."


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by holdenk <gi...@git.apache.org>.
Github user holdenk commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r149485321
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,192 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +"""
    +.. attribute:: ImageSchema
    +
    +    A singleton-like attribute of :class:`_ImageSchema` in this module.
    +
    +.. autoclass:: _ImageSchema
    +   :members:
    +"""
    +
    +from pyspark import SparkContext
    +from pyspark.sql.types import Row, _create_row, _parse_datatype_json_string
    +from pyspark.sql import DataFrame, SparkSession
    +import numpy as np
    +
    +
    +class _ImageSchema(object):
    +    """
    +    Internal class for `pyspark.ml.image.ImageSchema` attribute. Meant to be private and
    +    not to be instantized. Use `pyspark.ml.image.ImageSchema` attribute to access the
    +    APIs of this class.
    +    """
    +
    +    def __init__(self):
    +        self._imageSchema = None
    +        self._ocvTypes = None
    +        self._imageFields = None
    +        self._undefinedImageType = None
    +
    +    @property
    +    def imageSchema(self):
    +        """
    +        Returns the image schema.
    +
    +        :rtype StructType: a DataFrame with a single column of images
    +               named "image" (nullable)
    +
    +        .. versionadded:: 2.3.0
    +        """
    +
    +        if self._imageSchema is None:
    +            ctx = SparkContext._active_spark_context
    +            jschema = ctx._jvm.org.apache.spark.ml.image.ImageSchema.imageSchema()
    +            self._imageSchema = _parse_datatype_json_string(jschema.json())
    +        return self._imageSchema
    +
    +    @property
    +    def ocvTypes(self):
    +        """
    +        Returns the OpenCV type mapping supported
    +
    +        :rtype dict: The OpenCV type mapping supported
    +
    +        .. versionadded:: 2.3.0
    +        """
    +
    +        if self._ocvTypes is None:
    +            ctx = SparkContext._active_spark_context
    +            self._ocvTypes = dict(ctx._jvm.org.apache.spark.ml.image.ImageSchema._ocvTypes())
    +        return self._ocvTypes
    +
    +    @property
    +    def imageFields(self):
    +        """
    +        Returns field names of image columns.
    +
    +        :rtype list: a list of field names.
    +
    +        .. versionadded:: 2.3.0
    +        """
    +
    +        if self._imageFields is None:
    +            ctx = SparkContext._active_spark_context
    +            self._imageFields = list(ctx._jvm.org.apache.spark.ml.image.ImageSchema.imageFields())
    +        return self._imageFields
    +
    +    @property
    +    def undefinedImageType(self):
    +        """
    +        Returns the name of undefined image type for the invalid image.
    +
    +        .. versionadded:: 2.3.0
    +        """
    +
    +        if self._undefinedImageType is None:
    +            ctx = SparkContext._active_spark_context
    +            self._undefinedImageType = \
    +                ctx._jvm.org.apache.spark.ml.image.ImageSchema.undefinedImageType()
    +        return self._undefinedImageType
    +
    +    def toNDArray(self, image):
    +        """
    +        Converts an image to a one-dimensional array.
    +
    +        :param image: The image to be converted
    +        :rtype array: The image as a one-dimensional array
    +
    +        .. versionadded:: 2.3.0
    +        """
    +
    +        height = image.height
    +        width = image.width
    +        nChannels = image.nChannels
    +        return np.ndarray(
    +            shape=(height, width, nChannels),
    +            dtype=np.uint8,
    +            buffer=image.data,
    +            strides=(width * nChannels, nChannels, 1))
    +
    +    def toImage(self, array, origin=""):
    +        """
    +        Converts a one-dimensional array to a two-dimensional image.
    +
    +        :param array array: The array to convert to image
    +        :param str origin: Path to the image
    +        :rtype object: Two dimensional image
    +
    +        .. versionadded:: 2.3.0
    +        """
    +
    +        if array.ndim != 3:
    +            raise ValueError("Invalid array shape")
    +        height, width, nChannels = array.shape
    +        ocvTypes = ImageSchema.ocvTypes
    +        if nChannels == 1:
    +            mode = ocvTypes["CV_8UC1"]
    +        elif nChannels == 3:
    +            mode = ocvTypes["CV_8UC3"]
    +        elif nChannels == 4:
    +            mode = ocvTypes["CV_8UC4"]
    +        else:
    +            raise ValueError("Invalid number of channels")
    +        data = bytearray(array.astype(dtype=np.uint8).ravel())
    +        # Creating new Row with _create_row(), because Row(name = value, ... )
    +        # orders fields by name, which conflicts with expected schema order
    +        # when the new DataFrame is created by UDF
    +        return _create_row(self.imageFields,
    +                           [origin, height, width, nChannels, mode, data])
    +
    +    def readImages(self, path, recursive=False, numPartitions=0,
    +                   dropImageFailures=False, sampleRatio=1.0):
    +        """
    +        Reads the directory of images from the local or remote source.
    +
    +        :param str path: Path to the image directory
    +        :param SparkSession spark: The current spark session
    +        :param bool recursive: Recursive search flag
    +        :param int numPartitions: Number of DataFrame partitions
    +        :param bool dropImageFailures: Drop the files that are not valid images
    +        :param float sampleRatio: Fraction of the images loaded
    +        :rtype DataFrame: DataFrame with a single column of "images",
    +               see ImageSchema for details
    +
    +        >>> df = ImageSchema.readImages('python/test_support/image/kittens', recursive=True)
    +        >>> df.count()
    +        4
    +
    +        .. versionadded:: 2.3.0
    +        """
    +
    +        ctx = SparkContext._active_spark_context
    +        spark = SparkSession(ctx)
    +        image_schema = ctx._jvm.org.apache.spark.ml.image.ImageSchema
    +        jsession = spark._jsparkSession
    +        jresult = image_schema.readImages(path, jsession, recursive, numPartitions,
    +                                          dropImageFailures, float(sampleRatio))
    +        return DataFrame(jresult, spark._wrapped)
    +
    +
    +ImageSchema = _ImageSchema()
    +
    +
    +# Monkey patch to disallow instantization of this class.
    +def _disallow_instance(_):
    --- End diff --
    
    +0 to removing the monkey patch (but not a strong feeling)


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    @HyukjinKwon and @jkbradley I've updated the documentation based on your latest comments.  I believe all comments have been resolved for this PR at this point, please let me know if I missed anything.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r150449107
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -48,7 +48,7 @@ def imageSchema(self):
             """
             Returns the image schema.
     
    -        :rtype StructType: a DataFrame with a single column of images
    +        :return: StructType: a DataFrame with a single column of images
    --- End diff --
    
    Let's remove the type specification here -> `` :return: a :class:`StructType` ... blabla ``


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r143685432
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/HadoopUtils.scala ---
    @@ -0,0 +1,107 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import scala.language.existentials
    +import scala.util.Random
    +
    +import org.apache.commons.io.FilenameUtils
    +import org.apache.hadoop.conf.{Configuration, Configured}
    +import org.apache.hadoop.fs.{Path, PathFilter}
    +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
    +
    +import org.apache.spark.sql.SparkSession
    +
    +private object RecursiveFlag {
    +
    +  /** Sets a value of spark recursive flag
    +   *
    +   * @param value value to set
    +   * @param spark existing spark session
    +   * @return previous value of this flag
    +   */
    +  def setRecursiveFlag(value: Option[String], spark: SparkSession): Option[String] = {
    +    val flagName = FileInputFormat.INPUT_DIR_RECURSIVE
    +    val hadoopConf = spark.sparkContext.hadoopConfiguration
    +    val old = Option(hadoopConf.get(flagName))
    +
    +    value match {
    +      case Some(v) => hadoopConf.set(flagName, v)
    +      case None => hadoopConf.unset(flagName)
    +    }
    +
    +    old
    +  }
    +}
    +
    +/** Filter that allows loading a fraction of HDFS files. */
    +private class SamplePathFilter extends Configured with PathFilter {
    +  val random = {
    +    val rd = new Random()
    +    rd.setSeed(0)
    +    rd
    +  }
    +
    +  // Ratio of files to be read from disk
    +  var sampleRatio: Double = 1
    +
    +  override def setConf(conf: Configuration): Unit = {
    +    if (conf != null) {
    +      sampleRatio = conf.getDouble(SamplePathFilter.ratioParam, 1)
    +    }
    +  }
    +
    +  override def accept(path: Path): Boolean = {
    +    // Note: checking fileSystem.isDirectory is very slow here, so we use basic rules instead
    +    !SamplePathFilter.isFile(path) ||
    +      random.nextDouble() < sampleRatio
    +  }
    +}
    +
    +private object SamplePathFilter {
    +  val ratioParam = "sampleRatio"
    +
    +  def isFile(path: Path): Boolean = FilenameUtils.getExtension(path.toString) != ""
    +
    +  /** Set/unset  hdfs PathFilter
    +   *
    +   * @param value       Filter class that is passed to HDFS
    +   * @param sampleRatio Fraction of the files that the filter picks
    +   * @param spark       Existing Spark session
    +   * @return
    +   */
    +  def setPathFilter(value: Option[Class[_]], sampleRatio: Option[Double] = None,
    +                    spark: SparkSession)
    +  : Option[Class[_]] = {
    --- End diff --
    
    Usually we don't break new line like this in Spark codes. We possibly do it like:
    
    ```scala
    def setPathFilter(
        value: Option[Class[_]],
        sampleRatio: Option[Double] = None,
        spark: SparkSession): Option[Class[_]] = {
    ```



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r144742423
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC2" -> 8, "CV_8UC3" -> 16, "CV_8UC4" -> 24,
    +    "CV_8S" -> 1, "CV_8SC1" -> 1, "CV_8SC2" -> 9, "CV_8SC3" -> 17, "CV_8SC4" -> 25,
    +    "CV_16U" -> 2, "CV_16UC1" -> 2, "CV_16UC2" -> 10, "CV_16UC3" -> 18, "CV_16UC4" -> 26,
    +    "CV_16S" -> 3, "CV_16SC1" -> 3, "CV_16SC2" -> 11, "CV_16SC3" -> 19, "CV_16SC4" -> 27,
    +    "CV_32S" -> 4, "CV_32SC1" -> 4, "CV_32SC2" -> 12, "CV_32SC3" -> 20, "CV_32SC4" -> 28,
    +    "CV_32F" -> 5, "CV_32FC1" -> 5, "CV_32FC2" -> 13, "CV_32FC3" -> 21, "CV_32FC4" -> 29,
    +    "CV_64F" -> 6, "CV_64FC1" -> 6, "CV_64FC2" -> 14, "CV_64FC3" -> 22, "CV_64FC4" -> 30
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField("origin", StringType, true) ::
    +      StructField("height", IntegerType, false) ::
    +      StructField("width", IntegerType, false) ::
    +      StructField("nChannels", IntegerType, false) ::
    +      // OpenCV-compatible type: CV_8UC3 in most cases
    +      StructField("mode", StringType, false) ::
    +      // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +      StructField("data", BinaryType, false) :: Nil)
    +
    +  // Dataframe with a single column of images named "image" (nullable)
    +  private val imageDFSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  @Since("2.3.0")
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  @Since("2.3.0")
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  @Since("2.3.0")
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  @Since("2.3.0")
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  @Since("2.3.0")
    +  def getMode(row: Row): String = row.getString(4)
    +
    +  @Since("2.3.0")
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * Check if the dataframe column contains images (i.e. has ImageSchema)
    +   *
    +   * @param df       Dataframe
    +   * @param column   Column name
    +   * @return         True if the given column matches the image schema
    +   */
    +  @Since("2.3.0")
    +  def isImageColumn(df: DataFrame, column: String): Boolean =
    +    df.schema(column).dataType == columnSchema
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return       Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row = Row(Row(origin, -1, -1, -1, undefinedImageType,
    +    Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in dataframe Row
    +   *
    +   * @param origin   Arbitrary string that identifies the image
    +   * @param bytes    Image bytes (for example, jpeg)
    +   * @return         Dataframe Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val is_gray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val has_alpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) = if (is_gray) {
    +        (1, "CV_8UC1")
    +      } else if (has_alpha) {
    +        (4, "CV_8UC4")
    +      } else {
    +        (3, "CV_8UC3")
    +      }
    +
    +      assert(height * width * nChannels < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](height*width*nChannels)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (is_gray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for(h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      }
    +      else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    +            if (nChannels == 4) {
    +              decoded(offset + 3) = color.getAlpha.toByte
    +            }
    +            offset += nChannels
    +          }
    +        }
    +      }
    +
    +      // the internal "Row" is needed, because the image is a single dataframe column
    +      Some(Row(Row(origin, height, width, nChannels, mode, decoded)))
    +    }
    +  }
    +
    +  /**
    +   * :: Experimental ::
    +   * Read the directory of images from the local or remote source
    +   *
    +   * @param path               Path to the image directory
    +   * @param sparkSession       Spark Session
    +   * @param recursive          Recursive path search flag
    +   * @param numPartitions      Number of the dataframe partitions
    +   * @param dropImageFailures  Drop the files that are not valid images from the result
    +   * @param sampleRatio        Fraction of the files loaded
    +   * @return                   Dataframe with a single column "image" of images;
    +   *                           see ImageSchema for the details
    +   */
    +  @Since("2.3.0")
    +  def readImages(path: String,
    +                 sparkSession: SparkSession = null,
    +                 recursive: Boolean = false,
    +                 numPartitions: Int = 0,
    +                 dropImageFailures: Boolean = false,
    +                 sampleRatio: Double = 1.0): DataFrame = {
    +    require(sampleRatio <= 1.0 && sampleRatio >= 0, "sampleRatio should be between 0 and 1")
    +
    +    val session = if (sparkSession != null) sparkSession else SparkSession.builder().getOrCreate
    +    val partitions =
    +      if (numPartitions > 0) {
    +        numPartitions
    +      } else {
    +        session.sparkContext.defaultParallelism
    +      }
    +
    +    val oldRecursiveFlag = RecursiveFlag.setRecursiveFlag(Some(recursive.toString), session)
    +    val oldPathFilter: Option[Class[_]] =
    +      if (sampleRatio < 1) {
    +        SamplePathFilter.setPathFilter(Some(classOf[SamplePathFilter]), sampleRatio, session)
    +      } else {
    +        None
    +      }
    +
    +    var result: DataFrame = null
    +    try {
    +      val streams = session.sparkContext.binaryFiles(path, partitions)
    +        .repartition(partitions)
    +
    +      val images = if (dropImageFailures) {
    +        streams.flatMap {
    +          case (origin, stream) => decode(origin, stream.toArray)
    +        }
    +      } else {
    +        streams.map {
    +          case (origin, stream) => decode(origin, stream.toArray).getOrElse(invalidImageRow(origin))
    +        }
    +      }
    +
    +      result = session.createDataFrame(images, imageDFSchema)
    +    }
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r150737174
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -168,7 +167,7 @@ def readImages(self, path, recursive=False, numPartitions=-1,
             :param bool dropImageFailures: Drop the files that are not valid images
             :param float sampleRatio: Fraction of the images loaded
             :param int seed: Random number seed
    -        :rtype DataFrame: DataFrame with a single column of "images",
    +        :return: DataFrame: DataFrame with a single column of "images",
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by MrBago <gi...@git.apache.org>.
Github user MrBago commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r148049352
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,139 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +from pyspark.ml.util import *
    +from pyspark.ml.param.shared import *
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame, SparkSession, SQLContext
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +imageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +
    +def getOcvTypes(spark=None):
    +    """
    +    Returns the OpenCV type mapping supported
    +
    +    :param sparkSession (SparkSession): The current spark session
    +    :rtype dict: The OpenCV type mapping supported
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.ocvTypes
    +
    +
    +# DataFrame with a single column of images named "image" (nullable)
    +def getImageSchema(spark=None):
    --- End diff --
    
    I believe @HyukjinKwon's suggestion uses a singleton object to avoid the need for static methods.
    
    Here is an example of a similar situation from graphframes, the class there uses a mix of class methods and static methods, https://github.com/graphframes/graphframes/pull/169/files#diff-e81e6b169c0aa35012a3263b2f31b330R381.
    
    I don't see a problem with either approach in this case.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r148174944
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,135 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +from pyspark import SparkContext
    +from pyspark.sql.types import Row, _create_row, _parse_datatype_json_string
    +from pyspark.sql import DataFrame, SparkSession
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +imageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +
    +class _ImageSchema(object):
    +    """
    +    Returns the image schema.
    +
    +    :rtype StructType: a DataFrame with a single column of images
    +           named "image" (nullable)
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    @property
    +    def imageSchema(self):
    +        ctx = SparkContext._active_spark_context
    +        jschema = ctx._jvm.org.apache.spark.ml.image.ImageSchema.imageSchema()
    +        return _parse_datatype_json_string(jschema.json())
    +
    +    """
    +    Returns the OpenCV type mapping supported
    +
    +    :rtype dict: The OpenCV type mapping supported
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    @property
    +    def ocvTypes(self):
    +        ctx = SparkContext._active_spark_context
    +        return ctx._jvm.org.apache.spark.ml.image.ImageSchema._ocvTypes()
    +
    +ImageSchema = _ImageSchema()
    --- End diff --
    
    Ah .. thing is, docstring .. let me try to think more tonight (KTS) and probably make a cleanup PR to your branch, @imatiach-msft so that that can be pused to this PR, or leave another comment here.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #83679 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83679/testReport)** for PR 19439 at commit [`04db0fd`](https://github.com/apache/spark/commit/04db0fd02ee1abacc65d20c8d12eab8b6539e09f).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r150449414
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -109,7 +109,7 @@ def toNDArray(self, image):
             Converts an image to a one-dimensional array.
     
             :param image: The image to be converted
    -        :rtype array: The image as a one-dimensional array
    +        :return: array: The image as a one-dimensional array
    --- End diff --
    
    ``:return: `numpy.ndarray` that's blabla``


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r143925678
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/image/ImageSchemaSuite.scala ---
    @@ -0,0 +1,124 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.nio.file.Paths
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.ml.image.ImageSchema._
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.types._
    +
    +class ImageSchemaSuite extends SparkFunSuite with MLlibTestSparkContext {
    +  // Single column of images named "image"
    +  private val imageDFSchema =
    +    StructType(StructField("image", ImageSchema.columnSchema, true) :: Nil)
    +  private lazy val imagePath =
    +    Thread.currentThread().getContextClassLoader.getResource("test-data/images").getPath
    +
    +  test("Smoke test: create basic ImageSchema dataframe") {
    +    val origin = "path"
    +    val width = 1
    +    val height = 1
    +    val nChannels = 3
    +    val data = Array[Byte](0, 0, 0)
    +    val mode = "CV_8UC3"
    +
    +    // Internal Row corresponds to image StructType
    +    val rows = Seq(Row(Row(origin, height, width, nChannels, mode, data)),
    +      Row(Row(null, height, width, nChannels, mode, data)))
    +    val rdd = sc.makeRDD(rows)
    +    val df = spark.createDataFrame(rdd, imageDFSchema)
    +
    +    assert(df.count == 2, "incorrect image count")
    +    assert(ImageSchema.isImageColumn(df, "image"), "data do not fit ImageSchema")
    +  }
    +
    +  test("readImages count test") {
    +    var df = readImages(imagePath, recursive = false)
    +    assert(df.count == 0)
    +
    +    df = readImages(imagePath, recursive = true, dropImageFailures = false)
    +    assert(df.count == 8)
    +
    +    df = readImages(imagePath, recursive = true, dropImageFailures = true)
    +    val count100 = df.count
    +    assert(count100 == 7)
    +
    +    df = readImages(imagePath, recursive = true, sampleRatio = 0.5, dropImageFailures = true)
    +    // Random number about half of the size of the original dataset
    +    val count50 = df.count
    +    assert(count50 > 0.2 * count100 && count50 < 0.8 * count100)
    +  }
    +
    +  test("readImages partition test") {
    +    val df = readImages(imagePath, recursive = true, dropImageFailures = true, numPartitions = 3)
    +    assert(df.rdd.getNumPartitions == 3)
    +  }
    +
    +  // Images with the different number of channels
    +  test("readImages pixel values test") {
    +
    +    val images = readImages(imagePath + "/multi-channel/", recursive = false).collect
    +
    +    images.foreach{
    +      rrow => {
    +        val row = rrow.getAs[Row](0)
    +        val filename = Paths.get(getOrigin(row)).getFileName().toString()
    +        if(firstBytes20.contains(filename)) {
    +          val mode = getMode(row)
    +          val bytes20 = getData(row).slice(0, 20)
    +
    +          val expectedMode = firstBytes20(filename)._1
    +          val expectedBytes = firstBytes20(filename)._2
    +
    +          assert(expectedMode == mode, "mode of the image is not read correctly")
    +
    +          if (!compareBytes(expectedBytes, bytes20)) {
    +            throw new Exception("incorrect numeric value for flattened image")
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  // number of channels and first 20 bytes of OpenCV representation
    +  // - default representation for 3-channel RGB images is BGR row-wise:
    +  //   (B00, G00, R00,      B10, G10, R10,      ...)
    +  // - default representation for 4-channel RGB images is BGRA row-wise:
    +  //   (B00, G00, R00, A00, B10, G10, R10, A00, ...)
    +  private val firstBytes20 = Map(
    +    "grayscale.png" ->
    +      (("CV_8UC1", Array[Byte](0, 0, 0, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 1, 0, 0, 3, 5, 2, 1))),
    +    "RGB.png" -> (("CV_8UC3",
    +      Array[Byte](-34, -66, -98, -38, -69, -98, -62, -90, -117,
    +        -70, -98, -124, -34, -63, -90, -20, -48, -74, -18, -45))),
    +    "RGBA.png" -> (("CV_8UC4",
    +      Array[Byte](-128, -128, -8, -1, -128, -128, -8, -1, -128,
    +        -128, -8, -1, 127, 127, -9, -1, 127, 127, -9, -1)))
    +  )
    +
    +  private def compareBytes(x: Array[Byte], y: Array[Byte]): Boolean = {
    +    val length = Math.min(x.length, y.length)
    --- End diff --
    
    Isn't there a JDK method that compares arrays?



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r144193367
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC2" -> 8, "CV_8UC3" -> 16, "CV_8UC4" -> 24,
    +    "CV_8S" -> 1, "CV_8SC1" -> 1, "CV_8SC2" -> 9, "CV_8SC3" -> 17, "CV_8SC4" -> 25,
    +    "CV_16U" -> 2, "CV_16UC1" -> 2, "CV_16UC2" -> 10, "CV_16UC3" -> 18, "CV_16UC4" -> 26,
    +    "CV_16S" -> 3, "CV_16SC1" -> 3, "CV_16SC2" -> 11, "CV_16SC3" -> 19, "CV_16SC4" -> 27,
    +    "CV_32S" -> 4, "CV_32SC1" -> 4, "CV_32SC2" -> 12, "CV_32SC3" -> 20, "CV_32SC4" -> 28,
    +    "CV_32F" -> 5, "CV_32FC1" -> 5, "CV_32FC2" -> 13, "CV_32FC3" -> 21, "CV_32FC4" -> 29,
    +    "CV_64F" -> 6, "CV_64FC1" -> 6, "CV_64FC2" -> 14, "CV_64FC3" -> 22, "CV_64FC4" -> 30
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField("origin", StringType, true) ::
    +      StructField("height", IntegerType, false) ::
    +      StructField("width", IntegerType, false) ::
    +      StructField("nChannels", IntegerType, false) ::
    +      // OpenCV-compatible type: CV_8UC3 in most cases
    +      StructField("mode", StringType, false) ::
    +      // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +      StructField("data", BinaryType, false) :: Nil)
    +
    +  // Dataframe with a single column of images named "image" (nullable)
    +  private val imageDFSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  @Since("2.3.0")
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r147989847
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,139 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +from pyspark.ml.util import *
    +from pyspark.ml.param.shared import *
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame, SparkSession, SQLContext
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +imageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +
    +def getOcvTypes(spark=None):
    +    """
    +    Returns the OpenCV type mapping supported
    +
    +    :param sparkSession (SparkSession): The current spark session
    +    :rtype dict: The OpenCV type mapping supported
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.ocvTypes
    +
    +
    +# DataFrame with a single column of images named "image" (nullable)
    +def getImageSchema(spark=None):
    +    """
    +    Returns the image schema
    +
    +    :param spark (SparkSession): The current spark session
    +    :rtype StructType: The image schema
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.imageSchema
    +
    +
    +def toNDArray(image):
    +    """
    +    Converts an image to a one-dimensional array.
    +
    +    :param image (object): The image to be converted
    +    :rtype array: The image as a one-dimensional array
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    height = image.height
    +    width = image.width
    +    nChannels = image.nChannels
    +    return np.ndarray(
    +        shape=(height, width, nChannels),
    +        dtype=np.uint8,
    +        buffer=image.data,
    +        strides=(width * nChannels, nChannels, 1))
    +
    +
    +def toImage(array, origin="", spark=None):
    +    """
    +    Converts a one-dimensional array to a two-dimensional image.
    +
    +    :param array (array): The array to convert to image
    +    :param origin (str): Path to the image
    +    :param spark (SparkSession): The current spark session
    +    :rtype object: Two dimensional image
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    if array.ndim != 3:
    +        raise
    +    height, width, nChannels = array.shape
    +    ocvTypes = getOcvTypes(spark)
    +    if nChannels == 1:
    +        mode = ocvTypes["CV_8UC1"]
    +    elif nChannels == 3:
    +        mode = ocvTypes["CV_8UC3"]
    +    elif nChannels == 4:
    +        mode = ocvTypes["CV_8UC4"]
    +    else:
    +        raise
    +    data = bytearray(array.astype(dtype=np.uint8).ravel())
    +    # Creating new Row with _create_row(), because Row(name = value, ... )
    +    # orders fields by name, which conflicts with expected schema order
    +    # when the new DataFrame is created by UDF
    +    return _create_row(imageFields,
    +                       [origin, height, width, nChannels, mode, data])
    +
    +
    +def readImages(path, spark=None, recursive=False, numPartitions=0,
    +               dropImageFailures=False, sampleRatio=1.0):
    +    """
    +    Reads the directory of images from the local or remote source.
    +
    +    :param path (str): Path to the image directory
    +    :param spark (SparkSession): The current spark session
    +    :param recursive (bool): Recursive search flag
    +    :param numPartitions (int): Number of DataFrame partitions
    +    :param dropImageFailures (bool): Drop the files that are not valid images
    +    :param sampleRatio (double): Fraction of the images loaded
    +    :rtype DataFrame: DataFrame with a single column of "images",
    +           see ImageSchema for details
    +
    +    Examples:
    +
    +    >>> df = readImages('python/test_support/image/kittens', recursive=True)
    +    >>> df.count
    +    4
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    image_schema = ctx._jvm.org.apache.spark.ml.image.ImageSchema
    +    sql_ctx = SQLContext(ctx)
    +    jsession = spark._jsparkSession
    +    jresult = image_schema.readImages(path, jsession, recursive, numPartitions,
    +                                      dropImageFailures, float(sampleRatio))
    +    return DataFrame(jresult, sql_ctx)
    --- End diff --
    
    I think we can simply pass `spark._wrapped` instead of `SQLContext(ctx)`. And also remove `SQLContext` import.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Thanks for the explanation!  Given the complexity here, I'm OK with the random seed approach but recommend we add a warning about sampling being more efficient but potentially non-deterministic. What do you think @imatiach-msft ?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #83845 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83845/testReport)** for PR 19439 at commit [`c4a6dfe`](https://github.com/apache/spark/commit/c4a6dfe736b49384b8744241b40219192e400051).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #3988 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/3988/testReport)** for PR 19439 at commit [`c4a6dfe`](https://github.com/apache/spark/commit/c4a6dfe736b49384b8744241b40219192e400051).
     * This patch passes all tests.
     * This patch **does not merge cleanly**.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r144298709
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/HadoopUtils.scala ---
    @@ -0,0 +1,119 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import scala.language.existentials
    +import scala.util.Random
    +
    +import org.apache.commons.io.FilenameUtils
    +import org.apache.hadoop.conf.{Configuration, Configured}
    +import org.apache.hadoop.fs.{Path, PathFilter}
    +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
    +
    +import org.apache.spark.sql.SparkSession
    +
    +private object RecursiveFlag {
    +
    +  /**
    +   * Sets a value of spark recursive flag.
    +   * If value is a None, it unsets the flag.
    +   *
    +   * @param value value to set
    +   * @param spark existing spark session
    +   * @return previous value of this flag
    +   */
    +  def setRecursiveFlag(value: Option[String], spark: SparkSession): Option[String] = {
    +    val flagName = FileInputFormat.INPUT_DIR_RECURSIVE
    +    val hadoopConf = spark.sparkContext.hadoopConfiguration
    +    val old = Option(hadoopConf.get(flagName))
    +
    +    value match {
    +      case Some(v) => hadoopConf.set(flagName, v)
    +      case None => hadoopConf.unset(flagName)
    +    }
    +
    +    old
    +  }
    +}
    +
    +/**
    + * Filter that allows loading a fraction of HDFS files.
    + */
    +private class SamplePathFilter extends Configured with PathFilter {
    +  val random = new Random()
    +
    +  // Ratio of files to be read from disk
    +  var sampleRatio: Double = 1
    +
    +  override def setConf(conf: Configuration): Unit = {
    +    if (conf != null) {
    +      sampleRatio = conf.getDouble(SamplePathFilter.ratioParam, 1)
    +    }
    +  }
    +
    +  override def accept(path: Path): Boolean = {
    +    // Note: checking fileSystem.isDirectory is very slow here, so we use basic rules instead
    +    !SamplePathFilter.isFile(path) ||
    +      random.nextDouble() < sampleRatio
    +  }
    +}
    +
    +private object SamplePathFilter {
    +  val ratioParam = "sampleRatio"
    +
    +  def isFile(path: Path): Boolean = FilenameUtils.getExtension(path.toString) != ""
    +
    +  /**
    +   * Sets HDFS PathFilter
    +   *
    +   * @param value       Filter class that is passed to HDFS
    +   * @param sampleRatio Fraction of the files that the filter picks
    +   * @param spark       Existing Spark session
    +   * @return            Returns the previous HDFS path filter
    +   */
    +  def setPathFilter(value: Option[Class[_]], sampleRatio: Double,
    +                    spark: SparkSession) : Option[Class[_]] = {
    +    val flagName = FileInputFormat.PATHFILTER_CLASS
    +    val hadoopConf = spark.sparkContext.hadoopConfiguration
    +    val old = Option(hadoopConf.getClass(flagName, null))
    +    hadoopConf.setDouble(SamplePathFilter.ratioParam, sampleRatio)
    +
    +    value match {
    +      case Some(v) => hadoopConf.setClass(flagName, v, classOf[PathFilter])
    +      case None => hadoopConf.unset(flagName)
    +    }
    +    old
    +  }
    +
    +  /**
    +   * Unsets HDFS PathFilter
    +   *
    +   * @param value       Filter class to restore to HDFS
    +   * @param spark       Existing Spark session
    +   */
    +  def unsetPathFilter(value: Option[Class[_]], spark: SparkSession) : Unit = {
    --- End diff --
    
    nit: `) :` -> `):`.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by MrBago <gi...@git.apache.org>.
Github user MrBago commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    @imatiach-msft just a few more comments. When I was looking over this I realized that the python and Scala name spaces are going to be a little different, eg `pyspark.ml.image.readImages` vs `spark.ml.image.ImageSchema.readImages`. Should we use an `ImageSchema` class in python as a namespace? Does spark do that in other pyspark modules?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r150164406
  
    --- Diff: python/pyspark/ml/tests.py ---
    @@ -1818,6 +1819,24 @@ def tearDown(self):
             del self.data
     
     
    +class ImageReaderTest(SparkSessionTestCase):
    +
    +    def test_read_images(self):
    +        data_path = 'python/test_support/image/kittens'
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r148027184
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,139 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +from pyspark.ml.util import *
    +from pyspark.ml.param.shared import *
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame, SparkSession, SQLContext
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +imageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +
    +def getOcvTypes(spark=None):
    +    """
    +    Returns the OpenCV type mapping supported
    +
    +    :param sparkSession (SparkSession): The current spark session
    +    :rtype dict: The OpenCV type mapping supported
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.ocvTypes
    +
    +
    +# DataFrame with a single column of images named "image" (nullable)
    +def getImageSchema(spark=None):
    --- End diff --
    
    Actually, I think I don't like this API inconsistency. Should we maybe match this to Scala one.
    
    Could we maybe do this as below:
    
    ```python
    class _ImageSchema(object):
        @property
        def imageSchema(self):
            jschema = ctx._jvm.org.apache.spark.ml.image.ImageSchema.imageSchema()
            return _parse_datatype_json_string(jschema.json())
    
    
    ImageSchema = _ImageSchema()
    ```
    
    so that we can call:
    
    ```python
    >>> from pyspark.ml.image import ImageSchema
    >>> ImageSchema.imageSchema
    StructType(List(StructField(image,StructType(List(StructField(origin,StringType,true),StructField(height,IntegerType,false),StructField(width,IntegerType,false),StructField(nChannels,IntegerType,false),StructField(mode,IntegerType,false),StructField(data,BinaryType,false))),true)))
    ```
    
    like Scala
    
    ```scala
    scala> import org.apache.spark.ml.image.ImageSchema
    import org.apache.spark.ml.image.ImageSchema
    
    scala> ImageSchema.imageSchema
    res0: org.apache.spark.sql.types.StructType = StructType(StructField(image,StructType(StructField(origin,StringType,true), StructField(height,IntegerType,false), StructField(width,IntegerType,false), StructField(nChannels,IntegerType,false), StructField(mode,IntegerType,false), StructField(data,BinaryType,false)),true))
    ```
    
    Please let me know if anyone has some opinions about this suggestion.
    
    cc @jkbradley too. I think had a talk about API consistency with you before if I remember correctly.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    @MrBago in regards to the question below -- I discussed with @drdarshan who is familiar with Python, and he said that it wouldn't be standard to create a class just to change the namespace and have static methods everywhere (having a class does not make sense here since there is no common state).  I don't think the namespace being slightly different is an issue.
    
    "@imatiach-msft just a few more comments. When I was looking over this I realized that the python and Scala name spaces are going to be a little different, eg pyspark.ml.image.readImages vs spark.ml.image.ImageSchema.readImages. Should we use an ImageSchema class in python as a namespace? Does spark do that in other pyspark modules?"



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #82552 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82552/testReport)** for PR 19439 at commit [`b0c4ace`](https://github.com/apache/spark/commit/b0c4ace0f82694d4f3a065c43e1d715d763703dc).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Jenkins retest this please


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/83097/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r143924657
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/HadoopUtils.scala ---
    @@ -0,0 +1,122 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import scala.language.existentials
    +import scala.util.Random
    +
    +import org.apache.commons.io.FilenameUtils
    +import org.apache.hadoop.conf.{Configuration, Configured}
    +import org.apache.hadoop.fs.{Path, PathFilter}
    +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
    +
    +import org.apache.spark.sql.SparkSession
    +
    +private object RecursiveFlag {
    +
    +  /**
    +   * Sets a value of spark recursive flag
    +   *
    +   * @param value value to set
    +   * @param spark existing spark session
    +   * @return previous value of this flag
    +   */
    +  def setRecursiveFlag(value: Option[String], spark: SparkSession): Option[String] = {
    +    val flagName = FileInputFormat.INPUT_DIR_RECURSIVE
    +    val hadoopConf = spark.sparkContext.hadoopConfiguration
    +    val old = Option(hadoopConf.get(flagName))
    +
    +    value match {
    +      case Some(v) => hadoopConf.set(flagName, v)
    +      case None => hadoopConf.unset(flagName)
    +    }
    +
    +    old
    +  }
    +}
    +
    +/**
    + * Filter that allows loading a fraction of HDFS files.
    + */
    +private class SamplePathFilter extends Configured with PathFilter {
    +  val random = {
    +    val rd = new Random()
    +    rd.setSeed(0)
    +    rd
    +  }
    +
    +  // Ratio of files to be read from disk
    +  var sampleRatio: Double = 1
    +
    +  override def setConf(conf: Configuration): Unit = {
    +    if (conf != null) {
    +      sampleRatio = conf.getDouble(SamplePathFilter.ratioParam, 1)
    +    }
    +  }
    +
    +  override def accept(path: Path): Boolean = {
    +    // Note: checking fileSystem.isDirectory is very slow here, so we use basic rules instead
    +    !SamplePathFilter.isFile(path) ||
    +      random.nextDouble() < sampleRatio
    +  }
    +}
    +
    +private object SamplePathFilter {
    +  val ratioParam = "sampleRatio"
    +
    +  def isFile(path: Path): Boolean = FilenameUtils.getExtension(path.toString) != ""
    +
    +  /**
    +   * Sets hdfs PathFilter
    +   *
    +   * @param value       Filter class that is passed to HDFS
    +   * @param sampleRatio Fraction of the files that the filter picks
    +   * @param spark       Existing Spark session
    +   * @return            Returns the previous hdfs path filter
    +   */
    +  def setPathFilter(value: Option[Class[_]], sampleRatio: Double,
    +                    spark: SparkSession) : Option[Class[_]] = {
    +    val flagName = FileInputFormat.PATHFILTER_CLASS
    +    val hadoopConf = spark.sparkContext.hadoopConfiguration
    +    val old = Option(hadoopConf.getClass(flagName, null))
    +    hadoopConf.setDouble(SamplePathFilter.ratioParam, sampleRatio)
    +
    +    value match {
    +      case Some(v) => hadoopConf.setClass(flagName, v, classOf[PathFilter])
    +      case None => hadoopConf.unset(flagName)
    +    }
    +    old
    +  }
    +
    +  /**
    +   * Unsets hdfs PathFilter
    --- End diff --
    
    Nit: look for typos like hdfs -> HDFS


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r143907752
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC2" -> 8, "CV_8UC3" -> 16, "CV_8UC4" -> 24,
    +    "CV_8S" -> 1, "CV_8SC1" -> 1, "CV_8SC2" -> 9, "CV_8SC3" -> 17, "CV_8SC4" -> 25,
    +    "CV_16U" -> 2, "CV_16UC1" -> 2, "CV_16UC2" -> 10, "CV_16UC3" -> 18, "CV_16UC4" -> 26,
    +    "CV_16S" -> 3, "CV_16SC1" -> 3, "CV_16SC2" -> 11, "CV_16SC3" -> 19, "CV_16SC4" -> 27,
    +    "CV_32S" -> 4, "CV_32SC1" -> 4, "CV_32SC2" -> 12, "CV_32SC3" -> 20, "CV_32SC4" -> 28,
    +    "CV_32F" -> 5, "CV_32FC1" -> 5, "CV_32FC2" -> 13, "CV_32FC3" -> 21, "CV_32FC4" -> 29,
    +    "CV_64F" -> 6, "CV_64FC1" -> 6, "CV_64FC2" -> 14, "CV_64FC3" -> 22, "CV_64FC4" -> 30
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField("origin", StringType, true) ::
    +      StructField("height", IntegerType, false) ::
    +      StructField("width", IntegerType, false) ::
    +      StructField("nChannels", IntegerType, false) ::
    +      // OpenCV-compatible type: CV_8UC3 in most cases
    +      StructField("mode", StringType, false) ::
    +      // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +      StructField("data", BinaryType, false) :: Nil)
    +
    +  // Dataframe with a single column of images named "image" (nullable)
    +  private val imageDFSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  @Since("2.3.0")
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  @Since("2.3.0")
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  @Since("2.3.0")
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  @Since("2.3.0")
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  @Since("2.3.0")
    +  def getMode(row: Row): String = row.getString(4)
    +
    +  @Since("2.3.0")
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * Check if the dataframe column contains images (i.e. has ImageSchema)
    +   *
    +   * @param df       Dataframe
    +   * @param column   Column name
    +   * @return         True if the given column matches the image schema
    +   */
    +  @Since("2.3.0")
    +  def isImageColumn(df: DataFrame, column: String): Boolean =
    +    df.schema(column).dataType == columnSchema
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return       Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row = Row(Row(origin, -1, -1, -1, undefinedImageType,
    +    Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in dataframe Row
    +   *
    +   * @param origin   Arbitrary string that identifies the image
    +   * @param bytes    Image bytes (for example, jpeg)
    +   * @return         Dataframe Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val is_gray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val has_alpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) = if (is_gray) {
    +        (1, "CV_8UC1")
    +      } else if (has_alpha) {
    +        (4, "CV_8UC4")
    +      } else {
    +        (3, "CV_8UC3")
    +      }
    +
    +      assert(height * width * nChannels < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](height*width*nChannels)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (is_gray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for(h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      }
    +      else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    --- End diff --
    
    Will overflow be an issue later?
    
    scala> val c = new Color(255, 255, 255)
    c: java.awt.Color = java.awt.Color[r=255,g=255,b=255]
    
    scala> c.getGreen
    res12: Int = 255
    
    scala> c.getGreen.toByte
    res13: Byte = -1



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by MrBago <gi...@git.apache.org>.
Github user MrBago commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r143798662
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,133 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +import pyspark
    +from pyspark import SparkContext
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame
    +from pyspark.ml.param.shared import *
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +ImageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +ocvTypes = {
    +    undefinedImageType: -1,
    +    "CV_8U": 0, "CV_8UC1": 0, "CV_8UC2": 8, "CV_8UC3": 16, "CV_8UC4": 24,
    +    "CV_8S": 1, "CV_8SC1": 1, "CV_8SC2": 9, "CV_8SC3": 17, "CV_8SC4": 25,
    +    "CV_16U": 2, "CV_16UC1": 2, "CV_16UC2": 10, "CV_16UC3": 18, "CV_16UC4": 26,
    +    "CV_16S": 3, "CV_16SC1": 3, "CV_16SC2": 11, "CV_16SC3": 19, "CV_16SC4": 27,
    +    "CV_32S": 4, "CV_32SC1": 4, "CV_32SC2": 12, "CV_32SC3": 20, "CV_32SC4": 28,
    +    "CV_32F": 5, "CV_32FC1": 5, "CV_32FC2": 13, "CV_32FC3": 21, "CV_32FC4": 29,
    +    "CV_64F": 6, "CV_64FC1": 6, "CV_64FC2": 14, "CV_64FC3": 22, "CV_64FC4": 30
    +}
    +
    +ImageSchema = StructType([
    +    StructField(ImageFields[0], StringType(),  True),
    +    StructField(ImageFields[1], IntegerType(), False),
    +    StructField(ImageFields[2], IntegerType(), False),
    +    StructField(ImageFields[3], IntegerType(), False),
    +    # OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(ImageFields[4], StringType(), False),
    +    # bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(ImageFields[5], BinaryType(), False)])
    +
    +
    +# TODO: generalize to other datatypes and number of channels
    +def toNDArray(image):
    +    """
    +    Converts an image to a 1-dimensional array
    +
    +    Args:
    +        image (object): The image to be converted
    +
    +    Returns:
    +        array: The image as a 1-dimensional array
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    height = image.height
    +    width = image.width
    +    return np.asarray(image.data, dtype=np.uint8) \
    +             .reshape((height, width, 3))[:, :, (2, 1, 0)]
    --- End diff --
    
    This code assumes `image` is 3-channel GBR image and the user wants an RGB image. I think we should try and support at least the 3 image types that `readImages` (CV_8UC1, CV_8UC3, and CV_8UC4) but it would be nice to also support 1, 3, and 4 channel float images.
    
    The ndarray constructor is quite flexible and might be easier to work with than calling `asarray` in this case because you want to treat the bytearray as a buffer not as a sequence of ints.
    
    ```
    np.ndarray(
      shape=(height, width, nChannels),
      dtype=np.uint8,
      buffer=image.data,
      strides=(width * nChannels, nChannels, 1))
    ```
    
    Also I have mixed feelings about re-ordering the channels. I think it's probably useful in the most common use-case, but (if my understanding is correct) the open cv types don't require a specific channel order so we can't just assume the input is RGB or RGBA. Maybe we should avoid re-ordering, document the ordering we use whenever appropriate, and then leave it up to the user to do any necessary re-order for themselves.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/83671/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r143778773
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/HadoopUtils.scala ---
    @@ -0,0 +1,107 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import scala.language.existentials
    +import scala.util.Random
    +
    +import org.apache.commons.io.FilenameUtils
    +import org.apache.hadoop.conf.{Configuration, Configured}
    +import org.apache.hadoop.fs.{Path, PathFilter}
    +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
    +
    +import org.apache.spark.sql.SparkSession
    +
    +private object RecursiveFlag {
    +
    +  /** Sets a value of spark recursive flag
    +   *
    +   * @param value value to set
    +   * @param spark existing spark session
    +   * @return previous value of this flag
    +   */
    +  def setRecursiveFlag(value: Option[String], spark: SparkSession): Option[String] = {
    +    val flagName = FileInputFormat.INPUT_DIR_RECURSIVE
    +    val hadoopConf = spark.sparkContext.hadoopConfiguration
    +    val old = Option(hadoopConf.get(flagName))
    +
    +    value match {
    +      case Some(v) => hadoopConf.set(flagName, v)
    +      case None => hadoopConf.unset(flagName)
    +    }
    +
    +    old
    +  }
    +}
    +
    +/** Filter that allows loading a fraction of HDFS files. */
    +private class SamplePathFilter extends Configured with PathFilter {
    +  val random = {
    +    val rd = new Random()
    +    rd.setSeed(0)
    +    rd
    +  }
    +
    +  // Ratio of files to be read from disk
    +  var sampleRatio: Double = 1
    +
    +  override def setConf(conf: Configuration): Unit = {
    +    if (conf != null) {
    +      sampleRatio = conf.getDouble(SamplePathFilter.ratioParam, 1)
    +    }
    +  }
    +
    +  override def accept(path: Path): Boolean = {
    +    // Note: checking fileSystem.isDirectory is very slow here, so we use basic rules instead
    +    !SamplePathFilter.isFile(path) ||
    +      random.nextDouble() < sampleRatio
    +  }
    +}
    +
    +private object SamplePathFilter {
    +  val ratioParam = "sampleRatio"
    +
    +  def isFile(path: Path): Boolean = FilenameUtils.getExtension(path.toString) != ""
    +
    +  /** Set/unset  hdfs PathFilter
    +   *
    +   * @param value       Filter class that is passed to HDFS
    +   * @param sampleRatio Fraction of the files that the filter picks
    +   * @param spark       Existing Spark session
    +   * @return
    +   */
    +  def setPathFilter(value: Option[Class[_]], sampleRatio: Option[Double] = None,
    --- End diff --
    
    done, added an unset method


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r148055894
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,139 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +from pyspark.ml.util import *
    +from pyspark.ml.param.shared import *
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame, SparkSession, SQLContext
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +imageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +
    +def getOcvTypes(spark=None):
    +    """
    +    Returns the OpenCV type mapping supported
    +
    +    :param sparkSession (SparkSession): The current spark session
    +    :rtype dict: The OpenCV type mapping supported
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.ocvTypes
    +
    +
    +# DataFrame with a single column of images named "image" (nullable)
    +def getImageSchema(spark=None):
    --- End diff --
    
    I see, I've updated the code to use this approach


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r147002808
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,122 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +import pyspark
    +from pyspark import SparkContext
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame
    +from pyspark.ml.param.shared import *
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +imageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +ocvTypes = {
    +    undefinedImageType: -1,
    +    "CV_8U": 0, "CV_8UC1": 0, "CV_8UC2": 8, "CV_8UC3": 16, "CV_8UC4": 24,
    +    "CV_8S": 1, "CV_8SC1": 1, "CV_8SC2": 9, "CV_8SC3": 17, "CV_8SC4": 25,
    +    "CV_16U": 2, "CV_16UC1": 2, "CV_16UC2": 10, "CV_16UC3": 18, "CV_16UC4": 26,
    +    "CV_16S": 3, "CV_16SC1": 3, "CV_16SC2": 11, "CV_16SC3": 19, "CV_16SC4": 27,
    +    "CV_32S": 4, "CV_32SC1": 4, "CV_32SC2": 12, "CV_32SC3": 20, "CV_32SC4": 28,
    +    "CV_32F": 5, "CV_32FC1": 5, "CV_32FC2": 13, "CV_32FC3": 21, "CV_32FC4": 29,
    +    "CV_64F": 6, "CV_64FC1": 6, "CV_64FC2": 14, "CV_64FC3": 22, "CV_64FC4": 30
    +}
    +
    +# DataFrame with a single column of images named "image" (nullable)
    +imageSchema = StructType(StructField("image", StructType([
    +    StructField(imageFields[0], StringType(),  True),
    +    StructField(imageFields[1], IntegerType(), False),
    +    StructField(imageFields[2], IntegerType(), False),
    +    StructField(imageFields[3], IntegerType(), False),
    +    # OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(imageFields[4], StringType(), False),
    +    # bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(imageFields[5], BinaryType(), False)]), True))
    +
    +
    +def toNDArray(image):
    +    """
    +    Converts an image to a one-dimensional array.
    +
    +    :param image (object): The image to be converted
    +    :rtype array: The image as a one-dimensional array
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    height = image.height
    +    width = image.width
    +    nChannels = image.nChannels
    +    return np.ndarray(
    +        shape=(height, width, nChannels),
    +        dtype=np.uint8,
    --- End diff --
    
    I limited the ocvType for now to resolve the issue


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #83106 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83106/testReport)** for PR 19439 at commit [`c4e2c48`](https://github.com/apache/spark/commit/c4e2c483c1bd8606ff2701ed4c0ca6e3ae455645).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #84118 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84118/testReport)** for PR 19439 at commit [`a76496b`](https://github.com/apache/spark/commit/a76496be9ebc8b4aba1cd1cd4e3132411649597e).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r143782986
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,217 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.Since
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC2" -> 8, "CV_8UC3" -> 16, "CV_8UC4" -> 24,
    +    "CV_8S" -> 1, "CV_8SC1" -> 1, "CV_8SC2" -> 9, "CV_8SC3" -> 17, "CV_8SC4" -> 25,
    +    "CV_16U" -> 2, "CV_16UC1" -> 2, "CV_16UC2" -> 10, "CV_16UC3" -> 18, "CV_16UC4" -> 26,
    +    "CV_16S" -> 3, "CV_16SC1" -> 3, "CV_16SC2" -> 11, "CV_16SC3" -> 19, "CV_16SC4" -> 27,
    +    "CV_32S" -> 4, "CV_32SC1" -> 4, "CV_32SC2" -> 12, "CV_32SC3" -> 20, "CV_32SC4" -> 28,
    +    "CV_32F" -> 5, "CV_32FC1" -> 5, "CV_32FC2" -> 13, "CV_32FC3" -> 21, "CV_32FC4" -> 29,
    +    "CV_64F" -> 6, "CV_64FC1" -> 6, "CV_64FC2" -> 14, "CV_64FC3" -> 22, "CV_64FC4" -> 30
    +  )
    +
    +  /** Schema for the image column: Row(String, Int, Int, Int, Array[Byte]) */
    +  val columnSchema = StructType(
    +    StructField("origin", StringType, true) ::
    +      StructField("height", IntegerType, false) ::
    +      StructField("width", IntegerType, false) ::
    +      StructField("nChannels", IntegerType, false) ::
    +      // OpenCV-compatible type: CV_8UC3 in most cases
    +      StructField("mode", StringType, false) ::
    +      // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +      StructField("data", BinaryType, false) :: Nil)
    +
    +  // Dataframe with a single column of images named "image" (nullable)
    +  private val imageDFSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  @Since("2.3.0")
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  @Since("2.3.0")
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  @Since("2.3.0")
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  @Since("2.3.0")
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  @Since("2.3.0")
    +  def getMode(row: Row): String = row.getString(4)
    +
    +  @Since("2.3.0")
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /** Check if the dataframe column contains images (i.e. has ImageSchema)
    +   *
    +   * @param df       Dataframe
    +   * @param column   Column name
    +   * @return         True if the given column matches the image schema
    +   */
    +  @Since("2.3.0")
    +  def isImageColumn(df: DataFrame, column: String): Boolean =
    +    df.schema(column).dataType == columnSchema
    +
    +  /** Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return       Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row = Row(Row(origin, -1, -1, -1, undefinedImageType,
    +    Array.ofDim[Byte](0)))
    +
    +  /** Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in dataframe Row
    +   *
    +   * @param origin   Arbitrary string that identifies the image
    +   * @param bytes    Image bytes (for example, jpeg)
    +   * @return         Dataframe Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val is_gray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val has_alpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) =
    +        if (is_gray) (1, "CV_8UC1")
    +        else if (has_alpha) (4, "CV_8UC4")
    +        else (3, "CV_8UC3")
    +
    +      assert(height*width*nChannels < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](height*width*nChannels)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (is_gray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for(h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      }
    +      else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    +            if (nChannels == 4) {
    +              decoded(offset + 3) = color.getAlpha.toByte
    +            }
    +            offset += nChannels
    +          }
    +        }
    +      }
    +
    +      // the internal "Row" is needed, because the image is a single dataframe column
    +      Some(Row(Row(origin, height, width, nChannels, mode, decoded)))
    +    }
    +  }
    +
    +  /** Read the directory of images from the local or remote source
    +   *
    +   * @param path               Path to the image directory
    +   * @param sparkSession       Spark Session
    +   * @param recursive          Recursive path search flag
    +   * @param numPartitions      Number of the dataframe partitions
    +   * @param dropImageFailures  Drop the files that are not valid images from the result
    +   * @param sampleRatio        Fraction of the files loaded
    +   * @return                   Dataframe with a single column "image" of images;
    +   *                           see ImageSchema for the details
    +   */
    +  @Since("2.3.0")
    +  def readImages(path: String,
    +                 sparkSession: SparkSession = null,
    +                 recursive: Boolean = false,
    +                 numPartitions: Int = 0,
    +                 dropImageFailures: Boolean = false,
    +                 sampleRatio: Double = 1.0): DataFrame = {
    +    require(sampleRatio <= 1.0 && sampleRatio >= 0, "sampleRatio should be between 0 and 1")
    +
    +    val session = if (sparkSession != null) sparkSession else SparkSession.builder().getOrCreate
    +    val partitions =
    +      if (numPartitions > 0) numPartitions
    +      else session.sparkContext.defaultParallelism
    +
    +    val oldRecursiveFlag = RecursiveFlag.setRecursiveFlag(Some(recursive.toString), session)
    +    val oldPathFilter: Option[Class[_]] =
    +      if (sampleRatio < 1) {
    +        SamplePathFilter.setPathFilter(Some(classOf[SamplePathFilter]), Some(sampleRatio), session)
    +      }
    +      else {
    +        None
    +      }
    +
    +    var result: DataFrame = null
    +    try {
    +      val streams = session.sparkContext.binaryFiles(path, partitions)
    +        .repartition(partitions)
    +
    +      val images = if (dropImageFailures) {
    +        streams.flatMap{
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r148035301
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,139 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +from pyspark.ml.util import *
    +from pyspark.ml.param.shared import *
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame, SparkSession, SQLContext
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +imageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +
    +def getOcvTypes(spark=None):
    +    """
    +    Returns the OpenCV type mapping supported
    +
    +    :param sparkSession (SparkSession): The current spark session
    +    :rtype dict: The OpenCV type mapping supported
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.ocvTypes
    +
    +
    +# DataFrame with a single column of images named "image" (nullable)
    +def getImageSchema(spark=None):
    --- End diff --
    
    according to @drdarshan we shouldn't have a python class with a static method.
    I don't think the APIs being slightly different is a problem, but if there is a different way to make the APIs use the same package/namespace (without using a static method on a class, which doesn't make sense to do) I can update


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r150737147
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -129,7 +129,7 @@ def toImage(self, array, origin=""):
     
             :param array array: The array to convert to image
             :param str origin: Path to the image, optional
    -        :rtype object: Two dimensional image
    +        :return: object: Two dimensional image
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    @viirya @srowen @MrBago @holdenk @HyukjinKwon @dakirsa @hhbyyh Thank you for taking a look at the PR merging the "spark-images" spark package into spark and your great comments.  I believe all comments have been resolved.  If you have time, please take another look at the PR in case I missed anything, and let me know if you have any concerns.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r147737843
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,252 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC3" -> 16, "CV_8UC4" -> 24
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField(imageFields(0), StringType, true) ::
    +    StructField(imageFields(1), IntegerType, false) ::
    +    StructField(imageFields(2), IntegerType, false) ::
    +    StructField(imageFields(3), IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(imageFields(4), IntegerType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(imageFields(5), BinaryType, false) :: Nil)
    +
    +  /**
    +   * DataFrame with a single column of images named "image" (nullable)
    +   */
    +  val imageSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the height of the image
    +   *
    +   * @return The height of the image
    +   */
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the width of the image
    +   *
    +   * @return The width of the image
    +   */
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the number of channels in the image
    +   *
    +   * @return The number of channels in the image
    +   */
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the OpenCV representation as an int
    +   *
    +   * @return The OpenCV representation as an int
    +   */
    +  def getMode(row: Row): Int = row.getInt(4)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the image data
    +   *
    +   * @return The image data
    +   */
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * :: Experimental ::
    +   * Check if the DataFrame column contains images (i.e. has ImageSchema)
    +   *
    +   * @param df DataFrame
    +   * @param column Column name
    +   * @return True if the given column matches the image schema
    +   */
    +  def isImageColumn(df: DataFrame, column: String): Boolean =
    +    df.schema(column).dataType == columnSchema
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row =
    +    Row(Row(origin, -1, -1, -1, ocvTypes(undefinedImageType), Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in DataFrame Row
    +   *
    +   * @param origin Arbitrary string that identifies the image
    +   * @param bytes Image bytes (for example, jpeg)
    +   * @return DataFrame Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val isGray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val hasAlpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) = if (isGray) {
    +        (1, ocvTypes("CV_8UC1"))
    +      } else if (hasAlpha) {
    +        (4, ocvTypes("CV_8UC4"))
    +      } else {
    +        (3, ocvTypes("CV_8UC3"))
    +      }
    +
    +      val imageSize = height * width * nChannels
    +      assert(imageSize < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](imageSize)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (isGray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      } else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    +            if (nChannels == 4) {
    +              decoded(offset + 3) = color.getAlpha.toByte
    +            }
    +            offset += nChannels
    +          }
    +        }
    +      }
    +
    +      // the internal "Row" is needed, because the image is a single DataFrame column
    +      Some(Row(Row(origin, height, width, nChannels, mode, decoded)))
    +    }
    +  }
    +
    +  /**
    +   * :: Experimental ::
    +   * Read the directory of images from the local or remote source
    +   *
    +   * @param path Path to the image directory
    +   * @param sparkSession Spark Session
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/82913/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r143686017
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/HadoopUtils.scala ---
    @@ -0,0 +1,107 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import scala.language.existentials
    +import scala.util.Random
    +
    +import org.apache.commons.io.FilenameUtils
    +import org.apache.hadoop.conf.{Configuration, Configured}
    +import org.apache.hadoop.fs.{Path, PathFilter}
    +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
    +
    +import org.apache.spark.sql.SparkSession
    +
    +private object RecursiveFlag {
    +
    +  /** Sets a value of spark recursive flag
    +   *
    +   * @param value value to set
    +   * @param spark existing spark session
    +   * @return previous value of this flag
    +   */
    +  def setRecursiveFlag(value: Option[String], spark: SparkSession): Option[String] = {
    +    val flagName = FileInputFormat.INPUT_DIR_RECURSIVE
    +    val hadoopConf = spark.sparkContext.hadoopConfiguration
    +    val old = Option(hadoopConf.get(flagName))
    +
    +    value match {
    +      case Some(v) => hadoopConf.set(flagName, v)
    +      case None => hadoopConf.unset(flagName)
    +    }
    +
    +    old
    +  }
    +}
    +
    +/** Filter that allows loading a fraction of HDFS files. */
    +private class SamplePathFilter extends Configured with PathFilter {
    +  val random = {
    +    val rd = new Random()
    +    rd.setSeed(0)
    +    rd
    +  }
    +
    +  // Ratio of files to be read from disk
    +  var sampleRatio: Double = 1
    +
    +  override def setConf(conf: Configuration): Unit = {
    +    if (conf != null) {
    +      sampleRatio = conf.getDouble(SamplePathFilter.ratioParam, 1)
    +    }
    +  }
    +
    +  override def accept(path: Path): Boolean = {
    +    // Note: checking fileSystem.isDirectory is very slow here, so we use basic rules instead
    +    !SamplePathFilter.isFile(path) ||
    +      random.nextDouble() < sampleRatio
    +  }
    +}
    +
    +private object SamplePathFilter {
    +  val ratioParam = "sampleRatio"
    +
    +  def isFile(path: Path): Boolean = FilenameUtils.getExtension(path.toString) != ""
    +
    +  /** Set/unset  hdfs PathFilter
    +   *
    +   * @param value       Filter class that is passed to HDFS
    +   * @param sampleRatio Fraction of the files that the filter picks
    +   * @param spark       Existing Spark session
    +   * @return
    +   */
    +  def setPathFilter(value: Option[Class[_]], sampleRatio: Option[Double] = None,
    +                    spark: SparkSession)
    +  : Option[Class[_]] = {
    +    val flagName = FileInputFormat.PATHFILTER_CLASS
    +    val hadoopConf = spark.sparkContext.hadoopConfiguration
    +    val old = Option(hadoopConf.getClass(flagName, null))
    +    if (sampleRatio.isDefined) {
    +      hadoopConf.setDouble(SamplePathFilter.ratioParam, sampleRatio.get)
    +    } else {
    +      hadoopConf.unset(SamplePathFilter.ratioParam)
    +      None
    --- End diff --
    
    Looks like `None` is redundant.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r144855218
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/image/ImageSchemaSuite.scala ---
    @@ -0,0 +1,124 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.nio.file.Paths
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.ml.image.ImageSchema._
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.types._
    +
    +class ImageSchemaSuite extends SparkFunSuite with MLlibTestSparkContext {
    +  // Single column of images named "image"
    +  private val imageDFSchema =
    +    StructType(StructField("image", ImageSchema.columnSchema, true) :: Nil)
    +  private lazy val imagePath =
    +    Thread.currentThread().getContextClassLoader.getResource("test-data/images").getPath
    +
    +  test("Smoke test: create basic ImageSchema dataframe") {
    +    val origin = "path"
    +    val width = 1
    +    val height = 1
    +    val nChannels = 3
    +    val data = Array[Byte](0, 0, 0)
    +    val mode = "CV_8UC3"
    +
    +    // Internal Row corresponds to image StructType
    +    val rows = Seq(Row(Row(origin, height, width, nChannels, mode, data)),
    +      Row(Row(null, height, width, nChannels, mode, data)))
    +    val rdd = sc.makeRDD(rows)
    +    val df = spark.createDataFrame(rdd, imageDFSchema)
    +
    +    assert(df.count == 2, "incorrect image count")
    +    assert(ImageSchema.isImageColumn(df, "image"), "data do not fit ImageSchema")
    +  }
    +
    +  test("readImages count test") {
    +    var df = readImages(imagePath, recursive = false)
    +    assert(df.count == 0)
    +
    +    df = readImages(imagePath, recursive = true, dropImageFailures = false)
    +    assert(df.count == 8)
    +
    +    df = readImages(imagePath, recursive = true, dropImageFailures = true)
    +    val count100 = df.count
    +    assert(count100 == 7)
    +
    +    df = readImages(imagePath, recursive = true, sampleRatio = 0.5, dropImageFailures = true)
    +    // Random number about half of the size of the original dataset
    +    val count50 = df.count
    +    assert(count50 > 0.2 * count100 && count50 < 0.8 * count100)
    +  }
    +
    +  test("readImages partition test") {
    +    val df = readImages(imagePath, recursive = true, dropImageFailures = true, numPartitions = 3)
    +    assert(df.rdd.getNumPartitions == 3)
    +  }
    +
    +  // Images with the different number of channels
    +  test("readImages pixel values test") {
    +
    +    val images = readImages(imagePath + "/multi-channel/", recursive = false).collect
    +
    +    images.foreach{
    +      rrow => {
    +        val row = rrow.getAs[Row](0)
    +        val filename = Paths.get(getOrigin(row)).getFileName().toString()
    +        if(firstBytes20.contains(filename)) {
    +          val mode = getMode(row)
    +          val bytes20 = getData(row).slice(0, 20)
    +
    +          val expectedMode = firstBytes20(filename)._1
    +          val expectedBytes = firstBytes20(filename)._2
    +
    +          assert(expectedMode == mode, "mode of the image is not read correctly")
    +
    +          if (!compareBytes(expectedBytes, bytes20)) {
    +            throw new Exception("incorrect numeric value for flattened image")
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  // number of channels and first 20 bytes of OpenCV representation
    +  // - default representation for 3-channel RGB images is BGR row-wise:
    +  //   (B00, G00, R00,      B10, G10, R10,      ...)
    +  // - default representation for 4-channel RGB images is BGRA row-wise:
    +  //   (B00, G00, R00, A00, B10, G10, R10, A00, ...)
    +  private val firstBytes20 = Map(
    +    "grayscale.png" ->
    +      (("CV_8UC1", Array[Byte](0, 0, 0, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 1, 0, 0, 3, 5, 2, 1))),
    +    "RGB.png" -> (("CV_8UC3",
    +      Array[Byte](-34, -66, -98, -38, -69, -98, -62, -90, -117,
    +        -70, -98, -124, -34, -63, -90, -20, -48, -74, -18, -45))),
    +    "RGBA.png" -> (("CV_8UC4",
    +      Array[Byte](-128, -128, -8, -1, -128, -128, -8, -1, -128,
    +        -128, -8, -1, 127, 127, -9, -1, 127, 127, -9, -1)))
    +  )
    +
    +  private def compareBytes(x: Array[Byte], y: Array[Byte]): Boolean = {
    +    val length = Math.min(x.length, y.length)
    --- End diff --
    
    done, used Arrays.equals()


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r148695330
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,236 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields: Array[String] = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes: Map[String, Int] = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC3" -> 16, "CV_8UC4" -> 24
    +  )
    +
    +  /**
    +   * Used for conversion to python
    +   */
    +  val _ocvTypes: java.util.Map[String, Int] = ocvTypes.asJava
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField(imageFields(0), StringType, true) ::
    +    StructField(imageFields(1), IntegerType, false) ::
    +    StructField(imageFields(2), IntegerType, false) ::
    +    StructField(imageFields(3), IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(imageFields(4), IntegerType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(imageFields(5), BinaryType, false) :: Nil)
    +
    +  /**
    +   * DataFrame with a single column of images named "image" (nullable)
    +   */
    +  val imageSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the height of the image
    +   *
    +   * @return The height of the image
    +   */
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the width of the image
    +   *
    +   * @return The width of the image
    +   */
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the number of channels in the image
    +   *
    +   * @return The number of channels in the image
    +   */
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the OpenCV representation as an int
    +   *
    +   * @return The OpenCV representation as an int
    +   */
    +  def getMode(row: Row): Int = row.getInt(4)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the image data
    +   *
    +   * @return The image data
    +   */
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row =
    +    Row(Row(origin, -1, -1, -1, ocvTypes(undefinedImageType), Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in DataFrame Row
    +   *
    +   * @param origin Arbitrary string that identifies the image
    +   * @param bytes Image bytes (for example, jpeg)
    +   * @return DataFrame Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val isGray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val hasAlpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) = if (isGray) {
    +        (1, ocvTypes("CV_8UC1"))
    +      } else if (hasAlpha) {
    +        (4, ocvTypes("CV_8UC4"))
    +      } else {
    +        (3, ocvTypes("CV_8UC3"))
    +      }
    +
    +      val imageSize = height * width * nChannels
    +      assert(imageSize < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](imageSize)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (isGray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      } else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    +            if (nChannels == 4) {
    +              decoded(offset + 3) = color.getAlpha.toByte
    +            }
    +            offset += nChannels
    +          }
    +        }
    +      }
    +
    +      // the internal "Row" is needed, because the image is a single DataFrame column
    +      Some(Row(Row(origin, height, width, nChannels, mode, decoded)))
    +    }
    +  }
    +
    +  /**
    +   * :: Experimental ::
    +   * Read the directory of images from the local or remote source
    +   *
    +   * @param path Path to the image directory
    +   * @param sparkSession Spark Session, if omitted gets or creates the session
    +   * @param recursive Recursive path search flag
    +   * @param numPartitions Number of the DataFrame partitions,
    +   *                      if omitted uses defaultParallelism instead
    +   * @param dropImageFailures Drop the files that are not valid images from the result
    +   * @param sampleRatio Fraction of the files loaded
    +   * @return DataFrame with a single column "image" of images;
    +   *         see ImageSchema for the details
    +   */
    +  def readImages(
    +      path: String,
    +      sparkSession: SparkSession = null,
    +      recursive: Boolean = false,
    +      numPartitions: Int = 0,
    +      dropImageFailures: Boolean = false,
    +      sampleRatio: Double = 1.0): DataFrame = {
    +    require(sampleRatio <= 1.0 && sampleRatio >= 0, "sampleRatio should be between 0 and 1")
    +
    +    val session = if (sparkSession != null) sparkSession else SparkSession.builder().getOrCreate
    +    val partitions =
    +      if (numPartitions > 0) {
    +        numPartitions
    +      } else {
    +        session.sparkContext.defaultParallelism
    +      }
    +
    +    RecursiveFlag.withRecursiveFlag(recursive, session) {
    +      SamplePathFilter.withPathFilter(sampleRatio, session) {
    +        val streams = session.sparkContext.binaryFiles(path, partitions).repartition(partitions)
    --- End diff --
    
    (I think) this will repartition regardless of whether binaryFiles returns the correct number of partitions.  I feel like it'd be good to leave it up to the user to decide whether to do this expensive repartitioning.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r150449520
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -129,7 +129,7 @@ def toImage(self, array, origin=""):
     
             :param array array: The array to convert to image
             :param str origin: Path to the image, optional
    -        :rtype object: Two dimensional image
    +        :return: object: Two dimensional image
    --- End diff --
    
    ``:return: a :class:`Row` of blabla``


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r147004943
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,122 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +import pyspark
    +from pyspark import SparkContext
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame
    +from pyspark.ml.param.shared import *
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +imageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +ocvTypes = {
    +    undefinedImageType: -1,
    +    "CV_8U": 0, "CV_8UC1": 0, "CV_8UC2": 8, "CV_8UC3": 16, "CV_8UC4": 24,
    +    "CV_8S": 1, "CV_8SC1": 1, "CV_8SC2": 9, "CV_8SC3": 17, "CV_8SC4": 25,
    +    "CV_16U": 2, "CV_16UC1": 2, "CV_16UC2": 10, "CV_16UC3": 18, "CV_16UC4": 26,
    +    "CV_16S": 3, "CV_16SC1": 3, "CV_16SC2": 11, "CV_16SC3": 19, "CV_16SC4": 27,
    +    "CV_32S": 4, "CV_32SC1": 4, "CV_32SC2": 12, "CV_32SC3": 20, "CV_32SC4": 28,
    +    "CV_32F": 5, "CV_32FC1": 5, "CV_32FC2": 13, "CV_32FC3": 21, "CV_32FC4": 29,
    +    "CV_64F": 6, "CV_64FC1": 6, "CV_64FC2": 14, "CV_64FC3": 22, "CV_64FC4": 30
    +}
    +
    +# DataFrame with a single column of images named "image" (nullable)
    +imageSchema = StructType(StructField("image", StructType([
    +    StructField(imageFields[0], StringType(),  True),
    +    StructField(imageFields[1], IntegerType(), False),
    +    StructField(imageFields[2], IntegerType(), False),
    +    StructField(imageFields[3], IntegerType(), False),
    +    # OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(imageFields[4], StringType(), False),
    +    # bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(imageFields[5], BinaryType(), False)]), True))
    +
    +
    +def toNDArray(image):
    +    """
    +    Converts an image to a one-dimensional array.
    +
    +    :param image (object): The image to be converted
    +    :rtype array: The image as a one-dimensional array
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    height = image.height
    +    width = image.width
    +    nChannels = image.nChannels
    +    return np.ndarray(
    +        shape=(height, width, nChannels),
    +        dtype=np.uint8,
    +        buffer=image.data,
    +        strides=(width * nChannels, nChannels, 1))
    +
    +
    +def toImage(array, origin="", mode=ocvTypes["CV_8UC3"]):
    +    """
    +    Converts a one-dimensional array to a two-dimensional image.
    +
    +    :param array (array): The array to convert to image
    +    :param origin (str): Path to the image
    +    :param mode (str): OpenCV compatible type
    +
    +    :rtype object: Two dimensional image
    +
    +    .. versionadded:: 2.3.0
    +    """
    --- End diff --
    
    Yes, I think RGB ordering is correct, we just use the opencv ordering.  @dakirsa could give more details


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r148695893
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,192 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +"""
    +.. attribute:: ImageSchema
    +
    +    A singleton-like attribute of :class:`_ImageSchema` in this module.
    +
    +.. autoclass:: _ImageSchema
    +   :members:
    +"""
    +
    +from pyspark import SparkContext
    +from pyspark.sql.types import Row, _create_row, _parse_datatype_json_string
    +from pyspark.sql import DataFrame, SparkSession
    +import numpy as np
    --- End diff --
    
    style: order imports with standard python libraries imported before pyspark


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r150161698
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,236 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields: Array[String] = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes: Map[String, Int] = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC3" -> 16, "CV_8UC4" -> 24
    +  )
    +
    +  /**
    +   * Used for conversion to python
    +   */
    +  val _ocvTypes: java.util.Map[String, Int] = ocvTypes.asJava
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    --- End diff --
    
    good idea, done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Quick comment: I see that data are being added under mllib/src/test/resources/  That appears to be a new directory, created recently.  The standard directory is https://github.com/apache/spark/tree/master/data/mllib --- could you please put the images there instead?  I'll ping on the PR which introduced the new data directory to see about correcting it.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by MrBago <gi...@git.apache.org>.
Github user MrBago commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r144128955
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC2" -> 8, "CV_8UC3" -> 16, "CV_8UC4" -> 24,
    +    "CV_8S" -> 1, "CV_8SC1" -> 1, "CV_8SC2" -> 9, "CV_8SC3" -> 17, "CV_8SC4" -> 25,
    +    "CV_16U" -> 2, "CV_16UC1" -> 2, "CV_16UC2" -> 10, "CV_16UC3" -> 18, "CV_16UC4" -> 26,
    +    "CV_16S" -> 3, "CV_16SC1" -> 3, "CV_16SC2" -> 11, "CV_16SC3" -> 19, "CV_16SC4" -> 27,
    +    "CV_32S" -> 4, "CV_32SC1" -> 4, "CV_32SC2" -> 12, "CV_32SC3" -> 20, "CV_32SC4" -> 28,
    +    "CV_32F" -> 5, "CV_32FC1" -> 5, "CV_32FC2" -> 13, "CV_32FC3" -> 21, "CV_32FC4" -> 29,
    +    "CV_64F" -> 6, "CV_64FC1" -> 6, "CV_64FC2" -> 14, "CV_64FC3" -> 22, "CV_64FC4" -> 30
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField("origin", StringType, true) ::
    +      StructField("height", IntegerType, false) ::
    +      StructField("width", IntegerType, false) ::
    +      StructField("nChannels", IntegerType, false) ::
    +      // OpenCV-compatible type: CV_8UC3 in most cases
    +      StructField("mode", StringType, false) ::
    --- End diff --
    
    String is great, sorry for the noise.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r144193350
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    --- End diff --
    
    I believe getOrigin/getHeight/getWidth/getNChannels/getMode/getData are all convenient functions that users should be able to use - but I can change them to private if you prefer.  I've added better documentation to the methods.  isImage is also a convenient function for them to use to tell if the column is an image column.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r148541073
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,236 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields: Array[String] = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes: Map[String, Int] = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC3" -> 16, "CV_8UC4" -> 24
    +  )
    +
    +  /**
    +   * Used for conversion to python
    +   */
    +  val _ocvTypes: java.util.Map[String, Int] = ocvTypes.asJava
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField(imageFields(0), StringType, true) ::
    +    StructField(imageFields(1), IntegerType, false) ::
    +    StructField(imageFields(2), IntegerType, false) ::
    +    StructField(imageFields(3), IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(imageFields(4), IntegerType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(imageFields(5), BinaryType, false) :: Nil)
    +
    +  /**
    +   * DataFrame with a single column of images named "image" (nullable)
    +   */
    +  val imageSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the height of the image
    +   *
    +   * @return The height of the image
    +   */
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the width of the image
    +   *
    +   * @return The width of the image
    +   */
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the number of channels in the image
    +   *
    +   * @return The number of channels in the image
    +   */
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the OpenCV representation as an int
    +   *
    +   * @return The OpenCV representation as an int
    +   */
    +  def getMode(row: Row): Int = row.getInt(4)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the image data
    +   *
    +   * @return The image data
    +   */
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row =
    +    Row(Row(origin, -1, -1, -1, ocvTypes(undefinedImageType), Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in DataFrame Row
    +   *
    +   * @param origin Arbitrary string that identifies the image
    +   * @param bytes Image bytes (for example, jpeg)
    +   * @return DataFrame Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val isGray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val hasAlpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) = if (isGray) {
    +        (1, ocvTypes("CV_8UC1"))
    +      } else if (hasAlpha) {
    +        (4, ocvTypes("CV_8UC4"))
    +      } else {
    +        (3, ocvTypes("CV_8UC3"))
    +      }
    +
    +      val imageSize = height * width * nChannels
    +      assert(imageSize < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](imageSize)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (isGray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      } else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    +            if (nChannels == 4) {
    +              decoded(offset + 3) = color.getAlpha.toByte
    +            }
    +            offset += nChannels
    +          }
    +        }
    +      }
    +
    +      // the internal "Row" is needed, because the image is a single DataFrame column
    +      Some(Row(Row(origin, height, width, nChannels, mode, decoded)))
    +    }
    +  }
    +
    +  /**
    +   * :: Experimental ::
    +   * Read the directory of images from the local or remote source
    +   *
    +   * @param path Path to the image directory
    +   * @param sparkSession Spark Session, if omitted gets or creates the session
    +   * @param recursive Recursive path search flag
    +   * @param numPartitions Number of the DataFrame partitions,
    +   *                      if omitted uses defaultParallelism instead
    +   * @param dropImageFailures Drop the files that are not valid images from the result
    +   * @param sampleRatio Fraction of the files loaded
    +   * @return DataFrame with a single column "image" of images;
    +   *         see ImageSchema for the details
    +   */
    +  def readImages(
    +      path: String,
    +      sparkSession: SparkSession = null,
    --- End diff --
    
    in Java api the user can just specify all values, I think that would be better than creating a ton of overloaded functions, considering there are so many parameters.  Or do we not have a choice when it comes to the style guide - is it actually forbidden to have default parameters?  That would be strange since it is not baked into the style checker.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r143773560
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/HadoopUtils.scala ---
    @@ -0,0 +1,107 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import scala.language.existentials
    +import scala.util.Random
    +
    +import org.apache.commons.io.FilenameUtils
    +import org.apache.hadoop.conf.{Configuration, Configured}
    +import org.apache.hadoop.fs.{Path, PathFilter}
    +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
    +
    +import org.apache.spark.sql.SparkSession
    +
    +private object RecursiveFlag {
    +
    +  /** Sets a value of spark recursive flag
    +   *
    +   * @param value value to set
    +   * @param spark existing spark session
    +   * @return previous value of this flag
    +   */
    +  def setRecursiveFlag(value: Option[String], spark: SparkSession): Option[String] = {
    +    val flagName = FileInputFormat.INPUT_DIR_RECURSIVE
    +    val hadoopConf = spark.sparkContext.hadoopConfiguration
    +    val old = Option(hadoopConf.get(flagName))
    +
    +    value match {
    +      case Some(v) => hadoopConf.set(flagName, v)
    +      case None => hadoopConf.unset(flagName)
    +    }
    +
    +    old
    +  }
    +}
    +
    +/** Filter that allows loading a fraction of HDFS files. */
    +private class SamplePathFilter extends Configured with PathFilter {
    +  val random = {
    +    val rd = new Random()
    +    rd.setSeed(0)
    +    rd
    +  }
    +
    +  // Ratio of files to be read from disk
    +  var sampleRatio: Double = 1
    +
    +  override def setConf(conf: Configuration): Unit = {
    +    if (conf != null) {
    +      sampleRatio = conf.getDouble(SamplePathFilter.ratioParam, 1)
    +    }
    +  }
    +
    +  override def accept(path: Path): Boolean = {
    +    // Note: checking fileSystem.isDirectory is very slow here, so we use basic rules instead
    +    !SamplePathFilter.isFile(path) ||
    +      random.nextDouble() < sampleRatio
    +  }
    +}
    +
    +private object SamplePathFilter {
    +  val ratioParam = "sampleRatio"
    +
    +  def isFile(path: Path): Boolean = FilenameUtils.getExtension(path.toString) != ""
    +
    +  /** Set/unset  hdfs PathFilter
    +   *
    +   * @param value       Filter class that is passed to HDFS
    +   * @param sampleRatio Fraction of the files that the filter picks
    +   * @param spark       Existing Spark session
    +   * @return
    +   */
    +  def setPathFilter(value: Option[Class[_]], sampleRatio: Option[Double] = None,
    +                    spark: SparkSession)
    +  : Option[Class[_]] = {
    +    val flagName = FileInputFormat.PATHFILTER_CLASS
    +    val hadoopConf = spark.sparkContext.hadoopConfiguration
    +    val old = Option(hadoopConf.getClass(flagName, null))
    +    if (sampleRatio.isDefined) {
    +      hadoopConf.setDouble(SamplePathFilter.ratioParam, sampleRatio.get)
    +    } else {
    +      hadoopConf.unset(SamplePathFilter.ratioParam)
    +      None
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r145343991
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,124 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +import pyspark
    +from pyspark import SparkContext
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame
    +from pyspark.ml.param.shared import *
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +ImageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +ocvTypes = {
    +    undefinedImageType: -1,
    +    "CV_8U": 0, "CV_8UC1": 0, "CV_8UC2": 8, "CV_8UC3": 16, "CV_8UC4": 24,
    +    "CV_8S": 1, "CV_8SC1": 1, "CV_8SC2": 9, "CV_8SC3": 17, "CV_8SC4": 25,
    +    "CV_16U": 2, "CV_16UC1": 2, "CV_16UC2": 10, "CV_16UC3": 18, "CV_16UC4": 26,
    +    "CV_16S": 3, "CV_16SC1": 3, "CV_16SC2": 11, "CV_16SC3": 19, "CV_16SC4": 27,
    +    "CV_32S": 4, "CV_32SC1": 4, "CV_32SC2": 12, "CV_32SC3": 20, "CV_32SC4": 28,
    +    "CV_32F": 5, "CV_32FC1": 5, "CV_32FC2": 13, "CV_32FC3": 21, "CV_32FC4": 29,
    +    "CV_64F": 6, "CV_64FC1": 6, "CV_64FC2": 14, "CV_64FC3": 22, "CV_64FC4": 30
    +}
    +
    +# DataFrame with a single column of images named "image" (nullable)
    +imageSchema = StructType(StructField("image", StructType([
    +    StructField(ImageFields[0], StringType(),  True),
    +    StructField(ImageFields[1], IntegerType(), False),
    +    StructField(ImageFields[2], IntegerType(), False),
    +    StructField(ImageFields[3], IntegerType(), False),
    +    # OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(ImageFields[4], StringType(), False),
    +    # bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(ImageFields[5], BinaryType(), False)]), True))
    +
    +
    +# TODO: generalize to other datatypes and number of channels
    +def toNDArray(image):
    +    """
    +    Converts an image to a one-dimensional array
    +
    +    :param image (object): The image to be converted
    +    :rtype array: The image as a one-dimensional array
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    height = image.height
    +    width = image.width
    +    nChannels = image.nChannels
    +    return np.ndarray(
    +        shape=(height, width, nChannels),
    +        dtype=np.uint8,
    +        buffer=image.data,
    +        strides=(width * nChannels, nChannels, 1))
    +
    +
    +# TODO: generalize to other datatypes and number of channels
    +def toImage(array, origin="", mode=ocvTypes["CV_8UC3"]):
    +    """
    +
    --- End diff --
    
    Not a big deal but I'd remove this extra newline.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #83105 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83105/testReport)** for PR 19439 at commit [`a5f83c1`](https://github.com/apache/spark/commit/a5f83c171c0e3f077946a9b406311dc797424327).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by MrBago <gi...@git.apache.org>.
Github user MrBago commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r143799083
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,133 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +import pyspark
    +from pyspark import SparkContext
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame
    +from pyspark.ml.param.shared import *
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +ImageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +ocvTypes = {
    +    undefinedImageType: -1,
    +    "CV_8U": 0, "CV_8UC1": 0, "CV_8UC2": 8, "CV_8UC3": 16, "CV_8UC4": 24,
    +    "CV_8S": 1, "CV_8SC1": 1, "CV_8SC2": 9, "CV_8SC3": 17, "CV_8SC4": 25,
    +    "CV_16U": 2, "CV_16UC1": 2, "CV_16UC2": 10, "CV_16UC3": 18, "CV_16UC4": 26,
    +    "CV_16S": 3, "CV_16SC1": 3, "CV_16SC2": 11, "CV_16SC3": 19, "CV_16SC4": 27,
    +    "CV_32S": 4, "CV_32SC1": 4, "CV_32SC2": 12, "CV_32SC3": 20, "CV_32SC4": 28,
    +    "CV_32F": 5, "CV_32FC1": 5, "CV_32FC2": 13, "CV_32FC3": 21, "CV_32FC4": 29,
    +    "CV_64F": 6, "CV_64FC1": 6, "CV_64FC2": 14, "CV_64FC3": 22, "CV_64FC4": 30
    +}
    +
    +ImageSchema = StructType([
    +    StructField(ImageFields[0], StringType(),  True),
    +    StructField(ImageFields[1], IntegerType(), False),
    +    StructField(ImageFields[2], IntegerType(), False),
    +    StructField(ImageFields[3], IntegerType(), False),
    +    # OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(ImageFields[4], StringType(), False),
    +    # bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(ImageFields[5], BinaryType(), False)])
    +
    +
    +# TODO: generalize to other datatypes and number of channels
    +def toNDArray(image):
    +    """
    +    Converts an image to a 1-dimensional array
    +
    +    Args:
    +        image (object): The image to be converted
    +
    +    Returns:
    +        array: The image as a 1-dimensional array
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    height = image.height
    +    width = image.width
    +    return np.asarray(image.data, dtype=np.uint8) \
    +             .reshape((height, width, 3))[:, :, (2, 1, 0)]
    +
    +
    +# TODO: generalize to other datatypes and number of channels
    +def toImage(array, origin="", mode="CV_8UC3"):
    +    """
    +
    +    Converts a one-dimensional array to a 2 dimensional image
    +
    +    Args:
    +        array (array):
    +        origin (str):
    +        mode (int):
    +
    +    Returns:
    +        object: 2 dimensional image
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    length = np.prod(array.shape)
    +
    +    data = bytearray(array.astype(dtype=np.int8)[:, :, (2, 1, 0)]
    +                          .reshape(length))
    --- End diff --
    
    nit: do you mind using `dtype=np.uint8`? I know it's technically the same in this case but it's more consistent with how python thinks of bytes.
    
    Also `ndarray.ravel()` is shorthand for `ndarray.reshape(lenght)` :).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/82483/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Jenkins retest this please


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r148695824
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,236 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields: Array[String] = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes: Map[String, Int] = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC3" -> 16, "CV_8UC4" -> 24
    +  )
    +
    +  /**
    +   * Used for conversion to python
    +   */
    +  val _ocvTypes: java.util.Map[String, Int] = ocvTypes.asJava
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField(imageFields(0), StringType, true) ::
    +    StructField(imageFields(1), IntegerType, false) ::
    +    StructField(imageFields(2), IntegerType, false) ::
    +    StructField(imageFields(3), IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(imageFields(4), IntegerType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(imageFields(5), BinaryType, false) :: Nil)
    +
    +  /**
    +   * DataFrame with a single column of images named "image" (nullable)
    +   */
    +  val imageSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the height of the image
    +   *
    +   * @return The height of the image
    +   */
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  /**
    +   * :: Experimental ::
    --- End diff --
    
    If this whole class is marked Experimental, then it's OK not to mark each field/method Experimental.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r148541487
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,236 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields: Array[String] = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes: Map[String, Int] = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC3" -> 16, "CV_8UC4" -> 24
    +  )
    +
    +  /**
    +   * Used for conversion to python
    +   */
    +  val _ocvTypes: java.util.Map[String, Int] = ocvTypes.asJava
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField(imageFields(0), StringType, true) ::
    +    StructField(imageFields(1), IntegerType, false) ::
    +    StructField(imageFields(2), IntegerType, false) ::
    +    StructField(imageFields(3), IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(imageFields(4), IntegerType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(imageFields(5), BinaryType, false) :: Nil)
    +
    +  /**
    +   * DataFrame with a single column of images named "image" (nullable)
    +   */
    +  val imageSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    --- End diff --
    
    @HyukjinKwon  @dakirsa agree, it would be nice to add them to python API as well eventually


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r147698310
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,140 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +from pyspark.ml.util import *
    +from pyspark.ml.param.shared import *
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame, SparkSession, SQLContext
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +imageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +
    +def getOcvTypes(spark=None):
    +    """
    +    Returns the OpenCV type mapping supported
    +
    +    :param sparkSession (SparkSession): The current spark session
    +    :rtype dict: The OpenCV type mapping supported
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.ocvTypes
    +
    +
    +# DataFrame with a single column of images named "image" (nullable)
    +def getImageSchema(spark=None):
    +    """
    +    Returns the image schema
    +
    +    :param spark (SparkSession): The current spark session
    +    :rtype StructType: The image schema
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.imageSchema
    +
    +
    +def toNDArray(image):
    +    """
    +    Converts an image to a one-dimensional array.
    +
    +    :param image (object): The image to be converted
    +    :rtype array: The image as a one-dimensional array
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    height = image.height
    +    width = image.width
    +    nChannels = image.nChannels
    +    return np.ndarray(
    +        shape=(height, width, nChannels),
    +        dtype=np.uint8,
    +        buffer=image.data,
    +        strides=(width * nChannels, nChannels, 1))
    +
    +
    +def toImage(array, origin="", spark=None):
    +    """
    +    Converts a one-dimensional array to a two-dimensional image.
    +
    +    :param array (array): The array to convert to image
    +    :param origin (str): Path to the image
    +    :param mode (str): OpenCV compatible type
    +    :param spark (SparkSession): The current spark session
    +    :rtype object: Two dimensional image
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    if array.ndim != 3:
    +        raise
    +    height, width, nChannels = array.shape
    +    ocvTypes = getOcvTypes(spark)
    +    if nChannels == 1:
    +        mode = ocvTypes["CV_8UC1"]
    +    elif nChannels == 3:
    +        mode = ocvTypes["CV_8UC3"]
    +    elif nChannels == 4:
    +        mode = ocvTypes["CV_8UC4"]
    +    else:
    +        raise
    +    data = bytearray(array.astype(dtype=np.uint8).ravel())
    +    # Creating new Row with _create_row(), because Row(name = value, ... )
    +    # orders fields by name, which conflicts with expected schema order
    +    # when the new DataFrame is created by UDF
    +    return _create_row(imageFields,
    +                       [origin, height, width, nChannels, mode, data])
    +
    +
    +def readImages(path, recursive=False, numPartitions=0,
    +               dropImageFailures=False, sampleRatio=1.0, spark=None):
    --- End diff --
    
    Could we match the parameter's order to the scala one?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r147991609
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,139 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +from pyspark.ml.util import *
    +from pyspark.ml.param.shared import *
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame, SparkSession, SQLContext
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +imageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +
    +def getOcvTypes(spark=None):
    +    """
    +    Returns the OpenCV type mapping supported
    +
    +    :param sparkSession (SparkSession): The current spark session
    +    :rtype dict: The OpenCV type mapping supported
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.ocvTypes
    --- End diff --
    
    This looks a JVM object too:
    
    ```python
    >>> spark.sparkContext._jvm.org.apache.spark.ml.image.ImageSchema.ocvTypes()
    JavaObject id=o41
    >>> spark.sparkContext._jvm.org.apache.spark.ml.image.ImageSchema.ocvTypes().getClass().toString()
    u'class scala.collection.immutable.HashMap$HashTrieMap'
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #83277 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83277/testReport)** for PR 19439 at commit [`61290cc`](https://github.com/apache/spark/commit/61290ccfcb56ea6787e033330351b27df4219e48).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Jenkins retest this please


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #84100 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84100/testReport)** for PR 19439 at commit [`aa3c704`](https://github.com/apache/spark/commit/aa3c704918d4b5585ac00771741fd7e86548a215).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #83098 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83098/testReport)** for PR 19439 at commit [`838c4c0`](https://github.com/apache/spark/commit/838c4c026bcd610e604a966262476d723e563264).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r148028749
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,139 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +from pyspark.ml.util import *
    +from pyspark.ml.param.shared import *
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame, SparkSession, SQLContext
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +imageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +
    +def getOcvTypes(spark=None):
    +    """
    +    Returns the OpenCV type mapping supported
    +
    +    :param sparkSession (SparkSession): The current spark session
    +    :rtype dict: The OpenCV type mapping supported
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    +    ctx = spark.sparkContext
    +    return ctx._jvm.org.apache.spark.ml.image.ImageSchema.ocvTypes
    +
    +
    +# DataFrame with a single column of images named "image" (nullable)
    +def getImageSchema(spark=None):
    +    """
    +    Returns the image schema
    +
    +    :param spark (SparkSession): The current spark session
    +    :rtype StructType: The image schema
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    spark = spark or SparkSession.builder.getOrCreate()
    --- End diff --
    
    the tests fail otherwise - if there are multiple spark contexts during tests (when tests are run in parallel), it seems one of the other ones gets picked up, and when it is stopped the code fails with error stating that it cannot use closed spark context


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r144193584
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC2" -> 8, "CV_8UC3" -> 16, "CV_8UC4" -> 24,
    +    "CV_8S" -> 1, "CV_8SC1" -> 1, "CV_8SC2" -> 9, "CV_8SC3" -> 17, "CV_8SC4" -> 25,
    +    "CV_16U" -> 2, "CV_16UC1" -> 2, "CV_16UC2" -> 10, "CV_16UC3" -> 18, "CV_16UC4" -> 26,
    +    "CV_16S" -> 3, "CV_16SC1" -> 3, "CV_16SC2" -> 11, "CV_16SC3" -> 19, "CV_16SC4" -> 27,
    +    "CV_32S" -> 4, "CV_32SC1" -> 4, "CV_32SC2" -> 12, "CV_32SC3" -> 20, "CV_32SC4" -> 28,
    +    "CV_32F" -> 5, "CV_32FC1" -> 5, "CV_32FC2" -> 13, "CV_32FC3" -> 21, "CV_32FC4" -> 29,
    +    "CV_64F" -> 6, "CV_64FC1" -> 6, "CV_64FC2" -> 14, "CV_64FC3" -> 22, "CV_64FC4" -> 30
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField("origin", StringType, true) ::
    +      StructField("height", IntegerType, false) ::
    +      StructField("width", IntegerType, false) ::
    +      StructField("nChannels", IntegerType, false) ::
    +      // OpenCV-compatible type: CV_8UC3 in most cases
    +      StructField("mode", StringType, false) ::
    +      // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +      StructField("data", BinaryType, false) :: Nil)
    +
    +  // Dataframe with a single column of images named "image" (nullable)
    +  private val imageDFSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  @Since("2.3.0")
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  @Since("2.3.0")
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  @Since("2.3.0")
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  @Since("2.3.0")
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  @Since("2.3.0")
    +  def getMode(row: Row): String = row.getString(4)
    +
    +  @Since("2.3.0")
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * Check if the dataframe column contains images (i.e. has ImageSchema)
    +   *
    +   * @param df       Dataframe
    +   * @param column   Column name
    +   * @return         True if the given column matches the image schema
    +   */
    +  @Since("2.3.0")
    +  def isImageColumn(df: DataFrame, column: String): Boolean =
    +    df.schema(column).dataType == columnSchema
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return       Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row = Row(Row(origin, -1, -1, -1, undefinedImageType,
    +    Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in dataframe Row
    +   *
    +   * @param origin   Arbitrary string that identifies the image
    +   * @param bytes    Image bytes (for example, jpeg)
    +   * @return         Dataframe Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val is_gray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val has_alpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) = if (is_gray) {
    +        (1, "CV_8UC1")
    +      } else if (has_alpha) {
    +        (4, "CV_8UC4")
    +      } else {
    +        (3, "CV_8UC3")
    +      }
    +
    +      assert(height * width * nChannels < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](height*width*nChannels)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (is_gray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for(h <- 0 until height) {
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r144297458
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/HadoopUtils.scala ---
    @@ -0,0 +1,119 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import scala.language.existentials
    +import scala.util.Random
    +
    +import org.apache.commons.io.FilenameUtils
    +import org.apache.hadoop.conf.{Configuration, Configured}
    +import org.apache.hadoop.fs.{Path, PathFilter}
    +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
    +
    +import org.apache.spark.sql.SparkSession
    +
    +private object RecursiveFlag {
    +
    +  /**
    +   * Sets a value of spark recursive flag.
    +   * If value is a None, it unsets the flag.
    +   *
    +   * @param value value to set
    +   * @param spark existing spark session
    +   * @return previous value of this flag
    +   */
    +  def setRecursiveFlag(value: Option[String], spark: SparkSession): Option[String] = {
    +    val flagName = FileInputFormat.INPUT_DIR_RECURSIVE
    +    val hadoopConf = spark.sparkContext.hadoopConfiguration
    +    val old = Option(hadoopConf.get(flagName))
    +
    +    value match {
    +      case Some(v) => hadoopConf.set(flagName, v)
    +      case None => hadoopConf.unset(flagName)
    +    }
    +
    +    old
    +  }
    +}
    +
    +/**
    + * Filter that allows loading a fraction of HDFS files.
    + */
    +private class SamplePathFilter extends Configured with PathFilter {
    +  val random = new Random()
    +
    +  // Ratio of files to be read from disk
    +  var sampleRatio: Double = 1
    +
    +  override def setConf(conf: Configuration): Unit = {
    +    if (conf != null) {
    +      sampleRatio = conf.getDouble(SamplePathFilter.ratioParam, 1)
    +    }
    +  }
    +
    +  override def accept(path: Path): Boolean = {
    +    // Note: checking fileSystem.isDirectory is very slow here, so we use basic rules instead
    +    !SamplePathFilter.isFile(path) ||
    +      random.nextDouble() < sampleRatio
    --- End diff --
    
    I think we could make this inlined.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r144302523
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/image/ImageSchemaSuite.scala ---
    @@ -0,0 +1,124 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.nio.file.Paths
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.ml.image.ImageSchema._
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.types._
    +
    +class ImageSchemaSuite extends SparkFunSuite with MLlibTestSparkContext {
    +  // Single column of images named "image"
    +  private val imageDFSchema =
    +    StructType(StructField("image", ImageSchema.columnSchema, true) :: Nil)
    +  private lazy val imagePath =
    +    Thread.currentThread().getContextClassLoader.getResource("test-data/images").getPath
    +
    +  test("Smoke test: create basic ImageSchema dataframe") {
    +    val origin = "path"
    +    val width = 1
    +    val height = 1
    +    val nChannels = 3
    +    val data = Array[Byte](0, 0, 0)
    +    val mode = "CV_8UC3"
    +
    +    // Internal Row corresponds to image StructType
    +    val rows = Seq(Row(Row(origin, height, width, nChannels, mode, data)),
    +      Row(Row(null, height, width, nChannels, mode, data)))
    +    val rdd = sc.makeRDD(rows)
    +    val df = spark.createDataFrame(rdd, imageDFSchema)
    +
    +    assert(df.count == 2, "incorrect image count")
    +    assert(ImageSchema.isImageColumn(df, "image"), "data do not fit ImageSchema")
    +  }
    +
    +  test("readImages count test") {
    +    var df = readImages(imagePath, recursive = false)
    +    assert(df.count == 0)
    +
    +    df = readImages(imagePath, recursive = true, dropImageFailures = false)
    +    assert(df.count == 8)
    +
    +    df = readImages(imagePath, recursive = true, dropImageFailures = true)
    +    val count100 = df.count
    +    assert(count100 == 7)
    +
    +    df = readImages(imagePath, recursive = true, sampleRatio = 0.5, dropImageFailures = true)
    +    // Random number about half of the size of the original dataset
    +    val count50 = df.count
    +    assert(count50 > 0.2 * count100 && count50 < 0.8 * count100)
    +  }
    +
    +  test("readImages partition test") {
    +    val df = readImages(imagePath, recursive = true, dropImageFailures = true, numPartitions = 3)
    +    assert(df.rdd.getNumPartitions == 3)
    +  }
    +
    +  // Images with the different number of channels
    +  test("readImages pixel values test") {
    +
    +    val images = readImages(imagePath + "/multi-channel/", recursive = false).collect
    +
    +    images.foreach{
    +      rrow => {
    +        val row = rrow.getAs[Row](0)
    +        val filename = Paths.get(getOrigin(row)).getFileName().toString()
    +        if(firstBytes20.contains(filename)) {
    --- End diff --
    
    `if(` -> `if (`.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r145524681
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,255 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC2" -> 8, "CV_8UC3" -> 16, "CV_8UC4" -> 24,
    +    "CV_8S" -> 1, "CV_8SC1" -> 1, "CV_8SC2" -> 9, "CV_8SC3" -> 17, "CV_8SC4" -> 25,
    +    "CV_16U" -> 2, "CV_16UC1" -> 2, "CV_16UC2" -> 10, "CV_16UC3" -> 18, "CV_16UC4" -> 26,
    +    "CV_16S" -> 3, "CV_16SC1" -> 3, "CV_16SC2" -> 11, "CV_16SC3" -> 19, "CV_16SC4" -> 27,
    +    "CV_32S" -> 4, "CV_32SC1" -> 4, "CV_32SC2" -> 12, "CV_32SC3" -> 20, "CV_32SC4" -> 28,
    +    "CV_32F" -> 5, "CV_32FC1" -> 5, "CV_32FC2" -> 13, "CV_32FC3" -> 21, "CV_32FC4" -> 29,
    +    "CV_64F" -> 6, "CV_64FC1" -> 6, "CV_64FC2" -> 14, "CV_64FC3" -> 22, "CV_64FC4" -> 30
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Array[Byte])
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Jenkins retest this please


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/83274/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/82851/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #82851 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82851/testReport)** for PR 19439 at commit [`dcc0d12`](https://github.com/apache/spark/commit/dcc0d12c3dce0dffb49d4f6b4f7a9e31f80b9b9e).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #83224 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83224/testReport)** for PR 19439 at commit [`e314327`](https://github.com/apache/spark/commit/e314327dd74c0092194c311a531c8a8bb90fdb86).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r144878549
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,129 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +import pyspark
    +from pyspark import SparkContext
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame
    +from pyspark.ml.param.shared import *
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +ImageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +ocvTypes = {
    +    undefinedImageType: -1,
    +    "CV_8U": 0, "CV_8UC1": 0, "CV_8UC2": 8, "CV_8UC3": 16, "CV_8UC4": 24,
    +    "CV_8S": 1, "CV_8SC1": 1, "CV_8SC2": 9, "CV_8SC3": 17, "CV_8SC4": 25,
    +    "CV_16U": 2, "CV_16UC1": 2, "CV_16UC2": 10, "CV_16UC3": 18, "CV_16UC4": 26,
    +    "CV_16S": 3, "CV_16SC1": 3, "CV_16SC2": 11, "CV_16SC3": 19, "CV_16SC4": 27,
    +    "CV_32S": 4, "CV_32SC1": 4, "CV_32SC2": 12, "CV_32SC3": 20, "CV_32SC4": 28,
    +    "CV_32F": 5, "CV_32FC1": 5, "CV_32FC2": 13, "CV_32FC3": 21, "CV_32FC4": 29,
    +    "CV_64F": 6, "CV_64FC1": 6, "CV_64FC2": 14, "CV_64FC3": 22, "CV_64FC4": 30
    +}
    +
    +ImageSchema = StructType([
    --- End diff --
    
    yep, we want to expose them.  What specifically doesn't match in scala?  There is an ocvTypes in scala just like in python that we expose to the user.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r147695860
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,252 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC3" -> 16, "CV_8UC4" -> 24
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField(imageFields(0), StringType, true) ::
    +    StructField(imageFields(1), IntegerType, false) ::
    +    StructField(imageFields(2), IntegerType, false) ::
    +    StructField(imageFields(3), IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(imageFields(4), IntegerType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(imageFields(5), BinaryType, false) :: Nil)
    +
    +  /**
    +   * DataFrame with a single column of images named "image" (nullable)
    +   */
    +  val imageSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the height of the image
    +   *
    +   * @return The height of the image
    +   */
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the width of the image
    +   *
    +   * @return The width of the image
    +   */
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the number of channels in the image
    +   *
    +   * @return The number of channels in the image
    +   */
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the OpenCV representation as an int
    +   *
    +   * @return The OpenCV representation as an int
    +   */
    +  def getMode(row: Row): Int = row.getInt(4)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the image data
    +   *
    +   * @return The image data
    +   */
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * :: Experimental ::
    +   * Check if the DataFrame column contains images (i.e. has ImageSchema)
    +   *
    +   * @param df DataFrame
    +   * @param column Column name
    +   * @return True if the given column matches the image schema
    +   */
    +  def isImageColumn(df: DataFrame, column: String): Boolean =
    +    df.schema(column).dataType == columnSchema
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row =
    +    Row(Row(origin, -1, -1, -1, ocvTypes(undefinedImageType), Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in DataFrame Row
    +   *
    +   * @param origin Arbitrary string that identifies the image
    +   * @param bytes Image bytes (for example, jpeg)
    +   * @return DataFrame Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val isGray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val hasAlpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) = if (isGray) {
    +        (1, ocvTypes("CV_8UC1"))
    +      } else if (hasAlpha) {
    +        (4, ocvTypes("CV_8UC4"))
    +      } else {
    +        (3, ocvTypes("CV_8UC3"))
    +      }
    +
    +      val imageSize = height * width * nChannels
    +      assert(imageSize < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](imageSize)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (isGray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      } else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    +            if (nChannels == 4) {
    +              decoded(offset + 3) = color.getAlpha.toByte
    +            }
    +            offset += nChannels
    +          }
    +        }
    +      }
    +
    +      // the internal "Row" is needed, because the image is a single DataFrame column
    +      Some(Row(Row(origin, height, width, nChannels, mode, decoded)))
    +    }
    +  }
    +
    +  /**
    +   * :: Experimental ::
    +   * Read the directory of images from the local or remote source
    +   *
    +   * @param path Path to the image directory
    +   * @param sparkSession Spark Session
    +   * @param recursive Recursive path search flag
    +   * @param numPartitions Number of the DataFrame partitions
    +   * @param dropImageFailures Drop the files that are not valid images from the result
    +   * @param sampleRatio Fraction of the files loaded
    +   * @return DataFrame with a single column "image" of images;
    +   *         see ImageSchema for the details
    +   */
    +  def readImages(
    +      path: String,
    +      sparkSession: SparkSession = null,
    +      recursive: Boolean = false,
    +      numPartitions: Int = 0,
    +      dropImageFailures: Boolean = false,
    +      sampleRatio: Double = 1.0): DataFrame = {
    +    require(sampleRatio <= 1.0 && sampleRatio >= 0, "sampleRatio should be between 0 and 1")
    +
    +    val session = if (sparkSession != null) sparkSession else SparkSession.builder().getOrCreate
    +    val partitions =
    +      if (numPartitions > 0) {
    +        numPartitions
    +      } else {
    +        session.sparkContext.defaultParallelism
    +      }
    +
    +    val oldRecursiveFlag = RecursiveFlag.setRecursiveFlag(Some(recursive.toString), session)
    +    val sampleImages = sampleRatio < 1
    +    val oldPathFilter: Option[Class[_]] =
    +      if (sampleImages) {
    +        SamplePathFilter.setPathFilter(Some(classOf[SamplePathFilter]), sampleRatio, session)
    +      } else {
    +        None
    +      }
    +
    +    try {
    +      val streams = session.sparkContext.binaryFiles(path, partitions)
    +        .repartition(partitions)
    +
    +      val convert = (stream: (String, PortableDataStream)) =>
    --- End diff --
    
    How about ... :
    
    ```scala
    val images = if (dropImageFailures) {
      streams.flatMap { case (origin, bytes) => 
        decode(origin, bytes.toArray())
      }
    } else {
      streams.map { case (origin, bytes) =>
        // Sets the value for the invalid image if the files that are not valid.
        decode(origin, bytes.toArray()).getOrElse(invalidImageRow(origin)) 
      }
    }
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #83672 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83672/testReport)** for PR 19439 at commit [`04db0fd`](https://github.com/apache/spark/commit/04db0fd02ee1abacc65d20c8d12eab8b6539e09f).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r150163944
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/image/ImageSchemaSuite.scala ---
    @@ -0,0 +1,108 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.nio.file.Paths
    +import java.util.Arrays
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.ml.image.ImageSchema._
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.types._
    +
    +class ImageSchemaSuite extends SparkFunSuite with MLlibTestSparkContext {
    +  // Single column of images named "image"
    +  private lazy val imagePath = "../data/mllib/images"
    +
    +  test("Smoke test: create basic ImageSchema dataframe") {
    +    val origin = "path"
    +    val width = 1
    +    val height = 1
    +    val nChannels = 3
    +    val data = Array[Byte](0, 0, 0)
    +    val mode = ocvTypes("CV_8UC3")
    +
    +    // Internal Row corresponds to image StructType
    +    val rows = Seq(Row(Row(origin, height, width, nChannels, mode, data)),
    +      Row(Row(null, height, width, nChannels, mode, data)))
    +    val rdd = sc.makeRDD(rows)
    +    val df = spark.createDataFrame(rdd, ImageSchema.imageSchema)
    +
    +    assert(df.count === 2, "incorrect image count")
    +    assert(df.schema("image").dataType == columnSchema, "data do not fit ImageSchema")
    +  }
    +
    +  test("readImages count test") {
    +    var df = readImages(imagePath, recursive = false)
    +    assert(df.count === 1)
    +
    +    df = readImages(imagePath, recursive = true, dropImageFailures = false)
    +    assert(df.count === 9)
    +
    +    df = readImages(imagePath, recursive = true, dropImageFailures = true)
    +    val countTotal = df.count
    +    assert(countTotal === 7)
    +
    +    df = readImages(imagePath, recursive = true, sampleRatio = 0.5, dropImageFailures = true)
    --- End diff --
    
    agreed +1


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/83679/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/83303/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by MrBago <gi...@git.apache.org>.
Github user MrBago commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r144365856
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,256 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC2" -> 8, "CV_8UC3" -> 16, "CV_8UC4" -> 24,
    +    "CV_8S" -> 1, "CV_8SC1" -> 1, "CV_8SC2" -> 9, "CV_8SC3" -> 17, "CV_8SC4" -> 25,
    +    "CV_16U" -> 2, "CV_16UC1" -> 2, "CV_16UC2" -> 10, "CV_16UC3" -> 18, "CV_16UC4" -> 26,
    +    "CV_16S" -> 3, "CV_16SC1" -> 3, "CV_16SC2" -> 11, "CV_16SC3" -> 19, "CV_16SC4" -> 27,
    +    "CV_32S" -> 4, "CV_32SC1" -> 4, "CV_32SC2" -> 12, "CV_32SC3" -> 20, "CV_32SC4" -> 28,
    +    "CV_32F" -> 5, "CV_32FC1" -> 5, "CV_32FC2" -> 13, "CV_32FC3" -> 21, "CV_32FC4" -> 29,
    +    "CV_64F" -> 6, "CV_64FC1" -> 6, "CV_64FC2" -> 14, "CV_64FC3" -> 22, "CV_64FC4" -> 30
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField("origin", StringType, true) ::
    +    StructField("height", IntegerType, false) ::
    +    StructField("width", IntegerType, false) ::
    +    StructField("nChannels", IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField("mode", StringType, false) ::
    --- End diff --
    
    After some more thought and conversation I actually think we should use an IntegerType here. There is one issue I had not noticed before that I think could be an issue down the road. The openCV string representation for some types is not unique, eg "CV_16U" and "CV_16SC1" map to type 2 (1 channel, 16 bit, unsigned). Having more than one identifier for each type is a potential minefield I think we should avoid.
    
    Alternatively I think we could stick to using strings if we restrict the supported types and pick only one representation to be valid when there are duplicates.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/83124/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r143686181
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/HadoopUtils.scala ---
    @@ -0,0 +1,107 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import scala.language.existentials
    +import scala.util.Random
    +
    +import org.apache.commons.io.FilenameUtils
    +import org.apache.hadoop.conf.{Configuration, Configured}
    +import org.apache.hadoop.fs.{Path, PathFilter}
    +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
    +
    +import org.apache.spark.sql.SparkSession
    +
    +private object RecursiveFlag {
    +
    +  /** Sets a value of spark recursive flag
    +   *
    +   * @param value value to set
    +   * @param spark existing spark session
    +   * @return previous value of this flag
    +   */
    +  def setRecursiveFlag(value: Option[String], spark: SparkSession): Option[String] = {
    +    val flagName = FileInputFormat.INPUT_DIR_RECURSIVE
    +    val hadoopConf = spark.sparkContext.hadoopConfiguration
    +    val old = Option(hadoopConf.get(flagName))
    +
    +    value match {
    +      case Some(v) => hadoopConf.set(flagName, v)
    +      case None => hadoopConf.unset(flagName)
    +    }
    +
    +    old
    +  }
    +}
    +
    +/** Filter that allows loading a fraction of HDFS files. */
    +private class SamplePathFilter extends Configured with PathFilter {
    +  val random = {
    +    val rd = new Random()
    +    rd.setSeed(0)
    +    rd
    +  }
    +
    +  // Ratio of files to be read from disk
    +  var sampleRatio: Double = 1
    +
    +  override def setConf(conf: Configuration): Unit = {
    +    if (conf != null) {
    +      sampleRatio = conf.getDouble(SamplePathFilter.ratioParam, 1)
    +    }
    +  }
    +
    +  override def accept(path: Path): Boolean = {
    +    // Note: checking fileSystem.isDirectory is very slow here, so we use basic rules instead
    +    !SamplePathFilter.isFile(path) ||
    +      random.nextDouble() < sampleRatio
    +  }
    +}
    +
    +private object SamplePathFilter {
    +  val ratioParam = "sampleRatio"
    +
    +  def isFile(path: Path): Boolean = FilenameUtils.getExtension(path.toString) != ""
    +
    +  /** Set/unset  hdfs PathFilter
    +   *
    +   * @param value       Filter class that is passed to HDFS
    +   * @param sampleRatio Fraction of the files that the filter picks
    +   * @param spark       Existing Spark session
    +   * @return
    +   */
    +  def setPathFilter(value: Option[Class[_]], sampleRatio: Option[Double] = None,
    --- End diff --
    
    Shall we have both set and unset methods instead of one method to do both set/unset?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #82913 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82913/testReport)** for PR 19439 at commit [`3759d46`](https://github.com/apache/spark/commit/3759d46135db096c5d0d225a1bc6f63dec1c0c6f).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r147001738
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,122 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +import pyspark
    +from pyspark import SparkContext
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame
    +from pyspark.ml.param.shared import *
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +imageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +ocvTypes = {
    +    undefinedImageType: -1,
    +    "CV_8U": 0, "CV_8UC1": 0, "CV_8UC2": 8, "CV_8UC3": 16, "CV_8UC4": 24,
    +    "CV_8S": 1, "CV_8SC1": 1, "CV_8SC2": 9, "CV_8SC3": 17, "CV_8SC4": 25,
    +    "CV_16U": 2, "CV_16UC1": 2, "CV_16UC2": 10, "CV_16UC3": 18, "CV_16UC4": 26,
    +    "CV_16S": 3, "CV_16SC1": 3, "CV_16SC2": 11, "CV_16SC3": 19, "CV_16SC4": 27,
    +    "CV_32S": 4, "CV_32SC1": 4, "CV_32SC2": 12, "CV_32SC3": 20, "CV_32SC4": 28,
    +    "CV_32F": 5, "CV_32FC1": 5, "CV_32FC2": 13, "CV_32FC3": 21, "CV_32FC4": 29,
    +    "CV_64F": 6, "CV_64FC1": 6, "CV_64FC2": 14, "CV_64FC3": 22, "CV_64FC4": 30
    +}
    +
    +# DataFrame with a single column of images named "image" (nullable)
    +imageSchema = StructType(StructField("image", StructType([
    +    StructField(imageFields[0], StringType(),  True),
    +    StructField(imageFields[1], IntegerType(), False),
    +    StructField(imageFields[2], IntegerType(), False),
    +    StructField(imageFields[3], IntegerType(), False),
    +    # OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(imageFields[4], StringType(), False),
    +    # bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(imageFields[5], BinaryType(), False)]), True))
    +
    +
    +def toNDArray(image):
    +    """
    +    Converts an image to a one-dimensional array.
    +
    +    :param image (object): The image to be converted
    +    :rtype array: The image as a one-dimensional array
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    height = image.height
    +    width = image.width
    +    nChannels = image.nChannels
    +    return np.ndarray(
    +        shape=(height, width, nChannels),
    +        dtype=np.uint8,
    --- End diff --
    
    in the scala code we only support (in ImageSchema.scala):
    
    if (isGray) {
            (1, ocvTypes("CV_8UC1"))
          } else if (hasAlpha) {
            (4, ocvTypes("CV_8UC4"))
          } else {
            (3, ocvTypes("CV_8UC3"))
          }
    
    maybe I should just limit the ocvTypes to just those 3 for now.
    Then you won't need to worry about floating point at all.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r144189582
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC2" -> 8, "CV_8UC3" -> 16, "CV_8UC4" -> 24,
    +    "CV_8S" -> 1, "CV_8SC1" -> 1, "CV_8SC2" -> 9, "CV_8SC3" -> 17, "CV_8SC4" -> 25,
    +    "CV_16U" -> 2, "CV_16UC1" -> 2, "CV_16UC2" -> 10, "CV_16UC3" -> 18, "CV_16UC4" -> 26,
    +    "CV_16S" -> 3, "CV_16SC1" -> 3, "CV_16SC2" -> 11, "CV_16SC3" -> 19, "CV_16SC4" -> 27,
    +    "CV_32S" -> 4, "CV_32SC1" -> 4, "CV_32SC2" -> 12, "CV_32SC3" -> 20, "CV_32SC4" -> 28,
    +    "CV_32F" -> 5, "CV_32FC1" -> 5, "CV_32FC2" -> 13, "CV_32FC3" -> 21, "CV_32FC4" -> 29,
    +    "CV_64F" -> 6, "CV_64FC1" -> 6, "CV_64FC2" -> 14, "CV_64FC3" -> 22, "CV_64FC4" -> 30
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField("origin", StringType, true) ::
    +      StructField("height", IntegerType, false) ::
    --- End diff --
    
    It is strange, it seems my intellij prefers this default but I can't tell why.  I've changed it to two spaces for all of them.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/83273/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r145344124
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,124 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +import pyspark
    +from pyspark import SparkContext
    +from pyspark.sql.types import *
    +from pyspark.sql.types import Row, _create_row
    +from pyspark.sql import DataFrame
    +from pyspark.ml.param.shared import *
    +import numpy as np
    +
    +undefinedImageType = "Undefined"
    +
    +ImageFields = ["origin", "height", "width", "nChannels", "mode", "data"]
    +
    +ocvTypes = {
    +    undefinedImageType: -1,
    +    "CV_8U": 0, "CV_8UC1": 0, "CV_8UC2": 8, "CV_8UC3": 16, "CV_8UC4": 24,
    +    "CV_8S": 1, "CV_8SC1": 1, "CV_8SC2": 9, "CV_8SC3": 17, "CV_8SC4": 25,
    +    "CV_16U": 2, "CV_16UC1": 2, "CV_16UC2": 10, "CV_16UC3": 18, "CV_16UC4": 26,
    +    "CV_16S": 3, "CV_16SC1": 3, "CV_16SC2": 11, "CV_16SC3": 19, "CV_16SC4": 27,
    +    "CV_32S": 4, "CV_32SC1": 4, "CV_32SC2": 12, "CV_32SC3": 20, "CV_32SC4": 28,
    +    "CV_32F": 5, "CV_32FC1": 5, "CV_32FC2": 13, "CV_32FC3": 21, "CV_32FC4": 29,
    +    "CV_64F": 6, "CV_64FC1": 6, "CV_64FC2": 14, "CV_64FC3": 22, "CV_64FC4": 30
    +}
    +
    +# DataFrame with a single column of images named "image" (nullable)
    +imageSchema = StructType(StructField("image", StructType([
    +    StructField(ImageFields[0], StringType(),  True),
    +    StructField(ImageFields[1], IntegerType(), False),
    +    StructField(ImageFields[2], IntegerType(), False),
    +    StructField(ImageFields[3], IntegerType(), False),
    +    # OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(ImageFields[4], StringType(), False),
    +    # bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(ImageFields[5], BinaryType(), False)]), True))
    +
    +
    +# TODO: generalize to other datatypes and number of channels
    +def toNDArray(image):
    +    """
    +    Converts an image to a one-dimensional array
    +
    +    :param image (object): The image to be converted
    +    :rtype array: The image as a one-dimensional array
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    height = image.height
    +    width = image.width
    +    nChannels = image.nChannels
    +    return np.ndarray(
    +        shape=(height, width, nChannels),
    +        dtype=np.uint8,
    +        buffer=image.data,
    +        strides=(width * nChannels, nChannels, 1))
    +
    +
    +# TODO: generalize to other datatypes and number of channels
    +def toImage(array, origin="", mode=ocvTypes["CV_8UC3"]):
    +    """
    +
    +    Converts a one-dimensional array to a two-dimensional image
    +
    +    :param array (array): The array to convert to image
    +    :param origin (str): Path to the image
    +    :param mode (str): OpenCV compatible type
    +
    +    :rtype object: Two dimensional image
    +
    +    .. versionadded:: 2.3.0
    +    """
    +    data = bytearray(array.astype(dtype=np.uint8).ravel())
    +    height = array.shape[0]
    +    width = array.shape[1]
    +    nChannels = array.shape[2]
    +    # Creating new Row with _create_row(), because Row(name = value, ... )
    +    # orders fields by name, which conflicts with expected schema order
    +    # when the new DataFrame is created by UDF
    +    return _create_row(ImageFields,
    +                       [origin, height, width, nChannels, mode, data])
    +
    +
    +def readImages(path, recursive=False, numPartitions=0,
    +               dropImageFailures=False, sampleRatio=1.0):
    +    """
    +    Reads the directory of images from the local or remote source.
    --- End diff --
    
    tiny nit: I'd add a newline between this description and `:param`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r143688286
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,217 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.Since
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC2" -> 8, "CV_8UC3" -> 16, "CV_8UC4" -> 24,
    +    "CV_8S" -> 1, "CV_8SC1" -> 1, "CV_8SC2" -> 9, "CV_8SC3" -> 17, "CV_8SC4" -> 25,
    +    "CV_16U" -> 2, "CV_16UC1" -> 2, "CV_16UC2" -> 10, "CV_16UC3" -> 18, "CV_16UC4" -> 26,
    +    "CV_16S" -> 3, "CV_16SC1" -> 3, "CV_16SC2" -> 11, "CV_16SC3" -> 19, "CV_16SC4" -> 27,
    +    "CV_32S" -> 4, "CV_32SC1" -> 4, "CV_32SC2" -> 12, "CV_32SC3" -> 20, "CV_32SC4" -> 28,
    +    "CV_32F" -> 5, "CV_32FC1" -> 5, "CV_32FC2" -> 13, "CV_32FC3" -> 21, "CV_32FC4" -> 29,
    +    "CV_64F" -> 6, "CV_64FC1" -> 6, "CV_64FC2" -> 14, "CV_64FC3" -> 22, "CV_64FC4" -> 30
    +  )
    +
    +  /** Schema for the image column: Row(String, Int, Int, Int, Array[Byte]) */
    +  val columnSchema = StructType(
    +    StructField("origin", StringType, true) ::
    +      StructField("height", IntegerType, false) ::
    +      StructField("width", IntegerType, false) ::
    +      StructField("nChannels", IntegerType, false) ::
    +      // OpenCV-compatible type: CV_8UC3 in most cases
    +      StructField("mode", StringType, false) ::
    +      // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +      StructField("data", BinaryType, false) :: Nil)
    +
    +  // Dataframe with a single column of images named "image" (nullable)
    +  private val imageDFSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  @Since("2.3.0")
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  @Since("2.3.0")
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  @Since("2.3.0")
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  @Since("2.3.0")
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  @Since("2.3.0")
    +  def getMode(row: Row): String = row.getString(4)
    +
    +  @Since("2.3.0")
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /** Check if the dataframe column contains images (i.e. has ImageSchema)
    +   *
    +   * @param df       Dataframe
    +   * @param column   Column name
    +   * @return         True if the given column matches the image schema
    +   */
    +  @Since("2.3.0")
    +  def isImageColumn(df: DataFrame, column: String): Boolean =
    +    df.schema(column).dataType == columnSchema
    +
    +  /** Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return       Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row = Row(Row(origin, -1, -1, -1, undefinedImageType,
    +    Array.ofDim[Byte](0)))
    +
    +  /** Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in dataframe Row
    +   *
    +   * @param origin   Arbitrary string that identifies the image
    +   * @param bytes    Image bytes (for example, jpeg)
    +   * @return         Dataframe Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val is_gray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val has_alpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) =
    +        if (is_gray) (1, "CV_8UC1")
    +        else if (has_alpha) (4, "CV_8UC4")
    +        else (3, "CV_8UC3")
    --- End diff --
    
    Recommended and usual style:
    
    ```scala
    val (nChannels, mode) = if (is_gray) {
      (1, "CV_8UC1")
    } else if (has_alpha) {
      (4, "CV_8UC4")
    } else {
      (3, "CV_8UC3")
    }
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r150242380
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,236 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields: Array[String] = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes: Map[String, Int] = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC3" -> 16, "CV_8UC4" -> 24
    +  )
    +
    +  /**
    +   * Used for conversion to python
    +   */
    +  val _ocvTypes: java.util.Map[String, Int] = ocvTypes.asJava
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField(imageFields(0), StringType, true) ::
    +    StructField(imageFields(1), IntegerType, false) ::
    +    StructField(imageFields(2), IntegerType, false) ::
    +    StructField(imageFields(3), IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(imageFields(4), IntegerType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(imageFields(5), BinaryType, false) :: Nil)
    +
    +  /**
    +   * DataFrame with a single column of images named "image" (nullable)
    +   */
    +  val imageSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the height of the image
    +   *
    +   * @return The height of the image
    +   */
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the width of the image
    +   *
    +   * @return The width of the image
    +   */
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the number of channels in the image
    +   *
    +   * @return The number of channels in the image
    +   */
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the OpenCV representation as an int
    +   *
    +   * @return The OpenCV representation as an int
    +   */
    +  def getMode(row: Row): Int = row.getInt(4)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the image data
    +   *
    +   * @return The image data
    +   */
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row =
    +    Row(Row(origin, -1, -1, -1, ocvTypes(undefinedImageType), Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in DataFrame Row
    +   *
    +   * @param origin Arbitrary string that identifies the image
    +   * @param bytes Image bytes (for example, jpeg)
    +   * @return DataFrame Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val isGray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val hasAlpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) = if (isGray) {
    +        (1, ocvTypes("CV_8UC1"))
    +      } else if (hasAlpha) {
    +        (4, ocvTypes("CV_8UC4"))
    +      } else {
    +        (3, ocvTypes("CV_8UC3"))
    +      }
    +
    +      val imageSize = height * width * nChannels
    +      assert(imageSize < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](imageSize)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (isGray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      } else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    +            if (nChannels == 4) {
    +              decoded(offset + 3) = color.getAlpha.toByte
    +            }
    +            offset += nChannels
    +          }
    +        }
    +      }
    +
    +      // the internal "Row" is needed, because the image is a single DataFrame column
    +      Some(Row(Row(origin, height, width, nChannels, mode, decoded)))
    +    }
    +  }
    +
    +  /**
    +   * :: Experimental ::
    +   * Read the directory of images from the local or remote source
    +   *
    +   * @param path Path to the image directory
    +   * @param sparkSession Spark Session, if omitted gets or creates the session
    +   * @param recursive Recursive path search flag
    +   * @param numPartitions Number of the DataFrame partitions,
    +   *                      if omitted uses defaultParallelism instead
    +   * @param dropImageFailures Drop the files that are not valid images from the result
    +   * @param sampleRatio Fraction of the files loaded
    +   * @return DataFrame with a single column "image" of images;
    +   *         see ImageSchema for the details
    +   */
    +  def readImages(
    +      path: String,
    +      sparkSession: SparkSession = null,
    --- End diff --
    
    agreed, it would be good to change this to the builder API, let me check with @drdarshan and @dakirsa first to see what they say


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r147691394
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,252 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC3" -> 16, "CV_8UC4" -> 24
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField(imageFields(0), StringType, true) ::
    +    StructField(imageFields(1), IntegerType, false) ::
    +    StructField(imageFields(2), IntegerType, false) ::
    +    StructField(imageFields(3), IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(imageFields(4), IntegerType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(imageFields(5), BinaryType, false) :: Nil)
    +
    +  /**
    +   * DataFrame with a single column of images named "image" (nullable)
    +   */
    +  val imageSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the height of the image
    +   *
    +   * @return The height of the image
    +   */
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the width of the image
    +   *
    +   * @return The width of the image
    +   */
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the number of channels in the image
    +   *
    +   * @return The number of channels in the image
    +   */
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the OpenCV representation as an int
    +   *
    +   * @return The OpenCV representation as an int
    +   */
    +  def getMode(row: Row): Int = row.getInt(4)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the image data
    +   *
    +   * @return The image data
    +   */
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * :: Experimental ::
    +   * Check if the DataFrame column contains images (i.e. has ImageSchema)
    +   *
    +   * @param df DataFrame
    +   * @param column Column name
    +   * @return True if the given column matches the image schema
    +   */
    +  def isImageColumn(df: DataFrame, column: String): Boolean =
    +    df.schema(column).dataType == columnSchema
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row =
    +    Row(Row(origin, -1, -1, -1, ocvTypes(undefinedImageType), Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in DataFrame Row
    +   *
    +   * @param origin Arbitrary string that identifies the image
    +   * @param bytes Image bytes (for example, jpeg)
    +   * @return DataFrame Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val isGray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val hasAlpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) = if (isGray) {
    +        (1, ocvTypes("CV_8UC1"))
    +      } else if (hasAlpha) {
    +        (4, ocvTypes("CV_8UC4"))
    +      } else {
    +        (3, ocvTypes("CV_8UC3"))
    +      }
    +
    +      val imageSize = height * width * nChannels
    +      assert(imageSize < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](imageSize)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (isGray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      } else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    +            if (nChannels == 4) {
    +              decoded(offset + 3) = color.getAlpha.toByte
    +            }
    +            offset += nChannels
    +          }
    +        }
    +      }
    +
    +      // the internal "Row" is needed, because the image is a single DataFrame column
    +      Some(Row(Row(origin, height, width, nChannels, mode, decoded)))
    +    }
    +  }
    +
    +  /**
    +   * :: Experimental ::
    +   * Read the directory of images from the local or remote source
    +   *
    +   * @param path Path to the image directory
    +   * @param sparkSession Spark Session
    +   * @param recursive Recursive path search flag
    +   * @param numPartitions Number of the DataFrame partitions
    +   * @param dropImageFailures Drop the files that are not valid images from the result
    +   * @param sampleRatio Fraction of the files loaded
    +   * @return DataFrame with a single column "image" of images;
    +   *         see ImageSchema for the details
    +   */
    +  def readImages(
    +      path: String,
    +      sparkSession: SparkSession = null,
    +      recursive: Boolean = false,
    +      numPartitions: Int = 0,
    +      dropImageFailures: Boolean = false,
    +      sampleRatio: Double = 1.0): DataFrame = {
    +    require(sampleRatio <= 1.0 && sampleRatio >= 0, "sampleRatio should be between 0 and 1")
    +
    +    val session = if (sparkSession != null) sparkSession else SparkSession.builder().getOrCreate
    +    val partitions =
    +      if (numPartitions > 0) {
    +        numPartitions
    +      } else {
    +        session.sparkContext.defaultParallelism
    +      }
    +
    +    val oldRecursiveFlag = RecursiveFlag.setRecursiveFlag(Some(recursive.toString), session)
    --- End diff --
    
    I think we could do the same thing for `SamplePathFilter.setPathFilter` and `SamplePathFilter.unsetPathFilter`.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r145342355
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/HadoopUtils.scala ---
    @@ -0,0 +1,119 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import scala.language.existentials
    +import scala.util.Random
    +
    +import org.apache.commons.io.FilenameUtils
    +import org.apache.hadoop.conf.{Configuration, Configured}
    +import org.apache.hadoop.fs.{Path, PathFilter}
    +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
    +
    +import org.apache.spark.sql.SparkSession
    +
    +private object RecursiveFlag {
    +
    +  /**
    +   * Sets a value of spark recursive flag.
    +   * If value is a None, it unsets the flag.
    +   *
    +   * @param value value to set
    +   * @param spark existing spark session
    +   * @return previous value of this flag
    +   */
    +  def setRecursiveFlag(value: Option[String], spark: SparkSession): Option[String] = {
    +    val flagName = FileInputFormat.INPUT_DIR_RECURSIVE
    +    val hadoopConf = spark.sparkContext.hadoopConfiguration
    +    val old = Option(hadoopConf.get(flagName))
    +
    +    value match {
    +      case Some(v) => hadoopConf.set(flagName, v)
    +      case None => hadoopConf.unset(flagName)
    +    }
    +
    +    old
    +  }
    +}
    +
    +/**
    + * Filter that allows loading a fraction of HDFS files.
    + */
    +private class SamplePathFilter extends Configured with PathFilter {
    +  val random = new Random()
    +
    +  // Ratio of files to be read from disk
    +  var sampleRatio: Double = 1
    +
    +  override def setConf(conf: Configuration): Unit = {
    +    if (conf != null) {
    +      sampleRatio = conf.getDouble(SamplePathFilter.ratioParam, 1)
    +    }
    +  }
    +
    +  override def accept(path: Path): Boolean = {
    +    // Note: checking fileSystem.isDirectory is very slow here, so we use basic rules instead
    +    !SamplePathFilter.isFile(path) || random.nextDouble() < sampleRatio
    +  }
    +}
    +
    +private object SamplePathFilter {
    +  val ratioParam = "sampleRatio"
    +
    +  def isFile(path: Path): Boolean = FilenameUtils.getExtension(path.toString) != ""
    +
    +  /**
    +   * Sets HDFS PathFilter
    +   *
    +   * @param value Filter class that is passed to HDFS
    +   * @param sampleRatio Fraction of the files that the filter picks
    +   * @param spark Existing Spark session
    +   * @return Returns the previous HDFS path filter
    +   */
    +  def setPathFilter(value: Option[Class[_]],
    +                    sampleRatio: Double,
    +                    spark: SparkSession): Option[Class[_]] = {
    --- End diff --
    
    nit:
    
    ```scala
    def setPathFilter(
        value: Option[Class[_]],
        sampleRatio: Double,
        spark: SparkSession): Option[Class[_]] = {
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #82480 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82480/testReport)** for PR 19439 at commit [`22baf02`](https://github.com/apache/spark/commit/22baf022b2f109bb1f5eba0b13ea34de894cd14c).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class SamplePathFilter extends Configured with PathFilter `


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #83344 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83344/testReport)** for PR 19439 at commit [`387d90c`](https://github.com/apache/spark/commit/387d90ca79ae840adb4c799a156f782dc9472207).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #82656 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82656/testReport)** for PR 19439 at commit [`d42636a`](https://github.com/apache/spark/commit/d42636a99dd045fa1003b4907b35796829c6efd5).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #82773 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82773/testReport)** for PR 19439 at commit [`52632cf`](https://github.com/apache/spark/commit/52632cf600303d6a3512efe3e7a1c159ee10f05f).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/83821/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r145525546
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,255 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC2" -> 8, "CV_8UC3" -> 16, "CV_8UC4" -> 24,
    +    "CV_8S" -> 1, "CV_8SC1" -> 1, "CV_8SC2" -> 9, "CV_8SC3" -> 17, "CV_8SC4" -> 25,
    +    "CV_16U" -> 2, "CV_16UC1" -> 2, "CV_16UC2" -> 10, "CV_16UC3" -> 18, "CV_16UC4" -> 26,
    +    "CV_16S" -> 3, "CV_16SC1" -> 3, "CV_16SC2" -> 11, "CV_16SC3" -> 19, "CV_16SC4" -> 27,
    +    "CV_32S" -> 4, "CV_32SC1" -> 4, "CV_32SC2" -> 12, "CV_32SC3" -> 20, "CV_32SC4" -> 28,
    +    "CV_32F" -> 5, "CV_32FC1" -> 5, "CV_32FC2" -> 13, "CV_32FC3" -> 21, "CV_32FC4" -> 29,
    +    "CV_64F" -> 6, "CV_64FC1" -> 6, "CV_64FC2" -> 14, "CV_64FC3" -> 22, "CV_64FC4" -> 30
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Array[Byte])
    +   */
    +  private val columnSchema = StructType(
    --- End diff --
    
    imageSchema StructType is already public, but I can make this public as well since you have a strong preference :)



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/83767/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    @thunterdb @jkbradley @yanboliang Would you be able to take a look at the new image reader?  This is taken from the spark-images package written by @dakirsa @drdarshan @imatiach-msft.  Thank you!


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    Thanks!  LGTM pending tests


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r150449573
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -168,7 +167,7 @@ def readImages(self, path, recursive=False, numPartitions=-1,
             :param bool dropImageFailures: Drop the files that are not valid images
             :param float sampleRatio: Fraction of the images loaded
             :param int seed: Random number seed
    -        :rtype DataFrame: DataFrame with a single column of "images",
    +        :return: DataFrame: DataFrame with a single column of "images",
    --- End diff --
    
    `:return: a DataFrame with a single column of "images",`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    @viirya @srowen @MrBago @holdenk @HyukjinKwon @dakirsa @hhbyyh Thank you for taking a look at the PR merging the "spark-images" spark package into spark and your great comments.  I've updated the PR based on your comments (excluding hhbyyh, which we are still discussing and will reply to shortly).  I've changed the images to use the CC0 license, which seems like the safest license possible (and I added a license file).  I removed the images channel reordering and modified the mode in the schema to use int instead of string.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #84118 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84118/testReport)** for PR 19439 at commit [`a76496b`](https://github.com/apache/spark/commit/a76496be9ebc8b4aba1cd1cd4e3132411649597e).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r143689831
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,217 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.Since
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC2" -> 8, "CV_8UC3" -> 16, "CV_8UC4" -> 24,
    +    "CV_8S" -> 1, "CV_8SC1" -> 1, "CV_8SC2" -> 9, "CV_8SC3" -> 17, "CV_8SC4" -> 25,
    +    "CV_16U" -> 2, "CV_16UC1" -> 2, "CV_16UC2" -> 10, "CV_16UC3" -> 18, "CV_16UC4" -> 26,
    +    "CV_16S" -> 3, "CV_16SC1" -> 3, "CV_16SC2" -> 11, "CV_16SC3" -> 19, "CV_16SC4" -> 27,
    +    "CV_32S" -> 4, "CV_32SC1" -> 4, "CV_32SC2" -> 12, "CV_32SC3" -> 20, "CV_32SC4" -> 28,
    +    "CV_32F" -> 5, "CV_32FC1" -> 5, "CV_32FC2" -> 13, "CV_32FC3" -> 21, "CV_32FC4" -> 29,
    +    "CV_64F" -> 6, "CV_64FC1" -> 6, "CV_64FC2" -> 14, "CV_64FC3" -> 22, "CV_64FC4" -> 30
    +  )
    +
    +  /** Schema for the image column: Row(String, Int, Int, Int, Array[Byte]) */
    +  val columnSchema = StructType(
    +    StructField("origin", StringType, true) ::
    +      StructField("height", IntegerType, false) ::
    +      StructField("width", IntegerType, false) ::
    +      StructField("nChannels", IntegerType, false) ::
    +      // OpenCV-compatible type: CV_8UC3 in most cases
    +      StructField("mode", StringType, false) ::
    +      // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +      StructField("data", BinaryType, false) :: Nil)
    +
    +  // Dataframe with a single column of images named "image" (nullable)
    +  private val imageDFSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  @Since("2.3.0")
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  @Since("2.3.0")
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  @Since("2.3.0")
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  @Since("2.3.0")
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  @Since("2.3.0")
    +  def getMode(row: Row): String = row.getString(4)
    +
    +  @Since("2.3.0")
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /** Check if the dataframe column contains images (i.e. has ImageSchema)
    +   *
    +   * @param df       Dataframe
    +   * @param column   Column name
    +   * @return         True if the given column matches the image schema
    +   */
    +  @Since("2.3.0")
    +  def isImageColumn(df: DataFrame, column: String): Boolean =
    +    df.schema(column).dataType == columnSchema
    +
    +  /** Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return       Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row = Row(Row(origin, -1, -1, -1, undefinedImageType,
    +    Array.ofDim[Byte](0)))
    +
    +  /** Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in dataframe Row
    +   *
    +   * @param origin   Arbitrary string that identifies the image
    +   * @param bytes    Image bytes (for example, jpeg)
    +   * @return         Dataframe Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val is_gray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val has_alpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) =
    +        if (is_gray) (1, "CV_8UC1")
    +        else if (has_alpha) (4, "CV_8UC4")
    +        else (3, "CV_8UC3")
    +
    +      assert(height*width*nChannels < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](height*width*nChannels)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (is_gray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for(h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      }
    +      else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    +            if (nChannels == 4) {
    +              decoded(offset + 3) = color.getAlpha.toByte
    +            }
    +            offset += nChannels
    +          }
    +        }
    +      }
    +
    +      // the internal "Row" is needed, because the image is a single dataframe column
    +      Some(Row(Row(origin, height, width, nChannels, mode, decoded)))
    +    }
    +  }
    +
    +  /** Read the directory of images from the local or remote source
    +   *
    +   * @param path               Path to the image directory
    +   * @param sparkSession       Spark Session
    +   * @param recursive          Recursive path search flag
    +   * @param numPartitions      Number of the dataframe partitions
    +   * @param dropImageFailures  Drop the files that are not valid images from the result
    +   * @param sampleRatio        Fraction of the files loaded
    +   * @return                   Dataframe with a single column "image" of images;
    +   *                           see ImageSchema for the details
    +   */
    +  @Since("2.3.0")
    +  def readImages(path: String,
    +                 sparkSession: SparkSession = null,
    +                 recursive: Boolean = false,
    +                 numPartitions: Int = 0,
    +                 dropImageFailures: Boolean = false,
    +                 sampleRatio: Double = 1.0): DataFrame = {
    +    require(sampleRatio <= 1.0 && sampleRatio >= 0, "sampleRatio should be between 0 and 1")
    +
    +    val session = if (sparkSession != null) sparkSession else SparkSession.builder().getOrCreate
    +    val partitions =
    +      if (numPartitions > 0) numPartitions
    +      else session.sparkContext.defaultParallelism
    +
    +    val oldRecursiveFlag = RecursiveFlag.setRecursiveFlag(Some(recursive.toString), session)
    +    val oldPathFilter: Option[Class[_]] =
    +      if (sampleRatio < 1) {
    +        SamplePathFilter.setPathFilter(Some(classOf[SamplePathFilter]), Some(sampleRatio), session)
    +      }
    +      else {
    +        None
    +      }
    +
    +    var result: DataFrame = null
    +    try {
    +      val streams = session.sparkContext.binaryFiles(path, partitions)
    +        .repartition(partitions)
    +
    +      val images = if (dropImageFailures) {
    +        streams.flatMap{
    --- End diff --
    
    `flatMap{` -> `flatMap {`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r144193840
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC2" -> 8, "CV_8UC3" -> 16, "CV_8UC4" -> 24,
    +    "CV_8S" -> 1, "CV_8SC1" -> 1, "CV_8SC2" -> 9, "CV_8SC3" -> 17, "CV_8SC4" -> 25,
    +    "CV_16U" -> 2, "CV_16UC1" -> 2, "CV_16UC2" -> 10, "CV_16UC3" -> 18, "CV_16UC4" -> 26,
    +    "CV_16S" -> 3, "CV_16SC1" -> 3, "CV_16SC2" -> 11, "CV_16SC3" -> 19, "CV_16SC4" -> 27,
    +    "CV_32S" -> 4, "CV_32SC1" -> 4, "CV_32SC2" -> 12, "CV_32SC3" -> 20, "CV_32SC4" -> 28,
    +    "CV_32F" -> 5, "CV_32FC1" -> 5, "CV_32FC2" -> 13, "CV_32FC3" -> 21, "CV_32FC4" -> 29,
    +    "CV_64F" -> 6, "CV_64FC1" -> 6, "CV_64FC2" -> 14, "CV_64FC3" -> 22, "CV_64FC4" -> 30
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField("origin", StringType, true) ::
    +      StructField("height", IntegerType, false) ::
    +      StructField("width", IntegerType, false) ::
    +      StructField("nChannels", IntegerType, false) ::
    +      // OpenCV-compatible type: CV_8UC3 in most cases
    +      StructField("mode", StringType, false) ::
    +      // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +      StructField("data", BinaryType, false) :: Nil)
    +
    +  // Dataframe with a single column of images named "image" (nullable)
    +  private val imageDFSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  @Since("2.3.0")
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  @Since("2.3.0")
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  @Since("2.3.0")
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  @Since("2.3.0")
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  @Since("2.3.0")
    +  def getMode(row: Row): String = row.getString(4)
    +
    +  @Since("2.3.0")
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * Check if the dataframe column contains images (i.e. has ImageSchema)
    +   *
    +   * @param df       Dataframe
    +   * @param column   Column name
    +   * @return         True if the given column matches the image schema
    +   */
    +  @Since("2.3.0")
    +  def isImageColumn(df: DataFrame, column: String): Boolean =
    +    df.schema(column).dataType == columnSchema
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return       Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row = Row(Row(origin, -1, -1, -1, undefinedImageType,
    +    Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in dataframe Row
    +   *
    +   * @param origin   Arbitrary string that identifies the image
    +   * @param bytes    Image bytes (for example, jpeg)
    +   * @return         Dataframe Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val is_gray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val has_alpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) = if (is_gray) {
    +        (1, "CV_8UC1")
    +      } else if (has_alpha) {
    +        (4, "CV_8UC4")
    +      } else {
    +        (3, "CV_8UC3")
    +      }
    +
    +      assert(height * width * nChannels < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](height*width*nChannels)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (is_gray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for(h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      }
    +      else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    +            if (nChannels == 4) {
    +              decoded(offset + 3) = color.getAlpha.toByte
    +            }
    +            offset += nChannels
    +          }
    +        }
    +      }
    +
    +      // the internal "Row" is needed, because the image is a single dataframe column
    +      Some(Row(Row(origin, height, width, nChannels, mode, decoded)))
    +    }
    +  }
    +
    +  /**
    +   * :: Experimental ::
    +   * Read the directory of images from the local or remote source
    +   *
    +   * @param path               Path to the image directory
    +   * @param sparkSession       Spark Session
    +   * @param recursive          Recursive path search flag
    +   * @param numPartitions      Number of the dataframe partitions
    +   * @param dropImageFailures  Drop the files that are not valid images from the result
    +   * @param sampleRatio        Fraction of the files loaded
    +   * @return                   Dataframe with a single column "image" of images;
    +   *                           see ImageSchema for the details
    +   */
    +  @Since("2.3.0")
    +  def readImages(path: String,
    +                 sparkSession: SparkSession = null,
    +                 recursive: Boolean = false,
    +                 numPartitions: Int = 0,
    +                 dropImageFailures: Boolean = false,
    +                 sampleRatio: Double = 1.0): DataFrame = {
    +    require(sampleRatio <= 1.0 && sampleRatio >= 0, "sampleRatio should be between 0 and 1")
    +
    +    val session = if (sparkSession != null) sparkSession else SparkSession.builder().getOrCreate
    +    val partitions =
    +      if (numPartitions > 0) {
    +        numPartitions
    +      } else {
    +        session.sparkContext.defaultParallelism
    +      }
    +
    +    val oldRecursiveFlag = RecursiveFlag.setRecursiveFlag(Some(recursive.toString), session)
    +    val oldPathFilter: Option[Class[_]] =
    +      if (sampleRatio < 1) {
    +        SamplePathFilter.setPathFilter(Some(classOf[SamplePathFilter]), sampleRatio, session)
    +      } else {
    +        None
    +      }
    +
    +    var result: DataFrame = null
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #82857 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82857/testReport)** for PR 19439 at commit [`fee2c16`](https://github.com/apache/spark/commit/fee2c16371074e894f2abb9a9e46ddf1d5b6c0ef).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by imatiach-msft <gi...@git.apache.org>.
Github user imatiach-msft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r148054322
  
    --- Diff: python/pyspark/ml/image.py ---
    @@ -0,0 +1,139 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +from pyspark.ml.util import *
    +from pyspark.ml.param.shared import *
    +from pyspark.sql.types import *
    --- End diff --
    
    I removed those imports and the tests still passed, so you may be right they they are not needed


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by WeichenXu123 <gi...@git.apache.org>.
Github user WeichenXu123 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r148700189
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,236 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.input.PortableDataStream
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val imageFields: Array[String] = Array("origin", "height", "width", "nChannels", "mode", "data")
    +
    +  val ocvTypes: Map[String, Int] = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC3" -> 16, "CV_8UC4" -> 24
    +  )
    +
    +  /**
    +   * Used for conversion to python
    +   */
    +  val _ocvTypes: java.util.Map[String, Int] = ocvTypes.asJava
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField(imageFields(0), StringType, true) ::
    +    StructField(imageFields(1), IntegerType, false) ::
    +    StructField(imageFields(2), IntegerType, false) ::
    +    StructField(imageFields(3), IntegerType, false) ::
    +    // OpenCV-compatible type: CV_8UC3 in most cases
    +    StructField(imageFields(4), IntegerType, false) ::
    +    // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +    StructField(imageFields(5), BinaryType, false) :: Nil)
    +
    +  /**
    +   * DataFrame with a single column of images named "image" (nullable)
    +   */
    +  val imageSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the origin of the image
    +   *
    +   * @return The origin of the image
    +   */
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the height of the image
    +   *
    +   * @return The height of the image
    +   */
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the width of the image
    +   *
    +   * @return The width of the image
    +   */
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the number of channels in the image
    +   *
    +   * @return The number of channels in the image
    +   */
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the OpenCV representation as an int
    +   *
    +   * @return The OpenCV representation as an int
    +   */
    +  def getMode(row: Row): Int = row.getInt(4)
    +
    +  /**
    +   * :: Experimental ::
    +   * Gets the image data
    +   *
    +   * @return The image data
    +   */
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row =
    +    Row(Row(origin, -1, -1, -1, ocvTypes(undefinedImageType), Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in DataFrame Row
    +   *
    +   * @param origin Arbitrary string that identifies the image
    +   * @param bytes Image bytes (for example, jpeg)
    +   * @return DataFrame Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val isGray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val hasAlpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) = if (isGray) {
    +        (1, ocvTypes("CV_8UC1"))
    +      } else if (hasAlpha) {
    +        (4, ocvTypes("CV_8UC4"))
    +      } else {
    +        (3, ocvTypes("CV_8UC3"))
    +      }
    +
    +      val imageSize = height * width * nChannels
    +      assert(imageSize < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](imageSize)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (isGray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      } else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    +            if (nChannels == 4) {
    +              decoded(offset + 3) = color.getAlpha.toByte
    +            }
    +            offset += nChannels
    +          }
    +        }
    +      }
    +
    +      // the internal "Row" is needed, because the image is a single DataFrame column
    +      Some(Row(Row(origin, height, width, nChannels, mode, decoded)))
    +    }
    +  }
    +
    +  /**
    +   * :: Experimental ::
    +   * Read the directory of images from the local or remote source
    +   *
    +   * @param path Path to the image directory
    +   * @param sparkSession Spark Session, if omitted gets or creates the session
    +   * @param recursive Recursive path search flag
    +   * @param numPartitions Number of the DataFrame partitions,
    +   *                      if omitted uses defaultParallelism instead
    +   * @param dropImageFailures Drop the files that are not valid images from the result
    +   * @param sampleRatio Fraction of the files loaded
    +   * @return DataFrame with a single column "image" of images;
    +   *         see ImageSchema for the details
    +   */
    +  def readImages(
    +      path: String,
    +      sparkSession: SparkSession = null,
    +      recursive: Boolean = false,
    +      numPartitions: Int = 0,
    +      dropImageFailures: Boolean = false,
    +      sampleRatio: Double = 1.0): DataFrame = {
    +    require(sampleRatio <= 1.0 && sampleRatio >= 0, "sampleRatio should be between 0 and 1")
    +
    +    val session = if (sparkSession != null) sparkSession else SparkSession.builder().getOrCreate
    +    val partitions =
    +      if (numPartitions > 0) {
    +        numPartitions
    +      } else {
    +        session.sparkContext.defaultParallelism
    +      }
    +
    +    RecursiveFlag.withRecursiveFlag(recursive, session) {
    +      SamplePathFilter.withPathFilter(sampleRatio, session) {
    +        val streams = session.sparkContext.binaryFiles(path, partitions).repartition(partitions)
    --- End diff --
    
    @jkbradley I think the `repartition` won't be expensive, because `.binaryFiles` returns `RDD[(String, PortableDataStream)]` which do not contain file content. (Only contain file split info in `PortableDataStream`.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19439: [SPARK-21866][ML][PySpark] Adding spark image reader

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19439
  
    **[Test build #83671 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83671/testReport)** for PR 19439 at commit [`a6c82ce`](https://github.com/apache/spark/commit/a6c82ceb1752345a2379e8e26f66bbf91b579991).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19439: [SPARK-21866][ML][PySpark] Adding spark image rea...

Posted by dakirsa <gi...@git.apache.org>.
Github user dakirsa commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19439#discussion_r145680529
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala ---
    @@ -0,0 +1,229 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.ml.image
    +
    +import java.awt.Color
    +import java.awt.color.ColorSpace
    +import java.io.ByteArrayInputStream
    +import javax.imageio.ImageIO
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.sql.{DataFrame, Row, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +@Experimental
    +@Since("2.3.0")
    +object ImageSchema {
    +
    +  val undefinedImageType = "Undefined"
    +
    +  val ocvTypes = Map(
    +    undefinedImageType -> -1,
    +    "CV_8U" -> 0, "CV_8UC1" -> 0, "CV_8UC2" -> 8, "CV_8UC3" -> 16, "CV_8UC4" -> 24,
    +    "CV_8S" -> 1, "CV_8SC1" -> 1, "CV_8SC2" -> 9, "CV_8SC3" -> 17, "CV_8SC4" -> 25,
    +    "CV_16U" -> 2, "CV_16UC1" -> 2, "CV_16UC2" -> 10, "CV_16UC3" -> 18, "CV_16UC4" -> 26,
    +    "CV_16S" -> 3, "CV_16SC1" -> 3, "CV_16SC2" -> 11, "CV_16SC3" -> 19, "CV_16SC4" -> 27,
    +    "CV_32S" -> 4, "CV_32SC1" -> 4, "CV_32SC2" -> 12, "CV_32SC3" -> 20, "CV_32SC4" -> 28,
    +    "CV_32F" -> 5, "CV_32FC1" -> 5, "CV_32FC2" -> 13, "CV_32FC3" -> 21, "CV_32FC4" -> 29,
    +    "CV_64F" -> 6, "CV_64FC1" -> 6, "CV_64FC2" -> 14, "CV_64FC3" -> 22, "CV_64FC4" -> 30
    +  )
    +
    +  /**
    +   * Schema for the image column: Row(String, Int, Int, Int, Array[Byte])
    +   */
    +  val columnSchema = StructType(
    +    StructField("origin", StringType, true) ::
    +      StructField("height", IntegerType, false) ::
    +      StructField("width", IntegerType, false) ::
    +      StructField("nChannels", IntegerType, false) ::
    +      // OpenCV-compatible type: CV_8UC3 in most cases
    +      StructField("mode", StringType, false) ::
    +      // Bytes in OpenCV-compatible order: row-wise BGR in most cases
    +      StructField("data", BinaryType, false) :: Nil)
    +
    +  // Dataframe with a single column of images named "image" (nullable)
    +  private val imageDFSchema = StructType(StructField("image", columnSchema, true) :: Nil)
    +
    +  @Since("2.3.0")
    +  def getOrigin(row: Row): String = row.getString(0)
    +
    +  @Since("2.3.0")
    +  def getHeight(row: Row): Int = row.getInt(1)
    +
    +  @Since("2.3.0")
    +  def getWidth(row: Row): Int = row.getInt(2)
    +
    +  @Since("2.3.0")
    +  def getNChannels(row: Row): Int = row.getInt(3)
    +
    +  @Since("2.3.0")
    +  def getMode(row: Row): String = row.getString(4)
    +
    +  @Since("2.3.0")
    +  def getData(row: Row): Array[Byte] = row.getAs[Array[Byte]](5)
    +
    +  /**
    +   * Check if the dataframe column contains images (i.e. has ImageSchema)
    +   *
    +   * @param df       Dataframe
    +   * @param column   Column name
    +   * @return         True if the given column matches the image schema
    +   */
    +  @Since("2.3.0")
    +  def isImageColumn(df: DataFrame, column: String): Boolean =
    +    df.schema(column).dataType == columnSchema
    +
    +  /**
    +   * Default values for the invalid image
    +   *
    +   * @param origin Origin of the invalid image
    +   * @return       Row with the default values
    +   */
    +  private def invalidImageRow(origin: String): Row = Row(Row(origin, -1, -1, -1, undefinedImageType,
    +    Array.ofDim[Byte](0)))
    +
    +  /**
    +   * Convert the compressed image (jpeg, png, etc.) into OpenCV
    +   * representation and store it in dataframe Row
    +   *
    +   * @param origin   Arbitrary string that identifies the image
    +   * @param bytes    Image bytes (for example, jpeg)
    +   * @return         Dataframe Row or None (if the decompression fails)
    +   */
    +  private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = {
    +
    +    val img = ImageIO.read(new ByteArrayInputStream(bytes))
    +
    +    if (img == null) {
    +      None
    +    } else {
    +      val is_gray = img.getColorModel.getColorSpace.getType == ColorSpace.TYPE_GRAY
    +      val has_alpha = img.getColorModel.hasAlpha
    +
    +      val height = img.getHeight
    +      val width = img.getWidth
    +      val (nChannels, mode) = if (is_gray) {
    +        (1, "CV_8UC1")
    +      } else if (has_alpha) {
    +        (4, "CV_8UC4")
    +      } else {
    +        (3, "CV_8UC3")
    +      }
    +
    +      assert(height * width * nChannels < 1e9, "image is too large")
    +      val decoded = Array.ofDim[Byte](height*width*nChannels)
    +
    +      // Grayscale images in Java require special handling to get the correct intensity
    +      if (is_gray) {
    +        var offset = 0
    +        val raster = img.getRaster
    +        for(h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            decoded(offset) = raster.getSample(w, h, 0).toByte
    +            offset += 1
    +          }
    +        }
    +      }
    +      else {
    +        var offset = 0
    +        for (h <- 0 until height) {
    +          for (w <- 0 until width) {
    +            val color = new Color(img.getRGB(w, h))
    +
    +            decoded(offset) = color.getBlue.toByte
    +            decoded(offset + 1) = color.getGreen.toByte
    +            decoded(offset + 2) = color.getRed.toByte
    --- End diff --
    
    I agree. The result is signed, but it is not an error for the Byte type.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org