avro-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From cutt...@apache.org
Subject svn commit: r898966 - in /hadoop/avro/trunk: CHANGES.txt lang/java/src/java/org/apache/avro/file/DataFileConstants.java lang/java/src/java/org/apache/avro/file/DataFileWriter.java lang/java/src/test/java/org/apache/avro/TestDataFile.java
Date Wed, 13 Jan 2010 22:06:57 GMT
Author: cutting
Date: Wed Jan 13 22:06:56 2010
New Revision: 898966

URL: http://svn.apache.org/viewvc?rev=898966&view=rev
Log:
AVRO-274.  Make Java's data file sync interval configurable.

Modified:
    hadoop/avro/trunk/CHANGES.txt
    hadoop/avro/trunk/lang/java/src/java/org/apache/avro/file/DataFileConstants.java
    hadoop/avro/trunk/lang/java/src/java/org/apache/avro/file/DataFileWriter.java
    hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/TestDataFile.java

Modified: hadoop/avro/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/CHANGES.txt?rev=898966&r1=898965&r2=898966&view=diff
==============================================================================
--- hadoop/avro/trunk/CHANGES.txt (original)
+++ hadoop/avro/trunk/CHANGES.txt Wed Jan 13 22:06:56 2010
@@ -62,6 +62,8 @@
     AVRO-152. Add support for documentation strings to schemas,
     protocols, and messages. (Philip Zeyliger via cutting)
 
+    AVRO-274. Make Java's data file sync interval configurable.  (cutting)
+
   IMPROVEMENTS
 
     AVRO-157. Changes from code review comments for C++. (sbanacho)

Modified: hadoop/avro/trunk/lang/java/src/java/org/apache/avro/file/DataFileConstants.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/java/org/apache/avro/file/DataFileConstants.java?rev=898966&r1=898965&r2=898966&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/java/org/apache/avro/file/DataFileConstants.java (original)
+++ hadoop/avro/trunk/lang/java/src/java/org/apache/avro/file/DataFileConstants.java Wed Jan
13 22:06:56 2010
@@ -28,7 +28,7 @@
   };
   public static final long FOOTER_BLOCK = -1;
   public static final int SYNC_SIZE = 16;
-  public static final int SYNC_INTERVAL = 1000*SYNC_SIZE; 
+  public static final int DEFAULT_SYNC_INTERVAL = 1000*SYNC_SIZE; 
 
   public static final String SCHEMA = "schema";
   public static final String COUNT = "count";

Modified: hadoop/avro/trunk/lang/java/src/java/org/apache/avro/file/DataFileWriter.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/java/org/apache/avro/file/DataFileWriter.java?rev=898966&r1=898965&r2=898966&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/java/org/apache/avro/file/DataFileWriter.java (original)
+++ hadoop/avro/trunk/lang/java/src/java/org/apache/avro/file/DataFileWriter.java Wed Jan
13 22:06:56 2010
@@ -63,11 +63,11 @@
 
   private int blockCount;                       // # entries in current block
 
-  private ByteArrayOutputStream buffer =
-    new ByteArrayOutputStream(DataFileConstants.SYNC_INTERVAL*2);
-  private Encoder bufOut = new BinaryEncoder(buffer);
+  private ByteArrayOutputStream buffer;
+  private Encoder bufOut;
 
   private byte[] sync;                          // 16 random bytes
+  private int syncInterval = DataFileConstants.DEFAULT_SYNC_INTERVAL;
 
   private boolean isOpen;
 
@@ -83,6 +83,12 @@
     if (isOpen) throw new AvroRuntimeException("already open");
   }
 
+  /** Set the synchronization interval for this file, in bytes. */
+  public DataFileWriter<D> setSyncInterval(int syncInterval) {
+    this.syncInterval = syncInterval;
+    return this;
+  }
+
   /** Open a new file for data matching a schema. */
   public DataFileWriter<D> create(Schema schema, File file) throws IOException {
     return create(schema, new FileOutputStream(file));
@@ -138,6 +144,8 @@
   }
 
   private void init(OutputStream outs) throws IOException {
+    this.buffer = new ByteArrayOutputStream(syncInterval*2);
+    this.bufOut = new BinaryEncoder(buffer);
     this.out = new BufferedFileOutputStream(outs);
     this.vout = new BinaryEncoder(out);
     dout.setSchema(schema);
@@ -179,7 +187,7 @@
     assertOpen();
     dout.write(datum, bufOut);
     blockCount++;
-    if (buffer.size() >= DataFileConstants.SYNC_INTERVAL)
+    if (buffer.size() >= syncInterval)
       writeBlock();
   }
 

Modified: hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/TestDataFile.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/TestDataFile.java?rev=898966&r1=898965&r2=898966&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/TestDataFile.java (original)
+++ hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/TestDataFile.java Wed Jan 13
22:06:56 2010
@@ -53,6 +53,7 @@
   public void testGenericWrite() throws IOException {
     DataFileWriter<Object> writer =
       new DataFileWriter<Object>(new GenericDatumWriter<Object>())
+      .setSyncInterval(100)
       .create(SCHEMA, FILE);
     try {
       int count = 0;



Mime
View raw message