Pārlūkot izejas kodu

HADOOP-3705. Fix mapred.join parser to accept InputFormats named with
underscore and static, inner classes.


git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@688960 13f79535-47bb-0310-9956-ffa450edef68

Christopher Douglas 17 gadi atpakaļ
vecāks
revīzija
02223fbb57

+ 3 - 0
CHANGES.txt

@@ -368,6 +368,9 @@ Trunk (unreleased changes)
     HADOOP-3506. Fix a rare NPE caused by error handling in S3. (Tom White via
     cdouglas)
 
+    HADOOP-3705. Fix mapred.join parser to accept InputFormats named with
+    underscore and static, inner classes. (cdouglas)
+
 Release 0.18.0 - 2008-08-19
 
   INCOMPATIBLE CHANGES

+ 2 - 0
src/mapred/org/apache/hadoop/mapred/join/Parser.java

@@ -135,6 +135,8 @@ public class Parser {
       tok.ordinaryChar(',');
       tok.ordinaryChar('(');
       tok.ordinaryChar(')');
+      tok.wordChars('$','$');
+      tok.wordChars('_','_');
     }
 
     Token next() throws IOException {

+ 0 - 85
src/test/org/apache/hadoop/mapred/join/FakeIF.java

@@ -1,85 +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.mapred.join;
-
-import java.io.IOException;
-import java.io.DataInput;
-import java.io.DataOutput;
-
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.JobConfigurable;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.util.ReflectionUtils;
-
-public class FakeIF<K,V>
-    implements InputFormat<K,V>, JobConfigurable {
-
-  public static class FakeSplit implements InputSplit {
-    public void write(DataOutput out) throws IOException { }
-    public void readFields(DataInput in) throws IOException { }
-    public long getLength() { return 0L; }
-    public String[] getLocations() { return new String[0]; }
-  }
-
-  public static void setKeyClass(JobConf job, Class<?> k) {
-    job.setClass("test.fakeif.keyclass", k, WritableComparable.class);
-  }
-
-  public static void setValClass(JobConf job, Class<?> v) {
-    job.setClass("test.fakeif.valclass", v, Writable.class);
-  }
-
-  private Class<? extends K> keyclass;
-  private Class<? extends V> valclass;
-
-  @SuppressWarnings("unchecked")
-  public void configure(JobConf job) {
-    keyclass = (Class<? extends K>) job.getClass("test.fakeif.keyclass",
-	IncomparableKey.class, WritableComparable.class);
-    valclass = (Class<? extends V>) job.getClass("test.fakeif.valclass",
-	NullWritable.class, WritableComparable.class);
-  }
-
-  public FakeIF() { }
-
-  public InputSplit[] getSplits(JobConf conf, int splits) {
-    return new InputSplit[] { new FakeSplit() };
-  }
-
-  public RecordReader<K,V> getRecordReader(
-      InputSplit ignored, JobConf conf, Reporter reporter) {
-    return new RecordReader<K,V>() {
-      public boolean next(K key, V value) throws IOException { return false; }
-      public K createKey() {
-        return ReflectionUtils.newInstance(keyclass, null);
-      }
-      public V createValue() {
-        return ReflectionUtils.newInstance(valclass, null);
-      }
-      public long getPos() throws IOException { return 0L; }
-      public void close() throws IOException { }
-      public float getProgress() throws IOException { return 0.0f; }
-    };
-  }
-}

+ 66 - 6
src/test/org/apache/hadoop/mapred/join/TestDatamerge.java

@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.mapred.join;
 
+import java.io.DataInput;
+import java.io.DataOutput;
 import java.io.IOException;
 import java.util.Iterator;
 
@@ -26,25 +28,31 @@ import junit.framework.TestSuite;
 import junit.extensions.TestSetup;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobConfigurable;
 import org.apache.hadoop.mapred.Mapper;
 import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hadoop.mapred.Reducer;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.SequenceFileInputFormat;
 import org.apache.hadoop.mapred.SequenceFileOutputFormat;
 import org.apache.hadoop.mapred.lib.IdentityMapper;
 import org.apache.hadoop.mapred.lib.IdentityReducer;
+import org.apache.hadoop.util.ReflectionUtils;
 
 public class TestDatamerge extends TestCase {
 
@@ -285,7 +293,7 @@ public class TestDatamerge extends TestCase {
       if (i + 1 != SOURCES) sb.append(",");
     }
     sb.append("),outer(");
-    sb.append(CompositeInputFormat.compose(FakeIF.class,"foobar"));
+    sb.append(CompositeInputFormat.compose(Fake_IF.class,"foobar"));
     sb.append(",");
     for (int i = 0; i < SOURCES; ++i) {
       sb.append(
@@ -293,13 +301,13 @@ public class TestDatamerge extends TestCase {
             src[i].toString()));
       sb.append(",");
     }
-    sb.append(CompositeInputFormat.compose(FakeIF.class,"raboof") + "))");
+    sb.append(CompositeInputFormat.compose(Fake_IF.class,"raboof") + "))");
     job.set("mapred.join.expr", sb.toString());
     job.setInputFormat(CompositeInputFormat.class);
     Path outf = new Path(base, "out");
     FileOutputFormat.setOutputPath(job, outf);
-    FakeIF.setKeyClass(job, IntWritable.class);
-    FakeIF.setValClass(job, IntWritable.class);
+    Fake_IF.setKeyClass(job, IntWritable.class);
+    Fake_IF.setValClass(job, IntWritable.class);
 
     job.setMapperClass(IdentityMapper.class);
     job.setReducerClass(IdentityReducer.class);
@@ -345,7 +353,7 @@ public class TestDatamerge extends TestCase {
     Path base = cluster.getFileSystem().makeQualified(new Path("/empty"));
     Path[] src = { new Path(base,"i0"), new Path("i1"), new Path("i2") };
     job.set("mapred.join.expr", CompositeInputFormat.compose("outer",
-        FakeIF.class, src));
+        Fake_IF.class, src));
     job.setInputFormat(CompositeInputFormat.class);
     FileOutputFormat.setOutputPath(job, new Path(base, "out"));
 
@@ -357,4 +365,56 @@ public class TestDatamerge extends TestCase {
     JobClient.runJob(job);
     base.getFileSystem(job).delete(base, true);
   }
+
+  public static class Fake_IF<K,V>
+      implements InputFormat<K,V>, JobConfigurable {
+
+    public static class FakeSplit implements InputSplit {
+      public void write(DataOutput out) throws IOException { }
+      public void readFields(DataInput in) throws IOException { }
+      public long getLength() { return 0L; }
+      public String[] getLocations() { return new String[0]; }
+    }
+
+    public static void setKeyClass(JobConf job, Class<?> k) {
+      job.setClass("test.fakeif.keyclass", k, WritableComparable.class);
+    }
+
+    public static void setValClass(JobConf job, Class<?> v) {
+      job.setClass("test.fakeif.valclass", v, Writable.class);
+    }
+
+    private Class<? extends K> keyclass;
+    private Class<? extends V> valclass;
+
+    @SuppressWarnings("unchecked")
+    public void configure(JobConf job) {
+      keyclass = (Class<? extends K>) job.getClass("test.fakeif.keyclass",
+    IncomparableKey.class, WritableComparable.class);
+      valclass = (Class<? extends V>) job.getClass("test.fakeif.valclass",
+    NullWritable.class, WritableComparable.class);
+    }
+
+    public Fake_IF() { }
+
+    public InputSplit[] getSplits(JobConf conf, int splits) {
+      return new InputSplit[] { new FakeSplit() };
+    }
+
+    public RecordReader<K,V> getRecordReader(
+        InputSplit ignored, JobConf conf, Reporter reporter) {
+      return new RecordReader<K,V>() {
+        public boolean next(K key, V value) throws IOException { return false; }
+        public K createKey() {
+          return ReflectionUtils.newInstance(keyclass, null);
+        }
+        public V createValue() {
+          return ReflectionUtils.newInstance(valclass, null);
+        }
+        public long getPos() throws IOException { return 0L; }
+        public void close() throws IOException { }
+        public float getProgress() throws IOException { return 0.0f; }
+      };
+    }
+  }
 }