5
0
mirror of https://github.com/apache/sqoop.git synced 2025-05-03 05:01:10 +08:00

SQOOP-354 SQOOP needs to be made compatible with Hadoop .23 release

git-svn-id: https://svn.apache.org/repos/asf/incubator/sqoop/trunk@1199149 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Bilung Lee 2011-11-08 08:05:02 +00:00
parent bfbfe757e8
commit b8de0e5026
3 changed files with 4 additions and 107 deletions

View File

@ -31,10 +31,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.MapContext;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import com.cloudera.sqoop.testutil.MockObjectFactory;
import com.cloudera.sqoop.testutil.MockResultSet;
/**
@ -45,7 +42,6 @@ public class TestLargeObjectLoader extends TestCase {
protected Configuration conf;
protected LargeObjectLoader loader;
protected Path outDir;
protected MapContext mapContext;
public void setUp() throws IOException, InterruptedException {
conf = new Configuration();
@ -60,17 +56,7 @@ public void setUp() throws IOException, InterruptedException {
}
fs.mkdirs(outDir);
/* A mock MapContext that uses FileOutputCommitter.
* This MapContext is actually serving two roles here; when writing the
* CLOB files, its OutputCommitter is used to determine where to write
* the CLOB data, as these are placed in the task output work directory.
* When reading the CLOB data back for verification, we use the
* getInputSplit() to determine where to read our source data from--the same
* directory. We are repurposing the same context for both output and input.
*/
mapContext = MockObjectFactory.getMapContextForIOPath(conf, outDir);
loader = new LargeObjectLoader(mapContext.getConfiguration(),
FileOutputFormat.getWorkOutputPath(mapContext));
loader = new LargeObjectLoader(conf, outDir);
}
public void testReadClobRef()
@ -88,7 +74,6 @@ public void testReadClobRef()
assertNotNull(clob);
assertTrue(clob.isExternal());
loader.close();
mapContext.getOutputCommitter().commitTask(mapContext);
Reader r = clob.getDataStream(conf, outDir);
char [] buf = new char[4096];
int chars = r.read(buf, 0, 4096);
@ -117,7 +102,6 @@ public void testReadBlobRef()
assertNotNull(blob);
assertTrue(blob.isExternal());
loader.close();
mapContext.getOutputCommitter().commitTask(mapContext);
InputStream is = blob.getDataStream(conf, outDir);
byte [] buf = new byte[4096];
int bytes = is.read(buf, 0, 4096);

View File

@ -23,13 +23,14 @@
import java.io.DataOutput;
import java.io.IOException;
import junit.framework.TestCase;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.*;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.WritableComparable;
import org.apache.hadoop.mapred.HadoopTestCase;
import org.apache.hadoop.mapreduce.*;
import org.apache.hadoop.mapreduce.lib.db.*;
import org.apache.hadoop.mapreduce.lib.output.*;
@ -38,7 +39,7 @@
/**
* Test aspects of DataDrivenDBInputFormat.
*/
public class TestDataDrivenDBInputFormat extends HadoopTestCase {
public class TestDataDrivenDBInputFormat extends TestCase {
private static final Log LOG = LogFactory.getLog(
TestDataDrivenDBInputFormat.class);
@ -52,10 +53,6 @@ public class TestDataDrivenDBInputFormat extends HadoopTestCase {
private static final String OUT_DIR;
public TestDataDrivenDBInputFormat() throws IOException {
super(LOCAL_MR, LOCAL_FS, 1, 1);
}
static {
OUT_DIR = System.getProperty("test.build.data", "/tmp") + "/dddbifout";
}

View File

@ -1,84 +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 com.cloudera.sqoop.testutil;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.Counters;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.MapContext;
import org.apache.hadoop.mapreduce.OutputCommitter;
import org.apache.hadoop.mapreduce.TaskAttemptID;
import org.apache.hadoop.mapreduce.lib.input.FileSplit;
import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
import org.apache.hadoop.mrunit.mapreduce.mock.MockReporter;
/**
* Allows the creation of various mock objects for testing purposes.
*/
public final class MockObjectFactory {
/**
* Returns a mock MapContext that has both an OutputCommitter and an
* InputSplit wired to the specified path.
* Used for testing LargeObjectLoader.
*/
public static MapContext getMapContextForIOPath(Configuration conf, Path p) {
return new MockMapContextWithCommitter(conf, p);
}
private static class MockMapContextWithCommitter
extends MapContext<Object, Object, Object, Object> {
private Path path;
private Configuration conf;
public MockMapContextWithCommitter(Configuration c, Path p) {
super(c, new TaskAttemptID("jt", 0, true, 0, 0),
null, null, null, new MockReporter(new Counters()), null);
this.path = p;
this.conf = c;
}
@Override
public OutputCommitter getOutputCommitter() {
try {
return new FileOutputCommitter(path, this);
} catch (IOException ioe) {
return null;
}
}
@Override
public InputSplit getInputSplit() {
return new FileSplit(new Path(path, "inputFile"), 0, 0, new String[0]);
}
@Override
public Configuration getConfiguration() {
return conf;
}
}
private MockObjectFactory() {
// Disable explicity object creation
}
}