Browse Source

HADOOP-2820. The following classes in streaming are removed : StreamLineRecordReader StreamOutputFormat StreamSequenceRecordReader. Contributed by Amareshwari Sri Ramadasu.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@633064 13f79535-47bb-0310-9956-ffa450edef68
Devaraj Das 17 years ago
parent
commit
c6b764e10f

+ 5 - 0
CHANGES.txt

@@ -116,6 +116,11 @@ Trunk (unreleased changes)
     HADOOP-2912. MiniDFSCluster restart should wait for namenode to exit
     HADOOP-2912. MiniDFSCluster restart should wait for namenode to exit
     safemode. This was causing TestFsck to fail.  (Mahadev Konar via dhruba)
     safemode. This was causing TestFsck to fail.  (Mahadev Konar via dhruba)
 
 
+    HADOOP-2820. The following classes in streaming are removed : 
+    StreamLineRecordReader StreamOutputFormat StreamSequenceRecordReader.
+    (Amareshwari Sri Ramadasu via ddas)
+
+
 Release 0.16.1 - Unreleased
 Release 0.16.1 - Unreleased
 
 
   IMPROVEMENTS
   IMPROVEMENTS

+ 0 - 1
src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamBaseRecordReader.java

@@ -36,7 +36,6 @@ import org.apache.commons.logging.*;
  * Shared functionality for hadoopStreaming formats.
  * Shared functionality for hadoopStreaming formats.
  * A custom reader can be defined to be a RecordReader with the constructor below
  * A custom reader can be defined to be a RecordReader with the constructor below
  * and is selected with the option bin/hadoopStreaming -inputreader ...
  * and is selected with the option bin/hadoopStreaming -inputreader ...
- * @see StreamLineRecordReader
  * @see StreamXmlRecordReader 
  * @see StreamXmlRecordReader 
  */
  */
 public abstract class StreamBaseRecordReader implements RecordReader<Text, Text> {
 public abstract class StreamBaseRecordReader implements RecordReader<Text, Text> {

+ 0 - 38
src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamLineRecordReader.java

@@ -1,38 +0,0 @@
-/**
- * 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.hadoop.streaming;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapred.KeyValueLineRecordReader;
-import org.apache.hadoop.mapred.FileSplit;
-
-/**
- * same as org.apache.hadoop.mapred.KeyValueLineRecordReader
- * 
- * @deprecated
- */
-public class StreamLineRecordReader extends KeyValueLineRecordReader {
-
-  public StreamLineRecordReader(Configuration job, FileSplit split)
-    throws IOException {
-    super(job, split);
-  }
-}

+ 0 - 28
src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamOutputFormat.java

@@ -1,28 +0,0 @@
-/**
- * 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.hadoop.streaming;
-
-import org.apache.hadoop.mapred.TextOutputFormat;
-
-/** Same as org.apache.hadoop.mapred.TextOutputFormat, 
- * @deprecated
- */
-public class StreamOutputFormat extends TextOutputFormat {
-
-}

+ 0 - 38
src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamSequenceRecordReader.java

@@ -1,38 +0,0 @@
-/**
- * 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.hadoop.streaming;
-
-import java.io.*;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapred.FileSplit;
-import org.apache.hadoop.mapred.SequenceFileRecordReader;
-
-/**
- * same as org.apache.hadoop.mapred.SequenceFileRecordReader
- * 
- * @deprecated
- */
-public class StreamSequenceRecordReader extends SequenceFileRecordReader {
-
-  public StreamSequenceRecordReader(Configuration conf, FileSplit split)
-    throws IOException {
-    super(conf, split);
-  }
-}