Author: cutting Date: Fri Sep 9 09:59:50 2005 New Revision: 279816 URL: http://svn.apache.org/viewcvs?rev=279816&view=rev Log: Consistently use configured buffer size; increase some NDFS timeouts.
Modified: lucene/nutch/branches/mapred/src/java/org/apache/nutch/fs/FileUtil.java lucene/nutch/branches/mapred/src/java/org/apache/nutch/fs/NDFSFileSystem.java lucene/nutch/branches/mapred/src/java/org/apache/nutch/ndfs/DataNode.java lucene/nutch/branches/mapred/src/java/org/apache/nutch/ndfs/FSConstants.java lucene/nutch/branches/mapred/src/java/org/apache/nutch/ndfs/NDFSClient.java lucene/nutch/branches/mapred/src/test/org/apache/nutch/fs/TestNutchFileSystem.java Modified: lucene/nutch/branches/mapred/src/java/org/apache/nutch/fs/FileUtil.java URL: http://svn.apache.org/viewcvs/lucene/nutch/branches/mapred/src/java/org/apache/nutch/fs/FileUtil.java?rev=279816&r1=279815&r2=279816&view=diff ============================================================================== --- lucene/nutch/branches/mapred/src/java/org/apache/nutch/fs/FileUtil.java (original) +++ lucene/nutch/branches/mapred/src/java/org/apache/nutch/fs/FileUtil.java Fri Sep 9 09:59:50 2005 @@ -18,6 +18,8 @@ import java.io.*; +import org.apache.nutch.util.NutchConf; + /** * A collection of file-processing util methods */ @@ -55,7 +57,7 @@ DataInputStream in = new DataInputStream(nfs.open(src)); try { DataOutputStream out = new DataOutputStream(nfs.create(dst)); - byte buf[] = new byte[2048]; + byte buf[] = new byte[NutchConf.get().getInt("io.file.buffer.size", 4096)]; try { int readBytes = in.read(buf); Modified: lucene/nutch/branches/mapred/src/java/org/apache/nutch/fs/NDFSFileSystem.java URL: http://svn.apache.org/viewcvs/lucene/nutch/branches/mapred/src/java/org/apache/nutch/fs/NDFSFileSystem.java?rev=279816&r1=279815&r2=279816&view=diff ============================================================================== --- lucene/nutch/branches/mapred/src/java/org/apache/nutch/fs/NDFSFileSystem.java (original) +++ lucene/nutch/branches/mapred/src/java/org/apache/nutch/fs/NDFSFileSystem.java Fri Sep 9 09:59:50 2005 @@ -22,6 +22,7 @@ import org.apache.nutch.io.*; import org.apache.nutch.ndfs.*; +import org.apache.nutch.util.NutchConf; /**************************************************************** * Implement the NutchFileSystem interface for the NDFS system. @@ -177,7 +178,7 @@ doFromLocalFile(contents[i], new File(dst, contents[i].getName()), deleteSource); } } else { - byte buf[] = new byte[4096]; + byte buf[] = new byte[NutchConf.get().getInt("io.file.buffer.size", 4096)]; InputStream in = new BufferedInputStream(new FileInputStream(src)); try { OutputStream out = create(dst); @@ -222,7 +223,7 @@ copyToLocalFile(contents[i], new File(dst, contents[i].getName())); } } else { - byte buf[] = new byte[4096]; + byte buf[] = new byte[NutchConf.get().getInt("io.file.buffer.size", 4096)]; InputStream in = open(src); try { dst.getParentFile().mkdirs(); Modified: lucene/nutch/branches/mapred/src/java/org/apache/nutch/ndfs/DataNode.java URL: http://svn.apache.org/viewcvs/lucene/nutch/branches/mapred/src/java/org/apache/nutch/ndfs/DataNode.java?rev=279816&r1=279815&r2=279816&view=diff ============================================================================== --- lucene/nutch/branches/mapred/src/java/org/apache/nutch/ndfs/DataNode.java (original) +++ lucene/nutch/branches/mapred/src/java/org/apache/nutch/ndfs/DataNode.java Fri Sep 9 09:59:50 2005 @@ -160,6 +160,7 @@ Block toDelete[] = namenode.blockReport(localName, data.getBlockReport()); data.invalidate(toDelete); lastBlockReport = now; + continue; } if (receivedBlockList.size() > 0) { // @@ -355,7 +356,7 @@ // try { boolean anotherChunk = true; - byte buf[] = new byte[2048]; + byte buf[] = new byte[BUFFER_SIZE]; while (anotherChunk) { while (len > 0) { @@ -489,7 +490,7 @@ out.writeLong(amtSkipped); } - byte buf[] = new byte[4096]; + byte buf[] = new byte[BUFFER_SIZE]; try { int bytesRead = 0; try { @@ -562,7 +563,7 @@ this.curTarget = createSocketAddr(targets[0].getName().toString()); this.targets = targets; this.b = b; - this.buf = new byte[2048]; + this.buf = new byte[BUFFER_SIZE]; } /** Modified: lucene/nutch/branches/mapred/src/java/org/apache/nutch/ndfs/FSConstants.java URL: http://svn.apache.org/viewcvs/lucene/nutch/branches/mapred/src/java/org/apache/nutch/ndfs/FSConstants.java?rev=279816&r1=279815&r2=279816&view=diff ============================================================================== --- lucene/nutch/branches/mapred/src/java/org/apache/nutch/ndfs/FSConstants.java (original) +++ lucene/nutch/branches/mapred/src/java/org/apache/nutch/ndfs/FSConstants.java Fri Sep 9 09:59:50 2005 @@ -15,6 +15,8 @@ */ package org.apache.nutch.ndfs; +import org.apache.nutch.util.NutchConf; + /************************************ * Some handy constants * @@ -99,9 +101,14 @@ // Timeouts, constants // public static long HEARTBEAT_INTERVAL = 3 * 1000; - public static long EXPIRE_INTERVAL = 60 * 1000; - public static long BLOCKREPORT_INTERVAL = 10 * 60 * 1000; - public static long DATANODE_STARTUP_PERIOD = 120 * 1000; - public static long LEASE_PERIOD = 16 * 1000; - public static int READ_TIMEOUT = 20 * 1000; + public static long EXPIRE_INTERVAL = 10 * 60 * 1000; + public static long BLOCKREPORT_INTERVAL = 60 * 60 * 1000; + public static long DATANODE_STARTUP_PERIOD = 2 * 60 * 1000; + public static long LEASE_PERIOD = 60 * 1000; + public static int READ_TIMEOUT = 60 * 1000; + + public static int BUFFER_SIZE = + NutchConf.get().getInt("io.file.buffer.size", 4096); + } + Modified: lucene/nutch/branches/mapred/src/java/org/apache/nutch/ndfs/NDFSClient.java URL: http://svn.apache.org/viewcvs/lucene/nutch/branches/mapred/src/java/org/apache/nutch/ndfs/NDFSClient.java?rev=279816&r1=279815&r2=279816&view=diff ============================================================================== --- lucene/nutch/branches/mapred/src/java/org/apache/nutch/ndfs/NDFSClient.java (original) +++ lucene/nutch/branches/mapred/src/java/org/apache/nutch/ndfs/NDFSClient.java Fri Sep 9 09:59:50 2005 @@ -33,7 +33,6 @@ ********************************************************/ public class NDFSClient implements FSConstants { public static final Logger LOG = LogFormatter.getLogger("org.apache.nutch.fs.NDFSClient"); - static int BUFFER_SIZE = 4096; static int MAX_BLOCK_ACQUIRE_FAILURES = 10; ClientProtocol namenode; boolean running = true; @@ -793,7 +792,7 @@ nextBlockOutputStream(false); InputStream in = new FileInputStream(backupFile); try { - byte buf[] = new byte[4096]; + byte buf[] = new byte[BUFFER_SIZE]; int bytesRead = in.read(buf); while (bytesRead >= 0) { blockStream.writeLong((long) bytesRead); 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=279816&r1=279815&r2=279816&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 Fri Sep 9 09:59:50 2005 @@ -30,6 +30,9 @@ public class TestNutchFileSystem extends TestCase { private static final Logger LOG = InputFormatBase.LOG; + private static int BUFFER_SIZE = + NutchConf.get().getInt("io.file.buffer.size", 4096); + private static final long MEGA = 1024 * 1024; private static final int SEEKS_PER_FILE = 4; @@ -98,7 +101,7 @@ public static class WriteMapper extends NutchConfigured implements Mapper { private Random random = new Random(); - private byte[] buffer = new byte[8192]; + private byte[] buffer = new byte[BUFFER_SIZE]; private NutchFileSystem fs; { @@ -173,8 +176,8 @@ public static class ReadMapper extends NutchConfigured implements Mapper { private Random random = new Random(); - private byte[] buffer = new byte[8192]; - private byte[] check = new byte[8192]; + private byte[] buffer = new byte[BUFFER_SIZE]; + private byte[] check = new byte[BUFFER_SIZE]; private NutchFileSystem fs; { @@ -257,7 +260,7 @@ public static class SeekMapper extends NutchConfigured implements Mapper { private Random random = new Random(); - private byte[] check = new byte[8192]; + private byte[] check = new byte[BUFFER_SIZE]; private NutchFileSystem fs; {