-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-1133] add small files input in MLlib #164
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 2 commits
fd93e59
9bf87d4
e3681f2
e7423d4
c27a7ab
2fa26ec
79e547f
e108b9a
f9d8a83
cc2655a
a18ea00
d55ec86
6112270
ab747d3
79d07d6
67fa71c
1678931
ffe272d
7d22941
66a03e5
ca76423
9aadcff
e09139d
78c0f25
7e17fe6
2c0aa22
dab5439
d780983
646e554
d348362
abf6714
57a4379
8265dc7
80c2968
eae90e4
839bd3f
21109fb
56db8a2
8043b7b
dc126f2
5140598
b637f2d
007a733
05ed628
134ace7
71d4ed2
f8111ea
8237df8
f87dab8
4f7d547
b859853
3b69987
a0853a3
345825d
b0ea02a
32cbdfd
e15e574
be6d96c
3e63d98
1fa48d9
4ed60d1
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -45,3 +45,4 @@ dist/ | |
| spark-*-bin.tar.gz | ||
| unit-tests.log | ||
| /lib/ | ||
| mllib/build/ | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,52 @@ | ||
| /* | ||
| * 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.mllib.util; | ||
|
|
||
| import java.io.IOException; | ||
|
|
||
| import org.apache.hadoop.fs.Path; | ||
| import org.apache.hadoop.io.Text; | ||
| import org.apache.hadoop.mapreduce.InputSplit; | ||
| import org.apache.hadoop.mapreduce.JobContext; | ||
| import org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat; | ||
| import org.apache.hadoop.mapreduce.lib.input.CombineFileRecordReader; | ||
| import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit; | ||
| import org.apache.hadoop.mapreduce.RecordReader; | ||
| import org.apache.hadoop.mapreduce.TaskAttemptContext; | ||
|
|
||
| /** | ||
| * The specific InputFormat reads files in HDFS or local disk. It will be called by | ||
| * HadoopRDD to generate new BatchFileRecordReader. | ||
| */ | ||
| public class BatchFileInputFormat | ||
|
||
| extends CombineFileInputFormat<String, Text> { | ||
|
||
|
|
||
| @Override | ||
| protected boolean isSplitable(JobContext context, Path file) { | ||
| return false; | ||
| } | ||
| @Override | ||
| public RecordReader<String, Text> createRecordReader( | ||
| InputSplit split, | ||
| TaskAttemptContext context) throws IOException { | ||
| return new CombineFileRecordReader<String, Text>( | ||
| (CombineFileSplit)split, | ||
|
||
| context, | ||
| (Class)BatchFileRecordReader.class); | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,117 @@ | ||
| /* | ||
| * 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.mllib.util; | ||
|
|
||
| import java.io.IOException; | ||
|
|
||
| import org.apache.hadoop.fs.Path; | ||
| import org.apache.hadoop.fs.FSDataInputStream; | ||
| import org.apache.hadoop.fs.FileSystem; | ||
| import org.apache.hadoop.io.Text; | ||
| import org.apache.hadoop.mapreduce.InputSplit; | ||
| import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit; | ||
| import org.apache.hadoop.mapreduce.RecordReader; | ||
| import org.apache.hadoop.mapreduce.TaskAttemptContext; | ||
|
|
||
| /** | ||
| * Reads an entire file out in bytes format in <filename, content> format. | ||
|
||
| */ | ||
|
|
||
| public class BatchFileRecordReader extends RecordReader<String, Text> { | ||
| private long startOffset; | ||
| private long end; | ||
| private long pos; | ||
| private Path path; | ||
|
|
||
| private static final int MAX_BYTES_ALLOCATION = 64 * 1024 * 1024; | ||
|
|
||
| private String key = null; | ||
| private Text value = null; | ||
|
|
||
| private FSDataInputStream fileIn; | ||
|
|
||
| public BatchFileRecordReader( | ||
| CombineFileSplit split, | ||
| TaskAttemptContext context, | ||
| Integer index) | ||
| throws IOException { | ||
| path = split.getPath(index); | ||
| startOffset = split.getOffset(index); | ||
| pos = startOffset; | ||
| end = startOffset + split.getLength(index); | ||
|
|
||
| FileSystem fs = path.getFileSystem(context.getConfiguration()); | ||
| fileIn = fs.open(path); | ||
| fileIn.seek(startOffset); | ||
| } | ||
|
|
||
| @Override | ||
| public void initialize(InputSplit arg0, TaskAttemptContext arg1) | ||
| throws IOException, InterruptedException {} | ||
|
|
||
| @Override | ||
| public void close() throws IOException { | ||
| if (fileIn != null) { | ||
| fileIn.close(); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public float getProgress() throws IOException { | ||
| if (startOffset == end) return 0; | ||
|
||
| return Math.min(1.0f, (pos - startOffset) / (float) (end - startOffset)); | ||
| } | ||
|
|
||
| @Override | ||
| public String getCurrentKey() throws IOException, InterruptedException { | ||
| return key; | ||
| } | ||
|
|
||
| @Override | ||
| public Text getCurrentValue() throws IOException, InterruptedException{ | ||
| return value; | ||
| } | ||
|
|
||
| @Override | ||
| public boolean nextKeyValue() throws IOException { | ||
| if (key == null) { | ||
| key = path.getName(); | ||
| } | ||
| if (value == null) { | ||
| value = new Text(); | ||
| } | ||
|
|
||
| if (pos >= end) { | ||
| return false; | ||
| } | ||
|
|
||
| int maxBufferLength = end - pos < Integer.MAX_VALUE ? (int) (end - pos) : Integer.MAX_VALUE; | ||
| if (maxBufferLength > MAX_BYTES_ALLOCATION) { | ||
| maxBufferLength = MAX_BYTES_ALLOCATION; | ||
| } | ||
|
|
||
| byte[] innerBuffer = new byte[maxBufferLength]; | ||
|
|
||
| int len = fileIn.read(pos, innerBuffer, 0, maxBufferLength); | ||
| pos += len; | ||
|
|
||
| value.set(innerBuffer, 0, len); | ||
|
|
||
| return true; | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,11 +17,12 @@ | |
|
|
||
| package org.apache.spark.mllib.util | ||
|
|
||
| import org.apache.hadoop.io.Text | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. organize imports
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think here
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. okay |
||
| import org.jblas.DoubleMatrix | ||
|
|
||
| import org.apache.spark.SparkContext | ||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.SparkContext._ | ||
|
|
||
| import org.jblas.DoubleMatrix | ||
| import org.apache.spark.mllib.regression.LabeledPoint | ||
|
|
||
| /** | ||
|
|
@@ -120,4 +121,22 @@ object MLUtils { | |
| } | ||
| sum | ||
| } | ||
|
|
||
| /** | ||
| * Reads a bunch of small files from HDFS, or a local file system (available on all nodes), or any | ||
| * Hadoop-supported file system URI, and return an RDD[(String, String)]. | ||
| * | ||
| * @param path The directory you should specified, such as | ||
| * hdfs://[address]:[port]/[dir] | ||
| * | ||
| * @return RDD[(fileName: String, content: String)] | ||
| * i.e. the first is the file name of a file, the second one is its content. | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. do not need to repeat |
||
| */ | ||
| def smallTextFiles(sc: SparkContext, path: String): RDD[(String, String)] = { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The current API uses |
||
| sc.newAPIHadoopFile( | ||
| path, | ||
| classOf[BatchFileInputFormat], | ||
| classOf[String], | ||
| classOf[Text]).mapValues(_.toString) | ||
| } | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Instead of
util, how aboutioorinput?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think
inputis better, because input of machine learning algorithm is usually datasets, batch/streaming files, in different format. Meantime, theoutputis usually models or prediction results, which has a different semantic meaning.