You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by cu...@apache.org on 2007/03/01 04:47:29 UTC
svn commit: r513122 [3/3] - in /lucene/hadoop/trunk: ./ src/c++/librecordio/
src/java/org/apache/hadoop/io/ src/java/org/apache/hadoop/record/
src/java/org/apache/hadoop/record/compiler/
src/java/org/apache/hadoop/record/compiler/ant/ src/java/org/apac...
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/generated/Rcc.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/generated/Rcc.java?view=diff&rev=513122&r1=513121&r2=513122
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/generated/Rcc.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/generated/Rcc.java Wed Feb 28 19:47:27 2007
@@ -31,9 +31,10 @@
public class Rcc implements RccConstants {
private static String language = "java";
private static String destDir = ".";
- private static ArrayList recFiles = new ArrayList();
+ private static ArrayList<String> recFiles = new ArrayList<String>();
+ private static ArrayList<String> cmdargs = new ArrayList<String>();
private static JFile curFile;
- private static Hashtable recTab;
+ private static Hashtable<String,JRecord> recTab;
private static String curDir = ".";
private static String curFileName;
private static String curModuleName;
@@ -52,26 +53,28 @@
"--destdir".equalsIgnoreCase(args[i])) {
destDir = args[i+1];
i++;
+ } else if (args[i].startsWith("-")) {
+ String arg = args[i].substring(1);
+ if (arg.startsWith("-")) {
+ arg = arg.substring(1);
+ }
+ cmdargs.add(arg.toLowerCase());
} else {
recFiles.add(args[i]);
}
}
- if (!"c++".equals(language) && !"java".equals(language)) {
- System.err.println("Cannot recognize language:" + language);
- return 1;
- }
if (recFiles.size() == 0) {
System.err.println("No record files specified. Exiting.");
return 1;
}
for (int i=0; i<recFiles.size(); i++) {
- curFileName = (String) recFiles.get(i);
+ curFileName = recFiles.get(i);
File file = new File(curFileName);
try {
FileReader reader = new FileReader(file);
Rcc parser = new Rcc(reader);
try {
- recTab = new Hashtable();
+ recTab = new Hashtable<String,JRecord>();
curFile = parser.Input();
} catch (ParseException e) {
System.err.println(e.toString());
@@ -87,7 +90,7 @@
return 1;
}
try {
- int retCode = curFile.genCode(language, destDir);
+ int retCode = curFile.genCode(language, destDir, cmdargs);
if (retCode != 0) { return retCode; }
} catch (IOException e) {
System.err.println(e.toString());
@@ -98,10 +101,10 @@
}
final public JFile Input() throws ParseException {
- ArrayList ilist = new ArrayList();
- ArrayList rlist = new ArrayList();
+ ArrayList<JFile> ilist = new ArrayList<JFile>();
+ ArrayList<JRecord> rlist = new ArrayList<JRecord>();
JFile i;
- ArrayList l;
+ ArrayList<JRecord> l;
label_1:
while (true) {
switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
@@ -170,9 +173,9 @@
throw new Error("Missing return statement in function");
}
- final public ArrayList Module() throws ParseException {
+ final public ArrayList<JRecord> Module() throws ParseException {
String mName;
- ArrayList rlist;
+ ArrayList<JRecord> rlist;
jj_consume_token(MODULE_TKN);
mName = ModuleName();
curModuleName = mName;
@@ -206,8 +209,8 @@
throw new Error("Missing return statement in function");
}
- final public ArrayList RecordList() throws ParseException {
- ArrayList rlist = new ArrayList();
+ final public ArrayList<JRecord> RecordList() throws ParseException {
+ ArrayList<JRecord> rlist = new ArrayList<JRecord>();
JRecord r;
label_3:
while (true) {
@@ -228,9 +231,9 @@
final public JRecord Record() throws ParseException {
String rname;
- ArrayList flist = new ArrayList();
+ ArrayList<JField<JType>> flist = new ArrayList<JField<JType>>();
Token t;
- JField f;
+ JField<JType> f;
jj_consume_token(RECORD_TKN);
t = jj_consume_token(IDENT_TKN);
rname = t.image;
@@ -267,12 +270,12 @@
throw new Error("Missing return statement in function");
}
- final public JField Field() throws ParseException {
+ final public JField<JType> Field() throws ParseException {
JType jt;
Token t;
jt = Type();
t = jj_consume_token(IDENT_TKN);
- {if (true) return new JField(jt, t.image);}
+ {if (true) return new JField<JType>(t.image, jt);}
throw new Error("Missing return statement in function");
}
@@ -326,7 +329,7 @@
if (rname.indexOf('.', 0) < 0) {
rname = curModuleName + "." + rname;
}
- JRecord r = (JRecord) recTab.get(rname);
+ JRecord r = recTab.get(rname);
if (r == null) {
System.out.println("Type " + rname + " not known. Exiting.");
System.exit(1);
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/generated/rcc.jj
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/generated/rcc.jj?view=diff&rev=513122&r1=513121&r2=513122
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/generated/rcc.jj (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/generated/rcc.jj Wed Feb 28 19:47:27 2007
@@ -35,9 +35,10 @@
public class Rcc {
private static String language = "java";
private static String destDir = ".";
- private static ArrayList recFiles = new ArrayList();
+ private static ArrayList<String> recFiles = new ArrayList<String>();
+ private static ArrayList<String> cmdargs = new ArrayList<String>();
private static JFile curFile;
- private static Hashtable recTab;
+ private static Hashtable<String,JRecord> recTab;
private static String curDir = ".";
private static String curFileName;
private static String curModuleName;
@@ -56,26 +57,28 @@
"--destdir".equalsIgnoreCase(args[i])) {
destDir = args[i+1];
i++;
+ } else if (args[i].startsWith("-")) {
+ String arg = args[i].substring(1);
+ if (arg.startsWith("-")) {
+ arg = arg.substring(1);
+ }
+ cmdargs.add(arg.toLowerCase());
} else {
recFiles.add(args[i]);
}
}
- if (!"c++".equals(language) && !"java".equals(language)) {
- System.err.println("Cannot recognize language:" + language);
- return 1;
- }
if (recFiles.size() == 0) {
System.err.println("No record files specified. Exiting.");
return 1;
}
for (int i=0; i<recFiles.size(); i++) {
- curFileName = (String) recFiles.get(i);
+ curFileName = recFiles.get(i);
File file = new File(curFileName);
try {
FileReader reader = new FileReader(file);
Rcc parser = new Rcc(reader);
try {
- recTab = new Hashtable();
+ recTab = new Hashtable<String,JRecord>();
curFile = parser.Input();
} catch (ParseException e) {
System.err.println(e.toString());
@@ -91,7 +94,7 @@
return 1;
}
try {
- int retCode = curFile.genCode(language, destDir);
+ int retCode = curFile.genCode(language, destDir, cmdargs);
if (retCode != 0) { return retCode; }
} catch (IOException e) {
System.err.println(e.toString());
@@ -170,10 +173,10 @@
JFile Input() :
{
- ArrayList ilist = new ArrayList();
- ArrayList rlist = new ArrayList();
+ ArrayList<JFile> ilist = new ArrayList<JFile>();
+ ArrayList<JRecord> rlist = new ArrayList<JRecord>();
JFile i;
- ArrayList l;
+ ArrayList<JRecord> l;
}
{
(
@@ -227,10 +230,10 @@
}
}
-ArrayList Module() :
+ArrayList<JRecord> Module() :
{
String mName;
- ArrayList rlist;
+ ArrayList<JRecord> rlist;
}
{
<MODULE_TKN>
@@ -258,9 +261,9 @@
{ return name; }
}
-ArrayList RecordList() :
+ArrayList<JRecord> RecordList() :
{
- ArrayList rlist = new ArrayList();
+ ArrayList<JRecord> rlist = new ArrayList<JRecord>();
JRecord r;
}
{
@@ -274,9 +277,9 @@
JRecord Record() :
{
String rname;
- ArrayList flist = new ArrayList();
+ ArrayList<JField<JType>> flist = new ArrayList<JField<JType>>();
Token t;
- JField f;
+ JField<JType> f;
}
{
<RECORD_TKN>
@@ -297,7 +300,7 @@
}
}
-JField Field() :
+JField<JType> Field() :
{
JType jt;
Token t;
@@ -305,7 +308,7 @@
{
jt = Type()
t = <IDENT_TKN>
- { return new JField(jt, t.image); }
+ { return new JField<JType>(t.image, jt); }
}
JType Type() :
@@ -340,7 +343,7 @@
if (rname.indexOf('.', 0) < 0) {
rname = curModuleName + "." + rname;
}
- JRecord r = (JRecord) recTab.get(rname);
+ JRecord r = recTab.get(rname);
if (r == null) {
System.out.println("Type " + rname + " not known. Exiting.");
System.exit(1);
Modified: lucene/hadoop/trunk/src/test/ddl/buffer.jr
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/test/ddl/buffer.jr?view=diff&rev=513122&r1=513121&r2=513122
==============================================================================
--- lucene/hadoop/trunk/src/test/ddl/buffer.jr (original)
+++ lucene/hadoop/trunk/src/test/ddl/buffer.jr Wed Feb 28 19:47:27 2007
@@ -1,6 +1,6 @@
-module org.apache.hadoop.record.test {
+module org.apache.hadoop.record {
class RecBuffer {
- buffer Data;
+ buffer data;
}
}
Modified: lucene/hadoop/trunk/src/test/ddl/int.jr
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/test/ddl/int.jr?view=diff&rev=513122&r1=513121&r2=513122
==============================================================================
--- lucene/hadoop/trunk/src/test/ddl/int.jr (original)
+++ lucene/hadoop/trunk/src/test/ddl/int.jr Wed Feb 28 19:47:27 2007
@@ -1,6 +1,6 @@
-module org.apache.hadoop.record.test {
+module org.apache.hadoop.record {
class RecInt {
- int Data;
+ int data;
}
}
Modified: lucene/hadoop/trunk/src/test/ddl/string.jr
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/test/ddl/string.jr?view=diff&rev=513122&r1=513121&r2=513122
==============================================================================
--- lucene/hadoop/trunk/src/test/ddl/string.jr (original)
+++ lucene/hadoop/trunk/src/test/ddl/string.jr Wed Feb 28 19:47:27 2007
@@ -1,6 +1,6 @@
-module org.apache.hadoop.record.test {
+module org.apache.hadoop.record {
class RecString {
- ustring Data;
+ ustring data;
}
}
Modified: lucene/hadoop/trunk/src/test/ddl/test.jr
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/test/ddl/test.jr?view=diff&rev=513122&r1=513121&r2=513122
==============================================================================
--- lucene/hadoop/trunk/src/test/ddl/test.jr (original)
+++ lucene/hadoop/trunk/src/test/ddl/test.jr Wed Feb 28 19:47:27 2007
@@ -1,21 +1,21 @@
-module org.apache.hadoop.record.test {
+module org.apache.hadoop.record {
class RecRecord0 {
- ustring StringVal;
+ ustring stringVal;
}
class RecRecord1 {
- boolean BoolVal;
- byte ByteVal;
- int IntVal;
- long LongVal;
- float FloatVal; // testing inline comment
- double DoubleVal; /* testing comment */
- ustring StringVal; /* testing multi-line
+ boolean boolVal;
+ byte byteVal;
+ int intVal;
+ long longVal;
+ float floatVal; // testing inline comment
+ double doubleVal; /* testing comment */
+ ustring stringVal; /* testing multi-line
* comment */
- buffer BufferVal; // testing another // inline comment
- vector<ustring> VectorVal;
- map<ustring,ustring> MapVal;
- RecRecord0 RecordVal;
+ buffer bufferVal; // testing another // inline comment
+ vector<ustring> vectorVal;
+ map<ustring, ustring> mapVal;
+ RecRecord0 recordVal;
}
}
Copied: lucene/hadoop/trunk/src/test/org/apache/hadoop/record/FromCpp.java (from r513042, lucene/hadoop/trunk/src/test/org/apache/hadoop/record/test/FromCpp.java)
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/test/org/apache/hadoop/record/FromCpp.java?view=diff&rev=513122&p1=lucene/hadoop/trunk/src/test/org/apache/hadoop/record/test/FromCpp.java&r1=513042&p2=lucene/hadoop/trunk/src/test/org/apache/hadoop/record/FromCpp.java&r2=513122
==============================================================================
--- lucene/hadoop/trunk/src/test/org/apache/hadoop/record/test/FromCpp.java (original)
+++ lucene/hadoop/trunk/src/test/org/apache/hadoop/record/FromCpp.java Wed Feb 28 19:47:27 2007
@@ -16,9 +16,8 @@
* limitations under the License.
*/
-package org.apache.hadoop.record.test;
+package org.apache.hadoop.record;
-import org.apache.hadoop.record.RecordReader;
import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.FileInputStream;
@@ -26,8 +25,6 @@
import java.util.ArrayList;
import java.util.TreeMap;
import junit.framework.*;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.Text;
/**
*
@@ -56,10 +53,10 @@
r1.setDoubleVal(1.5234);
r1.setIntVal(4567);
r1.setLongVal(0x5a5a5a5a5a5aL);
- r1.setStringVal(new Text("random text"));
- r1.setBufferVal(new BytesWritable());
- r1.setVectorVal(new ArrayList());
- r1.setMapVal(new TreeMap());
+ r1.setStringVal("random text");
+ r1.setBufferVal(new Buffer());
+ r1.setVectorVal(new ArrayList<String>());
+ r1.setMapVal(new TreeMap<String,String>());
FileInputStream istream = new FileInputStream(tmpfile);
RecordReader in = new RecordReader(istream, "binary");
RecRecord1 r2 = new RecRecord1();
@@ -82,10 +79,10 @@
r1.setDoubleVal(1.5234);
r1.setIntVal(4567);
r1.setLongVal(0x5a5a5a5a5a5aL);
- r1.setStringVal(new Text("random text"));
- r1.setBufferVal(new BytesWritable());
- r1.setVectorVal(new ArrayList());
- r1.setMapVal(new TreeMap());
+ r1.setStringVal("random text");
+ r1.setBufferVal(new Buffer());
+ r1.setVectorVal(new ArrayList<String>());
+ r1.setMapVal(new TreeMap<String,String>());
FileInputStream istream = new FileInputStream(tmpfile);
RecordReader in = new RecordReader(istream, "csv");
RecRecord1 r2 = new RecRecord1();
@@ -108,10 +105,10 @@
r1.setDoubleVal(1.5234);
r1.setIntVal(4567);
r1.setLongVal(0x5a5a5a5a5a5aL);
- r1.setStringVal(new Text("random text"));
- r1.setBufferVal(new BytesWritable());
- r1.setVectorVal(new ArrayList());
- r1.setMapVal(new TreeMap());
+ r1.setStringVal("random text");
+ r1.setBufferVal(new Buffer());
+ r1.setVectorVal(new ArrayList<String>());
+ r1.setMapVal(new TreeMap<String,String>());
FileInputStream istream = new FileInputStream(tmpfile);
RecordReader in = new RecordReader(istream, "xml");
RecRecord1 r2 = new RecRecord1();
Added: lucene/hadoop/trunk/src/test/org/apache/hadoop/record/TestBuffer.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/test/org/apache/hadoop/record/TestBuffer.java?view=auto&rev=513122
==============================================================================
--- lucene/hadoop/trunk/src/test/org/apache/hadoop/record/TestBuffer.java (added)
+++ lucene/hadoop/trunk/src/test/org/apache/hadoop/record/TestBuffer.java Wed Feb 28 19:47:27 2007
@@ -0,0 +1,126 @@
+/**
+ * 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.record;
+
+import junit.framework.*;
+
+/**
+ * A Unit test for Record I/O Buffer class
+ *
+ * @author milindb
+ */
+public class TestBuffer extends TestCase {
+
+ public TestBuffer(String testName) {
+ super(testName);
+ }
+
+ /**
+ * Test of set method, of class org.apache.hadoop.record.Buffer.
+ */
+ public void testSet() {
+ final byte[] bytes = new byte[10];
+ final Buffer instance = new Buffer();
+
+ instance.set(bytes);
+
+ assertEquals("set failed", bytes, instance.get());
+ }
+
+ /**
+ * Test of copy method, of class org.apache.hadoop.record.Buffer.
+ */
+ public void testCopy() {
+ final byte[] bytes = new byte[10];
+ final int offset = 6;
+ final int length = 3;
+ for (int idx = 0; idx < 10; idx ++) {
+ bytes[idx] = (byte) idx;
+ }
+ final Buffer instance = new Buffer();
+
+ instance.copy(bytes, offset, length);
+
+ assertEquals("copy failed", 3, instance.getCapacity());
+ assertEquals("copy failed", 3, instance.get().length);
+ for (int idx = 0; idx < 3; idx++) {
+ assertEquals("Buffer content corrupted", idx+6, instance.get()[idx]);
+ }
+ }
+
+ /**
+ * Test of getCount method, of class org.apache.hadoop.record.Buffer.
+ */
+ public void testGetCount() {
+ final Buffer instance = new Buffer();
+
+ final int expResult = 0;
+ final int result = instance.getCount();
+ assertEquals("getSize failed", expResult, result);
+ }
+
+ /**
+ * Test of getCapacity method, of class org.apache.hadoop.record.Buffer.
+ */
+ public void testGetCapacity() {
+ final Buffer instance = new Buffer();
+
+ final int expResult = 0;
+ final int result = instance.getCapacity();
+ assertEquals("getCapacity failed", expResult, result);
+
+ instance.setCapacity(100);
+ assertEquals("setCapacity failed", 100, instance.getCapacity());
+ }
+
+ /**
+ * Test of truncate method, of class org.apache.hadoop.record.Buffer.
+ */
+ public void testTruncate() {
+ final Buffer instance = new Buffer();
+ instance.setCapacity(100);
+ assertEquals("setCapacity failed", 100, instance.getCapacity());
+
+ instance.truncate();
+ assertEquals("truncate failed", 0, instance.getCapacity());
+ }
+
+ /**
+ * Test of append method, of class org.apache.hadoop.record.Buffer.
+ */
+ public void testAppend() {
+ final byte[] bytes = new byte[100];
+ final int offset = 0;
+ final int length = 100;
+ for (int idx = 0; idx < 100; idx++) {
+ bytes[idx] = (byte) (100-idx);
+ }
+
+ final Buffer instance = new Buffer();
+
+ instance.append(bytes, offset, length);
+
+ assertEquals("Buffer size mismatch", 100, instance.getCount());
+
+ for (int idx = 0; idx < 100; idx++) {
+ assertEquals("Buffer contents corrupted", 100-idx, instance.get()[idx]);
+ }
+
+ }
+}
Copied: lucene/hadoop/trunk/src/test/org/apache/hadoop/record/TestRecordIO.java (from r513042, lucene/hadoop/trunk/src/test/org/apache/hadoop/record/test/TestRecordIO.java)
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/test/org/apache/hadoop/record/TestRecordIO.java?view=diff&rev=513122&p1=lucene/hadoop/trunk/src/test/org/apache/hadoop/record/test/TestRecordIO.java&r1=513042&p2=lucene/hadoop/trunk/src/test/org/apache/hadoop/record/TestRecordIO.java&r2=513122
==============================================================================
--- lucene/hadoop/trunk/src/test/org/apache/hadoop/record/test/TestRecordIO.java (original)
+++ lucene/hadoop/trunk/src/test/org/apache/hadoop/record/TestRecordIO.java Wed Feb 28 19:47:27 2007
@@ -16,19 +16,15 @@
* limitations under the License.
*/
-package org.apache.hadoop.record.test;
+package org.apache.hadoop.record;
import java.io.IOException;
import junit.framework.*;
-import org.apache.hadoop.record.RecordWriter;
-import org.apache.hadoop.record.RecordReader;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.util.ArrayList;
import java.util.TreeMap;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.Text;
/**
*
@@ -57,14 +53,14 @@
r1.setByteVal((byte)0x66);
r1.setFloatVal(3.145F);
r1.setDoubleVal(1.5234);
- r1.setIntVal(4567);
- r1.setLongVal(0x5a5a5a5a5a5aL);
- r1.setStringVal(new Text("random text"));
- r1.setBufferVal(new BytesWritable());
- r1.setVectorVal(new ArrayList());
- r1.setMapVal(new TreeMap());
+ r1.setIntVal(-4567);
+ r1.setLongVal(-2367L);
+ r1.setStringVal("random text");
+ r1.setBufferVal(new Buffer());
+ r1.setVectorVal(new ArrayList<String>());
+ r1.setMapVal(new TreeMap<String,String>());
RecRecord0 r0 = new RecRecord0();
- r0.setStringVal(new Text("other random text"));
+ r0.setStringVal("other random text");
r1.setRecordVal(r0);
out.write(r1);
ostream.close();
@@ -93,12 +89,12 @@
r1.setDoubleVal(1.5234);
r1.setIntVal(4567);
r1.setLongVal(0x5a5a5a5a5a5aL);
- r1.setStringVal(new Text("random text"));
- r1.setBufferVal(new BytesWritable());
- r1.setVectorVal(new ArrayList());
- r1.setMapVal(new TreeMap());
+ r1.setStringVal("random text");
+ r1.setBufferVal(new Buffer());
+ r1.setVectorVal(new ArrayList<String>());
+ r1.setMapVal(new TreeMap<String,String>());
RecRecord0 r0 = new RecRecord0();
- r0.setStringVal(new Text("other random text"));
+ r0.setStringVal("other random text");
r1.setRecordVal(r0);
out.write(r1);
ostream.close();
@@ -124,12 +120,12 @@
r1.setDoubleVal(1.5234);
r1.setIntVal(4567);
r1.setLongVal(0x5a5a5a5a5a5aL);
- r1.setStringVal(new Text("random text"));
- r1.setBufferVal(new BytesWritable());
- r1.setVectorVal(new ArrayList());
- r1.setMapVal(new TreeMap());
+ r1.setStringVal("random text");
+ r1.setBufferVal(new Buffer());
+ r1.setVectorVal(new ArrayList<String>());
+ r1.setMapVal(new TreeMap<String,String>());
RecRecord0 r0 = new RecRecord0();
- r0.setStringVal(new Text("other random text"));
+ r0.setStringVal("other random text");
r1.setRecordVal(r0);
System.err.println("Illustrating toString bug"+r1.toString());
System.err.println("Illustrating toString bug"+r1.toString());
@@ -152,12 +148,12 @@
r1.setDoubleVal(1.5234);
r1.setIntVal(4567);
r1.setLongVal(0x5a5a5a5a5a5aL);
- r1.setStringVal(new Text("ran\002dom < %text<&more"));
- r1.setBufferVal(new BytesWritable());
- r1.setVectorVal(new ArrayList());
- r1.setMapVal(new TreeMap());
+ r1.setStringVal("ran\002dom < %text<&more");
+ r1.setBufferVal(new Buffer());
+ r1.setVectorVal(new ArrayList<String>());
+ r1.setMapVal(new TreeMap<String,String>());
RecRecord0 r0 = new RecRecord0();
- r0.setStringVal(new Text("other %rando\007m & >&more text"));
+ r0.setStringVal("other %rando\007m & >&more text");
r1.setRecordVal(r0);
out.write(r1);
ostream.close();
@@ -171,5 +167,31 @@
} catch (IOException ex) {
ex.printStackTrace();
}
+ }
+
+ public void testCloneable() {
+ RecRecord1 r1 = new RecRecord1();
+ r1.setBoolVal(true);
+ r1.setByteVal((byte)0x66);
+ r1.setFloatVal(3.145F);
+ r1.setDoubleVal(1.5234);
+ r1.setIntVal(-4567);
+ r1.setLongVal(-2367L);
+ r1.setStringVal("random text");
+ r1.setBufferVal(new Buffer());
+ r1.setVectorVal(new ArrayList<String>());
+ r1.setMapVal(new TreeMap<String,String>());
+ RecRecord0 r0 = new RecRecord0();
+ r0.setStringVal("other random text");
+ r1.setRecordVal(r0);
+ try {
+ RecRecord1 r2 = (RecRecord1) r1.clone();
+ assertTrue("Cloneable semantics violated. r1==r2", r1 != r2);
+ assertTrue("Cloneable semantics violated. r1.getClass() != r2.getClass()",
+ r1.getClass() == r2.getClass());
+ assertTrue("Cloneable semantics violated. !r2.equals(r1)", r2.equals(r1));
+ } catch (final CloneNotSupportedException ex) {
+ ex.printStackTrace();
+ }
}
}
Added: lucene/hadoop/trunk/src/test/org/apache/hadoop/record/TestRecordMR.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/test/org/apache/hadoop/record/TestRecordMR.java?view=auto&rev=513122
==============================================================================
--- lucene/hadoop/trunk/src/test/org/apache/hadoop/record/TestRecordMR.java (added)
+++ lucene/hadoop/trunk/src/test/org/apache/hadoop/record/TestRecordMR.java Wed Feb 28 19:47:27 2007
@@ -0,0 +1,453 @@
+/**
+ * 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.record;
+
+import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.io.SequenceFile.CompressionType;
+import org.apache.hadoop.conf.*;
+import junit.framework.TestCase;
+import java.io.*;
+import java.util.*;
+
+
+/**********************************************************
+ * MapredLoadTest generates a bunch of work that exercises
+ * a Hadoop Map-Reduce system (and DFS, too). It goes through
+ * the following steps:
+ *
+ * 1) Take inputs 'range' and 'counts'.
+ * 2) Generate 'counts' random integers between 0 and range-1.
+ * 3) Create a file that lists each integer between 0 and range-1,
+ * and lists the number of times that integer was generated.
+ * 4) Emit a (very large) file that contains all the integers
+ * in the order generated.
+ * 5) After the file has been generated, read it back and count
+ * how many times each int was generated.
+ * 6) Compare this big count-map against the original one. If
+ * they match, then SUCCESS! Otherwise, FAILURE!
+ *
+ * OK, that's how we can think about it. What are the map-reduce
+ * steps that get the job done?
+ *
+ * 1) In a non-mapred thread, take the inputs 'range' and 'counts'.
+ * 2) In a non-mapread thread, generate the answer-key and write to disk.
+ * 3) In a mapred job, divide the answer key into K jobs.
+ * 4) A mapred 'generator' task consists of K map jobs. Each reads
+ * an individual "sub-key", and generates integers according to
+ * to it (though with a random ordering).
+ * 5) The generator's reduce task agglomerates all of those files
+ * into a single one.
+ * 6) A mapred 'reader' task consists of M map jobs. The output
+ * file is cut into M pieces. Each of the M jobs counts the
+ * individual ints in its chunk and creates a map of all seen ints.
+ * 7) A mapred job integrates all the count files into a single one.
+ *
+ **********************************************************/
+public class TestRecordMR extends TestCase {
+ /**
+ * Modified to make it a junit test.
+ * The RandomGen Job does the actual work of creating
+ * a huge file of assorted numbers. It receives instructions
+ * as to how many times each number should be counted. Then
+ * it emits those numbers in a crazy order.
+ *
+ * The map() function takes a key/val pair that describes
+ * a value-to-be-emitted (the key) and how many times it
+ * should be emitted (the value), aka "numtimes". map() then
+ * emits a series of intermediate key/val pairs. It emits
+ * 'numtimes' of these. The key is a random number and the
+ * value is the 'value-to-be-emitted'.
+ *
+ * The system collates and merges these pairs according to
+ * the random number. reduce() function takes in a key/value
+ * pair that consists of a crazy random number and a series
+ * of values that should be emitted. The random number key
+ * is now dropped, and reduce() emits a pair for every intermediate value.
+ * The emitted key is an intermediate value. The emitted value
+ * is just a blank string. Thus, we've created a huge file
+ * of numbers in random order, but where each number appears
+ * as many times as we were instructed.
+ */
+ static public class RandomGenMapper implements Mapper {
+ Random r = new Random();
+ public void configure(JobConf job) {
+ }
+
+ public void map(WritableComparable key, Writable val, OutputCollector out, Reporter reporter) throws IOException {
+ int randomVal = ((RecInt) key).getData();
+ int randomCount = ((RecInt) val).getData();
+
+ for (int i = 0; i < randomCount; i++) {
+ out.collect(new RecInt(Math.abs(r.nextInt())),
+ new RecString(Integer.toString(randomVal)));
+ }
+ }
+ public void close() {
+ }
+ }
+ /**
+ */
+ static public class RandomGenReducer implements Reducer {
+ public void configure(JobConf job) {
+ }
+
+ public void reduce(WritableComparable key,
+ Iterator it,
+ OutputCollector out,
+ Reporter reporter)
+ throws IOException {
+ int keyint = ((RecInt) key).getData();
+ while (it.hasNext()) {
+ String val = ((RecString) it.next()).getData();
+ out.collect(new RecInt(Integer.parseInt(val)),
+ new RecString(""));
+ }
+ }
+ public void close() {
+ }
+ }
+
+ /**
+ * The RandomCheck Job does a lot of our work. It takes
+ * in a num/string keyspace, and transforms it into a
+ * key/count(int) keyspace.
+ *
+ * The map() function just emits a num/1 pair for every
+ * num/string input pair.
+ *
+ * The reduce() function sums up all the 1s that were
+ * emitted for a single key. It then emits the key/total
+ * pair.
+ *
+ * This is used to regenerate the random number "answer key".
+ * Each key here is a random number, and the count is the
+ * number of times the number was emitted.
+ */
+ static public class RandomCheckMapper implements Mapper {
+ public void configure(JobConf job) {
+ }
+
+ public void map(WritableComparable key, Writable val, OutputCollector out, Reporter reporter) throws IOException {
+ int pos = ((RecInt) key).getData();
+ String str = ((RecString) val).getData();
+ out.collect(new RecInt(pos), new RecString("1"));
+ }
+ public void close() {
+ }
+ }
+ /**
+ */
+ static public class RandomCheckReducer implements Reducer {
+ public void configure(JobConf job) {
+ }
+
+ public void reduce(WritableComparable key, Iterator it, OutputCollector out, Reporter reporter) throws IOException {
+ int keyint = ((RecInt) key).getData();
+ int count = 0;
+ while (it.hasNext()) {
+ it.next();
+ count++;
+ }
+ out.collect(new RecInt(keyint), new RecString(Integer.toString(count)));
+ }
+ public void close() {
+ }
+ }
+
+ /**
+ * The Merge Job is a really simple one. It takes in
+ * an int/int key-value set, and emits the same set.
+ * But it merges identical keys by adding their values.
+ *
+ * Thus, the map() function is just the identity function
+ * and reduce() just sums. Nothing to see here!
+ */
+ static public class MergeMapper implements Mapper {
+ public void configure(JobConf job) {
+ }
+
+ public void map(WritableComparable key, Writable val, OutputCollector out, Reporter reporter) throws IOException {
+ int keyint = ((RecInt) key).getData();
+ String valstr = ((RecString) val).getData();
+ out.collect(new RecInt(keyint), new RecInt(Integer.parseInt(valstr)));
+ }
+ public void close() {
+ }
+ }
+ static public class MergeReducer implements Reducer {
+ public void configure(JobConf job) {
+ }
+
+ public void reduce(WritableComparable key, Iterator it, OutputCollector out, Reporter reporter) throws IOException {
+ int keyint = ((RecInt) key).getData();
+ int total = 0;
+ while (it.hasNext()) {
+ total += ((RecInt) it.next()).getData();
+ }
+ out.collect(new RecInt(keyint), new RecInt(total));
+ }
+ public void close() {
+ }
+ }
+
+ private static int range = 10;
+ private static int counts = 100;
+ private static Random r = new Random();
+ private static Configuration conf = new Configuration();
+
+ public void testMapred() throws Exception {
+ launch();
+ }
+
+ /**
+ *
+ */
+ public static void launch() throws Exception {
+ //
+ // Generate distribution of ints. This is the answer key.
+ //
+ int countsToGo = counts;
+ int dist[] = new int[range];
+ for (int i = 0; i < range; i++) {
+ double avgInts = (1.0 * countsToGo) / (range - i);
+ dist[i] = (int) Math.max(0, Math.round(avgInts + (Math.sqrt(avgInts) * r.nextGaussian())));
+ countsToGo -= dist[i];
+ }
+ if (countsToGo > 0) {
+ dist[dist.length-1] += countsToGo;
+ }
+
+ //
+ // Write the answer key to a file.
+ //
+ FileSystem fs = FileSystem.get(conf);
+ Path testdir = new Path("mapred.loadtest");
+ if (!fs.mkdirs(testdir)) {
+ throw new IOException("Mkdirs failed to create directory " + testdir.toString());
+ }
+
+ Path randomIns = new Path(testdir, "genins");
+ if (!fs.mkdirs(randomIns)) {
+ throw new IOException("Mkdirs failed to create directory " + randomIns.toString());
+ }
+
+ Path answerkey = new Path(randomIns, "answer.key");
+ SequenceFile.Writer out = SequenceFile.createWriter(fs, conf,
+ answerkey, RecInt.class, RecInt.class,
+ CompressionType.NONE);
+ try {
+ for (int i = 0; i < range; i++) {
+ RecInt k = new RecInt();
+ RecInt v = new RecInt();
+ k.setData(i);
+ v.setData(dist[i]);
+ out.append(k, v);
+ }
+ } finally {
+ out.close();
+ }
+
+ //
+ // Now we need to generate the random numbers according to
+ // the above distribution.
+ //
+ // We create a lot of map tasks, each of which takes at least
+ // one "line" of the distribution. (That is, a certain number
+ // X is to be generated Y number of times.)
+ //
+ // A map task emits Y key/val pairs. The val is X. The key
+ // is a randomly-generated number.
+ //
+ // The reduce task gets its input sorted by key. That is, sorted
+ // in random order. It then emits a single line of text that
+ // for the given values. It does not emit the key.
+ //
+ // Because there's just one reduce task, we emit a single big
+ // file of random numbers.
+ //
+ Path randomOuts = new Path(testdir, "genouts");
+ fs.delete(randomOuts);
+
+
+ JobConf genJob = new JobConf(conf,TestRecordMR.class);
+ genJob.setInputPath(randomIns);
+ genJob.setInputKeyClass(RecInt.class);
+ genJob.setInputValueClass(RecInt.class);
+ genJob.setInputFormat(SequenceFileInputFormat.class);
+ genJob.setMapperClass(RandomGenMapper.class);
+
+ genJob.setOutputPath(randomOuts);
+ genJob.setOutputKeyClass(RecInt.class);
+ genJob.setOutputValueClass(RecString.class);
+ genJob.setOutputFormat(SequenceFileOutputFormat.class);
+ genJob.setReducerClass(RandomGenReducer.class);
+ genJob.setNumReduceTasks(1);
+
+ JobClient.runJob(genJob);
+
+ //
+ // Next, we read the big file in and regenerate the
+ // original map. It's split into a number of parts.
+ // (That number is 'intermediateReduces'.)
+ //
+ // We have many map tasks, each of which read at least one
+ // of the output numbers. For each number read in, the
+ // map task emits a key/value pair where the key is the
+ // number and the value is "1".
+ //
+ // We have a single reduce task, which receives its input
+ // sorted by the key emitted above. For each key, there will
+ // be a certain number of "1" values. The reduce task sums
+ // these values to compute how many times the given key was
+ // emitted.
+ //
+ // The reduce task then emits a key/val pair where the key
+ // is the number in question, and the value is the number of
+ // times the key was emitted. This is the same format as the
+ // original answer key (except that numbers emitted zero times
+ // will not appear in the regenerated key.) The answer set
+ // is split into a number of pieces. A final MapReduce job
+ // will merge them.
+ //
+ // There's not really a need to go to 10 reduces here
+ // instead of 1. But we want to test what happens when
+ // you have multiple reduces at once.
+ //
+ int intermediateReduces = 10;
+ Path intermediateOuts = new Path(testdir, "intermediateouts");
+ fs.delete(intermediateOuts);
+ JobConf checkJob = new JobConf(conf,TestRecordMR.class);
+ checkJob.setInputPath(randomOuts);
+ checkJob.setInputKeyClass(RecInt.class);
+ checkJob.setInputValueClass(RecString.class);
+ checkJob.setInputFormat(SequenceFileInputFormat.class);
+ checkJob.setMapperClass(RandomCheckMapper.class);
+
+ checkJob.setOutputPath(intermediateOuts);
+ checkJob.setOutputKeyClass(RecInt.class);
+ checkJob.setOutputValueClass(RecString.class);
+ checkJob.setOutputFormat(SequenceFileOutputFormat.class);
+ checkJob.setReducerClass(RandomCheckReducer.class);
+ checkJob.setNumReduceTasks(intermediateReduces);
+
+ JobClient.runJob(checkJob);
+
+ //
+ // OK, now we take the output from the last job and
+ // merge it down to a single file. The map() and reduce()
+ // functions don't really do anything except reemit tuples.
+ // But by having a single reduce task here, we end up merging
+ // all the files.
+ //
+ Path finalOuts = new Path(testdir, "finalouts");
+ fs.delete(finalOuts);
+ JobConf mergeJob = new JobConf(conf,TestRecordMR.class);
+ mergeJob.setInputPath(intermediateOuts);
+ mergeJob.setInputKeyClass(RecInt.class);
+ mergeJob.setInputValueClass(RecString.class);
+ mergeJob.setInputFormat(SequenceFileInputFormat.class);
+ mergeJob.setMapperClass(MergeMapper.class);
+
+ mergeJob.setOutputPath(finalOuts);
+ mergeJob.setOutputKeyClass(RecInt.class);
+ mergeJob.setOutputValueClass(RecInt.class);
+ mergeJob.setOutputFormat(SequenceFileOutputFormat.class);
+ mergeJob.setReducerClass(MergeReducer.class);
+ mergeJob.setNumReduceTasks(1);
+
+ JobClient.runJob(mergeJob);
+
+
+ //
+ // Finally, we compare the reconstructed answer key with the
+ // original one. Remember, we need to ignore zero-count items
+ // in the original key.
+ //
+ boolean success = true;
+ Path recomputedkey = new Path(finalOuts, "part-00000");
+ SequenceFile.Reader in = new SequenceFile.Reader(fs, recomputedkey, conf);
+ int totalseen = 0;
+ try {
+ RecInt key = new RecInt();
+ RecInt val = new RecInt();
+ for (int i = 0; i < range; i++) {
+ if (dist[i] == 0) {
+ continue;
+ }
+ if (! in.next(key, val)) {
+ System.err.println("Cannot read entry " + i);
+ success = false;
+ break;
+ } else {
+ if ( !((key.getData() == i ) && (val.getData() == dist[i]))) {
+ System.err.println("Mismatch! Pos=" + key.getData() + ", i=" + i + ", val=" + val.getData() + ", dist[i]=" + dist[i]);
+ success = false;
+ }
+ totalseen += val.getData();
+ }
+ }
+ if (success) {
+ if (in.next(key, val)) {
+ System.err.println("Unnecessary lines in recomputed key!");
+ success = false;
+ }
+ }
+ } finally {
+ in.close();
+ }
+ int originalTotal = 0;
+ for (int i = 0; i < dist.length; i++) {
+ originalTotal += dist[i];
+ }
+ System.out.println("Original sum: " + originalTotal);
+ System.out.println("Recomputed sum: " + totalseen);
+
+ //
+ // Write to "results" whether the test succeeded or not.
+ //
+ Path resultFile = new Path(testdir, "results");
+ BufferedWriter bw = new BufferedWriter(new OutputStreamWriter(fs.create(resultFile)));
+ try {
+ bw.write("Success=" + success + "\n");
+ System.out.println("Success=" + success);
+ } finally {
+ bw.close();
+ }
+ fs.delete(testdir);
+ }
+
+ /**
+ * Launches all the tasks in order.
+ */
+ public static void main(String[] argv) throws Exception {
+ if (argv.length < 2) {
+ System.err.println("Usage: TestRecordMR <range> <counts>");
+ System.err.println();
+ System.err.println("Note: a good test will have a <counts> value that is substantially larger than the <range>");
+ return;
+ }
+
+ int i = 0;
+ int range = Integer.parseInt(argv[i++]);
+ int counts = Integer.parseInt(argv[i++]);
+ launch();
+ }
+}
Added: lucene/hadoop/trunk/src/test/org/apache/hadoop/record/TestRecordWritable.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/test/org/apache/hadoop/record/TestRecordWritable.java?view=auto&rev=513122
==============================================================================
--- lucene/hadoop/trunk/src/test/org/apache/hadoop/record/TestRecordWritable.java (added)
+++ lucene/hadoop/trunk/src/test/org/apache/hadoop/record/TestRecordWritable.java Wed Feb 28 19:47:27 2007
@@ -0,0 +1,119 @@
+/**
+ * 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.record;
+
+import java.io.*;
+import java.util.*;
+import junit.framework.TestCase;
+
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.conf.*;
+import org.apache.commons.logging.*;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputFormatBase;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.SequenceFileInputFormat;
+
+public class TestRecordWritable extends TestCase {
+ private static final Log LOG = InputFormatBase.LOG;
+
+ private static int MAX_LENGTH = 10000;
+ private static Configuration conf = new Configuration();
+
+ public void testFormat() throws Exception {
+ JobConf job = new JobConf(conf);
+ FileSystem fs = FileSystem.getLocal(conf);
+ Path dir = new Path(System.getProperty("test.build.data",".") + "/mapred");
+ Path file = new Path(dir, "test.seq");
+
+ Reporter reporter = new Reporter() {
+ public void setStatus(String status) throws IOException {}
+ public void progress() throws IOException {}
+ public void incrCounter(Enum key, long amount) {}
+ };
+
+ int seed = new Random().nextInt();
+ //LOG.info("seed = "+seed);
+ Random random = new Random(seed);
+
+ fs.delete(dir);
+
+ job.setInputPath(dir);
+
+ // for a variety of lengths
+ for (int length = 0; length < MAX_LENGTH;
+ length+= random.nextInt(MAX_LENGTH/10)+1) {
+
+ // create a file with length entries
+ SequenceFile.Writer writer =
+ new SequenceFile.Writer(fs, conf, file,
+ RecInt.class, RecBuffer.class);
+ try {
+ for (int i = 0; i < length; i++) {
+ RecInt key = new RecInt();
+ key.setData(i);
+ byte[] data = new byte[random.nextInt(10)];
+ random.nextBytes(data);
+ RecBuffer value = new RecBuffer();
+ value.setData(new Buffer(data));
+ writer.append(key, value);
+ }
+ } finally {
+ writer.close();
+ }
+
+ // try splitting the file in a variety of sizes
+ InputFormat format = new SequenceFileInputFormat();
+ RecInt key = new RecInt();
+ RecBuffer value = new RecBuffer();
+ for (int i = 0; i < 3; i++) {
+ int numSplits =
+ random.nextInt(MAX_LENGTH/(SequenceFile.SYNC_INTERVAL/20))+1;
+ InputSplit[] splits = format.getSplits(job, numSplits);
+
+ // check each split
+ BitSet bits = new BitSet(length);
+ for (int j = 0; j < splits.length; j++) {
+ RecordReader reader =
+ format.getRecordReader(splits[j], job, reporter);
+ try {
+ int count = 0;
+ while (reader.next(key, value)) {
+ assertFalse("Key in multiple partitions.", bits.get(key.getData()));
+ bits.set(key.getData());
+ count++;
+ }
+ } finally {
+ reader.close();
+ }
+ }
+ assertEquals("Some keys in no partition.", length, bits.cardinality());
+ }
+
+ }
+ }
+
+ public static void main(String[] args) throws Exception {
+ new TestRecordWritable().testFormat();
+ }
+}
Copied: lucene/hadoop/trunk/src/test/org/apache/hadoop/record/ToCpp.java (from r513042, lucene/hadoop/trunk/src/test/org/apache/hadoop/record/test/ToCpp.java)
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/test/org/apache/hadoop/record/ToCpp.java?view=diff&rev=513122&p1=lucene/hadoop/trunk/src/test/org/apache/hadoop/record/test/ToCpp.java&r1=513042&p2=lucene/hadoop/trunk/src/test/org/apache/hadoop/record/ToCpp.java&r2=513122
==============================================================================
--- lucene/hadoop/trunk/src/test/org/apache/hadoop/record/test/ToCpp.java (original)
+++ lucene/hadoop/trunk/src/test/org/apache/hadoop/record/ToCpp.java Wed Feb 28 19:47:27 2007
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.hadoop.record.test;
+package org.apache.hadoop.record;
import java.io.IOException;
import junit.framework.*;
@@ -26,8 +26,6 @@
import java.io.FileOutputStream;
import java.util.ArrayList;
import java.util.TreeMap;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.Text;
/**
*
@@ -58,10 +56,10 @@
r1.setDoubleVal(1.5234);
r1.setIntVal(4567);
r1.setLongVal(0x5a5a5a5a5a5aL);
- r1.setStringVal(new Text("random text"));
- r1.setBufferVal(new BytesWritable());
- r1.setVectorVal(new ArrayList());
- r1.setMapVal(new TreeMap());
+ r1.setStringVal("random text");
+ r1.setBufferVal(new Buffer());
+ r1.setVectorVal(new ArrayList<String>());
+ r1.setMapVal(new TreeMap<String,String>());
out.write(r1);
ostream.close();
} catch (IOException ex) {
@@ -82,10 +80,10 @@
r1.setDoubleVal(1.5234);
r1.setIntVal(4567);
r1.setLongVal(0x5a5a5a5a5a5aL);
- r1.setStringVal(new Text("random text"));
- r1.setBufferVal(new BytesWritable());
- r1.setVectorVal(new ArrayList());
- r1.setMapVal(new TreeMap());
+ r1.setStringVal("random text");
+ r1.setBufferVal(new Buffer());
+ r1.setVectorVal(new ArrayList<String>());
+ r1.setMapVal(new TreeMap<String,String>());
out.write(r1);
ostream.close();
} catch (IOException ex) {
@@ -106,10 +104,10 @@
r1.setDoubleVal(1.5234);
r1.setIntVal(4567);
r1.setLongVal(0x5a5a5a5a5a5aL);
- r1.setStringVal(new Text("random text"));
- r1.setBufferVal(new BytesWritable());
- r1.setVectorVal(new ArrayList());
- r1.setMapVal(new TreeMap());
+ r1.setStringVal("random text");
+ r1.setBufferVal(new Buffer());
+ r1.setVectorVal(new ArrayList<String>());
+ r1.setMapVal(new TreeMap<String,String>());
out.write(r1);
ostream.close();
} catch (IOException ex) {