You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@avro.apache.org by cu...@apache.org on 2011/05/19 17:53:05 UTC
svn commit: r1124972 - in /avro/branches/branch-1.5: ./ CHANGES.txt
lang/java/avro/src/main/java/org/apache/avro/file/DataFileWriter.java
lang/java/avro/src/test/java/org/apache/avro/TestDataFile.java
Author: cutting
Date: Thu May 19 15:53:04 2011
New Revision: 1124972
URL: http://svn.apache.org/viewvc?rev=1124972&view=rev
Log:
Merge -c 1124971 from trunk to 1.5 branch. Fixes: AVRO-820.
Modified:
avro/branches/branch-1.5/ (props changed)
avro/branches/branch-1.5/CHANGES.txt
avro/branches/branch-1.5/lang/java/avro/src/main/java/org/apache/avro/file/DataFileWriter.java
avro/branches/branch-1.5/lang/java/avro/src/test/java/org/apache/avro/TestDataFile.java
Propchange: avro/branches/branch-1.5/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Thu May 19 15:53:04 2011
@@ -1 +1 @@
-/avro/trunk:1075938,1075993,1078917,1079055,1079060,1079063,1083246,1085921,1086727,1086730,1086866,1087076,1087129,1087136,1087439-1087440,1087463,1087472,1087792,1089128,1089131,1089550,1094812,1095206-1095208,1095493,1095529,1095548,1095550,1096798,1097916,1097927,1097968,1097974,1102332,1102335,1124127
+/avro/trunk:1075938,1075993,1078917,1079055,1079060,1079063,1083246,1085921,1086727,1086730,1086866,1087076,1087129,1087136,1087439-1087440,1087463,1087472,1087792,1089128,1089131,1089550,1094812,1095206-1095208,1095493,1095529,1095548,1095550,1096798,1097916,1097927,1097968,1097974,1102332,1102335,1124127,1124971
Modified: avro/branches/branch-1.5/CHANGES.txt
URL: http://svn.apache.org/viewvc/avro/branches/branch-1.5/CHANGES.txt?rev=1124972&r1=1124971&r2=1124972&view=diff
==============================================================================
--- avro/branches/branch-1.5/CHANGES.txt (original)
+++ avro/branches/branch-1.5/CHANGES.txt Thu May 19 15:53:04 2011
@@ -6,6 +6,10 @@ Avro 1.5.2 (unreleased)
IMPROVEMENTS
+ AVRO-820. Java: Permit applications to catch exceptions thrown
+ while writing data to a file and then continue writing to that
+ file. (scottcarey & cutting)
+
BUG FIXES
AVRO-818. C: Fix data file corruption bug in C library (dcreager)
Modified: avro/branches/branch-1.5/lang/java/avro/src/main/java/org/apache/avro/file/DataFileWriter.java
URL: http://svn.apache.org/viewvc/avro/branches/branch-1.5/lang/java/avro/src/main/java/org/apache/avro/file/DataFileWriter.java?rev=1124972&r1=1124971&r2=1124972&view=diff
==============================================================================
--- avro/branches/branch-1.5/lang/java/avro/src/main/java/org/apache/avro/file/DataFileWriter.java (original)
+++ avro/branches/branch-1.5/lang/java/avro/src/main/java/org/apache/avro/file/DataFileWriter.java Thu May 19 15:53:04 2011
@@ -238,13 +238,41 @@ public class DataFileWriter<D> implement
return setMeta(key, Long.toString(value));
}
- /** Append a datum to the file. */
+ /** Thrown by {@link #append(Object)} when an exception occurs while writing a
+ * datum to the buffer. When this is thrown, the file is unaltered and may
+ * continue to be appended to. */
+ public static class AppendWriteException extends RuntimeException {
+ public AppendWriteException(Exception e) { super(e); }
+ }
+
+ /** Append a datum to the file.
+ * @see AppendWriteException
+ */
public void append(D datum) throws IOException {
assertOpen();
- dout.write(datum, bufOut);
+ int usedBuffer = bufferInUse();
+ try {
+ dout.write(datum, bufOut);
+ } catch (IOException e) {
+ resetBufferTo(usedBuffer);
+ throw new AppendWriteException(e);
+ } catch (RuntimeException re) {
+ resetBufferTo(usedBuffer);
+ throw new AppendWriteException(re);
+ }
blockCount++;
writeIfBlockFull();
}
+
+ // if there is an error encoding, flush the encoder and then
+ // reset the buffer position to contain size bytes, discarding the rest.
+ // Otherwise the file will be corrupt with a partial record.
+ private void resetBufferTo(int size) throws IOException {
+ bufOut.flush();
+ byte[] data = buffer.toByteArray();
+ buffer.reset();
+ buffer.write(data, 0, size);
+ }
/** Expert: Append a pre-encoded datum to the file. No validation is
* performed to check that the encoding conforms to the file's schema.
@@ -256,9 +284,13 @@ public class DataFileWriter<D> implement
blockCount++;
writeIfBlockFull();
}
+
+ private int bufferInUse() {
+ return (buffer.size() + bufOut.bytesBuffered());
+ }
private void writeIfBlockFull() throws IOException {
- if ((buffer.size() + bufOut.bytesBuffered()) >= syncInterval)
+ if (bufferInUse() >= syncInterval)
writeBlock();
}
Modified: avro/branches/branch-1.5/lang/java/avro/src/test/java/org/apache/avro/TestDataFile.java
URL: http://svn.apache.org/viewvc/avro/branches/branch-1.5/lang/java/avro/src/test/java/org/apache/avro/TestDataFile.java?rev=1124972&r1=1124971&r2=1124972&view=diff
==============================================================================
--- avro/branches/branch-1.5/lang/java/avro/src/test/java/org/apache/avro/TestDataFile.java (original)
+++ avro/branches/branch-1.5/lang/java/avro/src/test/java/org/apache/avro/TestDataFile.java Thu May 19 15:53:04 2011
@@ -26,12 +26,15 @@ import java.util.ArrayList;
import java.util.List;
import java.util.Random;
+import junit.framework.Assert;
+
import org.apache.avro.file.CodecFactory;
import org.apache.avro.file.FileReader;
import org.apache.avro.file.DataFileReader;
import org.apache.avro.file.DataFileStream;
import org.apache.avro.file.DataFileWriter;
import org.apache.avro.file.SeekableFileInput;
+import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericDatumReader;
import org.apache.avro.generic.GenericDatumWriter;
import org.apache.avro.io.DatumReader;
@@ -97,6 +100,20 @@ public class TestDataFile {
writer.append(datum);
if (++count%(COUNT/3) == 0)
writer.sync(); // force some syncs mid-file
+ if (count == 5) {
+ // force a write of an invalid record
+ boolean threwProperly = false;
+ try {
+ GenericData.Record record = (GenericData.Record) datum;
+ record.put(1, null);
+ threwProperly = true;
+ writer.append(record);
+ threwProperly = false;
+ } catch (DataFileWriter.AppendWriteException e) {
+ System.out.println("Ignoring: "+e);
+ }
+ Assert.assertTrue("failed to throw when expected", threwProperly);
+ }
}
} finally {
writer.close();