Skip to content
Closed
Show file tree
Hide file tree
Changes from 18 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 @@
org.apache.spark.ml.source.libsvm.DefaultSource
Original file line number Diff line number Diff line change
@@ -0,0 +1,96 @@
/*
* 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.mllib.linalg.VectorUDT
import org.apache.spark.mllib.util.MLUtils
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.types.{StructType, StructField, DoubleType}
import org.apache.spark.sql.{Row, SQLContext}
import org.apache.spark.sql.sources._

/**
* LibSVMRelation provides the DataFrame constructed from LibSVM format data.
* @param path File path of LibSVM format
* @param numFeatures The number of features
* @param vectorType The type of vector. It can be 'sparse' or 'dense'
* @param sqlContext The Spark SQLContext
*/
private[ml] class LibSVMRelation(val path: String, val numFeatures: Int, val vectorType: String)
(@transient val sqlContext: SQLContext)
extends BaseRelation with TableScan with Logging with Serializable {

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

override def buildScan(): RDD[Row] = {
val sc = sqlContext.sparkContext
val baseRdd = MLUtils.loadLibSVMFile(sc, path, numFeatures)

baseRdd.map { pt =>
val features = if (vectorType == "dense") pt.features.toDense else pt.features.toSparse
Row(pt.label, features)
}
}

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
}

}

/**
* This is used for creating DataFrame from LibSVM format file.
* The LibSVM file path must be specified to DefaultSource.
*/
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 = {
require(parameters.contains("path"), "'path' must be specified")
parameters.get("path").get
}

/**
* 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 vectorType = parameters.getOrElse("vectorType", "sparse")
new LibSVMRelation(path, numFeatures, vectorType)(sqlContext)
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,79 @@
/*
* 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 java.io.IOException;

import com.google.common.base.Charsets;
import com.google.common.io.Files;

import org.apache.spark.mllib.linalg.DenseVector;
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 (com.google and org.junit should be in the same group). If you use Intellij, you can try this plugin: https://plugins.jetbrains.com/plugin/7350.

import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;

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;


/**
* 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);

File tmpDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "datasource");
path = File.createTempFile("datasource", "libsvm-relation", tmpDir);
Copy link
Contributor

Choose a reason for hiding this comment

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

The directory is already unique. We don't need another temp filename. path = new File(tempDir.getPath, "part-00000"); should work.


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;
path.delete();
Copy link
Contributor

Choose a reason for hiding this comment

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

Should call Utils.deleteRecursively on tmpDir.

}

@Test
public void verifyLibSVMDF() {
dataset = jsql.read().format("libsvm").option("vectorType", "dense").load(path.getPath());
Assert.assertEquals("label", dataset.columns()[0]);
Assert.assertEquals("features", dataset.columns()[1]);
Row r = dataset.first();
Assert.assertEquals(Double.valueOf(1.0), Double.valueOf(r.getDouble(0)));
Copy link
Contributor

Choose a reason for hiding this comment

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

Did you try assertEquals(1.0, r.getDouble(0))?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, I tried. assertEquals does not seem to be able to compare primitive floating-point numbers.

java.lang.AssertionError: Use assertEquals(expected, actual, delta) to compare floating-point numbers
        at org.junit.Assert.fail(Assert.java:93)
        at org.junit.Assert.assertEquals(Assert.java:493)
        at org.junit.Assert.assertEquals(Assert.java:482)
        at org.apache.spark.ml.source.JavaLibSVMRelationSuite.verifyLibSVMDF(JavaLibSVMRelationSuite.java:75)

Copy link
Contributor

Choose a reason for hiding this comment

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

You can use a small delta here like 1e-15.

DenseVector v = r.getAs(1);
Assert.assertEquals(Vectors.dense(1.0, 0.0, 2.0, 0.0, 3.0, 0.0), v);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,85 @@
/*
* 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.format("libsvm").load(path)
assert(df.columns(0) == "label")
assert(df.columns(1) == "features")
val row1 = df.first()
assert(row1.getDouble(0) == 1.0)
val v = row1.getAs[SparseVector](1)
assert(v == Vectors.sparse(6, Seq((0, 1.0), (2, 2.0), (4, 3.0))))
}

test("select as dense vector") {
val df = sqlContext.read.format("libsvm").options(Map("vectorType" -> "dense"))
.load(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)
val v = row1.getAs[DenseVector](1)
assert(v == Vectors.dense(1.0, 0.0, 2.0, 0.0, 3.0, 0.0))
}

test("select long vector with specifying the number of features") {
val lines =
"""
|1 1:1 10:2 20:3 30:4 40:5 50:6 60:7 70:8 80:9 90:10 100:1
|0 1:1 10:10 20:9 30:8 40:7 50:6 60:5 70:4 80:3 90:2 100:1
""".stripMargin
Copy link
Contributor

Choose a reason for hiding this comment

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

100 is the correct number of features. We can use the same data file created in beforeAll but set numFeatures greater than 6 and verify the output vector size. It is not necessary to create a separate file.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

So sorry, I misunderstood that. Thank you.

val tempDir = Utils.createTempDir()
val file = new File(tempDir.getPath, "part-00001")
Files.write(lines, file, Charsets.US_ASCII)
val df = sqlContext.read.option("numFeatures", "100").format("libsvm")
.load(tempDir.toURI.toString)
val row1 = df.first()
val v = row1.getAs[SparseVector](1)
assert(v == Vectors.sparse(100, Seq((0, 1.0), (9, 2.0), (19, 3.0), (29, 4.0), (39, 5.0),
(49, 6.0), (59, 7.0), (69, 8.0), (79, 9.0), (89, 10.0), (99, 1.0))))
}
}