Skip to content
Closed
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,97 @@
/*
* 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.source.libsvm

import com.google.common.base.Objects
import org.apache.spark.Logging
import org.apache.spark.annotation.Since
import org.apache.spark.mllib.linalg.{VectorUDT, Vector}
import org.apache.spark.mllib.regression.LabeledPoint
import org.apache.spark.mllib.util.MLUtils
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.types._
import org.apache.spark.sql.{Row, SQLContext}
import org.apache.spark.sql.sources.{DataSourceRegister, PrunedScan, BaseRelation, RelationProvider}

/**
* LibSVMRelation provides the DataFrame constructed from LibSVM format data.
* @param path
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fill in documentation

* @param numFeatures
* @param vectorType
* @param sqlContext
*/
private[ml] class LibSVMRelation(val path: String, val numFeatures: Int, val vectorType: String)
(@transient val sqlContext: SQLContext)
extends BaseRelation with PrunedScan with Logging {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Since we need to read the entire file anyway, it doesn't save much with PrunedScan. Maybe TableScan is simpler but sufficient.


override def schema: StructType = StructType(
StructField("label", DoubleType, nullable = false) ::
StructField("features", new VectorUDT(), nullable = false) :: Nil
)

override def buildScan(requiredColumns: Array[String]): RDD[Row] = {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

See my comments above about PrunedScan vs. TableScan.

val sc = sqlContext.sparkContext
val baseRdd = MLUtils.loadLibSVMFile(sc, path, numFeatures)

val rowBuilders = requiredColumns.map {
case "label" => (pt: LabeledPoint) => Seq(pt.label)
case "features" if vectorType == "sparse" => (pt: LabeledPoint) => Seq(pt.features.toSparse)
case "features" if vectorType == "dense" => (pt: LabeledPoint) => Seq(pt.features.toDense)
}

baseRdd.map(pt => {
Row.fromSeq(rowBuilders.map(_(pt)).reduceOption(_ ++ _).getOrElse(Seq.empty))
})
}

override def hashCode(): Int = {
Objects.hashCode(path, schema)
}

override def equals(other: Any): Boolean = other match {
case that: LibSVMRelation => (this.path == that.path) && this.schema.equals(that.schema)
case _ => false
}

}

class DefaultSource extends RelationProvider with DataSourceRegister {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

also missing doc

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.


override def shortName(): String = "libsvm"
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

add @Since("1.6.0")


private def checkPath(parameters: Map[String, String]): String = {
parameters.getOrElse("path", sys.error("'path' must be specified"))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

`require(parameters.contains("path"), "'path' must be specified"`)

}

/**
* Returns a new base relation with the given parameters.
* Note: the parameters' keywords are case insensitive and this insensitivity is enforced
* by the Map that is passed to the function.
*/
override def createRelation(sqlContext: SQLContext, parameters: Map[String, String])
: BaseRelation = {
val path = checkPath(parameters)
val numFeatures = parameters.getOrElse("numFeatures", "-1").toInt
/**
* featuresType can be selected "dense" or "sparse".
* This parameter decides the type of returned feature vector.
*/
val featuresType = parameters.getOrElse("featuresType", "sparse")
new LibSVMRelation(path, numFeatures, featuresType)(sqlContext)
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/*
* 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.source

import org.apache.spark.sql.{DataFrame, DataFrameReader}

package object libsvm {

/**
* Implicit declaration in order to be used from SQLContext.
* It is necessary to import org.apache.spark.ml.source.libsvm._
* @param read
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

missing doc

*/
implicit class LibSVMReader(read: DataFrameReader) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we should just remove all the implicits. It was a mistake in the beginning to encourage them.

Now the preferred format is just

sqlContext.read.format("libsvm").load(...)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Simply I did sqlContext.read.format("libsvm").load(...). However, not class found exception was occurred. Is there some way to register resource shortname to DaraFrameReader?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

def libsvm(filePath: String): DataFrame
= read.format(classOf[DefaultSource].getName).load(filePath)
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
package org.apache.spark.ml.source;

import com.google.common.base.Charsets;
import com.google.common.io.Files;
import org.apache.spark.api.java.JavaSparkContext;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ditto. organize imports

import org.apache.spark.mllib.linalg.Vectors;
import org.apache.spark.sql.DataFrame;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SQLContext;
import org.apache.spark.util.Utils;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;

import java.io.File;
import java.io.IOException;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

organize imports: java, scala, 3rd-party, spark.


/**
* Test LibSVMRelation in Java.
*/
public class JavaLibSVMRelationSuite {
private transient JavaSparkContext jsc;
private transient SQLContext jsql;
private transient DataFrame dataset;

private File path;

@Before
public void setUp() throws IOException {
jsc = new JavaSparkContext("local", "JavaLibSVMRelationSuite");
jsql = new SQLContext(jsc);

path = Utils.createTempDir(System.getProperty("java.io.tmpdir"),
"datasource").getCanonicalFile();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

path = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "datasource")
  .getCanonicalFile();

if (path.exists()) {
path.delete();
}

String s = "1 1:1.0 3:2.0 5:3.0\n0\n0 2:4.0 4:5.0 6:6.0";
Files.write(s, path, Charsets.US_ASCII);
}

@After
public void tearDown() {
jsc.stop();
jsc = null;
}

@Test
public void verifyLibSvmDF() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LibSVM to be consistent.

dataset = jsql.read().format("libsvm").load();
Assert.assertEquals(dataset.columns()[0], "label");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In JUnit, assertEquals takes the expected value as the first arg. So it should be "label", dataset.columns()[0] here.

Assert.assertEquals(dataset.columns()[1], "features");
Row r = dataset.first();
Assert.assertTrue(r.getDouble(0) == 1.0);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

use assertEquals

Assert.assertEquals(r.getAs(1), Vectors.dense(1.0, 0.0, 2.0, 0.0, 3.0, 0.0));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We need to check the class name first or cast it to DenseVector directly:

DenseVector v = r.getAs(1)
Assert.assertEquals(Vectors.dense(...), v)

If it is a sparse vector, the first line will throw an error.

}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,72 @@
/*
* 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.source

import java.io.File

import com.google.common.base.Charsets
import com.google.common.io.Files
import org.apache.spark.SparkFunSuite
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ditto. organize imports

import org.apache.spark.ml.source.libsvm._
import org.apache.spark.mllib.linalg.{SparseVector, Vectors, DenseVector}
import org.apache.spark.mllib.util.MLlibTestSparkContext
import org.apache.spark.util.Utils

class LibSVMRelationSuite extends SparkFunSuite with MLlibTestSparkContext {
var path: String = _

override def beforeAll(): Unit = {
super.beforeAll()
val lines =
"""
|1 1:1.0 3:2.0 5:3.0
|0
|0 2:4.0 4:5.0 6:6.0
""".stripMargin
val tempDir = Utils.createTempDir()
val file = new File(tempDir.getPath, "part-00000")
Files.write(lines, file, Charsets.US_ASCII)
path = tempDir.toURI.toString
}

test("select as sparse vector") {
val df = sqlContext.read.options(Map("numFeatures" -> "6")).libsvm(path)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can remove "numFeatures" -> 6 in one test.

assert(df.columns(0) == "label")
assert(df.columns(1) == "features")
val row1 = df.first()
assert(row1.getDouble(0) == 1.0)
assert(row1.getAs[SparseVector](1) == Vectors.sparse(6, Seq((0, 1.0), (2, 2.0), (4, 3.0))))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This doesn't verify the result is a sparse vector because runtime type erasure. We need

val v = row1.getAs[SparseVector](1)
assert(v == Vectors.sparse(...))

to force check.

}

test("select as dense vector") {
val df = sqlContext.read.options(Map("numFeatures" -> "6", "featuresType" -> "dense"))
.libsvm(path)
assert(df.columns(0) == "label")
assert(df.columns(1) == "features")
assert(df.count() == 3)
val row1 = df.first()
assert(row1.getDouble(0) == 1.0)
assert(row1.getAs[DenseVector](1) == Vectors.dense(1.0, 0.0, 2.0, 0.0, 3.0, 0.0))
}

test("select without any option") {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should add another test that sets numFeatures to a larger number and verify it.

val df = sqlContext.read.libsvm(path)
val row1 = df.first()
assert(row1.getAs[SparseVector](1) == Vectors.sparse(6, Seq((0, 1.0), (2, 2.0), (4, 3.0))))
}
}