You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by MrBago <gi...@git.apache.org> on 2017/11/14 15:09:33 UTC

[GitHub] spark pull request #19746: [SPARK-22346][ML]

GitHub user MrBago opened a pull request:

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

    [SPARK-22346][ML]

    ## What changes were proposed in this pull request?
    
    A new VectorSizeHint transformer was added. This transformer is meant to be used as a pipeline stage ahead of VectorAssembler, on vector columns, so that VectorAssembler can join vectors in a streaming context where the size of the input vectors is otherwise not known.
    
    ## How was this patch tested?
    
    Unit tests.
    
    Please review http://spark.apache.org/contributing.html before opening a pull request.


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

    $ git pull https://github.com/MrBago/spark vector-size-hint

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

    https://github.com/apache/spark/pull/19746.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 #19746
    
----
commit df990ed771d9d773b6a8a1b8751ae8c1c475b3d5
Author: Bago Amirbekian <ba...@databricks.com>
Date:   2017-11-14T15:01:57Z

    Added VectorSizeHint Transformer in ml.feature.

----


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r153420429
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala ---
    @@ -0,0 +1,173 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, Vectors}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.execution.streaming.MemoryStream
    +import org.apache.spark.sql.streaming.StreamTest
    +
    +class VectorSizeHintSuite
    +  extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  import testImplicits._
    +
    +  test("Test Param Validators") {
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setHandleInvalid("invalidValue"))
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setSize(-3))
    +  }
    +
    +  test("Adding size to column of vectors.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val denseVector = Vectors.dense(1, 2, 3)
    +    val sparseVector = Vectors.sparse(size, Array(), Array())
    +
    +    val data = Seq(denseVector, denseVector, sparseVector).map(Tuple1.apply)
    +    val dataFrame = data.toDF(vectorColName)
    +    assert(
    +      AttributeGroup.fromStructField(dataFrame.schema(vectorColName)).size == -1,
    +      "Transformer did not add expected size data.")
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrame)
    +      assert(
    +        AttributeGroup.fromStructField(withSize.schema(vectorColName)).size == size,
    +        "Transformer did not add expected size data.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size hint preserves attributes.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +    val group = AttributeGroup.fromStructField(dataFrameWithMetadata.schema(vectorColName))
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrameWithMetadata)
    +
    +      val newGroup = AttributeGroup.fromStructField(withSize.schema(vectorColName))
    +      assert(newGroup.size === size, "Transformer did not add expected size data.")
    +      assert(
    +        newGroup.attributes.get.deep === group.attributes.get.deep,
    +        "SizeHintTransformer did not preserve attributes.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size miss-match between current and target size raises an error.") {
    +    val size = 4
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      intercept[SparkException](transformer.transform(dataFrameWithMetadata))
    +    }
    +  }
    +
    +  test("Handle invalid does the right thing.") {
    +
    +    val vector = Vectors.dense(1, 2, 3)
    +    val short = Vectors.dense(2)
    +    val dataWithNull = Seq(vector, null).map(Tuple1.apply).toDF("vector")
    +    val dataWithShort = Seq(vector, short).map(Tuple1.apply).toDF("vector")
    +
    +    val sizeHint = new VectorSizeHint()
    +      .setInputCol("vector")
    +      .setHandleInvalid("error")
    +      .setSize(3)
    +
    +    intercept[SparkException](sizeHint.transform(dataWithNull).collect)
    +    intercept[SparkException](sizeHint.transform(dataWithShort).collect)
    +
    +    sizeHint.setHandleInvalid("skip")
    +    assert(sizeHint.transform(dataWithNull).count() === 1)
    +    assert(sizeHint.transform(dataWithShort).count() === 1)
    +  }
    +
    +  test("read/write") {
    +    val sizeHint = new VectorSizeHint()
    +      .setInputCol("myInputCol")
    +      .setSize(11)
    +      .setHandleInvalid("skip")
    +    testDefaultReadWrite(sizeHint)
    +  }
    +}
    +
    +class VectorSizeHintStreamingSuite extends StreamTest {
    +
    +  import testImplicits._
    +
    +  test("Test assemble vectors with size hint in steaming.") {
    +    val a = Vectors.dense(0, 1, 2)
    +    val b = Vectors.sparse(4, Array(0, 3), Array(3, 6))
    +
    +    val stream = MemoryStream[(Vector, Vector)]
    +    val streamingDF = stream.toDS.toDF("a", "b")
    +    val sizeHintA = new VectorSizeHint()
    +      .setSize(3)
    +      .setInputCol("a")
    +    val sizeHintB = new VectorSizeHint()
    +      .setSize(4)
    +      .setInputCol("b")
    +    val vectorAssembler = new VectorAssembler()
    +      .setInputCols(Array("a", "b"))
    +      .setOutputCol("assembled")
    +    val output = Seq(sizeHintA, sizeHintB, vectorAssembler).foldLeft(streamingDF) {
    +      case (data, transform) => transform.transform(data)
    +    }.select("assembled")
    +
    +    val expected = Vectors.dense(0, 1, 2, 3, 0, 0, 6)
    +
    +    testStream (output) (
    +      AddData(stream, (a, b), (a, b)),
    +      CheckAnswerRows(Seq(Row(expected), Row(expected)), false, false)
    --- End diff --
    
    Use `CheckAnswer(expected, expected)` will be simpler.


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    **[Test build #84036 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84036/testReport)** for PR 19746 at commit [`fb51cbf`](https://github.com/apache/spark/commit/fb51cbfb118c2754f354cec9d5685eecf289836e).
     * 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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/83859/
    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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r155860707
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala ---
    @@ -0,0 +1,173 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, Vectors}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.execution.streaming.MemoryStream
    +import org.apache.spark.sql.streaming.StreamTest
    +
    +class VectorSizeHintSuite
    +  extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  import testImplicits._
    +
    +  test("Test Param Validators") {
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setHandleInvalid("invalidValue"))
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setSize(-3))
    +  }
    +
    +  test("Adding size to column of vectors.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val denseVector = Vectors.dense(1, 2, 3)
    +    val sparseVector = Vectors.sparse(size, Array(), Array())
    +
    +    val data = Seq(denseVector, denseVector, sparseVector).map(Tuple1.apply)
    +    val dataFrame = data.toDF(vectorColName)
    +    assert(
    +      AttributeGroup.fromStructField(dataFrame.schema(vectorColName)).size == -1,
    +      "Transformer did not add expected size data.")
    --- End diff --
    
    incorrect error message


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

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


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    **[Test build #84089 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84089/testReport)** for PR 19746 at commit [`2b1ed0a`](https://github.com/apache/spark/commit/2b1ed0a3a85385f9b4042415889335942b65b9c9).
     * This patch **fails Spark 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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r156216171
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.Transformer
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, VectorUDT}
    +import org.apache.spark.ml.param.{IntParam, Param, ParamMap, ParamValidators}
    +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol}
    +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable}
    +import org.apache.spark.sql.{Column, DataFrame, Dataset}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.StructType
    +
    +/**
    + * A feature transformer that adds vector size information to a vector column.
    + */
    +@Experimental
    +@Since("2.3.0")
    +class VectorSizeHint @Since("2.3.0") (@Since("2.3.0") override val uid: String)
    +  extends Transformer with HasInputCol with HasHandleInvalid with DefaultParamsWritable {
    +
    +  @Since("2.3.0")
    +  def this() = this(Identifiable.randomUID("vectSizeHint"))
    +
    +  @Since("2.3.0")
    +  val size = new IntParam(this, "size", "Size of vectors in column.", {s: Int => s >= 0})
    +
    +  @Since("2.3.0")
    +  def getSize: Int = getOrDefault(size)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setSize(value: Int): this.type = set(size, value)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  @Since("2.3.0")
    +  override val handleInvalid: Param[String] = new Param[String](
    +    this,
    +    "handleInvalid",
    +    "How to handle invalid vectors in inputCol, (invalid vectors include nulls and vectors with " +
    +      "the wrong size. The options are `skip` (filter out rows with invalid vectors), `error` " +
    +      "(throw an error) and `optimistic` (don't check the vector size).",
    +    ParamValidators.inArray(VectorSizeHint.supportedHandleInvalids))
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setHandleInvalid(value: String): this.type = set(handleInvalid, value)
    +  setDefault(handleInvalid, VectorSizeHint.ERROR_INVALID)
    +
    +  @Since("2.3.0")
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    val localInputCol = getInputCol
    +    val localSize = getSize
    +    val localHandleInvalid = getHandleInvalid
    +
    +    val group = AttributeGroup.fromStructField(dataset.schema(localInputCol))
    +    if (localHandleInvalid == VectorSizeHint.OPTIMISTIC_INVALID && group.size == localSize) {
    +      dataset.toDF
    +    } else {
    +      val newGroup = group.size match {
    +        case `localSize` => group
    +        case -1 => new AttributeGroup(localInputCol, localSize)
    +        case _ =>
    +          val msg = s"Trying to set size of vectors in `$localInputCol` to $localSize but size " +
    +            s"already set to ${group.size}."
    +          throw new SparkException(msg)
    +      }
    +
    +      val newCol: Column = localHandleInvalid match {
    +        case VectorSizeHint.OPTIMISTIC_INVALID => col(localInputCol)
    +        case VectorSizeHint.ERROR_INVALID =>
    +          val checkVectorSizeUDF = udf { vector: Vector =>
    +            if (vector == null) {
    +              throw new SparkException(s"Got null vector in VectorSizeHint, set `handleInvalid` " +
    +                s"to 'skip' to filter invalid rows.")
    +            }
    +            if (vector.size != localSize) {
    +              throw new SparkException(s"VectorSizeHint Expecting a vector of size $localSize but" +
    +                s" got ${vector.size}")
    +            }
    +            vector
    +          }.asNondeterministic
    +          checkVectorSizeUDF(col(localInputCol))
    +        case VectorSizeHint.SKIP_INVALID =>
    +          val checkVectorSizeUDF = udf { vector: Vector =>
    +            if (vector != null && vector.size == localSize) {
    +              vector
    +            } else {
    +              null
    +            }
    +          }
    +          checkVectorSizeUDF(col(localInputCol))
    +      }
    +
    +      val res = dataset.withColumn(localInputCol, newCol.as(localInputCol, newGroup.toMetadata))
    +      if (localHandleInvalid == VectorSizeHint.SKIP_INVALID) {
    +        res.na.drop(Array(localInputCol))
    +      } else {
    +        res
    +      }
    +    }
    +  }
    +
    +  @Since("2.3.0")
    +  override def transformSchema(schema: StructType): StructType = {
    +    val inputColType = schema(getInputCol).dataType
    +    require(
    +      inputColType.isInstanceOf[VectorUDT],
    +      s"Input column, $getInputCol must be of Vector type, got $inputColType"
    +    )
    +    schema
    +  }
    +
    +  @Since("2.3.0")
    +  override def copy(extra: ParamMap): this.type = defaultCopy(extra)
    +}
    +
    +@Experimental
    +@Since("2.3.0")
    +object VectorSizeHint extends DefaultParamsReadable[VectorSizeHint] {
    +
    +  private[feature] val OPTIMISTIC_INVALID = "optimistic"
    --- End diff --
    
    I'm open to changing this to keep, but I wanted to lay out the argument for having it not be keep. The way that keep is used in other transformers is generally to map unknown or new values to some valid representation. In other words in an instruction to the transformer to make a "best effort" to deal with invalid values. The important thing here is that not only does keep not error on invalid values, but existing `keep` implementations actually ensure that invalid values produce a valid result.
    
    The behaviour of this transformer is subtly different. It doesn't do anything to "correct" invalid vectors and as a result using the `keep/optimistic` option can lead to an invalid state of the DataFrame, namely the metadata is wrong about the contents of the column. Users who are accustomed to using `keep` on other transformers maybe confused or frustrated by this difference.


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r155687965
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.Transformer
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, VectorUDT}
    +import org.apache.spark.ml.param.{IntParam, Param, ParamMap, ParamValidators}
    +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol}
    +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable}
    +import org.apache.spark.sql.{Column, DataFrame, Dataset}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.StructType
    +
    +/**
    + * A feature transformer that adds vector size information to a vector column.
    --- End diff --
    
    Add ```:: Experimental ::``` note here so it shows up properly in docs.  Look at other uses of Experimental for examples.  (Same for the companion object)


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    **[Test build #83918 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83918/testReport)** for PR 19746 at commit [`03bd63c`](https://github.com/apache/spark/commit/03bd63c654cc3f8982ec726c50472759f704918d).


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    **[Test build #84451 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84451/testReport)** for PR 19746 at commit [`c3d1c5e`](https://github.com/apache/spark/commit/c3d1c5eb2a81124b05a9f7b066f9e8ce46efe217).
     * 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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r151955287
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.Transformer
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, VectorUDT}
    +import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
    +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol}
    +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable}
    +import org.apache.spark.sql.{Column, DataFrame, Dataset}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.StructType
    +
    +/**
    + * A feature transformer that adds vector size information to a vector column.
    + */
    +@Experimental
    +@Since("2.3.0")
    +class VectorSizeHint @Since("2.3.0") (@Since("2.3.0") override val uid: String)
    +  extends Transformer with HasInputCol with HasHandleInvalid with DefaultParamsWritable {
    +
    +  @Since("2.3.0")
    +  def this() = this(Identifiable.randomUID("vectSizeHint"))
    +
    +  @Since("2.3.0")
    +  val size = new Param[Int](this, "size", "Size of vectors in column.", {s: Int => s >= 0})
    +
    +  @Since("2.3.0")
    +  def getSize: Int = getOrDefault(size)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setSize(value: Int): this.type = set(size, value)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  @Since("2.3.0")
    +  override val handleInvalid: Param[String] = new Param[String](
    +    this,
    +    "handleInvalid",
    +    "How to handle invalid vectors in inputCol, (invalid vectors include nulls and vectors with " +
    +      "the wrong size. The options are `skip` (filter out rows with invalid vectors), `error` " +
    +      "(throw an error) and `optimistic` (don't check the vector size).",
    +    ParamValidators.inArray(VectorSizeHint.supportedHandleInvalids))
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setHandleInvalid(value: String): this.type = set(handleInvalid, value)
    +  setDefault(handleInvalid, VectorSizeHint.ERROR_INVALID)
    +
    +  @Since("2.3.0")
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    val localInputCol = getInputCol
    +    val localSize = getSize
    +    val localHandleInvalid = getHandleInvalid
    +
    +    val group = AttributeGroup.fromStructField(dataset.schema(localInputCol))
    +    if (localHandleInvalid == VectorSizeHint.OPTIMISTIC_INVALID && group.size == localSize) {
    +      dataset.toDF
    +    } else {
    +      val newGroup = if (group.size == localSize) {
    +        // Pass along any existing metadata about vector.
    +        group
    +      } else {
    +        new AttributeGroup(localInputCol, localSize)
    +      }
    +
    +      val newCol: Column = localHandleInvalid match {
    +        case VectorSizeHint.OPTIMISTIC_INVALID => col(localInputCol)
    +        case VectorSizeHint.ERROR_INVALID =>
    +          val checkVectorSize = { vector: Vector =>
    +            if (vector == null) {
    +              throw new VectorSizeHint.InvalidEntryException(s"Got null vector in VectorSizeHint," +
    +                s" set `handleInvalid` to 'skip' to filter invalid rows.")
    +            }
    +            if (vector.size != localSize) {
    +              throw new VectorSizeHint.InvalidEntryException(s"VectorSizeHint Expecting a vector " +
    +                s"of size $localSize but got ${vector.size}")
    +            }
    +            vector
    +          }
    +          udf(checkVectorSize, new VectorUDT)(col(localInputCol))
    +        case VectorSizeHint.SKIP_INVALID =>
    +          val checkVectorSize = { vector: Vector =>
    +            if (vector != null && vector.size == localSize) {
    +              vector
    +            } else {
    +              null
    +            }
    +          }
    +          udf(checkVectorSize, new VectorUDT)(col(localInputCol))
    +      }
    +
    +      val res = dataset.withColumn(localInputCol, newCol.as(localInputCol, newGroup.toMetadata))
    +      if (localHandleInvalid == VectorSizeHint.SKIP_INVALID) {
    +        res.filter(col(localInputCol).isNotNull)
    --- End diff --
    
    I think here use `res.na.drop(Array(localInputCol))` will be better.


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r153420330
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala ---
    @@ -0,0 +1,173 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, Vectors}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.execution.streaming.MemoryStream
    +import org.apache.spark.sql.streaming.StreamTest
    +
    +class VectorSizeHintSuite
    +  extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  import testImplicits._
    +
    +  test("Test Param Validators") {
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setHandleInvalid("invalidValue"))
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setSize(-3))
    +  }
    +
    +  test("Adding size to column of vectors.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val denseVector = Vectors.dense(1, 2, 3)
    +    val sparseVector = Vectors.sparse(size, Array(), Array())
    +
    +    val data = Seq(denseVector, denseVector, sparseVector).map(Tuple1.apply)
    +    val dataFrame = data.toDF(vectorColName)
    +    assert(
    +      AttributeGroup.fromStructField(dataFrame.schema(vectorColName)).size == -1,
    +      "Transformer did not add expected size data.")
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrame)
    +      assert(
    +        AttributeGroup.fromStructField(withSize.schema(vectorColName)).size == size,
    +        "Transformer did not add expected size data.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size hint preserves attributes.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +    val group = AttributeGroup.fromStructField(dataFrameWithMetadata.schema(vectorColName))
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrameWithMetadata)
    +
    +      val newGroup = AttributeGroup.fromStructField(withSize.schema(vectorColName))
    +      assert(newGroup.size === size, "Transformer did not add expected size data.")
    +      assert(
    +        newGroup.attributes.get.deep === group.attributes.get.deep,
    +        "SizeHintTransformer did not preserve attributes.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size miss-match between current and target size raises an error.") {
    +    val size = 4
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      intercept[SparkException](transformer.transform(dataFrameWithMetadata))
    +    }
    +  }
    +
    +  test("Handle invalid does the right thing.") {
    +
    +    val vector = Vectors.dense(1, 2, 3)
    +    val short = Vectors.dense(2)
    +    val dataWithNull = Seq(vector, null).map(Tuple1.apply).toDF("vector")
    +    val dataWithShort = Seq(vector, short).map(Tuple1.apply).toDF("vector")
    +
    +    val sizeHint = new VectorSizeHint()
    +      .setInputCol("vector")
    +      .setHandleInvalid("error")
    +      .setSize(3)
    +
    +    intercept[SparkException](sizeHint.transform(dataWithNull).collect)
    +    intercept[SparkException](sizeHint.transform(dataWithShort).collect)
    +
    +    sizeHint.setHandleInvalid("skip")
    +    assert(sizeHint.transform(dataWithNull).count() === 1)
    +    assert(sizeHint.transform(dataWithShort).count() === 1)
    +  }
    +
    +  test("read/write") {
    +    val sizeHint = new VectorSizeHint()
    +      .setInputCol("myInputCol")
    +      .setSize(11)
    +      .setHandleInvalid("skip")
    +    testDefaultReadWrite(sizeHint)
    +  }
    +}
    +
    +class VectorSizeHintStreamingSuite extends StreamTest {
    +
    +  import testImplicits._
    +
    +  test("Test assemble vectors with size hint in steaming.") {
    +    val a = Vectors.dense(0, 1, 2)
    +    val b = Vectors.sparse(4, Array(0, 3), Array(3, 6))
    +
    +    val stream = MemoryStream[(Vector, Vector)]
    +    val streamingDF = stream.toDS.toDF("a", "b")
    +    val sizeHintA = new VectorSizeHint()
    +      .setSize(3)
    +      .setInputCol("a")
    +    val sizeHintB = new VectorSizeHint()
    +      .setSize(4)
    +      .setInputCol("b")
    +    val vectorAssembler = new VectorAssembler()
    +      .setInputCols(Array("a", "b"))
    +      .setOutputCol("assembled")
    +    val output = Seq(sizeHintA, sizeHintB, vectorAssembler).foldLeft(streamingDF) {
    +      case (data, transform) => transform.transform(data)
    --- End diff --
    
    `case (data, transform)` ==> `case (data, transformer)`


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r155852062
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.Transformer
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, VectorUDT}
    +import org.apache.spark.ml.param.{IntParam, Param, ParamMap, ParamValidators}
    +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol}
    +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable}
    +import org.apache.spark.sql.{Column, DataFrame, Dataset}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.StructType
    +
    +/**
    + * A feature transformer that adds vector size information to a vector column.
    + */
    +@Experimental
    +@Since("2.3.0")
    +class VectorSizeHint @Since("2.3.0") (@Since("2.3.0") override val uid: String)
    +  extends Transformer with HasInputCol with HasHandleInvalid with DefaultParamsWritable {
    +
    +  @Since("2.3.0")
    +  def this() = this(Identifiable.randomUID("vectSizeHint"))
    +
    +  @Since("2.3.0")
    +  val size = new IntParam(this, "size", "Size of vectors in column.", {s: Int => s >= 0})
    +
    +  @Since("2.3.0")
    +  def getSize: Int = getOrDefault(size)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setSize(value: Int): this.type = set(size, value)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  @Since("2.3.0")
    --- End diff --
    
    As long as you're overriding this val, can you please override the docstring and specify the default value here?


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    **[Test build #83918 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83918/testReport)** for PR 19746 at commit [`03bd63c`](https://github.com/apache/spark/commit/03bd63c654cc3f8982ec726c50472759f704918d).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class VectorSizeHint @Since(\"2.3.0\") (@Since(\"2.3.0\") override val uid: String)`
      * `  class InvalidEntryException(msg: String) extends Exception(msg)`


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    **[Test build #85071 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85071/testReport)** for PR 19746 at commit [`d63f077`](https://github.com/apache/spark/commit/d63f07789069a8f643dc41448e5f456233fc3499).
     * 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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r151956112
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.Transformer
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, VectorUDT}
    +import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
    +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol}
    +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable}
    +import org.apache.spark.sql.{Column, DataFrame, Dataset}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.StructType
    +
    +/**
    + * A feature transformer that adds vector size information to a vector column.
    + */
    +@Experimental
    +@Since("2.3.0")
    +class VectorSizeHint @Since("2.3.0") (@Since("2.3.0") override val uid: String)
    +  extends Transformer with HasInputCol with HasHandleInvalid with DefaultParamsWritable {
    +
    +  @Since("2.3.0")
    +  def this() = this(Identifiable.randomUID("vectSizeHint"))
    +
    +  @Since("2.3.0")
    +  val size = new Param[Int](this, "size", "Size of vectors in column.", {s: Int => s >= 0})
    +
    +  @Since("2.3.0")
    +  def getSize: Int = getOrDefault(size)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setSize(value: Int): this.type = set(size, value)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  @Since("2.3.0")
    +  override val handleInvalid: Param[String] = new Param[String](
    +    this,
    +    "handleInvalid",
    +    "How to handle invalid vectors in inputCol, (invalid vectors include nulls and vectors with " +
    +      "the wrong size. The options are `skip` (filter out rows with invalid vectors), `error` " +
    +      "(throw an error) and `optimistic` (don't check the vector size).",
    +    ParamValidators.inArray(VectorSizeHint.supportedHandleInvalids))
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setHandleInvalid(value: String): this.type = set(handleInvalid, value)
    +  setDefault(handleInvalid, VectorSizeHint.ERROR_INVALID)
    +
    +  @Since("2.3.0")
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    val localInputCol = getInputCol
    +    val localSize = getSize
    +    val localHandleInvalid = getHandleInvalid
    +
    +    val group = AttributeGroup.fromStructField(dataset.schema(localInputCol))
    +    if (localHandleInvalid == VectorSizeHint.OPTIMISTIC_INVALID && group.size == localSize) {
    +      dataset.toDF
    +    } else {
    +      val newGroup = if (group.size == localSize) {
    +        // Pass along any existing metadata about vector.
    +        group
    +      } else {
    +        new AttributeGroup(localInputCol, localSize)
    +      }
    +
    +      val newCol: Column = localHandleInvalid match {
    +        case VectorSizeHint.OPTIMISTIC_INVALID => col(localInputCol)
    +        case VectorSizeHint.ERROR_INVALID =>
    +          val checkVectorSize = { vector: Vector =>
    +            if (vector == null) {
    +              throw new VectorSizeHint.InvalidEntryException(s"Got null vector in VectorSizeHint," +
    +                s" set `handleInvalid` to 'skip' to filter invalid rows.")
    +            }
    +            if (vector.size != localSize) {
    +              throw new VectorSizeHint.InvalidEntryException(s"VectorSizeHint Expecting a vector " +
    +                s"of size $localSize but got ${vector.size}")
    +            }
    +            vector
    +          }
    +          udf(checkVectorSize, new VectorUDT)(col(localInputCol))
    +        case VectorSizeHint.SKIP_INVALID =>
    +          val checkVectorSize = { vector: Vector =>
    +            if (vector != null && vector.size == localSize) {
    +              vector
    +            } else {
    +              null
    +            }
    +          }
    +          udf(checkVectorSize, new VectorUDT)(col(localInputCol))
    +      }
    +
    +      val res = dataset.withColumn(localInputCol, newCol.as(localInputCol, newGroup.toMetadata))
    +      if (localHandleInvalid == VectorSizeHint.SKIP_INVALID) {
    +        res.filter(col(localInputCol).isNotNull)
    +      } else {
    +        res
    +      }
    +    }
    +  }
    +
    +  @Since("2.3.0")
    +  override def transformSchema(schema: StructType): StructType = {
    +    val inputColType = schema(getInputCol).dataType
    +    require(
    +      inputColType.isInstanceOf[VectorUDT],
    +      s"Input column, $getInputCol must be of Vector type, got $inputColType"
    +    )
    +    schema
    +  }
    +
    +  @Since("2.3.0")
    +  override def copy(extra: ParamMap): VectorAssembler = defaultCopy(extra)
    +}
    +
    +@Experimental
    +@Since("2.3.0")
    +object VectorSizeHint extends DefaultParamsReadable[VectorSizeHint] {
    +
    +  private[feature] val OPTIMISTIC_INVALID = "optimistic"
    +  private[feature] val ERROR_INVALID = "error"
    +  private[feature] val SKIP_INVALID = "skip"
    +  private[feature] val supportedHandleInvalids: Array[String] =
    +    Array(OPTIMISTIC_INVALID, ERROR_INVALID, SKIP_INVALID)
    +
    +  @Since("2.3.0")
    +  class InvalidEntryException(msg: String) extends Exception(msg)
    --- End diff --
    
    Do we need define a new exception class ? Or directly use `SparkException` ?


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    reviewing now


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r155862447
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala ---
    @@ -0,0 +1,173 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, Vectors}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.execution.streaming.MemoryStream
    +import org.apache.spark.sql.streaming.StreamTest
    +
    +class VectorSizeHintSuite
    +  extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  import testImplicits._
    +
    +  test("Test Param Validators") {
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setHandleInvalid("invalidValue"))
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setSize(-3))
    +  }
    +
    +  test("Adding size to column of vectors.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val denseVector = Vectors.dense(1, 2, 3)
    +    val sparseVector = Vectors.sparse(size, Array(), Array())
    +
    +    val data = Seq(denseVector, denseVector, sparseVector).map(Tuple1.apply)
    +    val dataFrame = data.toDF(vectorColName)
    +    assert(
    +      AttributeGroup.fromStructField(dataFrame.schema(vectorColName)).size == -1,
    +      "Transformer did not add expected size data.")
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrame)
    +      assert(
    +        AttributeGroup.fromStructField(withSize.schema(vectorColName)).size == size,
    +        "Transformer did not add expected size data.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size hint preserves attributes.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +    val group = AttributeGroup.fromStructField(dataFrameWithMetadata.schema(vectorColName))
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrameWithMetadata)
    +
    +      val newGroup = AttributeGroup.fromStructField(withSize.schema(vectorColName))
    +      assert(newGroup.size === size, "Transformer did not add expected size data.")
    +      assert(
    +        newGroup.attributes.get.deep === group.attributes.get.deep,
    --- End diff --
    
    nit: Triple-equals actually calls .deep under the hood


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r152159939
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala ---
    @@ -0,0 +1,185 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, Vectors}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.execution.streaming.MemoryStream
    +import org.apache.spark.sql.streaming.StreamTest
    +
    +class VectorSizeHintSuite
    +  extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  import testImplicits._
    +
    +  test("Test Param Validators") {
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setHandleInvalid("invalidValue"))
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setSize(-3))
    +  }
    +
    +  test("Adding size to column of vectors.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val denseVector = Vectors.dense(1, 2, 3)
    +    val sparseVector = Vectors.sparse(size, Array(), Array())
    +
    +    val data = Seq(denseVector, denseVector, sparseVector).map(Tuple1.apply)
    +    val dataFrame = data.toDF(vectorColName)
    +    assert(
    +      AttributeGroup.fromStructField(dataFrame.schema(vectorColName)).size == -1,
    +      "Transformer did not add expected size data.")
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrame)
    +      assert(
    +        AttributeGroup.fromStructField(withSize.schema(vectorColName)).size == size,
    +        "Transformer did not add expected size data.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size hint preserves attributes.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +    val group = AttributeGroup.fromStructField(dataFrameWithMetadata.schema(vectorColName))
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrameWithMetadata)
    +
    +      val newGroup = AttributeGroup.fromStructField(withSize.schema(vectorColName))
    +      assert(newGroup.size === size, "Transformer did not add expected size data.")
    +      assert(
    +        newGroup.attributes.get.deep === group.attributes.get.deep,
    +        "SizeHintTransformer did not preserve attributes.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size miss-match between current and target size raises an error.") {
    +    val size = 4
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      intercept[SparkException](transformer.transform(dataFrameWithMetadata))
    +    }
    +  }
    +
    +  test("Handle invalid does the right thing.") {
    +
    +    val vector = Vectors.dense(1, 2, 3)
    +    val short = Vectors.dense(2)
    +    val dataWithNull = Seq(vector, null).map(Tuple1.apply).toDF("vector")
    +    val dataWithShort = Seq(vector, short).map(Tuple1.apply).toDF("vector")
    +
    +    val sizeHint = new VectorSizeHint()
    +      .setInputCol("vector")
    +      .setHandleInvalid("error")
    +      .setSize(3)
    +
    +    intercept[SparkException](sizeHint.transform(dataWithNull).collect)
    +    intercept[SparkException](sizeHint.transform(dataWithShort).collect)
    +
    +    sizeHint.setHandleInvalid("skip")
    +    assert(sizeHint.transform(dataWithNull).count() === 1)
    +    assert(sizeHint.transform(dataWithShort).count() === 1)
    +  }
    +
    +  test("Test correct behaviour for handleInvalid == optimistic") {
    +    val vector = Vectors.dense(1, 2, 3)
    +    val data = Seq(vector, vector).map(Tuple1.apply).toDF("vector")
    +
    +    val sizeHint = new VectorSizeHint()
    +      .setInputCol("vector")
    +      .setHandleInvalid("optimistic")
    +      .setSize(3)
    +
    +    val transformed = sizeHint.transform(data)
    --- End diff --
    
    Can I just remove this test? I feel like all of that is tested in the first 3 tests.


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r155863630
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala ---
    @@ -0,0 +1,173 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, Vectors}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.execution.streaming.MemoryStream
    +import org.apache.spark.sql.streaming.StreamTest
    +
    +class VectorSizeHintSuite
    +  extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  import testImplicits._
    +
    +  test("Test Param Validators") {
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setHandleInvalid("invalidValue"))
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setSize(-3))
    +  }
    +
    +  test("Adding size to column of vectors.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val denseVector = Vectors.dense(1, 2, 3)
    +    val sparseVector = Vectors.sparse(size, Array(), Array())
    +
    +    val data = Seq(denseVector, denseVector, sparseVector).map(Tuple1.apply)
    +    val dataFrame = data.toDF(vectorColName)
    +    assert(
    +      AttributeGroup.fromStructField(dataFrame.schema(vectorColName)).size == -1,
    +      "Transformer did not add expected size data.")
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrame)
    +      assert(
    +        AttributeGroup.fromStructField(withSize.schema(vectorColName)).size == size,
    +        "Transformer did not add expected size data.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size hint preserves attributes.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +    val group = AttributeGroup.fromStructField(dataFrameWithMetadata.schema(vectorColName))
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrameWithMetadata)
    +
    +      val newGroup = AttributeGroup.fromStructField(withSize.schema(vectorColName))
    +      assert(newGroup.size === size, "Transformer did not add expected size data.")
    +      assert(
    +        newGroup.attributes.get.deep === group.attributes.get.deep,
    +        "SizeHintTransformer did not preserve attributes.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size miss-match between current and target size raises an error.") {
    +    val size = 4
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      intercept[SparkException](transformer.transform(dataFrameWithMetadata))
    +    }
    +  }
    +
    +  test("Handle invalid does the right thing.") {
    +
    +    val vector = Vectors.dense(1, 2, 3)
    +    val short = Vectors.dense(2)
    +    val dataWithNull = Seq(vector, null).map(Tuple1.apply).toDF("vector")
    +    val dataWithShort = Seq(vector, short).map(Tuple1.apply).toDF("vector")
    +
    +    val sizeHint = new VectorSizeHint()
    +      .setInputCol("vector")
    +      .setHandleInvalid("error")
    +      .setSize(3)
    +
    +    intercept[SparkException](sizeHint.transform(dataWithNull).collect)
    +    intercept[SparkException](sizeHint.transform(dataWithShort).collect)
    +
    +    sizeHint.setHandleInvalid("skip")
    +    assert(sizeHint.transform(dataWithNull).count() === 1)
    +    assert(sizeHint.transform(dataWithShort).count() === 1)
    +  }
    +
    +  test("read/write") {
    +    val sizeHint = new VectorSizeHint()
    +      .setInputCol("myInputCol")
    +      .setSize(11)
    +      .setHandleInvalid("skip")
    +    testDefaultReadWrite(sizeHint)
    +  }
    +}
    +
    +class VectorSizeHintStreamingSuite extends StreamTest {
    +
    +  import testImplicits._
    +
    +  test("Test assemble vectors with size hint in steaming.") {
    --- End diff --
    
    steaming streaming


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r155850183
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.Transformer
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, VectorUDT}
    +import org.apache.spark.ml.param.{IntParam, Param, ParamMap, ParamValidators}
    +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol}
    +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable}
    +import org.apache.spark.sql.{Column, DataFrame, Dataset}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.StructType
    +
    +/**
    + * A feature transformer that adds vector size information to a vector column.
    + */
    +@Experimental
    +@Since("2.3.0")
    +class VectorSizeHint @Since("2.3.0") (@Since("2.3.0") override val uid: String)
    +  extends Transformer with HasInputCol with HasHandleInvalid with DefaultParamsWritable {
    +
    +  @Since("2.3.0")
    +  def this() = this(Identifiable.randomUID("vectSizeHint"))
    +
    +  @Since("2.3.0")
    --- End diff --
    
    Add a docstring and mark with ```@group param```


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r156186135
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.Transformer
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, VectorUDT}
    +import org.apache.spark.ml.param.{IntParam, Param, ParamMap, ParamValidators}
    +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol}
    +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable}
    +import org.apache.spark.sql.{Column, DataFrame, Dataset}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.StructType
    +
    +/**
    + * A feature transformer that adds vector size information to a vector column.
    + */
    +@Experimental
    +@Since("2.3.0")
    +class VectorSizeHint @Since("2.3.0") (@Since("2.3.0") override val uid: String)
    +  extends Transformer with HasInputCol with HasHandleInvalid with DefaultParamsWritable {
    +
    +  @Since("2.3.0")
    +  def this() = this(Identifiable.randomUID("vectSizeHint"))
    +
    +  @Since("2.3.0")
    +  val size = new IntParam(this, "size", "Size of vectors in column.", {s: Int => s >= 0})
    +
    +  @Since("2.3.0")
    +  def getSize: Int = getOrDefault(size)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setSize(value: Int): this.type = set(size, value)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  @Since("2.3.0")
    +  override val handleInvalid: Param[String] = new Param[String](
    +    this,
    +    "handleInvalid",
    +    "How to handle invalid vectors in inputCol, (invalid vectors include nulls and vectors with " +
    +      "the wrong size. The options are `skip` (filter out rows with invalid vectors), `error` " +
    +      "(throw an error) and `optimistic` (don't check the vector size).",
    +    ParamValidators.inArray(VectorSizeHint.supportedHandleInvalids))
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setHandleInvalid(value: String): this.type = set(handleInvalid, value)
    +  setDefault(handleInvalid, VectorSizeHint.ERROR_INVALID)
    +
    +  @Since("2.3.0")
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    val localInputCol = getInputCol
    +    val localSize = getSize
    +    val localHandleInvalid = getHandleInvalid
    +
    +    val group = AttributeGroup.fromStructField(dataset.schema(localInputCol))
    +    if (localHandleInvalid == VectorSizeHint.OPTIMISTIC_INVALID && group.size == localSize) {
    +      dataset.toDF
    +    } else {
    +      val newGroup = group.size match {
    +        case `localSize` => group
    +        case -1 => new AttributeGroup(localInputCol, localSize)
    +        case _ =>
    +          val msg = s"Trying to set size of vectors in `$localInputCol` to $localSize but size " +
    +            s"already set to ${group.size}."
    +          throw new SparkException(msg)
    +      }
    +
    +      val newCol: Column = localHandleInvalid match {
    +        case VectorSizeHint.OPTIMISTIC_INVALID => col(localInputCol)
    +        case VectorSizeHint.ERROR_INVALID =>
    +          val checkVectorSizeUDF = udf { vector: Vector =>
    +            if (vector == null) {
    +              throw new SparkException(s"Got null vector in VectorSizeHint, set `handleInvalid` " +
    +                s"to 'skip' to filter invalid rows.")
    +            }
    +            if (vector.size != localSize) {
    +              throw new SparkException(s"VectorSizeHint Expecting a vector of size $localSize but" +
    +                s" got ${vector.size}")
    +            }
    +            vector
    +          }.asNondeterministic
    +          checkVectorSizeUDF(col(localInputCol))
    +        case VectorSizeHint.SKIP_INVALID =>
    +          val checkVectorSizeUDF = udf { vector: Vector =>
    --- End diff --
    
    Or not?  Maybe the analyzer won't allow you to treat a UDF column as a generic struct.  Scratch this comment.


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    jenkins test


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    What about supporting multiple columns ? VectorAssembler require multiple input columns, they all need VectorSizeHint to transform first. There's no need to use multiple VectorSizeHint transformer.


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

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


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

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


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

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


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    **[Test build #83859 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83859/testReport)** for PR 19746 at commit [`df990ed`](https://github.com/apache/spark/commit/df990ed771d9d773b6a8a1b8751ae8c1c475b3d5).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class VectorSizeHint @Since(\"2.3.0\") (@Since(\"2.3.0\") override val uid: String)`


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r156793561
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.Transformer
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, VectorUDT}
    +import org.apache.spark.ml.param.{IntParam, Param, ParamMap, ParamValidators}
    +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol}
    +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable}
    +import org.apache.spark.sql.{Column, DataFrame, Dataset}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.StructType
    +
    +/**
    + * A feature transformer that adds vector size information to a vector column.
    + */
    +@Experimental
    +@Since("2.3.0")
    +class VectorSizeHint @Since("2.3.0") (@Since("2.3.0") override val uid: String)
    +  extends Transformer with HasInputCol with HasHandleInvalid with DefaultParamsWritable {
    +
    +  @Since("2.3.0")
    +  def this() = this(Identifiable.randomUID("vectSizeHint"))
    +
    +  @Since("2.3.0")
    +  val size = new IntParam(this, "size", "Size of vectors in column.", {s: Int => s >= 0})
    +
    +  @Since("2.3.0")
    +  def getSize: Int = getOrDefault(size)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setSize(value: Int): this.type = set(size, value)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  @Since("2.3.0")
    +  override val handleInvalid: Param[String] = new Param[String](
    +    this,
    +    "handleInvalid",
    +    "How to handle invalid vectors in inputCol, (invalid vectors include nulls and vectors with " +
    +      "the wrong size. The options are `skip` (filter out rows with invalid vectors), `error` " +
    +      "(throw an error) and `optimistic` (don't check the vector size).",
    +    ParamValidators.inArray(VectorSizeHint.supportedHandleInvalids))
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setHandleInvalid(value: String): this.type = set(handleInvalid, value)
    +  setDefault(handleInvalid, VectorSizeHint.ERROR_INVALID)
    +
    +  @Since("2.3.0")
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    val localInputCol = getInputCol
    +    val localSize = getSize
    +    val localHandleInvalid = getHandleInvalid
    +
    +    val group = AttributeGroup.fromStructField(dataset.schema(localInputCol))
    +    if (localHandleInvalid == VectorSizeHint.OPTIMISTIC_INVALID && group.size == localSize) {
    +      dataset.toDF
    +    } else {
    +      val newGroup = group.size match {
    +        case `localSize` => group
    +        case -1 => new AttributeGroup(localInputCol, localSize)
    +        case _ =>
    +          val msg = s"Trying to set size of vectors in `$localInputCol` to $localSize but size " +
    +            s"already set to ${group.size}."
    +          throw new SparkException(msg)
    +      }
    +
    +      val newCol: Column = localHandleInvalid match {
    +        case VectorSizeHint.OPTIMISTIC_INVALID => col(localInputCol)
    +        case VectorSizeHint.ERROR_INVALID =>
    +          val checkVectorSizeUDF = udf { vector: Vector =>
    +            if (vector == null) {
    +              throw new SparkException(s"Got null vector in VectorSizeHint, set `handleInvalid` " +
    +                s"to 'skip' to filter invalid rows.")
    +            }
    +            if (vector.size != localSize) {
    +              throw new SparkException(s"VectorSizeHint Expecting a vector of size $localSize but" +
    +                s" got ${vector.size}")
    +            }
    +            vector
    +          }.asNondeterministic
    +          checkVectorSizeUDF(col(localInputCol))
    +        case VectorSizeHint.SKIP_INVALID =>
    +          val checkVectorSizeUDF = udf { vector: Vector =>
    +            if (vector != null && vector.size == localSize) {
    +              vector
    +            } else {
    +              null
    +            }
    +          }
    +          checkVectorSizeUDF(col(localInputCol))
    +      }
    +
    +      val res = dataset.withColumn(localInputCol, newCol.as(localInputCol, newGroup.toMetadata))
    +      if (localHandleInvalid == VectorSizeHint.SKIP_INVALID) {
    +        res.na.drop(Array(localInputCol))
    +      } else {
    +        res
    +      }
    +    }
    +  }
    +
    +  @Since("2.3.0")
    +  override def transformSchema(schema: StructType): StructType = {
    +    val inputColType = schema(getInputCol).dataType
    +    require(
    +      inputColType.isInstanceOf[VectorUDT],
    +      s"Input column, $getInputCol must be of Vector type, got $inputColType"
    +    )
    +    schema
    +  }
    +
    +  @Since("2.3.0")
    +  override def copy(extra: ParamMap): this.type = defaultCopy(extra)
    +}
    +
    +@Experimental
    +@Since("2.3.0")
    +object VectorSizeHint extends DefaultParamsReadable[VectorSizeHint] {
    +
    +  private[feature] val OPTIMISTIC_INVALID = "optimistic"
    --- End diff --
    
    OK that's a great argument.  Let's keep it "optimistic."


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r156259691
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.Transformer
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, VectorUDT}
    +import org.apache.spark.ml.param.{IntParam, Param, ParamMap, ParamValidators}
    +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol}
    +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable}
    +import org.apache.spark.sql.{Column, DataFrame, Dataset}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.StructType
    +
    +/**
    + * A feature transformer that adds vector size information to a vector column.
    + */
    +@Experimental
    +@Since("2.3.0")
    +class VectorSizeHint @Since("2.3.0") (@Since("2.3.0") override val uid: String)
    +  extends Transformer with HasInputCol with HasHandleInvalid with DefaultParamsWritable {
    +
    +  @Since("2.3.0")
    +  def this() = this(Identifiable.randomUID("vectSizeHint"))
    +
    +  @Since("2.3.0")
    +  val size = new IntParam(this, "size", "Size of vectors in column.", {s: Int => s >= 0})
    +
    +  @Since("2.3.0")
    +  def getSize: Int = getOrDefault(size)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setSize(value: Int): this.type = set(size, value)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  @Since("2.3.0")
    +  override val handleInvalid: Param[String] = new Param[String](
    +    this,
    +    "handleInvalid",
    +    "How to handle invalid vectors in inputCol, (invalid vectors include nulls and vectors with " +
    +      "the wrong size. The options are `skip` (filter out rows with invalid vectors), `error` " +
    +      "(throw an error) and `optimistic` (don't check the vector size).",
    +    ParamValidators.inArray(VectorSizeHint.supportedHandleInvalids))
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setHandleInvalid(value: String): this.type = set(handleInvalid, value)
    +  setDefault(handleInvalid, VectorSizeHint.ERROR_INVALID)
    +
    +  @Since("2.3.0")
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    val localInputCol = getInputCol
    +    val localSize = getSize
    +    val localHandleInvalid = getHandleInvalid
    +
    +    val group = AttributeGroup.fromStructField(dataset.schema(localInputCol))
    +    if (localHandleInvalid == VectorSizeHint.OPTIMISTIC_INVALID && group.size == localSize) {
    +      dataset.toDF
    +    } else {
    +      val newGroup = group.size match {
    +        case `localSize` => group
    +        case -1 => new AttributeGroup(localInputCol, localSize)
    +        case _ =>
    +          val msg = s"Trying to set size of vectors in `$localInputCol` to $localSize but size " +
    +            s"already set to ${group.size}."
    +          throw new SparkException(msg)
    +      }
    +
    +      val newCol: Column = localHandleInvalid match {
    +        case VectorSizeHint.OPTIMISTIC_INVALID => col(localInputCol)
    +        case VectorSizeHint.ERROR_INVALID =>
    +          val checkVectorSizeUDF = udf { vector: Vector =>
    +            if (vector == null) {
    +              throw new SparkException(s"Got null vector in VectorSizeHint, set `handleInvalid` " +
    +                s"to 'skip' to filter invalid rows.")
    +            }
    +            if (vector.size != localSize) {
    +              throw new SparkException(s"VectorSizeHint Expecting a vector of size $localSize but" +
    +                s" got ${vector.size}")
    +            }
    +            vector
    +          }.asNondeterministic
    +          checkVectorSizeUDF(col(localInputCol))
    +        case VectorSizeHint.SKIP_INVALID =>
    +          val checkVectorSizeUDF = udf { vector: Vector =>
    --- End diff --
    
    internally UDT column is stored as `UserDefinedType.sqlType`, so if your UDT is mapped to sql struct type, we can use it as struct type column via pure SQL/DataFrame operations.


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r155862816
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala ---
    @@ -0,0 +1,173 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, Vectors}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.execution.streaming.MemoryStream
    +import org.apache.spark.sql.streaming.StreamTest
    +
    +class VectorSizeHintSuite
    +  extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  import testImplicits._
    +
    +  test("Test Param Validators") {
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setHandleInvalid("invalidValue"))
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setSize(-3))
    +  }
    +
    +  test("Adding size to column of vectors.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val denseVector = Vectors.dense(1, 2, 3)
    +    val sparseVector = Vectors.sparse(size, Array(), Array())
    +
    +    val data = Seq(denseVector, denseVector, sparseVector).map(Tuple1.apply)
    +    val dataFrame = data.toDF(vectorColName)
    +    assert(
    +      AttributeGroup.fromStructField(dataFrame.schema(vectorColName)).size == -1,
    +      "Transformer did not add expected size data.")
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrame)
    +      assert(
    +        AttributeGroup.fromStructField(withSize.schema(vectorColName)).size == size,
    +        "Transformer did not add expected size data.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size hint preserves attributes.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +    val group = AttributeGroup.fromStructField(dataFrameWithMetadata.schema(vectorColName))
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrameWithMetadata)
    +
    +      val newGroup = AttributeGroup.fromStructField(withSize.schema(vectorColName))
    +      assert(newGroup.size === size, "Transformer did not add expected size data.")
    +      assert(
    +        newGroup.attributes.get.deep === group.attributes.get.deep,
    +        "SizeHintTransformer did not preserve attributes.")
    --- End diff --
    
    SizeHintTransformer -> VectorSizeHint


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    **[Test build #84449 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84449/testReport)** for PR 19746 at commit [`0837b76`](https://github.com/apache/spark/commit/0837b760530a217e5fc8ea694edc8e737c3f34a4).


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r155861030
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala ---
    @@ -0,0 +1,173 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, Vectors}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.execution.streaming.MemoryStream
    +import org.apache.spark.sql.streaming.StreamTest
    +
    +class VectorSizeHintSuite
    +  extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  import testImplicits._
    +
    +  test("Test Param Validators") {
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setHandleInvalid("invalidValue"))
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setSize(-3))
    +  }
    +
    +  test("Adding size to column of vectors.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val denseVector = Vectors.dense(1, 2, 3)
    +    val sparseVector = Vectors.sparse(size, Array(), Array())
    +
    +    val data = Seq(denseVector, denseVector, sparseVector).map(Tuple1.apply)
    +    val dataFrame = data.toDF(vectorColName)
    +    assert(
    +      AttributeGroup.fromStructField(dataFrame.schema(vectorColName)).size == -1,
    +      "Transformer did not add expected size data.")
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrame)
    +      assert(
    +        AttributeGroup.fromStructField(withSize.schema(vectorColName)).size == size,
    +        "Transformer did not add expected size data.")
    +      withSize.collect
    --- End diff --
    
    Might as well assert ```withSize.collect().length === 3``` to make sure no Rows were incorrectly filtered


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r155851617
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.Transformer
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, VectorUDT}
    +import org.apache.spark.ml.param.{IntParam, Param, ParamMap, ParamValidators}
    +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol}
    +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable}
    +import org.apache.spark.sql.{Column, DataFrame, Dataset}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.StructType
    +
    +/**
    + * A feature transformer that adds vector size information to a vector column.
    + */
    +@Experimental
    +@Since("2.3.0")
    +class VectorSizeHint @Since("2.3.0") (@Since("2.3.0") override val uid: String)
    +  extends Transformer with HasInputCol with HasHandleInvalid with DefaultParamsWritable {
    +
    +  @Since("2.3.0")
    +  def this() = this(Identifiable.randomUID("vectSizeHint"))
    +
    +  @Since("2.3.0")
    +  val size = new IntParam(this, "size", "Size of vectors in column.", {s: Int => s >= 0})
    +
    +  @Since("2.3.0")
    +  def getSize: Int = getOrDefault(size)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setSize(value: Int): this.type = set(size, value)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  @Since("2.3.0")
    +  override val handleInvalid: Param[String] = new Param[String](
    +    this,
    +    "handleInvalid",
    +    "How to handle invalid vectors in inputCol, (invalid vectors include nulls and vectors with " +
    --- End diff --
    
    The writing here is formatted strangely.  How about:
    "How to handle invalid vectors in inputCol.  Invalid vectors include nulls and vectors with the wrong size. The options are `skip` (filter out rows with invalid vectors), `error` (throw an error) and `keep` (do not check the vector size, and keep all rows)."


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML]

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

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


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r154815581
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala ---
    @@ -0,0 +1,173 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, Vectors}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.execution.streaming.MemoryStream
    +import org.apache.spark.sql.streaming.StreamTest
    +
    +class VectorSizeHintSuite
    +  extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  import testImplicits._
    +
    +  test("Test Param Validators") {
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setHandleInvalid("invalidValue"))
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setSize(-3))
    +  }
    +
    +  test("Adding size to column of vectors.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val denseVector = Vectors.dense(1, 2, 3)
    +    val sparseVector = Vectors.sparse(size, Array(), Array())
    +
    +    val data = Seq(denseVector, denseVector, sparseVector).map(Tuple1.apply)
    +    val dataFrame = data.toDF(vectorColName)
    +    assert(
    +      AttributeGroup.fromStructField(dataFrame.schema(vectorColName)).size == -1,
    +      "Transformer did not add expected size data.")
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrame)
    +      assert(
    +        AttributeGroup.fromStructField(withSize.schema(vectorColName)).size == size,
    +        "Transformer did not add expected size data.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size hint preserves attributes.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +    val group = AttributeGroup.fromStructField(dataFrameWithMetadata.schema(vectorColName))
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrameWithMetadata)
    +
    +      val newGroup = AttributeGroup.fromStructField(withSize.schema(vectorColName))
    +      assert(newGroup.size === size, "Transformer did not add expected size data.")
    +      assert(
    +        newGroup.attributes.get.deep === group.attributes.get.deep,
    +        "SizeHintTransformer did not preserve attributes.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size miss-match between current and target size raises an error.") {
    +    val size = 4
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      intercept[SparkException](transformer.transform(dataFrameWithMetadata))
    +    }
    +  }
    +
    +  test("Handle invalid does the right thing.") {
    +
    +    val vector = Vectors.dense(1, 2, 3)
    +    val short = Vectors.dense(2)
    +    val dataWithNull = Seq(vector, null).map(Tuple1.apply).toDF("vector")
    +    val dataWithShort = Seq(vector, short).map(Tuple1.apply).toDF("vector")
    +
    +    val sizeHint = new VectorSizeHint()
    +      .setInputCol("vector")
    +      .setHandleInvalid("error")
    +      .setSize(3)
    +
    +    intercept[SparkException](sizeHint.transform(dataWithNull).collect)
    +    intercept[SparkException](sizeHint.transform(dataWithShort).collect)
    +
    +    sizeHint.setHandleInvalid("skip")
    +    assert(sizeHint.transform(dataWithNull).count() === 1)
    +    assert(sizeHint.transform(dataWithShort).count() === 1)
    +  }
    +
    +  test("read/write") {
    +    val sizeHint = new VectorSizeHint()
    +      .setInputCol("myInputCol")
    +      .setSize(11)
    +      .setHandleInvalid("skip")
    +    testDefaultReadWrite(sizeHint)
    +  }
    +}
    +
    +class VectorSizeHintStreamingSuite extends StreamTest {
    +
    +  import testImplicits._
    +
    +  test("Test assemble vectors with size hint in steaming.") {
    +    val a = Vectors.dense(0, 1, 2)
    +    val b = Vectors.sparse(4, Array(0, 3), Array(3, 6))
    +
    +    val stream = MemoryStream[(Vector, Vector)]
    +    val streamingDF = stream.toDS.toDF("a", "b")
    +    val sizeHintA = new VectorSizeHint()
    +      .setSize(3)
    +      .setInputCol("a")
    +    val sizeHintB = new VectorSizeHint()
    +      .setSize(4)
    +      .setInputCol("b")
    +    val vectorAssembler = new VectorAssembler()
    +      .setInputCols(Array("a", "b"))
    +      .setOutputCol("assembled")
    +    val output = Seq(sizeHintA, sizeHintB, vectorAssembler).foldLeft(streamingDF) {
    +      case (data, transform) => transform.transform(data)
    +    }.select("assembled")
    +
    +    val expected = Vectors.dense(0, 1, 2, 3, 0, 0, 6)
    +
    +    testStream (output) (
    +      AddData(stream, (a, b), (a, b)),
    +      CheckAnswerRows(Seq(Row(expected), Row(expected)), false, false)
    --- End diff --
    
    The reason I didn't use `CheckAnswer` is because there isn't an implicit encoder in `testImplicits` that handles `Vector`. I tried `CheckAnswer[Vector](expected, expected)` but that doesn't work either :(. Is there an encoder that works for Vectors?


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r156151486
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.Transformer
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, VectorUDT}
    +import org.apache.spark.ml.param.{IntParam, Param, ParamMap, ParamValidators}
    +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol}
    +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable}
    +import org.apache.spark.sql.{Column, DataFrame, Dataset}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.StructType
    +
    +/**
    + * A feature transformer that adds vector size information to a vector column.
    + */
    +@Experimental
    +@Since("2.3.0")
    +class VectorSizeHint @Since("2.3.0") (@Since("2.3.0") override val uid: String)
    +  extends Transformer with HasInputCol with HasHandleInvalid with DefaultParamsWritable {
    +
    +  @Since("2.3.0")
    +  def this() = this(Identifiable.randomUID("vectSizeHint"))
    +
    +  @Since("2.3.0")
    +  val size = new IntParam(this, "size", "Size of vectors in column.", {s: Int => s >= 0})
    +
    +  @Since("2.3.0")
    +  def getSize: Int = getOrDefault(size)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setSize(value: Int): this.type = set(size, value)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  @Since("2.3.0")
    +  override val handleInvalid: Param[String] = new Param[String](
    +    this,
    +    "handleInvalid",
    +    "How to handle invalid vectors in inputCol, (invalid vectors include nulls and vectors with " +
    +      "the wrong size. The options are `skip` (filter out rows with invalid vectors), `error` " +
    +      "(throw an error) and `optimistic` (don't check the vector size).",
    +    ParamValidators.inArray(VectorSizeHint.supportedHandleInvalids))
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setHandleInvalid(value: String): this.type = set(handleInvalid, value)
    +  setDefault(handleInvalid, VectorSizeHint.ERROR_INVALID)
    +
    +  @Since("2.3.0")
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    val localInputCol = getInputCol
    +    val localSize = getSize
    +    val localHandleInvalid = getHandleInvalid
    +
    +    val group = AttributeGroup.fromStructField(dataset.schema(localInputCol))
    +    if (localHandleInvalid == VectorSizeHint.OPTIMISTIC_INVALID && group.size == localSize) {
    +      dataset.toDF
    +    } else {
    +      val newGroup = group.size match {
    +        case `localSize` => group
    +        case -1 => new AttributeGroup(localInputCol, localSize)
    +        case _ =>
    +          val msg = s"Trying to set size of vectors in `$localInputCol` to $localSize but size " +
    +            s"already set to ${group.size}."
    +          throw new SparkException(msg)
    +      }
    +
    +      val newCol: Column = localHandleInvalid match {
    +        case VectorSizeHint.OPTIMISTIC_INVALID => col(localInputCol)
    +        case VectorSizeHint.ERROR_INVALID =>
    +          val checkVectorSizeUDF = udf { vector: Vector =>
    +            if (vector == null) {
    +              throw new SparkException(s"Got null vector in VectorSizeHint, set `handleInvalid` " +
    +                s"to 'skip' to filter invalid rows.")
    +            }
    +            if (vector.size != localSize) {
    +              throw new SparkException(s"VectorSizeHint Expecting a vector of size $localSize but" +
    +                s" got ${vector.size}")
    +            }
    +            vector
    +          }.asNondeterministic
    +          checkVectorSizeUDF(col(localInputCol))
    +        case VectorSizeHint.SKIP_INVALID =>
    +          val checkVectorSizeUDF = udf { vector: Vector =>
    --- End diff --
    
    Something like:
    ```
    val isSparse = col(localInputCol)(0) === lit(0)
    val sparseSize = col(localInputCol)(1)
    val denseSize = size(col(localInputCol)(3))
    val vecSize = when(isSparse, sparseSize).otherwise(denseSize)
    val sizeMatches = vecSize === lit(localSize)
    when(col(localInputCol).isNotNull && sizeMatches,
      col(localInputCol),
      lit(null))
    ```
    
    That should be 90% correct I think  : )


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r152111084
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala ---
    @@ -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.
    + */
    +
    +package org.apache.spark.ml.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, Vectors}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.execution.streaming.MemoryStream
    +import org.apache.spark.sql.streaming.StreamTest
    +
    +class VectorSizeHintSuite
    +  extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  import testImplicits._
    +
    +  test("Test Param Validators") {
    +    assertThrows[IllegalArgumentException] (new VectorSizeHint().setHandleInvalid("invalidValue"))
    +    assertThrows[IllegalArgumentException] (new VectorSizeHint().setSize(-3))
    +  }
    +
    +  test("Adding size to column of vectors.") {
    +
    +    val size = 3
    +    val denseVector = Vectors.dense(1, 2, 3)
    +    val sparseVector = Vectors.sparse(size, Array(), Array())
    +
    +    val data = Seq(denseVector, denseVector, sparseVector).map(Tuple1.apply)
    +    val dataFrame = data.toDF("vector")
    +
    +    val transformer = new VectorSizeHint()
    +      .setInputCol("vector")
    +      .setSize(3)
    +      .setHandleInvalid("error")
    +    val withSize = transformer.transform(dataFrame)
    +    assert(
    +      AttributeGroup.fromStructField(withSize.schema("vector")).size == size,
    +      "Transformer did not add expected size data.")
    +  }
    +
    +  test("Size hint preserves attributes.") {
    +
    +    case class Foo(x: Double, y: Double, z: Double)
    +    val size = 3
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val boo = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol("vector")
    +    val dataFrameWithMeatadata = assembler.transform(boo)
    +    val group = AttributeGroup.fromStructField(dataFrameWithMeatadata.schema("vector"))
    +
    +    val transformer = new VectorSizeHint()
    +      .setInputCol("vector")
    +      .setSize(3)
    +      .setHandleInvalid("error")
    +    val withSize = transformer.transform(dataFrameWithMeatadata)
    +
    +    val newGroup = AttributeGroup.fromStructField(withSize.schema("vector"))
    +    assert(newGroup.size == size, "Transformer did not add expected size data.")
    +    assert(
    +      newGroup.attributes.get.deep === group.attributes.get.deep,
    +      "SizeHintTransformer did not preserve attributes.")
    +  }
    +
    +  test("Handle invalid does the right thing.") {
    --- End diff --
    
    I talked offline to @jkbradley and I think it's better to throw an exception unless if the column includes metadata & the there is a mismatch between the new and original size.
    
    I've added a new test for this exception and made sure the other tests are run with all `handleInvalid` cases. Does it look ok now?


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

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


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    LGTM
    Merged to master
    Thanks @MrBago and @WeichenXu123 !


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r156793824
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala ---
    @@ -0,0 +1,173 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, Vectors}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.execution.streaming.MemoryStream
    +import org.apache.spark.sql.streaming.StreamTest
    +
    +class VectorSizeHintSuite
    +  extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  import testImplicits._
    +
    +  test("Test Param Validators") {
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setHandleInvalid("invalidValue"))
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setSize(-3))
    +  }
    +
    +  test("Adding size to column of vectors.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val denseVector = Vectors.dense(1, 2, 3)
    +    val sparseVector = Vectors.sparse(size, Array(), Array())
    +
    +    val data = Seq(denseVector, denseVector, sparseVector).map(Tuple1.apply)
    +    val dataFrame = data.toDF(vectorColName)
    +    assert(
    +      AttributeGroup.fromStructField(dataFrame.schema(vectorColName)).size == -1,
    +      "Transformer did not add expected size data.")
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrame)
    +      assert(
    +        AttributeGroup.fromStructField(withSize.schema(vectorColName)).size == size,
    +        "Transformer did not add expected size data.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size hint preserves attributes.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +    val group = AttributeGroup.fromStructField(dataFrameWithMetadata.schema(vectorColName))
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrameWithMetadata)
    +
    +      val newGroup = AttributeGroup.fromStructField(withSize.schema(vectorColName))
    +      assert(newGroup.size === size, "Transformer did not add expected size data.")
    +      assert(
    +        newGroup.attributes.get.deep === group.attributes.get.deep,
    +        "SizeHintTransformer did not preserve attributes.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size miss-match between current and target size raises an error.") {
    +    val size = 4
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      intercept[SparkException](transformer.transform(dataFrameWithMetadata))
    +    }
    +  }
    +
    +  test("Handle invalid does the right thing.") {
    +
    +    val vector = Vectors.dense(1, 2, 3)
    +    val short = Vectors.dense(2)
    +    val dataWithNull = Seq(vector, null).map(Tuple1.apply).toDF("vector")
    +    val dataWithShort = Seq(vector, short).map(Tuple1.apply).toDF("vector")
    +
    +    val sizeHint = new VectorSizeHint()
    +      .setInputCol("vector")
    +      .setHandleInvalid("error")
    +      .setSize(3)
    +
    +    intercept[SparkException](sizeHint.transform(dataWithNull).collect)
    +    intercept[SparkException](sizeHint.transform(dataWithShort).collect)
    +
    +    sizeHint.setHandleInvalid("skip")
    +    assert(sizeHint.transform(dataWithNull).count() === 1)
    +    assert(sizeHint.transform(dataWithShort).count() === 1)
    +  }
    --- End diff --
    
    Yep, that's what I had in mind.  That is the expected behavior, so we can test that behavior...even if it's not what most use cases would need.


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r155853763
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.Transformer
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, VectorUDT}
    +import org.apache.spark.ml.param.{IntParam, Param, ParamMap, ParamValidators}
    +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol}
    +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable}
    +import org.apache.spark.sql.{Column, DataFrame, Dataset}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.StructType
    +
    +/**
    + * A feature transformer that adds vector size information to a vector column.
    + */
    +@Experimental
    +@Since("2.3.0")
    +class VectorSizeHint @Since("2.3.0") (@Since("2.3.0") override val uid: String)
    +  extends Transformer with HasInputCol with HasHandleInvalid with DefaultParamsWritable {
    +
    +  @Since("2.3.0")
    +  def this() = this(Identifiable.randomUID("vectSizeHint"))
    +
    +  @Since("2.3.0")
    +  val size = new IntParam(this, "size", "Size of vectors in column.", {s: Int => s >= 0})
    +
    +  @Since("2.3.0")
    +  def getSize: Int = getOrDefault(size)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setSize(value: Int): this.type = set(size, value)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  @Since("2.3.0")
    +  override val handleInvalid: Param[String] = new Param[String](
    +    this,
    +    "handleInvalid",
    +    "How to handle invalid vectors in inputCol, (invalid vectors include nulls and vectors with " +
    +      "the wrong size. The options are `skip` (filter out rows with invalid vectors), `error` " +
    +      "(throw an error) and `optimistic` (don't check the vector size).",
    +    ParamValidators.inArray(VectorSizeHint.supportedHandleInvalids))
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setHandleInvalid(value: String): this.type = set(handleInvalid, value)
    +  setDefault(handleInvalid, VectorSizeHint.ERROR_INVALID)
    +
    +  @Since("2.3.0")
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    val localInputCol = getInputCol
    +    val localSize = getSize
    +    val localHandleInvalid = getHandleInvalid
    +
    +    val group = AttributeGroup.fromStructField(dataset.schema(localInputCol))
    +    if (localHandleInvalid == VectorSizeHint.OPTIMISTIC_INVALID && group.size == localSize) {
    +      dataset.toDF
    --- End diff --
    
    note IntelliJ style warning: Call toDF() with parentheses.


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    **[Test build #84046 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84046/testReport)** for PR 19746 at commit [`2b1ed0a`](https://github.com/apache/spark/commit/2b1ed0a3a85385f9b4042415889335942b65b9c9).


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r152111218
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala ---
    @@ -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.
    + */
    +
    +package org.apache.spark.ml.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, Vectors}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.execution.streaming.MemoryStream
    +import org.apache.spark.sql.streaming.StreamTest
    +
    +class VectorSizeHintSuite
    +  extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  import testImplicits._
    +
    +  test("Test Param Validators") {
    +    assertThrows[IllegalArgumentException] (new VectorSizeHint().setHandleInvalid("invalidValue"))
    +    assertThrows[IllegalArgumentException] (new VectorSizeHint().setSize(-3))
    +  }
    +
    +  test("Adding size to column of vectors.") {
    +
    +    val size = 3
    +    val denseVector = Vectors.dense(1, 2, 3)
    +    val sparseVector = Vectors.sparse(size, Array(), Array())
    +
    +    val data = Seq(denseVector, denseVector, sparseVector).map(Tuple1.apply)
    +    val dataFrame = data.toDF("vector")
    +
    +    val transformer = new VectorSizeHint()
    +      .setInputCol("vector")
    +      .setSize(3)
    +      .setHandleInvalid("error")
    +    val withSize = transformer.transform(dataFrame)
    +    assert(
    +      AttributeGroup.fromStructField(withSize.schema("vector")).size == size,
    +      "Transformer did not add expected size data.")
    +  }
    +
    +  test("Size hint preserves attributes.") {
    +
    +    case class Foo(x: Double, y: Double, z: Double)
    +    val size = 3
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val boo = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol("vector")
    +    val dataFrameWithMeatadata = assembler.transform(boo)
    +    val group = AttributeGroup.fromStructField(dataFrameWithMeatadata.schema("vector"))
    +
    +    val transformer = new VectorSizeHint()
    +      .setInputCol("vector")
    +      .setSize(3)
    +      .setHandleInvalid("error")
    +    val withSize = transformer.transform(dataFrameWithMeatadata)
    +
    +    val newGroup = AttributeGroup.fromStructField(withSize.schema("vector"))
    +    assert(newGroup.size == size, "Transformer did not add expected size data.")
    +    assert(
    +      newGroup.attributes.get.deep === group.attributes.get.deep,
    +      "SizeHintTransformer did not preserve attributes.")
    +  }
    +
    +  test("Handle invalid does the right thing.") {
    +
    +    val vector = Vectors.dense(1, 2, 3)
    +    val short = Vectors.dense(2)
    +    val dataWithNull = Seq(vector, null).map(Tuple1.apply).toDF("vector")
    +    val dataWithShort = Seq(vector, short).map(Tuple1.apply).toDF("vector")
    +
    +    val sizeHint = new VectorSizeHint()
    +      .setInputCol("vector")
    +      .setHandleInvalid("error")
    +      .setSize(3)
    +
    +    assertThrows[SparkException](sizeHint.transform(dataWithNull).collect)
    --- End diff --
    
    I've made the change. Just out of curiosity, why is `intercept` better than `assertThrows`?


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r155854403
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.Transformer
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, VectorUDT}
    +import org.apache.spark.ml.param.{IntParam, Param, ParamMap, ParamValidators}
    +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol}
    +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable}
    +import org.apache.spark.sql.{Column, DataFrame, Dataset}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.StructType
    +
    +/**
    + * A feature transformer that adds vector size information to a vector column.
    + */
    +@Experimental
    +@Since("2.3.0")
    +class VectorSizeHint @Since("2.3.0") (@Since("2.3.0") override val uid: String)
    +  extends Transformer with HasInputCol with HasHandleInvalid with DefaultParamsWritable {
    +
    +  @Since("2.3.0")
    +  def this() = this(Identifiable.randomUID("vectSizeHint"))
    +
    +  @Since("2.3.0")
    +  val size = new IntParam(this, "size", "Size of vectors in column.", {s: Int => s >= 0})
    +
    +  @Since("2.3.0")
    +  def getSize: Int = getOrDefault(size)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setSize(value: Int): this.type = set(size, value)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  @Since("2.3.0")
    +  override val handleInvalid: Param[String] = new Param[String](
    +    this,
    +    "handleInvalid",
    +    "How to handle invalid vectors in inputCol, (invalid vectors include nulls and vectors with " +
    +      "the wrong size. The options are `skip` (filter out rows with invalid vectors), `error` " +
    +      "(throw an error) and `optimistic` (don't check the vector size).",
    +    ParamValidators.inArray(VectorSizeHint.supportedHandleInvalids))
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setHandleInvalid(value: String): this.type = set(handleInvalid, value)
    +  setDefault(handleInvalid, VectorSizeHint.ERROR_INVALID)
    +
    +  @Since("2.3.0")
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    val localInputCol = getInputCol
    +    val localSize = getSize
    +    val localHandleInvalid = getHandleInvalid
    +
    +    val group = AttributeGroup.fromStructField(dataset.schema(localInputCol))
    +    if (localHandleInvalid == VectorSizeHint.OPTIMISTIC_INVALID && group.size == localSize) {
    +      dataset.toDF
    +    } else {
    +      val newGroup = group.size match {
    +        case `localSize` => group
    +        case -1 => new AttributeGroup(localInputCol, localSize)
    +        case _ =>
    +          val msg = s"Trying to set size of vectors in `$localInputCol` to $localSize but size " +
    +            s"already set to ${group.size}."
    +          throw new SparkException(msg)
    --- End diff --
    
    Usually, SparkException is used for exceptions within tasks.  I'd use IllegalArgumentException.


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

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


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML]

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

    https://github.com/apache/spark/pull/19746
  
    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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/83869/
    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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r156279322
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.Transformer
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, VectorUDT}
    +import org.apache.spark.ml.param.{IntParam, Param, ParamMap, ParamValidators}
    +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol}
    +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable}
    +import org.apache.spark.sql.{Column, DataFrame, Dataset}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.StructType
    +
    +/**
    + * A feature transformer that adds vector size information to a vector column.
    + */
    +@Experimental
    +@Since("2.3.0")
    +class VectorSizeHint @Since("2.3.0") (@Since("2.3.0") override val uid: String)
    +  extends Transformer with HasInputCol with HasHandleInvalid with DefaultParamsWritable {
    +
    +  @Since("2.3.0")
    +  def this() = this(Identifiable.randomUID("vectSizeHint"))
    +
    +  @Since("2.3.0")
    +  val size = new IntParam(this, "size", "Size of vectors in column.", {s: Int => s >= 0})
    +
    +  @Since("2.3.0")
    +  def getSize: Int = getOrDefault(size)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setSize(value: Int): this.type = set(size, value)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  @Since("2.3.0")
    +  override val handleInvalid: Param[String] = new Param[String](
    +    this,
    +    "handleInvalid",
    +    "How to handle invalid vectors in inputCol, (invalid vectors include nulls and vectors with " +
    +      "the wrong size. The options are `skip` (filter out rows with invalid vectors), `error` " +
    +      "(throw an error) and `optimistic` (don't check the vector size).",
    +    ParamValidators.inArray(VectorSizeHint.supportedHandleInvalids))
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setHandleInvalid(value: String): this.type = set(handleInvalid, value)
    +  setDefault(handleInvalid, VectorSizeHint.ERROR_INVALID)
    +
    +  @Since("2.3.0")
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    val localInputCol = getInputCol
    +    val localSize = getSize
    +    val localHandleInvalid = getHandleInvalid
    +
    +    val group = AttributeGroup.fromStructField(dataset.schema(localInputCol))
    +    if (localHandleInvalid == VectorSizeHint.OPTIMISTIC_INVALID && group.size == localSize) {
    +      dataset.toDF
    +    } else {
    +      val newGroup = group.size match {
    +        case `localSize` => group
    +        case -1 => new AttributeGroup(localInputCol, localSize)
    +        case _ =>
    +          val msg = s"Trying to set size of vectors in `$localInputCol` to $localSize but size " +
    +            s"already set to ${group.size}."
    +          throw new SparkException(msg)
    +      }
    +
    +      val newCol: Column = localHandleInvalid match {
    +        case VectorSizeHint.OPTIMISTIC_INVALID => col(localInputCol)
    +        case VectorSizeHint.ERROR_INVALID =>
    +          val checkVectorSizeUDF = udf { vector: Vector =>
    +            if (vector == null) {
    +              throw new SparkException(s"Got null vector in VectorSizeHint, set `handleInvalid` " +
    +                s"to 'skip' to filter invalid rows.")
    +            }
    +            if (vector.size != localSize) {
    +              throw new SparkException(s"VectorSizeHint Expecting a vector of size $localSize but" +
    +                s" got ${vector.size}")
    +            }
    +            vector
    +          }.asNondeterministic
    +          checkVectorSizeUDF(col(localInputCol))
    +        case VectorSizeHint.SKIP_INVALID =>
    +          val checkVectorSizeUDF = udf { vector: Vector =>
    --- End diff --
    
    I feel this is a bug, let me look into it


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    **[Test build #83869 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83869/testReport)** for PR 19746 at commit [`73fe1d8`](https://github.com/apache/spark/commit/73fe1d8087cfc2d59ac5b9af48b4cf5f5b86f920).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  class InvalidEntryException(msg: String) extends Exception(msg) `


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r155863377
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala ---
    @@ -0,0 +1,173 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, Vectors}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.execution.streaming.MemoryStream
    +import org.apache.spark.sql.streaming.StreamTest
    +
    +class VectorSizeHintSuite
    +  extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  import testImplicits._
    +
    +  test("Test Param Validators") {
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setHandleInvalid("invalidValue"))
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setSize(-3))
    +  }
    +
    +  test("Adding size to column of vectors.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val denseVector = Vectors.dense(1, 2, 3)
    +    val sparseVector = Vectors.sparse(size, Array(), Array())
    +
    +    val data = Seq(denseVector, denseVector, sparseVector).map(Tuple1.apply)
    +    val dataFrame = data.toDF(vectorColName)
    +    assert(
    +      AttributeGroup.fromStructField(dataFrame.schema(vectorColName)).size == -1,
    +      "Transformer did not add expected size data.")
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrame)
    +      assert(
    +        AttributeGroup.fromStructField(withSize.schema(vectorColName)).size == size,
    +        "Transformer did not add expected size data.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size hint preserves attributes.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +    val group = AttributeGroup.fromStructField(dataFrameWithMetadata.schema(vectorColName))
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrameWithMetadata)
    +
    +      val newGroup = AttributeGroup.fromStructField(withSize.schema(vectorColName))
    +      assert(newGroup.size === size, "Transformer did not add expected size data.")
    +      assert(
    +        newGroup.attributes.get.deep === group.attributes.get.deep,
    +        "SizeHintTransformer did not preserve attributes.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size miss-match between current and target size raises an error.") {
    +    val size = 4
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      intercept[SparkException](transformer.transform(dataFrameWithMetadata))
    +    }
    +  }
    +
    +  test("Handle invalid does the right thing.") {
    +
    +    val vector = Vectors.dense(1, 2, 3)
    +    val short = Vectors.dense(2)
    +    val dataWithNull = Seq(vector, null).map(Tuple1.apply).toDF("vector")
    +    val dataWithShort = Seq(vector, short).map(Tuple1.apply).toDF("vector")
    +
    +    val sizeHint = new VectorSizeHint()
    +      .setInputCol("vector")
    +      .setHandleInvalid("error")
    +      .setSize(3)
    +
    +    intercept[SparkException](sizeHint.transform(dataWithNull).collect)
    --- End diff --
    
    style nit: Call collect() with parentheses


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    **[Test build #85134 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85134/testReport)** for PR 19746 at commit [`9c3dcec`](https://github.com/apache/spark/commit/9c3dcec30d152ab2c5c242db03ed466ae1d320ae).
     * 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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r151958073
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala ---
    @@ -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.
    + */
    +
    +package org.apache.spark.ml.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, Vectors}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.execution.streaming.MemoryStream
    +import org.apache.spark.sql.streaming.StreamTest
    +
    +class VectorSizeHintSuite
    +  extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  import testImplicits._
    +
    +  test("Test Param Validators") {
    +    assertThrows[IllegalArgumentException] (new VectorSizeHint().setHandleInvalid("invalidValue"))
    +    assertThrows[IllegalArgumentException] (new VectorSizeHint().setSize(-3))
    +  }
    +
    +  test("Adding size to column of vectors.") {
    +
    +    val size = 3
    +    val denseVector = Vectors.dense(1, 2, 3)
    +    val sparseVector = Vectors.sparse(size, Array(), Array())
    +
    +    val data = Seq(denseVector, denseVector, sparseVector).map(Tuple1.apply)
    +    val dataFrame = data.toDF("vector")
    +
    +    val transformer = new VectorSizeHint()
    +      .setInputCol("vector")
    +      .setSize(3)
    +      .setHandleInvalid("error")
    +    val withSize = transformer.transform(dataFrame)
    +    assert(
    +      AttributeGroup.fromStructField(withSize.schema("vector")).size == size,
    +      "Transformer did not add expected size data.")
    +  }
    +
    +  test("Size hint preserves attributes.") {
    +
    +    case class Foo(x: Double, y: Double, z: Double)
    +    val size = 3
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val boo = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol("vector")
    +    val dataFrameWithMeatadata = assembler.transform(boo)
    +    val group = AttributeGroup.fromStructField(dataFrameWithMeatadata.schema("vector"))
    +
    +    val transformer = new VectorSizeHint()
    +      .setInputCol("vector")
    +      .setSize(3)
    +      .setHandleInvalid("error")
    +    val withSize = transformer.transform(dataFrameWithMeatadata)
    +
    +    val newGroup = AttributeGroup.fromStructField(withSize.schema("vector"))
    +    assert(newGroup.size == size, "Transformer did not add expected size data.")
    +    assert(
    +      newGroup.attributes.get.deep === group.attributes.get.deep,
    +      "SizeHintTransformer did not preserve attributes.")
    +  }
    +
    +  test("Handle invalid does the right thing.") {
    --- End diff --
    
    I don't find a test for `optimistic` option. We should test:
    If input dataset vector column do not include metadata, the `VectorSizeHint` should add metadata with proper size, or input vector column include metadata with different `size`, the `VectorSizeHint` should replace it.


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    **[Test build #83860 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83860/testReport)** for PR 19746 at commit [`7f6ab98`](https://github.com/apache/spark/commit/7f6ab98310d6c2bf43599b6a38c8ead91c6534e9).
     * This patch **fails Spark 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 issue #19746: [SPARK-22346][ML]

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

    https://github.com/apache/spark/pull/19746
  
    **[Test build #83850 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83850/testReport)** for PR 19746 at commit [`df990ed`](https://github.com/apache/spark/commit/df990ed771d9d773b6a8a1b8751ae8c1c475b3d5).
     * This patch **fails RAT tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class VectorSizeHint @Since(\"2.3.0\") (@Since(\"2.3.0\") override val uid: String)`


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r155863960
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala ---
    @@ -0,0 +1,173 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, Vectors}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.execution.streaming.MemoryStream
    +import org.apache.spark.sql.streaming.StreamTest
    +
    +class VectorSizeHintSuite
    +  extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  import testImplicits._
    +
    +  test("Test Param Validators") {
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setHandleInvalid("invalidValue"))
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setSize(-3))
    +  }
    +
    +  test("Adding size to column of vectors.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val denseVector = Vectors.dense(1, 2, 3)
    +    val sparseVector = Vectors.sparse(size, Array(), Array())
    +
    +    val data = Seq(denseVector, denseVector, sparseVector).map(Tuple1.apply)
    +    val dataFrame = data.toDF(vectorColName)
    +    assert(
    +      AttributeGroup.fromStructField(dataFrame.schema(vectorColName)).size == -1,
    +      "Transformer did not add expected size data.")
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrame)
    +      assert(
    +        AttributeGroup.fromStructField(withSize.schema(vectorColName)).size == size,
    +        "Transformer did not add expected size data.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size hint preserves attributes.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +    val group = AttributeGroup.fromStructField(dataFrameWithMetadata.schema(vectorColName))
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrameWithMetadata)
    +
    +      val newGroup = AttributeGroup.fromStructField(withSize.schema(vectorColName))
    +      assert(newGroup.size === size, "Transformer did not add expected size data.")
    +      assert(
    +        newGroup.attributes.get.deep === group.attributes.get.deep,
    +        "SizeHintTransformer did not preserve attributes.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size miss-match between current and target size raises an error.") {
    +    val size = 4
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      intercept[SparkException](transformer.transform(dataFrameWithMetadata))
    +    }
    +  }
    +
    +  test("Handle invalid does the right thing.") {
    +
    +    val vector = Vectors.dense(1, 2, 3)
    +    val short = Vectors.dense(2)
    +    val dataWithNull = Seq(vector, null).map(Tuple1.apply).toDF("vector")
    +    val dataWithShort = Seq(vector, short).map(Tuple1.apply).toDF("vector")
    +
    +    val sizeHint = new VectorSizeHint()
    +      .setInputCol("vector")
    +      .setHandleInvalid("error")
    +      .setSize(3)
    +
    +    intercept[SparkException](sizeHint.transform(dataWithNull).collect)
    +    intercept[SparkException](sizeHint.transform(dataWithShort).collect)
    +
    +    sizeHint.setHandleInvalid("skip")
    +    assert(sizeHint.transform(dataWithNull).count() === 1)
    +    assert(sizeHint.transform(dataWithShort).count() === 1)
    +  }
    +
    +  test("read/write") {
    +    val sizeHint = new VectorSizeHint()
    +      .setInputCol("myInputCol")
    +      .setSize(11)
    +      .setHandleInvalid("skip")
    +    testDefaultReadWrite(sizeHint)
    +  }
    +}
    +
    +class VectorSizeHintStreamingSuite extends StreamTest {
    +
    +  import testImplicits._
    +
    +  test("Test assemble vectors with size hint in steaming.") {
    +    val a = Vectors.dense(0, 1, 2)
    +    val b = Vectors.sparse(4, Array(0, 3), Array(3, 6))
    +
    +    val stream = MemoryStream[(Vector, Vector)]
    +    val streamingDF = stream.toDS.toDF("a", "b")
    +    val sizeHintA = new VectorSizeHint()
    +      .setSize(3)
    +      .setInputCol("a")
    +    val sizeHintB = new VectorSizeHint()
    +      .setSize(4)
    +      .setInputCol("b")
    +    val vectorAssembler = new VectorAssembler()
    +      .setInputCols(Array("a", "b"))
    +      .setOutputCol("assembled")
    +    val output = Seq(sizeHintA, sizeHintB, vectorAssembler).foldLeft(streamingDF) {
    --- End diff --
    
    You can just put these in a PipelineModel to avoid using foldLeft.


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r156797156
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala ---
    @@ -0,0 +1,190 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.Pipeline
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, Vectors}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.execution.streaming.MemoryStream
    +import org.apache.spark.sql.streaming.StreamTest
    +
    +class VectorSizeHintSuite
    +  extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  import testImplicits._
    +
    +  test("Test Param Validators") {
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setHandleInvalid("invalidValue"))
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setSize(-3))
    +  }
    +
    +  test("Required params must be set before transform.") {
    +    val data = Seq((Vectors.dense(1, 2), 0)).toDF("vector", "intValue")
    +
    +    val noSizeTransformer = new VectorSizeHint().setInputCol("vector")
    +    intercept[NoSuchElementException] (noSizeTransformer.transform(data))
    +    intercept[NoSuchElementException] (noSizeTransformer.transformSchema(data.schema))
    +
    +    val noInputColTransformer = new VectorSizeHint().setSize(2)
    +    intercept[NoSuchElementException] (noInputColTransformer.transform(data))
    +    intercept[NoSuchElementException] (noInputColTransformer.transformSchema(data.schema))
    +  }
    +
    +  test("Adding size to column of vectors.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val denseVector = Vectors.dense(1, 2, 3)
    +    val sparseVector = Vectors.sparse(size, Array(), Array())
    +
    +    val data = Seq(denseVector, denseVector, sparseVector).map(Tuple1.apply)
    +    val dataFrame = data.toDF(vectorColName)
    +    assert(
    +      AttributeGroup.fromStructField(dataFrame.schema(vectorColName)).size == -1,
    +      s"This test requires that column '$vectorColName' not have size metadata.")
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrame)
    +      assert(
    +        AttributeGroup.fromStructField(withSize.schema(vectorColName)).size == size,
    +        "Transformer did not add expected size data.")
    +      val numRows = withSize.collect().length
    +      assert(numRows === data.length, s"Expecting ${data.length} rows, got $numRows.")
    +    }
    +  }
    +
    +  test("Size hint preserves attributes.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +    val group = AttributeGroup.fromStructField(dataFrameWithMetadata.schema(vectorColName))
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrameWithMetadata)
    +
    +      val newGroup = AttributeGroup.fromStructField(withSize.schema(vectorColName))
    +      assert(newGroup.size === size, "Column has incorrect size metadata.")
    +      assert(
    +        newGroup.attributes.get === group.attributes.get,
    +        "VectorSizeHint did not preserve attributes.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size mismatch between current and target size raises an error.") {
    +    val size = 4
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      intercept[IllegalArgumentException](transformer.transform(dataFrameWithMetadata))
    +    }
    +  }
    +
    +  test("Handle invalid does the right thing.") {
    +
    +    val vector = Vectors.dense(1, 2, 3)
    +    val short = Vectors.dense(2)
    +    val dataWithNull = Seq(vector, null).map(Tuple1.apply).toDF("vector")
    +    val dataWithShort = Seq(vector, short).map(Tuple1.apply).toDF("vector")
    +
    +    val sizeHint = new VectorSizeHint()
    +      .setInputCol("vector")
    +      .setHandleInvalid("error")
    +      .setSize(3)
    +
    +    intercept[SparkException](sizeHint.transform(dataWithNull).collect())
    +    intercept[SparkException](sizeHint.transform(dataWithShort).collect())
    +
    +    sizeHint.setHandleInvalid("skip")
    +    assert(sizeHint.transform(dataWithNull).count() === 1)
    +    assert(sizeHint.transform(dataWithShort).count() === 1)
    +  }
    +
    +  test("read/write") {
    +    val sizeHint = new VectorSizeHint()
    +      .setInputCol("myInputCol")
    +      .setSize(11)
    +      .setHandleInvalid("skip")
    +    testDefaultReadWrite(sizeHint)
    +  }
    +}
    +
    +class VectorSizeHintStreamingSuite extends StreamTest {
    +
    +  import testImplicits._
    +
    +  test("Test assemble vectors with size hint in streaming.") {
    +    val a = Vectors.dense(0, 1, 2)
    +    val b = Vectors.sparse(4, Array(0, 3), Array(3, 6))
    +
    +    val stream = MemoryStream[(Vector, Vector)]
    +    val streamingDF = stream.toDS.toDF("a", "b")
    +    val sizeHintA = new VectorSizeHint()
    +      .setSize(3)
    +      .setInputCol("a")
    +    val sizeHintB = new VectorSizeHint()
    +      .setSize(4)
    +      .setInputCol("b")
    +    val vectorAssembler = new VectorAssembler()
    +      .setInputCols(Array("a", "b"))
    +      .setOutputCol("assembled")
    +    val pipeline = new Pipeline().setStages(Array(sizeHintA, sizeHintB, vectorAssembler))
    +    /**
    --- End diff --
    
    remove unused code?


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

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


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r156268308
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.Transformer
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, VectorUDT}
    +import org.apache.spark.ml.param.{IntParam, Param, ParamMap, ParamValidators}
    +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol}
    +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable}
    +import org.apache.spark.sql.{Column, DataFrame, Dataset}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.StructType
    +
    +/**
    + * A feature transformer that adds vector size information to a vector column.
    + */
    +@Experimental
    +@Since("2.3.0")
    +class VectorSizeHint @Since("2.3.0") (@Since("2.3.0") override val uid: String)
    +  extends Transformer with HasInputCol with HasHandleInvalid with DefaultParamsWritable {
    +
    +  @Since("2.3.0")
    +  def this() = this(Identifiable.randomUID("vectSizeHint"))
    +
    +  @Since("2.3.0")
    +  val size = new IntParam(this, "size", "Size of vectors in column.", {s: Int => s >= 0})
    +
    +  @Since("2.3.0")
    +  def getSize: Int = getOrDefault(size)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setSize(value: Int): this.type = set(size, value)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  @Since("2.3.0")
    +  override val handleInvalid: Param[String] = new Param[String](
    +    this,
    +    "handleInvalid",
    +    "How to handle invalid vectors in inputCol, (invalid vectors include nulls and vectors with " +
    +      "the wrong size. The options are `skip` (filter out rows with invalid vectors), `error` " +
    +      "(throw an error) and `optimistic` (don't check the vector size).",
    +    ParamValidators.inArray(VectorSizeHint.supportedHandleInvalids))
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setHandleInvalid(value: String): this.type = set(handleInvalid, value)
    +  setDefault(handleInvalid, VectorSizeHint.ERROR_INVALID)
    +
    +  @Since("2.3.0")
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    val localInputCol = getInputCol
    +    val localSize = getSize
    +    val localHandleInvalid = getHandleInvalid
    +
    +    val group = AttributeGroup.fromStructField(dataset.schema(localInputCol))
    +    if (localHandleInvalid == VectorSizeHint.OPTIMISTIC_INVALID && group.size == localSize) {
    +      dataset.toDF
    +    } else {
    +      val newGroup = group.size match {
    +        case `localSize` => group
    +        case -1 => new AttributeGroup(localInputCol, localSize)
    +        case _ =>
    +          val msg = s"Trying to set size of vectors in `$localInputCol` to $localSize but size " +
    +            s"already set to ${group.size}."
    +          throw new SparkException(msg)
    +      }
    +
    +      val newCol: Column = localHandleInvalid match {
    +        case VectorSizeHint.OPTIMISTIC_INVALID => col(localInputCol)
    +        case VectorSizeHint.ERROR_INVALID =>
    +          val checkVectorSizeUDF = udf { vector: Vector =>
    +            if (vector == null) {
    +              throw new SparkException(s"Got null vector in VectorSizeHint, set `handleInvalid` " +
    +                s"to 'skip' to filter invalid rows.")
    +            }
    +            if (vector.size != localSize) {
    +              throw new SparkException(s"VectorSizeHint Expecting a vector of size $localSize but" +
    +                s" got ${vector.size}")
    +            }
    +            vector
    +          }.asNondeterministic
    +          checkVectorSizeUDF(col(localInputCol))
    +        case VectorSizeHint.SKIP_INVALID =>
    +          val checkVectorSizeUDF = udf { vector: Vector =>
    --- End diff --
    
    @cloud-fan I tried, but got such error:
    `org.apache.spark.sql.AnalysisException: Can't extract value from a#3: need struct type but got vector`, anywhere wrong ? test code:
    ```
    import spark.implicits._
    import org.apache.spark.ml.linalg._
    val df1 = Seq(Tuple1(Vectors.dense(1.0, 2.0))).toDF("a")
    df1.select(col("a")(0)).show
    ```


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/83860/
    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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r157668450
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala ---
    @@ -0,0 +1,195 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.Transformer
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, VectorUDT}
    +import org.apache.spark.ml.param.{IntParam, Param, ParamMap, ParamValidators}
    +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol}
    +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable}
    +import org.apache.spark.sql.{Column, DataFrame, Dataset}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.StructType
    +
    +/**
    + * :: Experimental ::
    + * A feature transformer that adds size information to the metadata of a vector column.
    + * VectorAssembler needs size information for its input columns and cannot be used on streaming
    + * dataframes without this metadata.
    + *
    + */
    +@Experimental
    +@Since("2.3.0")
    +class VectorSizeHint @Since("2.3.0") (@Since("2.3.0") override val uid: String)
    +  extends Transformer with HasInputCol with HasHandleInvalid with DefaultParamsWritable {
    +
    +  @Since("2.3.0")
    +  def this() = this(Identifiable.randomUID("vectSizeHint"))
    +
    +  /**
    +   * The size of Vectors in `inputCol`.
    +   * @group param
    +   */
    +  @Since("2.3.0")
    +  val size: IntParam = new IntParam(
    +    this,
    +    "size",
    +    "Size of vectors in column.",
    +    {s: Int => s >= 0})
    +
    +  /** group getParam */
    +  @Since("2.3.0")
    +  def getSize: Int = getOrDefault(size)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setSize(value: Int): this.type = set(size, value)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /**
    +   * Param for how to handle invalid entries. Invalid vectors include nulls and vectors with the
    +   * wrong size. The options are `skip` (filter out rows with invalid vectors), `error` (throw an
    +   * error) and `optimistic` (do not check the vector size, and keep all row\). `error` by default.
    --- End diff --
    
    "row\" ==> "rows"


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r155862041
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala ---
    @@ -0,0 +1,173 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, Vectors}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.execution.streaming.MemoryStream
    +import org.apache.spark.sql.streaming.StreamTest
    +
    +class VectorSizeHintSuite
    +  extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  import testImplicits._
    +
    +  test("Test Param Validators") {
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setHandleInvalid("invalidValue"))
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setSize(-3))
    +  }
    +
    +  test("Adding size to column of vectors.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val denseVector = Vectors.dense(1, 2, 3)
    +    val sparseVector = Vectors.sparse(size, Array(), Array())
    +
    +    val data = Seq(denseVector, denseVector, sparseVector).map(Tuple1.apply)
    +    val dataFrame = data.toDF(vectorColName)
    +    assert(
    +      AttributeGroup.fromStructField(dataFrame.schema(vectorColName)).size == -1,
    +      "Transformer did not add expected size data.")
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrame)
    +      assert(
    +        AttributeGroup.fromStructField(withSize.schema(vectorColName)).size == size,
    +        "Transformer did not add expected size data.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size hint preserves attributes.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +    val group = AttributeGroup.fromStructField(dataFrameWithMetadata.schema(vectorColName))
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrameWithMetadata)
    +
    +      val newGroup = AttributeGroup.fromStructField(withSize.schema(vectorColName))
    +      assert(newGroup.size === size, "Transformer did not add expected size data.")
    --- End diff --
    
    misleading error message


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

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


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

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


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

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


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    **[Test build #84449 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84449/testReport)** for PR 19746 at commit [`0837b76`](https://github.com/apache/spark/commit/0837b760530a217e5fc8ea694edc8e737c3f34a4).
     * 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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r156750447
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala ---
    @@ -0,0 +1,173 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, Vectors}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.execution.streaming.MemoryStream
    +import org.apache.spark.sql.streaming.StreamTest
    +
    +class VectorSizeHintSuite
    +  extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  import testImplicits._
    +
    +  test("Test Param Validators") {
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setHandleInvalid("invalidValue"))
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setSize(-3))
    +  }
    +
    +  test("Adding size to column of vectors.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val denseVector = Vectors.dense(1, 2, 3)
    +    val sparseVector = Vectors.sparse(size, Array(), Array())
    +
    +    val data = Seq(denseVector, denseVector, sparseVector).map(Tuple1.apply)
    +    val dataFrame = data.toDF(vectorColName)
    +    assert(
    +      AttributeGroup.fromStructField(dataFrame.schema(vectorColName)).size == -1,
    +      "Transformer did not add expected size data.")
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrame)
    +      assert(
    +        AttributeGroup.fromStructField(withSize.schema(vectorColName)).size == size,
    +        "Transformer did not add expected size data.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size hint preserves attributes.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +    val group = AttributeGroup.fromStructField(dataFrameWithMetadata.schema(vectorColName))
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrameWithMetadata)
    +
    +      val newGroup = AttributeGroup.fromStructField(withSize.schema(vectorColName))
    +      assert(newGroup.size === size, "Transformer did not add expected size data.")
    +      assert(
    +        newGroup.attributes.get.deep === group.attributes.get.deep,
    +        "SizeHintTransformer did not preserve attributes.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size miss-match between current and target size raises an error.") {
    +    val size = 4
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      intercept[SparkException](transformer.transform(dataFrameWithMetadata))
    +    }
    +  }
    +
    +  test("Handle invalid does the right thing.") {
    +
    +    val vector = Vectors.dense(1, 2, 3)
    +    val short = Vectors.dense(2)
    +    val dataWithNull = Seq(vector, null).map(Tuple1.apply).toDF("vector")
    +    val dataWithShort = Seq(vector, short).map(Tuple1.apply).toDF("vector")
    +
    +    val sizeHint = new VectorSizeHint()
    +      .setInputCol("vector")
    +      .setHandleInvalid("error")
    +      .setSize(3)
    +
    +    intercept[SparkException](sizeHint.transform(dataWithNull).collect)
    +    intercept[SparkException](sizeHint.transform(dataWithShort).collect)
    +
    +    sizeHint.setHandleInvalid("skip")
    +    assert(sizeHint.transform(dataWithNull).count() === 1)
    +    assert(sizeHint.transform(dataWithShort).count() === 1)
    +  }
    --- End diff --
    
    Did you a thought on how to test `keep`/`optimistic`. I could verify that the invalid data is not removed but that's a little bit weird to test. It's ensuring that this option allows the column to get into a "bad state" where the metadata doesn't match the contents. Is that what you had in mind?


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r151954590
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.Transformer
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, VectorUDT}
    +import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
    +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol}
    +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable}
    +import org.apache.spark.sql.{Column, DataFrame, Dataset}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.StructType
    +
    +/**
    + * A feature transformer that adds vector size information to a vector column.
    + */
    +@Experimental
    +@Since("2.3.0")
    +class VectorSizeHint @Since("2.3.0") (@Since("2.3.0") override val uid: String)
    +  extends Transformer with HasInputCol with HasHandleInvalid with DefaultParamsWritable {
    +
    +  @Since("2.3.0")
    +  def this() = this(Identifiable.randomUID("vectSizeHint"))
    +
    +  @Since("2.3.0")
    +  val size = new Param[Int](this, "size", "Size of vectors in column.", {s: Int => s >= 0})
    +
    +  @Since("2.3.0")
    +  def getSize: Int = getOrDefault(size)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setSize(value: Int): this.type = set(size, value)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  @Since("2.3.0")
    +  override val handleInvalid: Param[String] = new Param[String](
    +    this,
    +    "handleInvalid",
    +    "How to handle invalid vectors in inputCol, (invalid vectors include nulls and vectors with " +
    +      "the wrong size. The options are `skip` (filter out rows with invalid vectors), `error` " +
    +      "(throw an error) and `optimistic` (don't check the vector size).",
    +    ParamValidators.inArray(VectorSizeHint.supportedHandleInvalids))
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setHandleInvalid(value: String): this.type = set(handleInvalid, value)
    +  setDefault(handleInvalid, VectorSizeHint.ERROR_INVALID)
    +
    +  @Since("2.3.0")
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    val localInputCol = getInputCol
    +    val localSize = getSize
    +    val localHandleInvalid = getHandleInvalid
    +
    +    val group = AttributeGroup.fromStructField(dataset.schema(localInputCol))
    +    if (localHandleInvalid == VectorSizeHint.OPTIMISTIC_INVALID && group.size == localSize) {
    +      dataset.toDF
    +    } else {
    +      val newGroup = if (group.size == localSize) {
    +        // Pass along any existing metadata about vector.
    +        group
    +      } else {
    +        new AttributeGroup(localInputCol, localSize)
    +      }
    +
    +      val newCol: Column = localHandleInvalid match {
    +        case VectorSizeHint.OPTIMISTIC_INVALID => col(localInputCol)
    +        case VectorSizeHint.ERROR_INVALID =>
    +          val checkVectorSize = { vector: Vector =>
    +            if (vector == null) {
    +              throw new VectorSizeHint.InvalidEntryException(s"Got null vector in VectorSizeHint," +
    --- End diff --
    
    The UDF which is possible to throw exception should be marked as `nondeterministic`, check this PR #19662 for more explanation.


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r151956715
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala ---
    @@ -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.
    + */
    +
    +package org.apache.spark.ml.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, Vectors}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.execution.streaming.MemoryStream
    +import org.apache.spark.sql.streaming.StreamTest
    +
    +class VectorSizeHintSuite
    +  extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  import testImplicits._
    +
    +  test("Test Param Validators") {
    +    assertThrows[IllegalArgumentException] (new VectorSizeHint().setHandleInvalid("invalidValue"))
    +    assertThrows[IllegalArgumentException] (new VectorSizeHint().setSize(-3))
    +  }
    +
    +  test("Adding size to column of vectors.") {
    +
    +    val size = 3
    +    val denseVector = Vectors.dense(1, 2, 3)
    +    val sparseVector = Vectors.sparse(size, Array(), Array())
    +
    +    val data = Seq(denseVector, denseVector, sparseVector).map(Tuple1.apply)
    +    val dataFrame = data.toDF("vector")
    +
    +    val transformer = new VectorSizeHint()
    +      .setInputCol("vector")
    +      .setSize(3)
    +      .setHandleInvalid("error")
    +    val withSize = transformer.transform(dataFrame)
    +    assert(
    +      AttributeGroup.fromStructField(withSize.schema("vector")).size == size,
    +      "Transformer did not add expected size data.")
    +  }
    +
    +  test("Size hint preserves attributes.") {
    +
    +    case class Foo(x: Double, y: Double, z: Double)
    +    val size = 3
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val boo = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol("vector")
    +    val dataFrameWithMeatadata = assembler.transform(boo)
    +    val group = AttributeGroup.fromStructField(dataFrameWithMeatadata.schema("vector"))
    +
    +    val transformer = new VectorSizeHint()
    +      .setInputCol("vector")
    +      .setSize(3)
    +      .setHandleInvalid("error")
    +    val withSize = transformer.transform(dataFrameWithMeatadata)
    +
    +    val newGroup = AttributeGroup.fromStructField(withSize.schema("vector"))
    +    assert(newGroup.size == size, "Transformer did not add expected size data.")
    +    assert(
    +      newGroup.attributes.get.deep === group.attributes.get.deep,
    +      "SizeHintTransformer did not preserve attributes.")
    +  }
    +
    +  test("Handle invalid does the right thing.") {
    +
    +    val vector = Vectors.dense(1, 2, 3)
    +    val short = Vectors.dense(2)
    +    val dataWithNull = Seq(vector, null).map(Tuple1.apply).toDF("vector")
    +    val dataWithShort = Seq(vector, short).map(Tuple1.apply).toDF("vector")
    +
    +    val sizeHint = new VectorSizeHint()
    +      .setInputCol("vector")
    +      .setHandleInvalid("error")
    +      .setSize(3)
    +
    +    assertThrows[SparkException](sizeHint.transform(dataWithNull).collect)
    --- End diff --
    
    Use `intercept[SparkException] {...}` is better.


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r152166365
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala ---
    @@ -0,0 +1,185 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, Vectors}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.execution.streaming.MemoryStream
    +import org.apache.spark.sql.streaming.StreamTest
    +
    +class VectorSizeHintSuite
    +  extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  import testImplicits._
    +
    +  test("Test Param Validators") {
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setHandleInvalid("invalidValue"))
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setSize(-3))
    +  }
    +
    +  test("Adding size to column of vectors.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val denseVector = Vectors.dense(1, 2, 3)
    +    val sparseVector = Vectors.sparse(size, Array(), Array())
    +
    +    val data = Seq(denseVector, denseVector, sparseVector).map(Tuple1.apply)
    +    val dataFrame = data.toDF(vectorColName)
    +    assert(
    +      AttributeGroup.fromStructField(dataFrame.schema(vectorColName)).size == -1,
    +      "Transformer did not add expected size data.")
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrame)
    +      assert(
    +        AttributeGroup.fromStructField(withSize.schema(vectorColName)).size == size,
    +        "Transformer did not add expected size data.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size hint preserves attributes.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +    val group = AttributeGroup.fromStructField(dataFrameWithMetadata.schema(vectorColName))
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrameWithMetadata)
    +
    +      val newGroup = AttributeGroup.fromStructField(withSize.schema(vectorColName))
    +      assert(newGroup.size === size, "Transformer did not add expected size data.")
    +      assert(
    +        newGroup.attributes.get.deep === group.attributes.get.deep,
    +        "SizeHintTransformer did not preserve attributes.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size miss-match between current and target size raises an error.") {
    +    val size = 4
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      intercept[SparkException](transformer.transform(dataFrameWithMetadata))
    +    }
    +  }
    +
    +  test("Handle invalid does the right thing.") {
    +
    +    val vector = Vectors.dense(1, 2, 3)
    +    val short = Vectors.dense(2)
    +    val dataWithNull = Seq(vector, null).map(Tuple1.apply).toDF("vector")
    +    val dataWithShort = Seq(vector, short).map(Tuple1.apply).toDF("vector")
    +
    +    val sizeHint = new VectorSizeHint()
    +      .setInputCol("vector")
    +      .setHandleInvalid("error")
    +      .setSize(3)
    +
    +    intercept[SparkException](sizeHint.transform(dataWithNull).collect)
    +    intercept[SparkException](sizeHint.transform(dataWithShort).collect)
    +
    +    sizeHint.setHandleInvalid("skip")
    +    assert(sizeHint.transform(dataWithNull).count() === 1)
    +    assert(sizeHint.transform(dataWithShort).count() === 1)
    +  }
    +
    +  test("Test correct behaviour for handleInvalid == optimistic") {
    +    val vector = Vectors.dense(1, 2, 3)
    +    val data = Seq(vector, vector).map(Tuple1.apply).toDF("vector")
    +
    +    val sizeHint = new VectorSizeHint()
    +      .setInputCol("vector")
    +      .setHandleInvalid("optimistic")
    +      .setSize(3)
    +
    +    val transformed = sizeHint.transform(data)
    --- End diff --
    
    OK. I agree. Other testcases already cover them.


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r152158938
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala ---
    @@ -0,0 +1,185 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, Vectors}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.execution.streaming.MemoryStream
    +import org.apache.spark.sql.streaming.StreamTest
    +
    +class VectorSizeHintSuite
    +  extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  import testImplicits._
    +
    +  test("Test Param Validators") {
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setHandleInvalid("invalidValue"))
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setSize(-3))
    +  }
    +
    +  test("Adding size to column of vectors.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val denseVector = Vectors.dense(1, 2, 3)
    +    val sparseVector = Vectors.sparse(size, Array(), Array())
    +
    +    val data = Seq(denseVector, denseVector, sparseVector).map(Tuple1.apply)
    +    val dataFrame = data.toDF(vectorColName)
    +    assert(
    +      AttributeGroup.fromStructField(dataFrame.schema(vectorColName)).size == -1,
    +      "Transformer did not add expected size data.")
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrame)
    +      assert(
    +        AttributeGroup.fromStructField(withSize.schema(vectorColName)).size == size,
    +        "Transformer did not add expected size data.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size hint preserves attributes.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +    val group = AttributeGroup.fromStructField(dataFrameWithMetadata.schema(vectorColName))
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrameWithMetadata)
    +
    +      val newGroup = AttributeGroup.fromStructField(withSize.schema(vectorColName))
    +      assert(newGroup.size === size, "Transformer did not add expected size data.")
    +      assert(
    +        newGroup.attributes.get.deep === group.attributes.get.deep,
    +        "SizeHintTransformer did not preserve attributes.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size miss-match between current and target size raises an error.") {
    +    val size = 4
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      intercept[SparkException](transformer.transform(dataFrameWithMetadata))
    +    }
    +  }
    +
    +  test("Handle invalid does the right thing.") {
    +
    +    val vector = Vectors.dense(1, 2, 3)
    +    val short = Vectors.dense(2)
    +    val dataWithNull = Seq(vector, null).map(Tuple1.apply).toDF("vector")
    +    val dataWithShort = Seq(vector, short).map(Tuple1.apply).toDF("vector")
    +
    +    val sizeHint = new VectorSizeHint()
    +      .setInputCol("vector")
    +      .setHandleInvalid("error")
    +      .setSize(3)
    +
    +    intercept[SparkException](sizeHint.transform(dataWithNull).collect)
    +    intercept[SparkException](sizeHint.transform(dataWithShort).collect)
    +
    +    sizeHint.setHandleInvalid("skip")
    +    assert(sizeHint.transform(dataWithNull).count() === 1)
    +    assert(sizeHint.transform(dataWithShort).count() === 1)
    +  }
    +
    +  test("Test correct behaviour for handleInvalid == optimistic") {
    +    val vector = Vectors.dense(1, 2, 3)
    +    val data = Seq(vector, vector).map(Tuple1.apply).toDF("vector")
    +
    +    val sizeHint = new VectorSizeHint()
    +      .setInputCol("vector")
    +      .setHandleInvalid("optimistic")
    +      .setSize(3)
    +
    +    val transformed = sizeHint.transform(data)
    --- End diff --
    
    Add checking for added metadata here ?
    
    And should test if metadata exists, but size do not match, exception will be thrown.


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    **[Test build #83869 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83869/testReport)** for PR 19746 at commit [`73fe1d8`](https://github.com/apache/spark/commit/73fe1d8087cfc2d59ac5b9af48b4cf5f5b86f920).


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    @WeichenXu123  From what I've seen, it's more common for people to use VectorAssembler to assemble a bunch of Numeric columns, rather than a bunch of Vector columns.  I'd recommend we do things incrementally, adding single-column support before multi-column support (especially since we're still trying to achieve consensus about design for multi-column support, per my recent comment in the umbrella JIRA).


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r154820556
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala ---
    @@ -0,0 +1,173 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, Vectors}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.execution.streaming.MemoryStream
    +import org.apache.spark.sql.streaming.StreamTest
    +
    +class VectorSizeHintSuite
    +  extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  import testImplicits._
    +
    +  test("Test Param Validators") {
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setHandleInvalid("invalidValue"))
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setSize(-3))
    +  }
    +
    +  test("Adding size to column of vectors.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val denseVector = Vectors.dense(1, 2, 3)
    +    val sparseVector = Vectors.sparse(size, Array(), Array())
    +
    +    val data = Seq(denseVector, denseVector, sparseVector).map(Tuple1.apply)
    +    val dataFrame = data.toDF(vectorColName)
    +    assert(
    +      AttributeGroup.fromStructField(dataFrame.schema(vectorColName)).size == -1,
    +      "Transformer did not add expected size data.")
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrame)
    +      assert(
    +        AttributeGroup.fromStructField(withSize.schema(vectorColName)).size == size,
    +        "Transformer did not add expected size data.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size hint preserves attributes.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +    val group = AttributeGroup.fromStructField(dataFrameWithMetadata.schema(vectorColName))
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrameWithMetadata)
    +
    +      val newGroup = AttributeGroup.fromStructField(withSize.schema(vectorColName))
    +      assert(newGroup.size === size, "Transformer did not add expected size data.")
    +      assert(
    +        newGroup.attributes.get.deep === group.attributes.get.deep,
    +        "SizeHintTransformer did not preserve attributes.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size miss-match between current and target size raises an error.") {
    +    val size = 4
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      intercept[SparkException](transformer.transform(dataFrameWithMetadata))
    +    }
    +  }
    +
    +  test("Handle invalid does the right thing.") {
    +
    +    val vector = Vectors.dense(1, 2, 3)
    +    val short = Vectors.dense(2)
    +    val dataWithNull = Seq(vector, null).map(Tuple1.apply).toDF("vector")
    +    val dataWithShort = Seq(vector, short).map(Tuple1.apply).toDF("vector")
    +
    +    val sizeHint = new VectorSizeHint()
    +      .setInputCol("vector")
    +      .setHandleInvalid("error")
    +      .setSize(3)
    +
    +    intercept[SparkException](sizeHint.transform(dataWithNull).collect)
    +    intercept[SparkException](sizeHint.transform(dataWithShort).collect)
    +
    +    sizeHint.setHandleInvalid("skip")
    +    assert(sizeHint.transform(dataWithNull).count() === 1)
    +    assert(sizeHint.transform(dataWithShort).count() === 1)
    +  }
    +
    +  test("read/write") {
    +    val sizeHint = new VectorSizeHint()
    +      .setInputCol("myInputCol")
    +      .setSize(11)
    +      .setHandleInvalid("skip")
    +    testDefaultReadWrite(sizeHint)
    +  }
    +}
    +
    +class VectorSizeHintStreamingSuite extends StreamTest {
    +
    +  import testImplicits._
    +
    +  test("Test assemble vectors with size hint in steaming.") {
    +    val a = Vectors.dense(0, 1, 2)
    +    val b = Vectors.sparse(4, Array(0, 3), Array(3, 6))
    +
    +    val stream = MemoryStream[(Vector, Vector)]
    +    val streamingDF = stream.toDS.toDF("a", "b")
    +    val sizeHintA = new VectorSizeHint()
    +      .setSize(3)
    +      .setInputCol("a")
    +    val sizeHintB = new VectorSizeHint()
    +      .setSize(4)
    +      .setInputCol("b")
    +    val vectorAssembler = new VectorAssembler()
    +      .setInputCols(Array("a", "b"))
    +      .setOutputCol("assembled")
    +    val output = Seq(sizeHintA, sizeHintB, vectorAssembler).foldLeft(streamingDF) {
    +      case (data, transform) => transform.transform(data)
    +    }.select("assembled")
    +
    +    val expected = Vectors.dense(0, 1, 2, 3, 0, 0, 6)
    +
    +    testStream (output) (
    +      AddData(stream, (a, b), (a, b)),
    +      CheckAnswerRows(Seq(Row(expected), Row(expected)), false, false)
    --- End diff --
    
    ah, sorry, it should be `CheckAnswer(Tuple1(expected), Tuple1(expected))`. It should work I think.


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

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


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

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


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    **[Test build #83909 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83909/testReport)** for PR 19746 at commit [`38e1c5c`](https://github.com/apache/spark/commit/38e1c5c70c407e29716aa91a2032a54ec20fb6e7).


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r155849923
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.Transformer
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, VectorUDT}
    +import org.apache.spark.ml.param.{IntParam, Param, ParamMap, ParamValidators}
    +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol}
    +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable}
    +import org.apache.spark.sql.{Column, DataFrame, Dataset}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.StructType
    +
    +/**
    + * A feature transformer that adds vector size information to a vector column.
    + */
    +@Experimental
    +@Since("2.3.0")
    +class VectorSizeHint @Since("2.3.0") (@Since("2.3.0") override val uid: String)
    +  extends Transformer with HasInputCol with HasHandleInvalid with DefaultParamsWritable {
    +
    +  @Since("2.3.0")
    +  def this() = this(Identifiable.randomUID("vectSizeHint"))
    +
    +  @Since("2.3.0")
    +  val size = new IntParam(this, "size", "Size of vectors in column.", {s: Int => s >= 0})
    --- End diff --
    
    style: always specify type explicitly  (There was some better reason for this which I forget...)


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r155856287
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.Transformer
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, VectorUDT}
    +import org.apache.spark.ml.param.{IntParam, Param, ParamMap, ParamValidators}
    +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol}
    +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable}
    +import org.apache.spark.sql.{Column, DataFrame, Dataset}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.StructType
    +
    +/**
    + * A feature transformer that adds vector size information to a vector column.
    + */
    +@Experimental
    +@Since("2.3.0")
    +class VectorSizeHint @Since("2.3.0") (@Since("2.3.0") override val uid: String)
    +  extends Transformer with HasInputCol with HasHandleInvalid with DefaultParamsWritable {
    +
    +  @Since("2.3.0")
    +  def this() = this(Identifiable.randomUID("vectSizeHint"))
    +
    +  @Since("2.3.0")
    +  val size = new IntParam(this, "size", "Size of vectors in column.", {s: Int => s >= 0})
    +
    +  @Since("2.3.0")
    +  def getSize: Int = getOrDefault(size)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setSize(value: Int): this.type = set(size, value)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  @Since("2.3.0")
    +  override val handleInvalid: Param[String] = new Param[String](
    +    this,
    +    "handleInvalid",
    +    "How to handle invalid vectors in inputCol, (invalid vectors include nulls and vectors with " +
    +      "the wrong size. The options are `skip` (filter out rows with invalid vectors), `error` " +
    +      "(throw an error) and `optimistic` (don't check the vector size).",
    +    ParamValidators.inArray(VectorSizeHint.supportedHandleInvalids))
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setHandleInvalid(value: String): this.type = set(handleInvalid, value)
    +  setDefault(handleInvalid, VectorSizeHint.ERROR_INVALID)
    +
    +  @Since("2.3.0")
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    val localInputCol = getInputCol
    +    val localSize = getSize
    +    val localHandleInvalid = getHandleInvalid
    +
    +    val group = AttributeGroup.fromStructField(dataset.schema(localInputCol))
    +    if (localHandleInvalid == VectorSizeHint.OPTIMISTIC_INVALID && group.size == localSize) {
    +      dataset.toDF
    +    } else {
    +      val newGroup = group.size match {
    +        case `localSize` => group
    +        case -1 => new AttributeGroup(localInputCol, localSize)
    +        case _ =>
    +          val msg = s"Trying to set size of vectors in `$localInputCol` to $localSize but size " +
    +            s"already set to ${group.size}."
    +          throw new SparkException(msg)
    +      }
    +
    +      val newCol: Column = localHandleInvalid match {
    +        case VectorSizeHint.OPTIMISTIC_INVALID => col(localInputCol)
    +        case VectorSizeHint.ERROR_INVALID =>
    +          val checkVectorSizeUDF = udf { vector: Vector =>
    +            if (vector == null) {
    +              throw new SparkException(s"Got null vector in VectorSizeHint, set `handleInvalid` " +
    +                s"to 'skip' to filter invalid rows.")
    +            }
    +            if (vector.size != localSize) {
    +              throw new SparkException(s"VectorSizeHint Expecting a vector of size $localSize but" +
    +                s" got ${vector.size}")
    +            }
    +            vector
    +          }.asNondeterministic
    +          checkVectorSizeUDF(col(localInputCol))
    +        case VectorSizeHint.SKIP_INVALID =>
    +          val checkVectorSizeUDF = udf { vector: Vector =>
    --- End diff --
    
    This case can be converted to use pure SQL operations to avoid SerDe costs.


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r156257397
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.Transformer
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, VectorUDT}
    +import org.apache.spark.ml.param.{IntParam, Param, ParamMap, ParamValidators}
    +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol}
    +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable}
    +import org.apache.spark.sql.{Column, DataFrame, Dataset}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.StructType
    +
    +/**
    + * A feature transformer that adds vector size information to a vector column.
    + */
    +@Experimental
    +@Since("2.3.0")
    +class VectorSizeHint @Since("2.3.0") (@Since("2.3.0") override val uid: String)
    +  extends Transformer with HasInputCol with HasHandleInvalid with DefaultParamsWritable {
    +
    +  @Since("2.3.0")
    +  def this() = this(Identifiable.randomUID("vectSizeHint"))
    +
    +  @Since("2.3.0")
    +  val size = new IntParam(this, "size", "Size of vectors in column.", {s: Int => s >= 0})
    +
    +  @Since("2.3.0")
    +  def getSize: Int = getOrDefault(size)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setSize(value: Int): this.type = set(size, value)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  @Since("2.3.0")
    +  override val handleInvalid: Param[String] = new Param[String](
    +    this,
    +    "handleInvalid",
    +    "How to handle invalid vectors in inputCol, (invalid vectors include nulls and vectors with " +
    +      "the wrong size. The options are `skip` (filter out rows with invalid vectors), `error` " +
    +      "(throw an error) and `optimistic` (don't check the vector size).",
    +    ParamValidators.inArray(VectorSizeHint.supportedHandleInvalids))
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setHandleInvalid(value: String): this.type = set(handleInvalid, value)
    +  setDefault(handleInvalid, VectorSizeHint.ERROR_INVALID)
    +
    +  @Since("2.3.0")
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    val localInputCol = getInputCol
    +    val localSize = getSize
    +    val localHandleInvalid = getHandleInvalid
    +
    +    val group = AttributeGroup.fromStructField(dataset.schema(localInputCol))
    +    if (localHandleInvalid == VectorSizeHint.OPTIMISTIC_INVALID && group.size == localSize) {
    +      dataset.toDF
    +    } else {
    +      val newGroup = group.size match {
    +        case `localSize` => group
    +        case -1 => new AttributeGroup(localInputCol, localSize)
    +        case _ =>
    +          val msg = s"Trying to set size of vectors in `$localInputCol` to $localSize but size " +
    +            s"already set to ${group.size}."
    +          throw new SparkException(msg)
    +      }
    +
    +      val newCol: Column = localHandleInvalid match {
    +        case VectorSizeHint.OPTIMISTIC_INVALID => col(localInputCol)
    +        case VectorSizeHint.ERROR_INVALID =>
    +          val checkVectorSizeUDF = udf { vector: Vector =>
    +            if (vector == null) {
    +              throw new SparkException(s"Got null vector in VectorSizeHint, set `handleInvalid` " +
    +                s"to 'skip' to filter invalid rows.")
    +            }
    +            if (vector.size != localSize) {
    +              throw new SparkException(s"VectorSizeHint Expecting a vector of size $localSize but" +
    +                s" got ${vector.size}")
    +            }
    +            vector
    +          }.asNondeterministic
    +          checkVectorSizeUDF(col(localInputCol))
    +        case VectorSizeHint.SKIP_INVALID =>
    +          val checkVectorSizeUDF = udf { vector: Vector =>
    --- End diff --
    
    But it doesn't work... UserDefinedType column cannot be used as `StructType`. @cloud-fan Is there any way we can directly extract "Struct" from UDT column ? (in pure sql way)


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r155862850
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala ---
    @@ -0,0 +1,173 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, Vectors}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.execution.streaming.MemoryStream
    +import org.apache.spark.sql.streaming.StreamTest
    +
    +class VectorSizeHintSuite
    +  extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  import testImplicits._
    +
    +  test("Test Param Validators") {
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setHandleInvalid("invalidValue"))
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setSize(-3))
    +  }
    +
    +  test("Adding size to column of vectors.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val denseVector = Vectors.dense(1, 2, 3)
    +    val sparseVector = Vectors.sparse(size, Array(), Array())
    +
    +    val data = Seq(denseVector, denseVector, sparseVector).map(Tuple1.apply)
    +    val dataFrame = data.toDF(vectorColName)
    +    assert(
    +      AttributeGroup.fromStructField(dataFrame.schema(vectorColName)).size == -1,
    +      "Transformer did not add expected size data.")
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrame)
    +      assert(
    +        AttributeGroup.fromStructField(withSize.schema(vectorColName)).size == size,
    +        "Transformer did not add expected size data.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size hint preserves attributes.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +    val group = AttributeGroup.fromStructField(dataFrameWithMetadata.schema(vectorColName))
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrameWithMetadata)
    +
    +      val newGroup = AttributeGroup.fromStructField(withSize.schema(vectorColName))
    +      assert(newGroup.size === size, "Transformer did not add expected size data.")
    +      assert(
    +        newGroup.attributes.get.deep === group.attributes.get.deep,
    +        "SizeHintTransformer did not preserve attributes.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size miss-match between current and target size raises an error.") {
    --- End diff --
    
    typo: mismatch


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    **[Test build #83860 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83860/testReport)** for PR 19746 at commit [`7f6ab98`](https://github.com/apache/spark/commit/7f6ab98310d6c2bf43599b6a38c8ead91c6534e9).


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML]

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

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


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    **[Test build #84880 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84880/testReport)** for PR 19746 at commit [`cafa875`](https://github.com/apache/spark/commit/cafa875d60c487d7df0d935f0a1808b30db3d05d).
     * 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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    **[Test build #84039 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84039/testReport)** for PR 19746 at commit [`7b51563`](https://github.com/apache/spark/commit/7b5156319155e002d317b34fecbc5a39eb6e839f).


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    **[Test build #83909 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83909/testReport)** for PR 19746 at commit [`38e1c5c`](https://github.com/apache/spark/commit/38e1c5c70c407e29716aa91a2032a54ec20fb6e7).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  class InvalidEntryException(msg: String) extends Exception(msg)`


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r156795068
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala ---
    @@ -0,0 +1,188 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.Transformer
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, VectorUDT}
    +import org.apache.spark.ml.param.{IntParam, Param, ParamMap, ParamValidators}
    +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol}
    +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable}
    +import org.apache.spark.sql.{Column, DataFrame, Dataset}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.StructType
    +
    +/**
    + * :: Experimental ::
    + * A feature transformer that adds size information to the metadata of a vector column.
    + * VectorAssembler needs size information for its input columns and cannot be used on streaming
    + * dataframes without this metadata.
    + *
    + */
    +@Experimental
    +@Since("2.3.0")
    +class VectorSizeHint @Since("2.3.0") (@Since("2.3.0") override val uid: String)
    +  extends Transformer with HasInputCol with HasHandleInvalid with DefaultParamsWritable {
    +
    +  @Since("2.3.0")
    +  def this() = this(Identifiable.randomUID("vectSizeHint"))
    +
    +  /**
    +   * The size of Vectors in `inputCol`.
    +   * @group param
    +   */
    +  @Since("2.3.0")
    +  val size: IntParam = new IntParam(
    +    this,
    +    "size",
    +    "Size of vectors in column.",
    +    {s: Int => s >= 0})
    +
    +  /** group getParam */
    +  @Since("2.3.0")
    +  def getSize: Int = getOrDefault(size)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setSize(value: Int): this.type = set(size, value)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /**
    +   * Param for how to handle invalid entries. Invalid vectors include nulls and vectors with the
    +   * wrong size. The options are `skip` (filter out rows with invalid vectors), `error` (throw an
    +   * error) and `keep` (do not check the vector size, and keep all rows). `error` by default.
    +   * @group param
    +   */
    +  @Since("2.3.0")
    +  override val handleInvalid: Param[String] = new Param[String](
    +    this,
    +    "handleInvalid",
    +    "How to handle invalid vectors in inputCol. Invalid vectors include nulls and vectors with " +
    +      "the wrong size. The options are skip (filter out rows with invalid vectors), error " +
    +      "(throw an error) and keep (do not check the vector size, and keep all rows). `error` by " +
    --- End diff --
    
    keep -> optimistic


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

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


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    **[Test build #85134 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85134/testReport)** for PR 19746 at commit [`9c3dcec`](https://github.com/apache/spark/commit/9c3dcec30d152ab2c5c242db03ed466ae1d320ae).


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    **[Test build #84122 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84122/testReport)** for PR 19746 at commit [`2b1ed0a`](https://github.com/apache/spark/commit/2b1ed0a3a85385f9b4042415889335942b65b9c9).


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84039/
    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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r155853428
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.Transformer
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, VectorUDT}
    +import org.apache.spark.ml.param.{IntParam, Param, ParamMap, ParamValidators}
    +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol}
    +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable}
    +import org.apache.spark.sql.{Column, DataFrame, Dataset}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.StructType
    +
    +/**
    + * A feature transformer that adds vector size information to a vector column.
    + */
    +@Experimental
    +@Since("2.3.0")
    +class VectorSizeHint @Since("2.3.0") (@Since("2.3.0") override val uid: String)
    +  extends Transformer with HasInputCol with HasHandleInvalid with DefaultParamsWritable {
    +
    +  @Since("2.3.0")
    +  def this() = this(Identifiable.randomUID("vectSizeHint"))
    +
    +  @Since("2.3.0")
    +  val size = new IntParam(this, "size", "Size of vectors in column.", {s: Int => s >= 0})
    +
    +  @Since("2.3.0")
    +  def getSize: Int = getOrDefault(size)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setSize(value: Int): this.type = set(size, value)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  @Since("2.3.0")
    +  override val handleInvalid: Param[String] = new Param[String](
    +    this,
    +    "handleInvalid",
    +    "How to handle invalid vectors in inputCol, (invalid vectors include nulls and vectors with " +
    +      "the wrong size. The options are `skip` (filter out rows with invalid vectors), `error` " +
    +      "(throw an error) and `optimistic` (don't check the vector size).",
    +    ParamValidators.inArray(VectorSizeHint.supportedHandleInvalids))
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setHandleInvalid(value: String): this.type = set(handleInvalid, value)
    +  setDefault(handleInvalid, VectorSizeHint.ERROR_INVALID)
    +
    +  @Since("2.3.0")
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    val localInputCol = getInputCol
    --- End diff --
    
    Since it's lightweight, let's call transformSchema here to validate the params.  That way, if users have not yet specified a required Param, we can throw an exception with a better error message.


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r155850236
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.Transformer
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, VectorUDT}
    +import org.apache.spark.ml.param.{IntParam, Param, ParamMap, ParamValidators}
    +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol}
    +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable}
    +import org.apache.spark.sql.{Column, DataFrame, Dataset}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.StructType
    +
    +/**
    + * A feature transformer that adds vector size information to a vector column.
    + */
    +@Experimental
    +@Since("2.3.0")
    +class VectorSizeHint @Since("2.3.0") (@Since("2.3.0") override val uid: String)
    +  extends Transformer with HasInputCol with HasHandleInvalid with DefaultParamsWritable {
    +
    +  @Since("2.3.0")
    +  def this() = this(Identifiable.randomUID("vectSizeHint"))
    +
    +  @Since("2.3.0")
    +  val size = new IntParam(this, "size", "Size of vectors in column.", {s: Int => s >= 0})
    +
    +  @Since("2.3.0")
    --- End diff --
    
    Mark with ```@group getParam```


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r156795030
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala ---
    @@ -0,0 +1,188 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.Transformer
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, VectorUDT}
    +import org.apache.spark.ml.param.{IntParam, Param, ParamMap, ParamValidators}
    +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol}
    +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable}
    +import org.apache.spark.sql.{Column, DataFrame, Dataset}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.StructType
    +
    +/**
    + * :: Experimental ::
    + * A feature transformer that adds size information to the metadata of a vector column.
    + * VectorAssembler needs size information for its input columns and cannot be used on streaming
    + * dataframes without this metadata.
    + *
    + */
    +@Experimental
    +@Since("2.3.0")
    +class VectorSizeHint @Since("2.3.0") (@Since("2.3.0") override val uid: String)
    +  extends Transformer with HasInputCol with HasHandleInvalid with DefaultParamsWritable {
    +
    +  @Since("2.3.0")
    +  def this() = this(Identifiable.randomUID("vectSizeHint"))
    +
    +  /**
    +   * The size of Vectors in `inputCol`.
    +   * @group param
    +   */
    +  @Since("2.3.0")
    +  val size: IntParam = new IntParam(
    +    this,
    +    "size",
    +    "Size of vectors in column.",
    +    {s: Int => s >= 0})
    +
    +  /** group getParam */
    +  @Since("2.3.0")
    +  def getSize: Int = getOrDefault(size)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setSize(value: Int): this.type = set(size, value)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /**
    +   * Param for how to handle invalid entries. Invalid vectors include nulls and vectors with the
    +   * wrong size. The options are `skip` (filter out rows with invalid vectors), `error` (throw an
    +   * error) and `keep` (do not check the vector size, and keep all rows). `error` by default.
    --- End diff --
    
    keep -> optimistic


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r156794633
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala ---
    @@ -0,0 +1,188 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.Transformer
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, VectorUDT}
    +import org.apache.spark.ml.param.{IntParam, Param, ParamMap, ParamValidators}
    +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol}
    +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable}
    +import org.apache.spark.sql.{Column, DataFrame, Dataset}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.StructType
    +
    +/**
    + * :: Experimental ::
    + * A feature transformer that adds size information to the metadata of a vector column.
    + * VectorAssembler needs size information for its input columns and cannot be used on streaming
    + * dataframes without this metadata.
    + *
    + */
    +@Experimental
    +@Since("2.3.0")
    +class VectorSizeHint @Since("2.3.0") (@Since("2.3.0") override val uid: String)
    +  extends Transformer with HasInputCol with HasHandleInvalid with DefaultParamsWritable {
    +
    +  @Since("2.3.0")
    +  def this() = this(Identifiable.randomUID("vectSizeHint"))
    +
    +  /**
    +   * The size of Vectors in `inputCol`.
    +   * @group param
    +   */
    +  @Since("2.3.0")
    +  val size: IntParam = new IntParam(
    +    this,
    +    "size",
    +    "Size of vectors in column.",
    +    {s: Int => s >= 0})
    +
    +  /** group getParam */
    +  @Since("2.3.0")
    +  def getSize: Int = getOrDefault(size)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setSize(value: Int): this.type = set(size, value)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /**
    +   * Param for how to handle invalid entries. Invalid vectors include nulls and vectors with the
    +   * wrong size. The options are `skip` (filter out rows with invalid vectors), `error` (throw an
    +   * error) and `keep` (do not check the vector size, and keep all rows). `error` by default.
    +   * @group param
    +   */
    +  @Since("2.3.0")
    +  override val handleInvalid: Param[String] = new Param[String](
    +    this,
    +    "handleInvalid",
    +    "How to handle invalid vectors in inputCol. Invalid vectors include nulls and vectors with " +
    +      "the wrong size. The options are skip (filter out rows with invalid vectors), error " +
    +      "(throw an error) and keep (do not check the vector size, and keep all rows). `error` by " +
    +      "default.",
    +    ParamValidators.inArray(VectorSizeHint.supportedHandleInvalids))
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setHandleInvalid(value: String): this.type = set(handleInvalid, value)
    +  setDefault(handleInvalid, VectorSizeHint.ERROR_INVALID)
    +
    +  @Since("2.3.0")
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    val localInputCol = getInputCol
    +    val localSize = getSize
    +    val localHandleInvalid = getHandleInvalid
    +
    +    val group = AttributeGroup.fromStructField(dataset.schema(localInputCol))
    +    val newGroup = validateSchemaAndSize(dataset.schema, group)
    +    if (localHandleInvalid == VectorSizeHint.OPTIMISTIC_INVALID && group.size == localSize) {
    +      dataset.toDF()
    +    } else {
    +      val newCol: Column = localHandleInvalid match {
    +        case VectorSizeHint.OPTIMISTIC_INVALID => col(localInputCol)
    +        case VectorSizeHint.ERROR_INVALID =>
    +          val checkVectorSizeUDF = udf { vector: Vector =>
    +            if (vector == null) {
    +              throw new SparkException(s"Got null vector in VectorSizeHint, set `handleInvalid` " +
    +                s"to 'skip' to filter invalid rows.")
    +            }
    +            if (vector.size != localSize) {
    +              throw new SparkException(s"VectorSizeHint Expecting a vector of size $localSize but" +
    +                s" got ${vector.size}")
    +            }
    +            vector
    +          }.asNondeterministic()
    +          checkVectorSizeUDF(col(localInputCol))
    +        case VectorSizeHint.SKIP_INVALID =>
    +          val checkVectorSizeUDF = udf { vector: Vector =>
    +            if (vector != null && vector.size == localSize) {
    +              vector
    +            } else {
    +              null
    +            }
    +          }
    +          checkVectorSizeUDF(col(localInputCol))
    +      }
    +
    +      val res = dataset.withColumn(localInputCol, newCol.as(localInputCol, newGroup.toMetadata()))
    +      if (localHandleInvalid == VectorSizeHint.SKIP_INVALID) {
    +        res.na.drop(Array(localInputCol))
    +      } else {
    +        res
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Checks that schema can be updated with new size and returns a new attribute group with
    +   * updated size.
    +   */
    +  private def validateSchemaAndSize(schema: StructType, group: AttributeGroup): AttributeGroup = {
    +    // This will throw a NoSuchElementException if params are not set.
    +    val localSize = getSize
    +    val localInputCol = getInputCol
    +
    +    val inputColType = schema(getInputCol).dataType
    +    require(
    +      inputColType.isInstanceOf[VectorUDT],
    +      s"Input column, $getInputCol must be of Vector type, got $inputColType"
    +    )
    +    group.size match {
    +      case `localSize` => group
    +      case -1 => new AttributeGroup(localInputCol, localSize)
    +      case _ =>
    +        val msg = s"Trying to set size of vectors in `$localInputCol` to $localSize but size " +
    +          s"already set to ${group.size}."
    +        throw new IllegalArgumentException(msg)
    +    }
    +  }
    +
    +  @Since("2.3.0")
    +  override def transformSchema(schema: StructType): StructType = {
    +    val fieldIndex = schema.fieldIndex(getInputCol)
    +    val fields = schema.fields.clone()
    +    val inputField = fields(fieldIndex)
    +    val group = AttributeGroup.fromStructField(inputField)
    +    val newGroup = validateSchemaAndSize(schema, group)
    +    fields(fieldIndex) = inputField.copy(metadata = newGroup.toMetadata())
    +    StructType(fields)
    +  }
    +
    +  @Since("2.3.0")
    +  override def copy(extra: ParamMap): this.type = defaultCopy(extra)
    +}
    +
    +@Experimental
    --- End diff --
    
    Add Scala docstring here with ```:: Experimental ::``` note.


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    **[Test build #84089 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84089/testReport)** for PR 19746 at commit [`2b1ed0a`](https://github.com/apache/spark/commit/2b1ed0a3a85385f9b4042415889335942b65b9c9).


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

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


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    **[Test build #84122 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84122/testReport)** for PR 19746 at commit [`2b1ed0a`](https://github.com/apache/spark/commit/2b1ed0a3a85385f9b4042415889335942b65b9c9).
     * This patch **fails from timeout after a configured wait of \`250m\`**.
     * 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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r155852524
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.Transformer
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, VectorUDT}
    +import org.apache.spark.ml.param.{IntParam, Param, ParamMap, ParamValidators}
    +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol}
    +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable}
    +import org.apache.spark.sql.{Column, DataFrame, Dataset}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.StructType
    +
    +/**
    + * A feature transformer that adds vector size information to a vector column.
    + */
    +@Experimental
    +@Since("2.3.0")
    +class VectorSizeHint @Since("2.3.0") (@Since("2.3.0") override val uid: String)
    +  extends Transformer with HasInputCol with HasHandleInvalid with DefaultParamsWritable {
    +
    +  @Since("2.3.0")
    +  def this() = this(Identifiable.randomUID("vectSizeHint"))
    +
    +  @Since("2.3.0")
    +  val size = new IntParam(this, "size", "Size of vectors in column.", {s: Int => s >= 0})
    +
    +  @Since("2.3.0")
    +  def getSize: Int = getOrDefault(size)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setSize(value: Int): this.type = set(size, value)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  @Since("2.3.0")
    +  override val handleInvalid: Param[String] = new Param[String](
    +    this,
    +    "handleInvalid",
    +    "How to handle invalid vectors in inputCol, (invalid vectors include nulls and vectors with " +
    +      "the wrong size. The options are `skip` (filter out rows with invalid vectors), `error` " +
    +      "(throw an error) and `optimistic` (don't check the vector size).",
    +    ParamValidators.inArray(VectorSizeHint.supportedHandleInvalids))
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setHandleInvalid(value: String): this.type = set(handleInvalid, value)
    +  setDefault(handleInvalid, VectorSizeHint.ERROR_INVALID)
    +
    +  @Since("2.3.0")
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    val localInputCol = getInputCol
    +    val localSize = getSize
    +    val localHandleInvalid = getHandleInvalid
    +
    +    val group = AttributeGroup.fromStructField(dataset.schema(localInputCol))
    +    if (localHandleInvalid == VectorSizeHint.OPTIMISTIC_INVALID && group.size == localSize) {
    +      dataset.toDF
    +    } else {
    +      val newGroup = group.size match {
    +        case `localSize` => group
    +        case -1 => new AttributeGroup(localInputCol, localSize)
    +        case _ =>
    +          val msg = s"Trying to set size of vectors in `$localInputCol` to $localSize but size " +
    +            s"already set to ${group.size}."
    +          throw new SparkException(msg)
    +      }
    +
    +      val newCol: Column = localHandleInvalid match {
    +        case VectorSizeHint.OPTIMISTIC_INVALID => col(localInputCol)
    +        case VectorSizeHint.ERROR_INVALID =>
    +          val checkVectorSizeUDF = udf { vector: Vector =>
    +            if (vector == null) {
    +              throw new SparkException(s"Got null vector in VectorSizeHint, set `handleInvalid` " +
    +                s"to 'skip' to filter invalid rows.")
    +            }
    +            if (vector.size != localSize) {
    +              throw new SparkException(s"VectorSizeHint Expecting a vector of size $localSize but" +
    +                s" got ${vector.size}")
    +            }
    +            vector
    +          }.asNondeterministic
    +          checkVectorSizeUDF(col(localInputCol))
    +        case VectorSizeHint.SKIP_INVALID =>
    +          val checkVectorSizeUDF = udf { vector: Vector =>
    +            if (vector != null && vector.size == localSize) {
    +              vector
    +            } else {
    +              null
    +            }
    +          }
    +          checkVectorSizeUDF(col(localInputCol))
    +      }
    +
    +      val res = dataset.withColumn(localInputCol, newCol.as(localInputCol, newGroup.toMetadata))
    +      if (localHandleInvalid == VectorSizeHint.SKIP_INVALID) {
    +        res.na.drop(Array(localInputCol))
    +      } else {
    +        res
    +      }
    +    }
    +  }
    +
    +  @Since("2.3.0")
    +  override def transformSchema(schema: StructType): StructType = {
    +    val inputColType = schema(getInputCol).dataType
    +    require(
    +      inputColType.isInstanceOf[VectorUDT],
    +      s"Input column, $getInputCol must be of Vector type, got $inputColType"
    +    )
    +    schema
    +  }
    +
    +  @Since("2.3.0")
    +  override def copy(extra: ParamMap): this.type = defaultCopy(extra)
    +}
    +
    +@Experimental
    +@Since("2.3.0")
    +object VectorSizeHint extends DefaultParamsReadable[VectorSizeHint] {
    +
    +  private[feature] val OPTIMISTIC_INVALID = "optimistic"
    --- End diff --
    
    Can we call this "keep" instead of "optimistic" in order to match handeInvalid Params in other Transformers?


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r155687996
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.Transformer
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, VectorUDT}
    +import org.apache.spark.ml.param.{IntParam, Param, ParamMap, ParamValidators}
    +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol}
    +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable}
    +import org.apache.spark.sql.{Column, DataFrame, Dataset}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.StructType
    +
    +/**
    + * A feature transformer that adds vector size information to a vector column.
    --- End diff --
    
    Also, it'd be good to add more docs about why/when people should use this.


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r155863529
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala ---
    @@ -0,0 +1,173 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, Vectors}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.execution.streaming.MemoryStream
    +import org.apache.spark.sql.streaming.StreamTest
    +
    +class VectorSizeHintSuite
    +  extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  import testImplicits._
    +
    +  test("Test Param Validators") {
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setHandleInvalid("invalidValue"))
    +    intercept[IllegalArgumentException] (new VectorSizeHint().setSize(-3))
    +  }
    +
    +  test("Adding size to column of vectors.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val denseVector = Vectors.dense(1, 2, 3)
    +    val sparseVector = Vectors.sparse(size, Array(), Array())
    +
    +    val data = Seq(denseVector, denseVector, sparseVector).map(Tuple1.apply)
    +    val dataFrame = data.toDF(vectorColName)
    +    assert(
    +      AttributeGroup.fromStructField(dataFrame.schema(vectorColName)).size == -1,
    +      "Transformer did not add expected size data.")
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrame)
    +      assert(
    +        AttributeGroup.fromStructField(withSize.schema(vectorColName)).size == size,
    +        "Transformer did not add expected size data.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size hint preserves attributes.") {
    +
    +    val size = 3
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +    val group = AttributeGroup.fromStructField(dataFrameWithMetadata.schema(vectorColName))
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      val withSize = transformer.transform(dataFrameWithMetadata)
    +
    +      val newGroup = AttributeGroup.fromStructField(withSize.schema(vectorColName))
    +      assert(newGroup.size === size, "Transformer did not add expected size data.")
    +      assert(
    +        newGroup.attributes.get.deep === group.attributes.get.deep,
    +        "SizeHintTransformer did not preserve attributes.")
    +      withSize.collect
    +    }
    +  }
    +
    +  test("Size miss-match between current and target size raises an error.") {
    +    val size = 4
    +    val vectorColName = "vector"
    +    val data = Seq((1, 2, 3), (2, 3, 3))
    +    val dataFrame = data.toDF("x", "y", "z")
    +
    +    val assembler = new VectorAssembler()
    +      .setInputCols(Array("x", "y", "z"))
    +      .setOutputCol(vectorColName)
    +    val dataFrameWithMetadata = assembler.transform(dataFrame)
    +
    +    for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) {
    +      val transformer = new VectorSizeHint()
    +        .setInputCol(vectorColName)
    +        .setSize(size)
    +        .setHandleInvalid(handleInvalid)
    +      intercept[SparkException](transformer.transform(dataFrameWithMetadata))
    +    }
    +  }
    +
    +  test("Handle invalid does the right thing.") {
    +
    +    val vector = Vectors.dense(1, 2, 3)
    +    val short = Vectors.dense(2)
    +    val dataWithNull = Seq(vector, null).map(Tuple1.apply).toDF("vector")
    +    val dataWithShort = Seq(vector, short).map(Tuple1.apply).toDF("vector")
    +
    +    val sizeHint = new VectorSizeHint()
    +      .setInputCol("vector")
    +      .setHandleInvalid("error")
    +      .setSize(3)
    +
    +    intercept[SparkException](sizeHint.transform(dataWithNull).collect)
    +    intercept[SparkException](sizeHint.transform(dataWithShort).collect)
    +
    +    sizeHint.setHandleInvalid("skip")
    +    assert(sizeHint.transform(dataWithNull).count() === 1)
    +    assert(sizeHint.transform(dataWithShort).count() === 1)
    +  }
    --- End diff --
    
    Test keep/optimistic too


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r151954037
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.Transformer
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, VectorUDT}
    +import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
    +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol}
    +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable}
    +import org.apache.spark.sql.{Column, DataFrame, Dataset}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.StructType
    +
    +/**
    + * A feature transformer that adds vector size information to a vector column.
    + */
    +@Experimental
    +@Since("2.3.0")
    +class VectorSizeHint @Since("2.3.0") (@Since("2.3.0") override val uid: String)
    +  extends Transformer with HasInputCol with HasHandleInvalid with DefaultParamsWritable {
    +
    +  @Since("2.3.0")
    +  def this() = this(Identifiable.randomUID("vectSizeHint"))
    +
    +  @Since("2.3.0")
    +  val size = new Param[Int](this, "size", "Size of vectors in column.", {s: Int => s >= 0})
    +
    +  @Since("2.3.0")
    +  def getSize: Int = getOrDefault(size)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setSize(value: Int): this.type = set(size, value)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  @Since("2.3.0")
    +  override val handleInvalid: Param[String] = new Param[String](
    +    this,
    +    "handleInvalid",
    +    "How to handle invalid vectors in inputCol, (invalid vectors include nulls and vectors with " +
    +      "the wrong size. The options are `skip` (filter out rows with invalid vectors), `error` " +
    +      "(throw an error) and `optimistic` (don't check the vector size).",
    +    ParamValidators.inArray(VectorSizeHint.supportedHandleInvalids))
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setHandleInvalid(value: String): this.type = set(handleInvalid, value)
    +  setDefault(handleInvalid, VectorSizeHint.ERROR_INVALID)
    +
    +  @Since("2.3.0")
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    val localInputCol = getInputCol
    +    val localSize = getSize
    +    val localHandleInvalid = getHandleInvalid
    +
    +    val group = AttributeGroup.fromStructField(dataset.schema(localInputCol))
    +    if (localHandleInvalid == VectorSizeHint.OPTIMISTIC_INVALID && group.size == localSize) {
    +      dataset.toDF
    +    } else {
    +      val newGroup = if (group.size == localSize) {
    +        // Pass along any existing metadata about vector.
    +        group
    +      } else {
    +        new AttributeGroup(localInputCol, localSize)
    +      }
    +
    +      val newCol: Column = localHandleInvalid match {
    +        case VectorSizeHint.OPTIMISTIC_INVALID => col(localInputCol)
    +        case VectorSizeHint.ERROR_INVALID =>
    +          val checkVectorSize = { vector: Vector =>
    --- End diff --
    
    I think here can simply use:
    ```
    val checkVectorSizeUDF = udf { vector: Vector => ...}
    checkVectorSizeUDF(col(localInputCol))
    ```
    So code will be clearer.


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

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


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

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


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    **[Test build #84039 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84039/testReport)** for PR 19746 at commit [`7b51563`](https://github.com/apache/spark/commit/7b5156319155e002d317b34fecbc5a39eb6e839f).
     * 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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    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 #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

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


---

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


[GitHub] spark pull request #19746: [SPARK-22346][ML] VectorSizeHint Transformer for ...

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

    https://github.com/apache/spark/pull/19746#discussion_r155852882
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala ---
    @@ -0,0 +1,151 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.Transformer
    +import org.apache.spark.ml.attribute.AttributeGroup
    +import org.apache.spark.ml.linalg.{Vector, VectorUDT}
    +import org.apache.spark.ml.param.{IntParam, Param, ParamMap, ParamValidators}
    +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol}
    +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable}
    +import org.apache.spark.sql.{Column, DataFrame, Dataset}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.StructType
    +
    +/**
    + * A feature transformer that adds vector size information to a vector column.
    + */
    +@Experimental
    +@Since("2.3.0")
    +class VectorSizeHint @Since("2.3.0") (@Since("2.3.0") override val uid: String)
    +  extends Transformer with HasInputCol with HasHandleInvalid with DefaultParamsWritable {
    +
    +  @Since("2.3.0")
    +  def this() = this(Identifiable.randomUID("vectSizeHint"))
    +
    +  @Since("2.3.0")
    +  val size = new IntParam(this, "size", "Size of vectors in column.", {s: Int => s >= 0})
    +
    +  @Since("2.3.0")
    +  def getSize: Int = getOrDefault(size)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setSize(value: Int): this.type = set(size, value)
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  @Since("2.3.0")
    +  override val handleInvalid: Param[String] = new Param[String](
    +    this,
    +    "handleInvalid",
    +    "How to handle invalid vectors in inputCol, (invalid vectors include nulls and vectors with " +
    +      "the wrong size. The options are `skip` (filter out rows with invalid vectors), `error` " +
    +      "(throw an error) and `optimistic` (don't check the vector size).",
    +    ParamValidators.inArray(VectorSizeHint.supportedHandleInvalids))
    +
    +  /** @group setParam */
    +  @Since("2.3.0")
    +  def setHandleInvalid(value: String): this.type = set(handleInvalid, value)
    +  setDefault(handleInvalid, VectorSizeHint.ERROR_INVALID)
    +
    +  @Since("2.3.0")
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    val localInputCol = getInputCol
    +    val localSize = getSize
    +    val localHandleInvalid = getHandleInvalid
    +
    +    val group = AttributeGroup.fromStructField(dataset.schema(localInputCol))
    +    if (localHandleInvalid == VectorSizeHint.OPTIMISTIC_INVALID && group.size == localSize) {
    +      dataset.toDF
    +    } else {
    +      val newGroup = group.size match {
    +        case `localSize` => group
    +        case -1 => new AttributeGroup(localInputCol, localSize)
    +        case _ =>
    +          val msg = s"Trying to set size of vectors in `$localInputCol` to $localSize but size " +
    +            s"already set to ${group.size}."
    +          throw new SparkException(msg)
    +      }
    +
    +      val newCol: Column = localHandleInvalid match {
    +        case VectorSizeHint.OPTIMISTIC_INVALID => col(localInputCol)
    +        case VectorSizeHint.ERROR_INVALID =>
    +          val checkVectorSizeUDF = udf { vector: Vector =>
    +            if (vector == null) {
    +              throw new SparkException(s"Got null vector in VectorSizeHint, set `handleInvalid` " +
    +                s"to 'skip' to filter invalid rows.")
    +            }
    +            if (vector.size != localSize) {
    +              throw new SparkException(s"VectorSizeHint Expecting a vector of size $localSize but" +
    +                s" got ${vector.size}")
    +            }
    +            vector
    +          }.asNondeterministic
    +          checkVectorSizeUDF(col(localInputCol))
    +        case VectorSizeHint.SKIP_INVALID =>
    +          val checkVectorSizeUDF = udf { vector: Vector =>
    +            if (vector != null && vector.size == localSize) {
    +              vector
    +            } else {
    +              null
    +            }
    +          }
    +          checkVectorSizeUDF(col(localInputCol))
    +      }
    +
    +      val res = dataset.withColumn(localInputCol, newCol.as(localInputCol, newGroup.toMetadata))
    +      if (localHandleInvalid == VectorSizeHint.SKIP_INVALID) {
    +        res.na.drop(Array(localInputCol))
    +      } else {
    +        res
    +      }
    +    }
    +  }
    +
    +  @Since("2.3.0")
    +  override def transformSchema(schema: StructType): StructType = {
    +    val inputColType = schema(getInputCol).dataType
    +    require(
    +      inputColType.isInstanceOf[VectorUDT],
    +      s"Input column, $getInputCol must be of Vector type, got $inputColType"
    +    )
    +    schema
    --- End diff --
    
    Since transformSchema does final Param validation checks, let's require that 'size' is specified here.  Also, add size to the metadata.


---

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


[GitHub] spark issue #19746: [SPARK-22346][ML] VectorSizeHint Transformer for using V...

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

    https://github.com/apache/spark/pull/19746
  
    **[Test build #84046 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84046/testReport)** for PR 19746 at commit [`2b1ed0a`](https://github.com/apache/spark/commit/2b1ed0a3a85385f9b4042415889335942b65b9c9).
     * 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