You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@avro.apache.org by sc...@apache.org on 2013/04/30 18:18:26 UTC

svn commit: r1477712 [2/2] - in /avro/trunk: ./ lang/java/avro/src/main/java/org/apache/avro/generic/ lang/java/avro/src/main/java/org/apache/avro/io/ lang/java/avro/src/main/java/org/apache/avro/reflect/ lang/java/avro/src/test/java/org/apache/avro/re...

Modified: avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/io/Perf.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/io/Perf.java?rev=1477712&r1=1477711&r2=1477712&view=diff
==============================================================================
--- avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/io/Perf.java (original)
+++ avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/io/Perf.java Tue Apr 30 16:18:26 2013
@@ -19,7 +19,7 @@ package org.apache.avro.io;
 
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
-import java.io.OutputStream;
+import java.lang.reflect.Array;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -36,11 +36,15 @@ import org.apache.avro.generic.GenericDa
 import org.apache.avro.generic.GenericDatumReader;
 import org.apache.avro.generic.GenericDatumWriter;
 import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.reflect.ReflectData;
+import org.apache.avro.reflect.ReflectDatumReader;
+import org.apache.avro.reflect.ReflectDatumWriter;
 import org.apache.avro.specific.SpecificDatumReader;
 import org.apache.avro.specific.SpecificDatumWriter;
 import org.apache.avro.specific.SpecificRecordBase;
 import org.apache.avro.util.Utf8;
 
+
 /**
  * Performance tests for various low level operations of
  * Avro encoding and decoding.
@@ -77,6 +81,7 @@ public class Perf {
   private static final List<TestDescriptor> GENERIC = new ArrayList<TestDescriptor>();
   private static final List<TestDescriptor> GENERIC_ONETIME = new ArrayList<TestDescriptor>();
   private static final List<TestDescriptor> SPECIFIC = new ArrayList<TestDescriptor>();
+  private static final List<TestDescriptor> REFLECT = new ArrayList<TestDescriptor>();
   private static final LinkedHashMap<String, TestDescriptor> ALL_TESTS;
   private static final LinkedHashMap<String, List<TestDescriptor>> BATCHES;
   static {
@@ -113,6 +118,19 @@ public class Perf {
     new TestDescriptor(GenericOneTimeReaderUse.class, "-Gotr").add(GENERIC_ONETIME);
     new TestDescriptor(GenericOneTimeUse.class, "-Got").add(GENERIC_ONETIME);
     new TestDescriptor(FooBarSpecificRecordTest.class, "-Sf").add(SPECIFIC);
+    BATCHES.put("-reflect", REFLECT);
+    new TestDescriptor(ReflectRecordTest.class, "-REFr").add(REFLECT);
+    new TestDescriptor(ReflectBigRecordTest.class, "-REFbr").add(REFLECT);
+    new TestDescriptor(ReflectFloatTest.class, "-REFf").add(REFLECT);
+    new TestDescriptor(ReflectDoubleTest.class, "-REFd").add(REFLECT);
+    new TestDescriptor(ReflectIntArrayTest.class, "-REFia").add(REFLECT);
+    new TestDescriptor(ReflectLongArrayTest.class, "-REFla").add(REFLECT);
+    new TestDescriptor(ReflectDoubleArrayTest.class, "-REFda").add(REFLECT);
+    new TestDescriptor(ReflectFloatArrayTest.class, "-REFfa").add(REFLECT);
+    new TestDescriptor(ReflectNestedFloatArrayTest.class, "-REFnf").add(REFLECT);
+    new TestDescriptor(ReflectNestedObjectArrayTest.class, "-REFno").add(REFLECT);
+    new TestDescriptor(ReflectNestedLargeFloatArrayTest.class, "-REFnlf").add(REFLECT);
+    new TestDescriptor(ReflectNestedLargeFloatArrayBlockedTest.class, "-REFnlfb").add(REFLECT);
   }
   
   private static void usage() {
@@ -149,7 +167,7 @@ public class Perf {
       }
       List<TestDescriptor> lt = BATCHES.get(a);
       if (null != lt) {
-        for(TestDescriptor td : lt) {
+        for (TestDescriptor td : lt) {
           tests.add(td.test.newInstance());
         }
         continue;
@@ -233,7 +251,7 @@ public class Perf {
   
   private static final void printHeader() {
     String header = String.format(
-        "%29s     time    M entries/sec   M bytes/sec  bytes/cycle",
+        "%60s     time    M entries/sec   M bytes/sec  bytes/cycle",
         "test name");
     System.out.println(header.toString());
   }
@@ -243,7 +261,7 @@ public class Perf {
     double entries = (t.cycles * (double) t.count);
     double bytes = t.cycles * (double) t.encodedSize;
     StringBuilder result = new StringBuilder();
-    result.append(String.format("%29s: %6d ms  ", name, (s/1000)));
+    result.append(String.format("%42s: %6d ms  ", name, (s/1000)));
     result.append(String.format("%10.3f   %11.3f   %11d", 
         (entries / s), (bytes/ s),  t.encodedSize));
     System.out.println(result.toString());
@@ -312,7 +330,7 @@ public class Perf {
     }
     BasicTest(String name, String json, int factor) throws IOException {
       super(name, CYCLES, COUNT/factor);
-      this.schema = Schema.parse(json);
+      this.schema = new Schema.Parser().parse(json);
     }
 
     @Override
@@ -336,31 +354,30 @@ public class Perf {
       return newDecoder();
     }
     
-    protected Encoder getEncoder() throws IOException {
-      return newEncoder();
+    private Encoder getEncoder() throws IOException {
+      return newEncoder(getOutputStream());
     }
 
     protected Decoder newDecoder() {
       return decoder_factory.binaryDecoder(data, null);
     }
     
-    protected Encoder newEncoder() {
-      OutputStream out = new ByteArrayOutputStream((int)(encodedSize > 0 ? encodedSize : count));
-      return newEncoder(out);
-    }
-    // switch out what is returned to test different encoders.
-    private Encoder newEncoder(OutputStream out) {
+    protected Encoder newEncoder(ByteArrayOutputStream out) throws IOException {
       Encoder e = encoder_factory.binaryEncoder(out, null);
-//      Encoder e = encoder_factory.directBinaryEncoder(out, null);
-//      Encoder e = encoder_factory.blockingBinaryEncoder(out, null);
-//      Encoder e = new LegacyBinaryEncoder(out);
+//    Encoder e = encoder_factory.directBinaryEncoder(out, null);
+//    Encoder e = encoder_factory.blockingBinaryEncoder(out, null);
+//    Encoder e = new LegacyBinaryEncoder(out);
       return e;
     }
+
+    private ByteArrayOutputStream getOutputStream() {
+      return new ByteArrayOutputStream((int)(encodedSize > 0 ? encodedSize : count));
+    }
     
     @Override
     void init() throws IOException {
       genSourceData();
-      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      ByteArrayOutputStream baos = getOutputStream();
       Encoder e = newEncoder(baos);
       writeInternal(e);
       e.flush();
@@ -520,7 +537,7 @@ public class Perf {
    
     @Override
     void readInternal(Decoder d) throws IOException {
-      for (int i = 0; i < count/4; i++) {
+      for (int i = 0; i < count; i+=4) {
         d.readFloat();
         d.readFloat();
         d.readFloat();
@@ -562,7 +579,7 @@ public class Perf {
    
     @Override
     void readInternal(Decoder d) throws IOException {
-      for (int i = 0; i < count/4; i++) {
+      for (int i = 0; i < count; i+=4) {
         d.readDouble();
         d.readDouble();
         d.readDouble();
@@ -744,7 +761,7 @@ public class Perf {
     @Override
     void readInternal(Decoder d) throws IOException {
       d.readArrayStart();
-      for(long i = d.readArrayStart(); i != 0; i = d.arrayNext()) {
+      for (long i = d.readArrayStart(); i != 0; i = d.arrayNext()) {
         for (long j = 0; j < i; j++) {
           d.readFloat();
           d.readFloat();
@@ -778,7 +795,7 @@ public class Perf {
   static class MapTest extends FloatTest {
     public MapTest() throws IOException {
       super("Map", "{ \"type\": \"map\", \"values\": " +
-      		"  { \"type\": \"record\", \"name\":\"Vals\", \"fields\": [" +
+          "  { \"type\": \"record\", \"name\":\"Vals\", \"fields\": [" +
           "   {\"name\":\"f1\", \"type\":\"float\"}," +
           "   {\"name\":\"f2\", \"type\":\"float\"}," +
           "   {\"name\":\"f3\", \"type\":\"float\"}," +
@@ -789,7 +806,7 @@ public class Perf {
     @Override
     void readInternal(Decoder d) throws IOException {
       Utf8 key = new Utf8();
-      for(long i = d.readMapStart(); i != 0; i = d.mapNext()) {
+      for (long i = d.readMapStart(); i != 0; i = d.mapNext()) {
         for (long j = 0; j < i; j++) {
           key = d.readString(key);
           d.readFloat();
@@ -806,7 +823,7 @@ public class Perf {
       e.writeMapStart();
       e.setItemCount(items);
       Utf8 foo = new Utf8("foo");
-      for (int i = 0, j = 0; i < sourceData.length;i+=4, j++) {
+      for (int i = 0; i < sourceData.length;i+=4) {
         e.startItem();
         e.writeString(foo);
         e.writeFloat(sourceData[i]);
@@ -831,8 +848,8 @@ public class Perf {
   private static final String NESTED_RECORD_SCHEMA =
     "{ \"type\": \"record\", \"name\": \"R\", \"fields\": [\n"
     + "{ \"name\": \"f1\", \"type\": \n" +
-    		"{ \"type\": \"record\", \"name\": \"D\", \"fields\": [\n" +
-    		  "{\"name\": \"dbl\", \"type\": \"double\" }]\n" +
+        "{ \"type\": \"record\", \"name\": \"D\", \"fields\": [\n" +
+          "{\"name\": \"dbl\", \"type\": \"double\" }]\n" +
         "} },\n"
     + "{ \"name\": \"f2\", \"type\": \"D\" },\n"
     + "{ \"name\": \"f3\", \"type\": \"D\" },\n"
@@ -848,6 +865,9 @@ public class Perf {
     int f4;
     int f5;
     int f6;
+    Rec() {
+      
+    }
     Rec(Random r) {
       f1 = r.nextDouble();
       f2 = r.nextDouble();
@@ -913,8 +933,8 @@ public class Perf {
       return new ValidatingDecoder(schema, super.getDecoder());
     }
     @Override
-    protected Encoder getEncoder() throws IOException {
-      return encoder_factory.validatingEncoder(schema, super.getEncoder());  
+    protected Encoder newEncoder(ByteArrayOutputStream out) throws IOException {
+      return encoder_factory.validatingEncoder(schema, super.newEncoder(out));  
     }
   }
   
@@ -971,7 +991,7 @@ public class Perf {
     private final Schema readerSchema;
     public RecordWithDefault() throws IOException {
       super("RecordWithDefault");
-      readerSchema = Schema.parse(RECORD_SCHEMA_WITH_DEFAULT);
+      readerSchema = new Schema.Parser().parse(RECORD_SCHEMA_WITH_DEFAULT);
       isWriteTest = false;
     }
     @Override
@@ -1013,7 +1033,7 @@ public class Perf {
     private final Schema readerSchema;
     public RecordWithOutOfOrder() throws IOException {
       super("RecordWithOutOfOrder");
-      readerSchema = Schema.parse(RECORD_SCHEMA_WITH_OUT_OF_ORDER);
+      readerSchema = new Schema.Parser().parse(RECORD_SCHEMA_WITH_OUT_OF_ORDER);
       isWriteTest = false;
     }
     @Override
@@ -1051,7 +1071,7 @@ public class Perf {
     private final Schema readerSchema;
     public RecordWithPromotion() throws IOException {
       super("RecordWithPromotion");
-      readerSchema = Schema.parse(RECORD_SCHEMA_WITH_PROMOTION);
+      readerSchema = new Schema.Parser().parse(RECORD_SCHEMA_WITH_PROMOTION);
       isWriteTest = false;
     }
     @Override
@@ -1269,7 +1289,7 @@ public class Perf {
     }
     @Override
     protected Schema getReaderSchema() {
-      return  Schema.parse(RECORD_SCHEMA_WITH_DEFAULT);
+      return  new Schema.Parser().parse(RECORD_SCHEMA_WITH_DEFAULT);
     }
   }
 
@@ -1279,7 +1299,7 @@ public class Perf {
     }
     @Override
     protected Schema getReaderSchema() {
-      return Schema.parse(RECORD_SCHEMA_WITH_OUT_OF_ORDER);
+      return new Schema.Parser().parse(RECORD_SCHEMA_WITH_OUT_OF_ORDER);
     }
   }
 
@@ -1289,7 +1309,7 @@ public class Perf {
     }
     @Override
     protected Schema getReaderSchema() {
-      return Schema.parse(RECORD_SCHEMA_WITH_PROMOTION);
+      return new Schema.Parser().parse(RECORD_SCHEMA_WITH_PROMOTION);
     }
   }
   
@@ -1336,7 +1356,7 @@ public class Perf {
     private Object[] sourceData;
 
     protected SpecificTest(String name, String writerSchema) throws IOException {
-      super(name, writerSchema, 12);
+      super(name, writerSchema, 48);
       reader = newReader();
       writer = newWriter();
     }
@@ -1384,7 +1404,8 @@ public class Perf {
     }
   }
 
-  static class FooBarSpecificRecordTest extends SpecificTest<FooBarSpecificRecord> {
+  static class FooBarSpecificRecordTest extends
+      SpecificTest<FooBarSpecificRecord> {
     public FooBarSpecificRecordTest() throws IOException {
       super("FooBarSpecificRecordTest", FooBarSpecificRecord.SCHEMA$.toString());
     }
@@ -1398,17 +1419,334 @@ public class Perf {
       }
 
       try {
-        return FooBarSpecificRecord.newBuilder().
-          setId(r.nextInt()).
-          setName(randomString(r)).
-          setNicknames(Arrays.asList(randomString(r), randomString(r))).
-          setTypeEnum(typeEnums[r.nextInt(typeEnums.length)]).
-          setRelatedids(relatedIds).
-          build();
-      }
-      catch (Exception e) {
+        return FooBarSpecificRecord.newBuilder().setId(r.nextInt())
+            .setName(randomString(r))
+            .setNicknames(Arrays.asList(randomString(r), randomString(r)))
+            .setTypeEnum(typeEnums[r.nextInt(typeEnums.length)])
+            .setRelatedids(relatedIds).build();
+      } catch (Exception e) {
         throw new RuntimeException(e);
       }
     }
   }
+
+  static abstract class ReflectTest<T> extends BasicTest {
+    T[] sourceData = null;
+    ReflectDatumReader<T> reader;
+    ReflectDatumWriter<T> writer;
+    Class<T> clazz;
+
+    @SuppressWarnings("unchecked")
+    ReflectTest(String name, T sample, int factor) throws IOException {
+      super(name, ReflectData.get().getSchema(sample.getClass()).toString(), factor);
+      clazz = (Class<T>) sample.getClass();
+      reader = new ReflectDatumReader<T>(schema);
+      writer = new ReflectDatumWriter<T>(schema);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    protected final void genSourceData() {
+      Random r = newRandom();
+      sourceData = (T[]) Array.newInstance(clazz, count);
+      for (int i = 0; i < sourceData.length; i++) {
+        sourceData[i] = createDatum(r);
+      }
+    }
+
+    protected abstract T createDatum(Random r);
+
+    @Override
+    protected final void readInternal(Decoder d) throws IOException {
+      for (int i = 0; i < count; i++) {
+        reader.read(null, d);
+      }
+    }
+
+    @Override
+    protected final void writeInternal(Encoder e) throws IOException {
+      for (int i = 0; i < sourceData.length; i++) {
+        writer.write(sourceData[i], e);
+      }
+    }
+
+    @Override
+    protected final void reset() {
+      sourceData = null;
+      data = null;
+    }
+  }
+
+  static class ReflectRecordTest extends ReflectTest<Rec> {
+    ReflectRecordTest() throws IOException {
+      super("ReflectRecord", new Rec(), 12);
+    }
+
+    @Override
+    protected Rec createDatum(Random r) {
+      return new Rec(r);
+    }
+  }
+  
+  static class ReflectFloatTest extends ReflectTest<float[]> {
+    ReflectFloatTest() throws IOException {
+      super("ReflectFloat", new float[0], COUNT);
+    }
+
+    @Override
+    protected float[] createDatum(Random r) {
+      return populateFloatArray(r, COUNT / count);
+    }
+  }
+
+  static class ReflectDoubleTest extends ReflectTest<double[]> {
+    ReflectDoubleTest() throws IOException {
+      super("ReflectDouble", new double[0], COUNT);
+    }
+
+    @Override
+    protected double[] createDatum(Random r) {
+      return populateDoubleArray(r, COUNT / count);
+    }
+  }
+
+  static class ReflectFloatArrayTest extends ReflectTest<float[]> {
+    ReflectFloatArrayTest() throws IOException {
+      super("ReflectFloatArray", new float[0], 10);
+    }
+
+    @Override
+    protected float[] createDatum(Random r) {
+      return populateFloatArray(r, false);
+    }
+  }
+
+  static class ReflectDoubleArrayTest extends ReflectTest<double[]> {
+    ReflectDoubleArrayTest() throws IOException {
+      super("ReflectDoubleArray", new double[0], 20);
+    }
+
+    @Override
+    protected double[] createDatum(Random r) {
+      return populateDoubleArray(r);
+    }
+  }
+  
+  static class ReflectIntArrayTest extends ReflectTest<int[]> {
+    ReflectIntArrayTest() throws IOException {
+      super("ReflectIntArray", new int[0], 12);
+    }
+
+    @Override
+    protected int[] createDatum(Random r) {
+      return populateIntArray(r);
+    }
+  }
+  
+  static class ReflectLongArrayTest extends ReflectTest<long[]> {
+    ReflectLongArrayTest() throws IOException {
+      super("ReflectLongArray", new long[0], 24);
+    }
+
+    @Override
+    protected long[] createDatum(Random r) {
+      return populateLongArray(r);
+    }
+  }
+
+
+  static class ReflectNestedObjectArrayTest extends
+      ReflectTest<ReflectNestedObjectArrayTest.Foo> {
+    ReflectNestedObjectArrayTest() throws IOException {
+      super("ReflectNestedObjectArray", new Foo(new Random()), 50);
+    }
+
+    @Override
+    protected Foo createDatum(Random r) {
+      return new Foo(r);
+    }
+
+    static public class Foo {
+      Vals[] bar;
+
+      Foo() {
+      }
+      
+      Foo(Random r) {
+        bar = new Vals[smallArraySize(r)];
+        for (int i = 0; i < bar.length; i++) {
+          bar[i] = new Vals(r);
+        }
+      }
+    }
+
+    static class Vals {
+      float f1;
+      float f2;
+      float f3;
+      float f4;
+
+      Vals(){
+      }
+      
+      Vals(Random r) {
+        this.f1 = r.nextFloat();
+        this.f2 = r.nextFloat();
+        this.f3 = r.nextFloat();
+        this.f4 = r.nextFloat();
+      }
+    }
+
+  }
+
+  static public class FloatFoo {
+    float[] floatBar;
+    
+    FloatFoo() {
+    }
+
+    FloatFoo(Random r, boolean large) {
+      floatBar = populateFloatArray(r, large);
+    }
+  }
+
+  // average of 8, between 1 and 15
+  private static int smallArraySize(Random r) {
+    return r.nextInt(15) + 1;
+  }
+
+  // average of 64, between 16 and 112
+  private static int largeArraySize(Random r) {
+    return r.nextInt(97) + 16;
+  }
+
+  static float[] populateFloatArray(Random r, boolean large) {
+    int size = large ? largeArraySize(r) : smallArraySize(r);
+    return populateFloatArray(r, size);
+  }
+  
+  static float[] populateFloatArray(Random r, int size) {
+    float[] result = new float[size];
+    for (int i = 0; i < result.length; i++) {
+      result[i] = r.nextFloat();
+    }
+    return result;
+  }
+  
+  static double[] populateDoubleArray(Random r) {
+    return populateDoubleArray(r, smallArraySize(r));
+  }
+  
+  static double[] populateDoubleArray(Random r, int size) {
+    double[] result = new double[size];
+    for (int i = 0; i < result.length; i++) {
+      result[i] = r.nextDouble();
+    }
+    return result;
+  }
+
+  static int[] populateIntArray(Random r) {
+    int size = smallArraySize(r);
+    int[] result = new int[size];
+    for (int i = 0; i < result.length; i++) {
+      result[i] = r.nextInt();
+    }
+    return result;
+  }
+  
+  static long[] populateLongArray(Random r) {
+    int size = smallArraySize(r);
+    long[] result = new long[size];
+    for (int i = 0; i < result.length; i++) {
+      result[i] = r.nextLong();
+    }
+    return result;
+  }
+  
+  static class ReflectNestedFloatArrayTest extends ReflectTest<FloatFoo> {
+    public ReflectNestedFloatArrayTest() throws IOException {
+      super("ReflectNestedFloatArray", new FloatFoo(new Random(), false), 10);
+    }
+
+    @Override
+    protected FloatFoo createDatum(Random r) {
+      return new FloatFoo(r, false);
+    }
+  }
+
+  static class ReflectNestedLargeFloatArrayTest extends ReflectTest<FloatFoo> {
+    public ReflectNestedLargeFloatArrayTest() throws IOException {
+      super("ReflectNestedLargeFloatArray", new FloatFoo(new Random(), true),
+          60);
+    }
+
+    @Override
+    protected FloatFoo createDatum(Random r) {
+      return new FloatFoo(r, true);
+    }
+
+  }
+  
+  static class ReflectNestedLargeFloatArrayBlockedTest extends ReflectTest<FloatFoo> {
+    public ReflectNestedLargeFloatArrayBlockedTest() throws IOException {
+      super("ReflectNestedLargeFloatArrayBlocked", new FloatFoo(new Random(), true),
+          60);
+    }
+
+    @Override
+    protected FloatFoo createDatum(Random r) {
+      return new FloatFoo(r, true);
+    }
+    
+    @Override
+    protected Encoder newEncoder(ByteArrayOutputStream out) throws IOException {
+      return new EncoderFactory().configureBlockSize(254).blockingBinaryEncoder(out, null);
+    }
+
+  }
+
+  @SuppressWarnings("unused")
+  private static class Rec1 {
+    double d1;
+    double d11;
+    float f2;
+    float f22;
+    int f3;
+    int f33;
+    long f4;
+    long f44;
+    byte f5;
+    byte f55;
+    short f6;
+    short f66;
+
+    Rec1() {
+    }
+    
+    Rec1(Random r) {
+      d1 = r.nextDouble();
+      d11 = r.nextDouble();
+      f2 = r.nextFloat();
+      f22 = r.nextFloat();
+      f3 = r.nextInt();
+      f33 = r.nextInt();
+      f4 = r.nextLong();
+      f44 = r.nextLong();
+      f5 = (byte) r.nextInt();
+      f55 = (byte) r.nextInt();
+      f6 = (short) r.nextInt();
+      f66 = (short) r.nextInt();
+    }
+  }
+
+  static class ReflectBigRecordTest extends ReflectTest<Rec1> {
+    public ReflectBigRecordTest() throws IOException {
+      super("ReflectBigRecord", new Rec1(new Random()), 20);
+    }
+
+    @Override
+    protected Rec1 createDatum(Random r) {
+      return new Rec1(r);
+    }
+  }
 }