Return-Path: X-Original-To: apmail-hive-commits-archive@www.apache.org Delivered-To: apmail-hive-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 29786C237 for ; Fri, 21 Jun 2013 14:56:44 +0000 (UTC) Received: (qmail 97994 invoked by uid 500); 21 Jun 2013 14:56:43 -0000 Delivered-To: apmail-hive-commits-archive@hive.apache.org Received: (qmail 97297 invoked by uid 500); 21 Jun 2013 14:56:37 -0000 Mailing-List: contact commits-help@hive.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hive-dev@hive.apache.org Delivered-To: mailing list commits@hive.apache.org Received: (qmail 97289 invoked by uid 99); 21 Jun 2013 14:56:35 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 21 Jun 2013 14:56:35 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 21 Jun 2013 14:56:30 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id B2CBD23889EC; Fri, 21 Jun 2013 14:56:09 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1495462 - in /hive/trunk/ql/src: java/org/apache/hadoop/hive/ql/io/ test/org/apache/hadoop/hive/ql/io/ Date: Fri, 21 Jun 2013 14:56:09 -0000 To: commits@hive.apache.org From: hashutosh@apache.org X-Mailer: svnmailer-1.0.8-patched Message-Id: <20130621145609.B2CBD23889EC@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: hashutosh Date: Fri Jun 21 14:56:08 2013 New Revision: 1495462 URL: http://svn.apache.org/r1495462 Log: HIVE-4743 : Improve test coverage of package org.apache.hadoop.hive.ql.io (Ivan Veselovsky via Ashutosh Chauhan) Added: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/TestPerformTestRCFileAndSeqFile.java Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/NonSyncDataInputBuffer.java hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/PerformTestRCFileAndSeqFile.java hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/TestHiveInputOutputBuffer.java hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/TestRCFile.java hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/TestSymlinkTextInputFormat.java Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/NonSyncDataInputBuffer.java URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/NonSyncDataInputBuffer.java?rev=1495462&r1=1495461&r2=1495462&view=diff ============================================================================== --- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/NonSyncDataInputBuffer.java (original) +++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/NonSyncDataInputBuffer.java Fri Jun 21 14:56:08 2013 @@ -118,6 +118,7 @@ public class NonSyncDataInputBuffer exte * If a problem occurs reading from this DataInputStream. * */ + @Override public final boolean readBoolean() throws IOException { int temp = in.read(); if (temp < 0) { @@ -135,6 +136,7 @@ public class NonSyncDataInputBuffer exte * If a problem occurs reading from this DataInputStream. * */ + @Override public final byte readByte() throws IOException { int temp = in.read(); if (temp < 0) { @@ -165,6 +167,7 @@ public class NonSyncDataInputBuffer exte return offset; } + @Override public final char readChar() throws IOException { if (readToBuff(2) < 0) { throw new EOFException(); @@ -182,6 +185,7 @@ public class NonSyncDataInputBuffer exte * If a problem occurs reading from this DataInputStream. * */ + @Override public final double readDouble() throws IOException { return Double.longBitsToDouble(readLong()); } @@ -195,6 +199,7 @@ public class NonSyncDataInputBuffer exte * If a problem occurs reading from this DataInputStream. * */ + @Override public final float readFloat() throws IOException { return Float.intBitsToFloat(readInt()); } @@ -211,6 +216,7 @@ public class NonSyncDataInputBuffer exte * If a problem occurs reading from this DataInputStream. * */ + @Override public final void readFully(byte[] buffer) throws IOException { readFully(buffer, 0, buffer.length); } @@ -233,6 +239,7 @@ public class NonSyncDataInputBuffer exte * if reaches the end of the stream before enough bytes have been * read */ + @Override public final void readFully(byte[] buffer, int offset, int length) throws IOException { if (length < 0) { @@ -267,6 +274,7 @@ public class NonSyncDataInputBuffer exte * If a problem occurs reading from this DataInputStream. * */ + @Override public final int readInt() throws IOException { if (readToBuff(4) < 0) { throw new EOFException(); @@ -292,6 +300,7 @@ public class NonSyncDataInputBuffer exte * @deprecated Use BufferedReader */ @Deprecated + @Override public final String readLine() throws IOException { StringBuilder line = new StringBuilder(80); // Typical line length boolean foundTerminator = false; @@ -335,6 +344,7 @@ public class NonSyncDataInputBuffer exte * If a problem occurs reading from this DataInputStream. * */ + @Override public final long readLong() throws IOException { if (readToBuff(8) < 0) { throw new EOFException(); @@ -356,6 +366,7 @@ public class NonSyncDataInputBuffer exte * If a problem occurs reading from this DataInputStream. * */ + @Override public final short readShort() throws IOException { if (readToBuff(2) < 0) { throw new EOFException(); @@ -373,6 +384,7 @@ public class NonSyncDataInputBuffer exte * If a problem occurs reading from this DataInputStream. * */ + @Override public final int readUnsignedByte() throws IOException { int temp = in.read(); if (temp < 0) { @@ -391,6 +403,7 @@ public class NonSyncDataInputBuffer exte * If a problem occurs reading from this DataInputStream. * */ + @Override public final int readUnsignedShort() throws IOException { if (readToBuff(2) < 0) { throw new EOFException(); @@ -407,6 +420,7 @@ public class NonSyncDataInputBuffer exte * If a problem occurs reading from this DataInputStream. * */ + @Override public final String readUTF() throws IOException { return decodeUTF(readUnsignedShort()); } @@ -450,6 +464,7 @@ public class NonSyncDataInputBuffer exte * @throws IOException * If the stream is already closed or another IOException occurs. */ + @Override public final int skipBytes(int count) throws IOException { int skipped = 0; long skip; Modified: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/PerformTestRCFileAndSeqFile.java URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/PerformTestRCFileAndSeqFile.java?rev=1495462&r1=1495461&r2=1495462&view=diff ============================================================================== --- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/PerformTestRCFileAndSeqFile.java (original) +++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/PerformTestRCFileAndSeqFile.java Fri Jun 21 14:56:08 2013 @@ -20,8 +20,6 @@ package org.apache.hadoop.hive.ql.io; import java.io.IOException; import java.util.Random; -import junit.framework.TestCase; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -34,12 +32,13 @@ import org.apache.hadoop.io.SequenceFile import org.apache.hadoop.io.SequenceFile.CompressionType; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.DefaultCodec; +import static org.junit.Assert.*; /** * PerformTestRCFileAndSeqFile. * */ -public class PerformTestRCFileAndSeqFile extends TestCase { +public class PerformTestRCFileAndSeqFile { private final Configuration conf = new Configuration(); Modified: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/TestHiveInputOutputBuffer.java URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/TestHiveInputOutputBuffer.java?rev=1495462&r1=1495461&r2=1495462&view=diff ============================================================================== --- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/TestHiveInputOutputBuffer.java (original) +++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/TestHiveInputOutputBuffer.java Fri Jun 21 14:56:08 2013 @@ -17,7 +17,13 @@ */ package org.apache.hadoop.hive.ql.io; +import static org.junit.Assert.assertArrayEquals; + +import java.io.DataOutput; import java.io.IOException; +import java.util.Random; + +import org.junit.Test; import junit.framework.TestCase; @@ -27,17 +33,196 @@ import junit.framework.TestCase; */ public class TestHiveInputOutputBuffer extends TestCase { + private static final int numCases = 14; + + private static final String asciiLine1 = "Foo 12345 moo"; + private static final String asciiLine2 = "Line two"; + private static final String asciiString = asciiLine1 + "\n" + asciiLine2 + "\r\n"; + public void testReadAndWrite() throws IOException { String testString = "test_hive_input_output_number_0"; byte[] string_bytes = testString.getBytes(); NonSyncDataInputBuffer inBuffer = new NonSyncDataInputBuffer(); NonSyncDataOutputBuffer outBuffer = new NonSyncDataOutputBuffer(); - outBuffer.write(string_bytes); - inBuffer.reset(outBuffer.getData(), 0, outBuffer.getLength()); - byte[] readBytes = new byte[string_bytes.length]; - inBuffer.read(readBytes); - String readString = new String(readBytes); - assertEquals("Field testReadAndWrite()", readString, testString); + try { + outBuffer.write(string_bytes); + inBuffer.reset(outBuffer.getData(), 0, outBuffer.getLength()); + byte[] readBytes = new byte[string_bytes.length]; + inBuffer.read(readBytes); + String readString = new String(readBytes); + assertEquals("Field testReadAndWrite()", readString, testString); + } finally { + inBuffer.close(); + outBuffer.close(); + } } + @SuppressWarnings("deprecation") + private static void readJunk(NonSyncDataInputBuffer in, Random r, long seed, int iter) + throws IOException { + r.setSeed(seed); + for (int i = 0; i < iter; ++i) { + switch (r.nextInt(numCases)) { + case 0: + assertEquals((byte)(r.nextInt() & 0xFF), in.readByte()); break; + case 1: + assertEquals((short)(r.nextInt() & 0xFFFF), in.readShort()); break; + case 2: + assertEquals(r.nextInt(), in.readInt()); break; + case 3: + assertEquals(r.nextLong(), in.readLong()); break; + case 4: + assertEquals(Double.doubleToLongBits(r.nextDouble()), + Double.doubleToLongBits(in.readDouble())); break; + case 5: + assertEquals(Float.floatToIntBits(r.nextFloat()), + Float.floatToIntBits(in.readFloat())); break; + case 6: + int len = r.nextInt(1024); + // 1 (test #readFully(3)): + final byte[] vb = new byte[len]; + r.nextBytes(vb); + final byte[] b = new byte[len]; + in.readFully(b, 0, len); + assertArrayEquals(vb, b); + // 2 (test #read(3)): + r.nextBytes(vb); + in.read(b, 0, len); + assertArrayEquals(vb, b); + // 3 (test #readFully(1)): + r.nextBytes(vb); + in.readFully(b); + assertArrayEquals(vb, b); + break; + case 7: + assertEquals(r.nextBoolean(), in.readBoolean()); + break; + case 8: + assertEquals((char)r.nextInt(), in.readChar()); + break; + case 9: + int actualUB = in.readUnsignedByte(); + assertTrue(actualUB >= 0); + assertTrue(actualUB <= 255); + assertEquals(r.nextInt() & 0xFF, actualUB); + break; + case 10: + int actualUS = in.readUnsignedShort(); + assertTrue(actualUS >= 0); + assertTrue(actualUS <= 0xFFFF); + assertEquals(r.nextInt() & 0xFFFF, actualUS); + break; + case 11: + String expectedString1 = composeString(1024, r); + assertEquals(expectedString1, in.readUTF()); + String expectedString2 = composeString(1024, r); + assertEquals(expectedString2, NonSyncDataInputBuffer.readUTF(in)); + break; + case 12: + assertEquals(asciiLine1, in.readLine()); + assertEquals(asciiLine2, in.readLine()); + break; + case 13: + in.skipBytes(8); + r.nextLong(); // ignore + assertEquals(r.nextLong(), in.readLong()); + break; + } + } + } + + private static void writeJunk(DataOutput out, Random r, long seed, int iter) + throws IOException { + r.setSeed(seed); + for (int i = 0; i < iter; ++i) { + switch (r.nextInt(numCases)) { + case 0: out.writeByte(r.nextInt()); break; + case 1: out.writeShort((short)(r.nextInt() & 0xFFFF)); break; + case 2: out.writeInt(r.nextInt()); break; + case 3: out.writeLong(r.nextLong()); break; + case 4: out.writeDouble(r.nextDouble()); break; + case 5: out.writeFloat(r.nextFloat()); break; + case 6: + byte[] b = new byte[r.nextInt(1024)]; + // 1: + r.nextBytes(b); + out.write(b); + // 2: + r.nextBytes(b); + out.write(b); + // 3: + r.nextBytes(b); + out.write(b); + break; + case 7: + out.writeBoolean(r.nextBoolean()); + break; + case 8: + out.writeChar((char)r.nextInt()); + break; + case 9: + out.writeByte((byte)r.nextInt()); + break; + case 10: + out.writeShort((short)r.nextInt()); + break; + case 11: + String string = composeString(1024, r); + out.writeUTF(string); + String string2 = composeString(1024, r); + out.writeUTF(string2); + break; + case 12: + byte[] bb = asciiString.getBytes("UTF-8"); + out.write(bb); + break; + case 13: + out.writeLong(r.nextLong()); + out.writeLong(r.nextLong()); + break; + } + } + } + + private static String composeString(int len, Random r) { + char[] cc = new char[len]; + char ch; + for (int i = 0; i