Skip to content

Commit

Permalink
remove useless code and consideration, neaten the code style
Browse files Browse the repository at this point in the history
  • Loading branch information
yinxusen committed Mar 21, 2014
1 parent 7d22941 commit 78c0f25
Show file tree
Hide file tree
Showing 6 changed files with 164 additions and 169 deletions.

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -31,22 +31,22 @@

/**
* The specific InputFormat reads files in HDFS or local disk. It will be called by
* HadoopRDD to generate new BatchFilesRecordReader.
* HadoopRDD to generate new WholeTextFileRecordReader.
*/
public class BatchFilesInputFormat
extends CombineFileInputFormat<String, Text> {
public class WholeTextFileInputFormat
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) BatchFilesRecordReader.class);
}
@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) WholeTextFileRecordReader.class);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,103 @@
/*
* 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.input;

import java.io.IOException;

import com.google.common.io.Closeables;
import org.apache.commons.io.IOUtils;
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 <filename, content> format.
*/

public class WholeTextFileRecordReader extends RecordReader<String, Text> {
private Path path;

private String key = null;
private Text value = null;

private boolean processed = false;

private FileSystem fs;

public WholeTextFileRecordReader(
CombineFileSplit split,
TaskAttemptContext context,
Integer index)
throws IOException {
path = split.getPath(index);
fs = path.getFileSystem(context.getConfiguration());
}

@Override
public void initialize(InputSplit arg0, TaskAttemptContext arg1)
throws IOException, InterruptedException {
}

@Override
public void close() throws IOException {
}

@Override
public float getProgress() throws IOException {
return processed ? 1.0f : 0.0f;
}

@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 (!processed) {
if (key == null) {
key = path.getName();
}
if (value == null) {
value = new Text();
}

FSDataInputStream fileIn = null;
try {
fileIn = fs.open(path);
byte[] innerBuffer = IOUtils.toByteArray(fileIn);
value.set(innerBuffer, 0, innerBuffer.length);
} finally {
Closeables.close(fileIn, false);
}
processed = true;
return true;
}
return false;
}
}
12 changes: 6 additions & 6 deletions mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -20,11 +20,11 @@ package org.apache.spark.mllib.util
import org.apache.hadoop.io.Text
import org.jblas.DoubleMatrix

import org.apache.spark.SparkContext
import org.apache.spark.mllib.input.WholeTextFileInputFormat
import org.apache.spark.mllib.regression.LabeledPoint
import org.apache.spark.rdd.RDD
import org.apache.spark.SparkContext
import org.apache.spark.SparkContext._
import org.apache.spark.mllib.regression.LabeledPoint
import org.apache.spark.mllib.input.BatchFilesInputFormat

/**
* Helper methods to load, save and pre-process data used in ML Lib.
Expand Down Expand Up @@ -124,7 +124,7 @@ object MLUtils {
}

/**
* Reads a bunch of small files from HDFS, or a local file system (available on all nodes), or any
* Reads a bunch of whole 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
Expand All @@ -133,10 +133,10 @@ object MLUtils {
* @return RDD[(fileName: String, content: String)]
* i.e. the first is the file name of a file, the second one is its content.
*/
def smallTextFiles(sc: SparkContext, path: String): RDD[(String, String)] = {
def wholeTextFile(sc: SparkContext, path: String): RDD[(String, String)] = {
sc.newAPIHadoopFile(
path,
classOf[BatchFilesInputFormat],
classOf[WholeTextFileInputFormat],
classOf[String],
classOf[Text]).mapValues(_.toString)
}
Expand Down
Loading

0 comments on commit 78c0f25

Please sign in to comment.