You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nutch.apache.org by cu...@apache.org on 2005/09/07 20:42:13 UTC
svn commit: r279397 -
/lucene/nutch/branches/mapred/src/test/org/apache/nutch/fs/TestNutchFileSystem.java
Author: cutting
Date: Wed Sep 7 11:42:11 2005
New Revision: 279397
URL: http://svn.apache.org/viewcvs?rev=279397&view=rev
Log:
Add seek test.
Modified:
lucene/nutch/branches/mapred/src/test/org/apache/nutch/fs/TestNutchFileSystem.java
Modified: lucene/nutch/branches/mapred/src/test/org/apache/nutch/fs/TestNutchFileSystem.java
URL: http://svn.apache.org/viewcvs/lucene/nutch/branches/mapred/src/test/org/apache/nutch/fs/TestNutchFileSystem.java?rev=279397&r1=279396&r2=279397&view=diff
==============================================================================
--- lucene/nutch/branches/mapred/src/test/org/apache/nutch/fs/TestNutchFileSystem.java (original)
+++ lucene/nutch/branches/mapred/src/test/org/apache/nutch/fs/TestNutchFileSystem.java Wed Sep 7 11:42:11 2005
@@ -31,6 +31,7 @@
private static final Logger LOG = InputFormatBase.LOG;
private static final long MEGA = 1024 * 1024;
+ private static final int SEEKS_PER_FILE = 4;
private static String ROOT = System.getProperty("test.build.data","fs_test");
private static File CONTROL_DIR = new File(ROOT, "fs_control");
@@ -253,11 +254,94 @@
}
+ public static class SeekMapper extends NutchConfigured implements Mapper {
+ private Random random = new Random();
+ private byte[] check = new byte[8192];
+ private NutchFileSystem fs;
+
+ {
+ try {
+ fs = NutchFileSystem.get();
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ public SeekMapper() { super(null); }
+
+ public SeekMapper(NutchConf conf) { super(conf); }
+
+ public void configure(JobConf job) {
+ setConf(job);
+ }
+
+ public void map(WritableComparable key, Writable value,
+ OutputCollector collector, Reporter reporter)
+ throws IOException {
+ String name = ((UTF8)key).toString();
+ long size = ((LongWritable)value).get();
+ long seed = Long.parseLong(name);
+
+ reporter.setStatus("opening " + name);
+
+ NFSDataInputStream in =
+ new NFSDataInputStream(fs.open(new File(DATA_DIR, name)));
+
+ try {
+ for (int i = 0; i < SEEKS_PER_FILE; i++) {
+ // generate a random position
+ long position = Math.abs(random.nextLong()) % size;
+
+ // advance random state to that position
+ random.setSeed(seed);
+ for (int p = 0; p <= position; p+= check.length) {
+ reporter.setStatus("generating data for " + name);
+ random.nextBytes(check);
+ }
+
+ // seek file to that position
+ reporter.setStatus("seeking " + name);
+ in.seek(position);
+
+ // check that byte matches
+ assertEquals(in.readByte(), check[(int)(position % check.length)]);
+
+ }
+ } finally {
+ in.close();
+ }
+ }
+ }
+
+ public static void seekTest(NutchFileSystem fs)
+ throws Exception {
+
+ fs.delete(READ_DIR);
+
+ JobConf job = new JobConf(NutchConf.get());
+
+ job.setInputDir(CONTROL_DIR);
+ job.setInputFormat(SequenceFileInputFormat.class);
+ job.setInputKeyClass(UTF8.class);
+ job.setInputValueClass(LongWritable.class);
+
+ job.setMapperClass(SeekMapper.class);
+ job.setReducerClass(LongSumReducer.class);
+
+ job.setOutputDir(READ_DIR);
+ job.setOutputKeyClass(UTF8.class);
+ job.setOutputValueClass(LongWritable.class);
+ job.setNumReduceTasks(1);
+ JobClient.runJob(job);
+ }
+
+
public static void main(String[] args) throws Exception {
int megaBytes = 10;
int files = 100;
boolean noRead = false;
boolean noWrite = false;
+ boolean noSeek = false;
long seed = new Random().nextLong();
String usage = "Usage: TestNutchFileSystem -files N -megaBytes M [-noread] [-nowrite]";
@@ -290,6 +374,9 @@
}
if (!noRead) {
readTest(fs);
+ }
+ if (!noSeek) {
+ seekTest(fs);
}
}