You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@avro.apache.org by th...@apache.org on 2011/10/01 20:37:15 UTC

svn commit: r1178066 - in /avro/trunk: ./ lang/java/avro/src/main/java/org/apache/avro/io/ lang/java/avro/src/main/java/org/apache/avro/io/parsing/ lang/java/avro/src/test/java/org/apache/avro/io/

Author: thiru
Date: Sat Oct  1 18:37:14 2011
New Revision: 1178066

URL: http://svn.apache.org/viewvc?rev=1178066&view=rev
Log:
AVRO-895. JsonDecoder does not tolerate JSON records with different field order

Modified:
    avro/trunk/CHANGES.txt
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/JsonDecoder.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/JsonEncoder.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/parsing/JsonGrammarGenerator.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/parsing/Symbol.java
    avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/TestEncoders.java

Modified: avro/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/avro/trunk/CHANGES.txt?rev=1178066&r1=1178065&r2=1178066&view=diff
==============================================================================
--- avro/trunk/CHANGES.txt (original)
+++ avro/trunk/CHANGES.txt Sat Oct  1 18:37:14 2011
@@ -94,6 +94,7 @@ Avro 1.6.0 (unreleased)
 
     AVRO-894. C: cmake build now works on Mac OS X Lion. (dcreager)
 
+    AVRO-895. JsonDecoder does not tolerate JSON records with different field order (thiru)
 Avro 1.5.4 (12 September 2011)
 
   IMPROVEMENTS

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/JsonDecoder.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/JsonDecoder.java?rev=1178066&r1=1178065&r2=1178066&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/JsonDecoder.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/JsonDecoder.java Sat Oct  1 18:37:14 2011
@@ -17,10 +17,17 @@
  */
 package org.apache.avro.io;
 
-import java.io.IOException;
 import java.io.EOFException;
+import java.io.IOException;
 import java.io.InputStream;
+import java.math.BigDecimal;
+import java.math.BigInteger;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Stack;
 
 import org.apache.avro.AvroTypeException;
 import org.apache.avro.Schema;
@@ -28,9 +35,13 @@ import org.apache.avro.io.parsing.JsonGr
 import org.apache.avro.io.parsing.Parser;
 import org.apache.avro.io.parsing.Symbol;
 import org.apache.avro.util.Utf8;
+import org.codehaus.jackson.Base64Variant;
 import org.codehaus.jackson.JsonFactory;
+import org.codehaus.jackson.JsonLocation;
 import org.codehaus.jackson.JsonParser;
+import org.codehaus.jackson.JsonStreamContext;
 import org.codehaus.jackson.JsonToken;
+import org.codehaus.jackson.ObjectCodec;
 
 /** A {@link Decoder} for Avro's JSON data encoding. 
  * </p>
@@ -42,6 +53,13 @@ public class JsonDecoder extends Parsing
   implements Parser.ActionHandler {
   private JsonParser in;
   private static JsonFactory jsonFactory = new JsonFactory();
+  Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+  ReorderBuffer currentReorderBuffer; 
+  
+  private static class ReorderBuffer {
+    public Map<String, List<JsonElement>> savedFields = new HashMap<String, List<JsonElement>>();
+    public JsonParser origParser = null; 
+  }
   
   static final String CHARSET = "ISO-8859-1";
 
@@ -427,26 +445,54 @@ public class JsonDecoder extends Parsing
   @Override
   public Symbol doAction(Symbol input, Symbol top) throws IOException {
     if (top instanceof Symbol.FieldAdjustAction) {
-      Symbol.FieldAdjustAction fa = (Symbol.FieldAdjustAction) top;
-      if (in.getCurrentToken() == JsonToken.FIELD_NAME) {
-        String fn = in.getCurrentName();
-        if (fa.fname.equals(fn)) {
-          in.nextToken();
+        Symbol.FieldAdjustAction fa = (Symbol.FieldAdjustAction) top;
+        String name = fa.fname;
+      if (currentReorderBuffer != null) {
+        List<JsonElement> node = currentReorderBuffer.savedFields.get(name);
+        if (node != null) {
+          currentReorderBuffer.savedFields.remove(name);
+          currentReorderBuffer.origParser = in;
+          in = makeParser(node);
           return null;
-        } else {
-          throw new AvroTypeException("Expected field name " + fa.fname +
-              " got " + in.getCurrentName());
         }
       }
+      if (in.getCurrentToken() == JsonToken.FIELD_NAME) {
+        do {
+          String fn = in.getText();
+          in.nextToken();
+          if (name.equals(fn)) {
+            return null;
+          } else {
+            if (currentReorderBuffer == null) {
+              currentReorderBuffer = new ReorderBuffer();
+            }
+            currentReorderBuffer.savedFields.put(fn, getVaueAsTree(in));
+          }
+        } while (in.getCurrentToken() == JsonToken.FIELD_NAME);
+        throw new AvroTypeException("Expected field name not found: " + fa.fname);
+      }
+    } else if (top == Symbol.FIELD_END) {
+      if (currentReorderBuffer != null && currentReorderBuffer.origParser != null) {
+        in = currentReorderBuffer.origParser;
+        currentReorderBuffer.origParser = null;
+      }
     } else if (top == Symbol.RECORD_START) {
       if (in.getCurrentToken() == JsonToken.START_OBJECT) {
         in.nextToken();
+        reorderBuffers.push(currentReorderBuffer);
+        currentReorderBuffer = null;
       } else {
         throw error("record-start");
       }
     } else if (top == Symbol.RECORD_END || top == Symbol.UNION_END) {
       if (in.getCurrentToken() == JsonToken.END_OBJECT) {
         in.nextToken();
+        if (top == Symbol.RECORD_END) {
+          if (currentReorderBuffer != null && !currentReorderBuffer.savedFields.isEmpty()) {
+            throw error("Unknown fields: " + currentReorderBuffer.savedFields.keySet());
+          }
+          currentReorderBuffer = reorderBuffers.pop();
+        }
       } else {
         throw error(top == Symbol.RECORD_END ? "record-end" : "union-end");
       }
@@ -456,6 +502,192 @@ public class JsonDecoder extends Parsing
     return null;
   }
 
+  private static class JsonElement {
+    public final JsonToken token;
+    public final String value;
+    public JsonElement(JsonToken t, String value) {
+      this.token = t;
+      this.value = value;
+    }
+    
+    public JsonElement(JsonToken t) {
+      this(t, null);
+    }
+  }
+  
+  private static List<JsonElement> getVaueAsTree(JsonParser in) throws IOException {
+    int level = 0;
+    List<JsonElement> result = new ArrayList<JsonElement>();
+    do {
+      JsonToken t = in.getCurrentToken();
+      switch (t) {
+      case START_OBJECT:
+      case START_ARRAY:
+        level++;
+        result.add(new JsonElement(t));
+        break;
+      case END_OBJECT:
+      case END_ARRAY:
+        level--;
+        result.add(new JsonElement(t));
+        break;
+      case FIELD_NAME:
+      case VALUE_STRING:
+      case VALUE_NUMBER_INT:
+      case VALUE_NUMBER_FLOAT:
+      case VALUE_TRUE:
+      case VALUE_FALSE:
+      case VALUE_NULL:
+        result.add(new JsonElement(t, in.getText()));
+        break;
+      }
+      in.nextToken();
+    } while (level != 0);
+    result.add(new JsonElement(null));
+    return result;
+  }
+
+  private JsonParser makeParser(final List<JsonElement> elements) throws IOException {
+    return new JsonParser() {
+      int pos = 0;
+
+      @Override
+      public ObjectCodec getCodec() {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public void setCodec(ObjectCodec c) {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public void close() throws IOException {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public JsonToken nextToken() throws IOException {
+        pos++;
+        return elements.get(pos).token;
+      }
+
+      @Override
+      public JsonParser skipChildren() throws IOException {
+        int level = 0;
+        do {
+          switch(elements.get(pos++).token) {
+          case START_ARRAY:
+          case START_OBJECT:
+            level++;
+            break;
+          case END_ARRAY:
+          case END_OBJECT:
+            level--;
+            break;
+          }
+        } while (level > 0);
+        return this;
+      }
+
+      @Override
+      public boolean isClosed() {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public String getCurrentName() throws IOException {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public JsonStreamContext getParsingContext() {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public JsonLocation getTokenLocation() {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public JsonLocation getCurrentLocation() {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public String getText() throws IOException {
+        return elements.get(pos).value;
+      }
+
+      @Override
+      public char[] getTextCharacters() throws IOException {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public int getTextLength() throws IOException {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public int getTextOffset() throws IOException {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public Number getNumberValue() throws IOException {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public NumberType getNumberType() throws IOException {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public int getIntValue() throws IOException {
+        return Integer.parseInt(getText());
+      }
+
+      @Override
+      public long getLongValue() throws IOException {
+        return Long.parseLong(getText());
+      }
+
+      @Override
+      public BigInteger getBigIntegerValue() throws IOException {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public float getFloatValue() throws IOException {
+        return Float.parseFloat(getText());
+      }
+
+      @Override
+      public double getDoubleValue() throws IOException {
+        return Double.parseDouble(getText());
+      }
+
+      @Override
+      public BigDecimal getDecimalValue() throws IOException {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public byte[] getBinaryValue(Base64Variant b64variant)
+        throws IOException {
+        throw new UnsupportedOperationException();
+      }
+      
+      @Override
+      public JsonToken getCurrentToken() {
+        return elements.get(pos).token;
+      }
+    };
+  }
+
   private AvroTypeException error(String type) {
     return new AvroTypeException("Expected " + type +
         ". Got " + in.getCurrentToken());

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/JsonEncoder.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/JsonEncoder.java?rev=1178066&r1=1178065&r2=1178066&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/JsonEncoder.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/JsonEncoder.java Sat Oct  1 18:37:14 2011
@@ -286,7 +286,7 @@ public class JsonEncoder extends Parsing
       out.writeStartObject();
     } else if (top == Symbol.RECORD_END || top == Symbol.UNION_END) {
       out.writeEndObject();
-    } else {
+    } else if (top != Symbol.FIELD_END) {
       throw new AvroTypeException("Unknown action symbol " + top);
     }
     return null;

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/parsing/JsonGrammarGenerator.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/parsing/JsonGrammarGenerator.java?rev=1178066&r1=1178065&r2=1178066&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/parsing/JsonGrammarGenerator.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/parsing/JsonGrammarGenerator.java Sat Oct  1 18:37:14 2011
@@ -76,7 +76,7 @@ public class JsonGrammarGenerator extend
       LitS wsc = new LitS(sc);
       Symbol rresult = seen.get(wsc);
       if (rresult == null) {
-        Symbol[] production = new Symbol[sc.getFields().size() * 2 + 2];
+        Symbol[] production = new Symbol[sc.getFields().size() * 3 + 2];
         rresult = Symbol.seq(production);
         seen.put(wsc, rresult);
 
@@ -86,6 +86,7 @@ public class JsonGrammarGenerator extend
         for (Field f : sc.getFields()) {
           production[--i] = new Symbol.FieldAdjustAction(n, f.name());
           production[--i] = generate(f.schema(), seen);
+          production[--i] = Symbol.FIELD_END;
           n++;
         }
         production[--i] = Symbol.RECORD_END;

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/parsing/Symbol.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/parsing/Symbol.java?rev=1178066&r1=1178065&r2=1178066&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/parsing/Symbol.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/parsing/Symbol.java Sat Oct  1 18:37:14 2011
@@ -543,6 +543,7 @@ public abstract class Symbol {
   public static final Symbol RECORD_START = new ImplicitAction(false);
   public static final Symbol RECORD_END = new ImplicitAction(true);
   public static final Symbol UNION_END = new ImplicitAction(true);
+  public static final Symbol FIELD_END = new ImplicitAction(true);
   
   public static final Symbol DEFAULT_END_ACTION = new ImplicitAction(true);
   public static final Symbol MAP_KEY_MARKER =

Modified: avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/TestEncoders.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/TestEncoders.java?rev=1178066&r1=1178065&r2=1178066&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/TestEncoders.java (original)
+++ avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/TestEncoders.java Sat Oct  1 18:37:14 2011
@@ -21,8 +21,10 @@ import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
 
+import org.apache.avro.AvroTypeException;
 import org.apache.avro.Schema;
 import org.apache.avro.Schema.Type;
+import org.apache.avro.generic.GenericDatumReader;
 import org.codehaus.jackson.JsonEncoding;
 import org.codehaus.jackson.JsonFactory;
 import org.codehaus.jackson.JsonGenerator;
@@ -98,4 +100,83 @@ public class TestEncoders {
     factory.validatingEncoder(s, e).configure(e);
   }
 
+  @Test
+  public void testJsonRecordOrdering() throws IOException {
+    String value = "{\"b\": 2, \"a\": 1}";
+    Schema schema = new Schema.Parser().parse("{\"type\": \"record\", \"name\": \"ab\", \"fields\": [" +
+        "{\"name\": \"a\", \"type\": \"int\"}, {\"name\": \"b\", \"type\": \"int\"}" +
+    		"]}");
+    GenericDatumReader<Object> reader = new GenericDatumReader<Object>(schema);
+    Decoder decoder = DecoderFactory.get().jsonDecoder(schema, value);
+    Object o = reader.read(null, decoder);
+    Assert.assertEquals("{\"a\": 1, \"b\": 2}", o.toString());
+  }
+
+  @Test(expected=AvroTypeException.class)
+  public void testJsonExcessFields() throws IOException {
+    String value = "{\"b\": { \"b3\": 1.4, \"b2\": 3.14, \"b1\": \"h\"}, \"a\": {\"a0\": 45, \"a2\":true, \"a1\": null}}";
+    Schema schema = new Schema.Parser().parse("{\"type\": \"record\", \"name\": \"ab\", \"fields\": [\n" +
+        "{\"name\": \"a\", \"type\": {\"type\":\"record\",\"name\":\"A\",\"fields\":\n" +
+        "[{\"name\":\"a1\", \"type\":\"null\"}, {\"name\":\"a2\", \"type\":\"boolean\"}]}},\n" +
+        "{\"name\": \"b\", \"type\": {\"type\":\"record\",\"name\":\"B\",\"fields\":\n" +
+        "[{\"name\":\"b1\", \"type\":\"string\"}, {\"name\":\"b2\", \"type\":\"float\"}, {\"name\":\"b3\", \"type\":\"double\"}]}}\n" +
+        "]}");
+    GenericDatumReader<Object> reader = new GenericDatumReader<Object>(schema);
+    Decoder decoder = DecoderFactory.get().jsonDecoder(schema, value);
+    reader.read(null, decoder);
+  }
+
+  @Test
+  public void testJsonRecordOrdering2() throws IOException {
+    String value = "{\"b\": { \"b3\": 1.4, \"b2\": 3.14, \"b1\": \"h\"}, \"a\": {\"a2\":true, \"a1\": null}}";
+    Schema schema = new Schema.Parser().parse("{\"type\": \"record\", \"name\": \"ab\", \"fields\": [\n" +
+        "{\"name\": \"a\", \"type\": {\"type\":\"record\",\"name\":\"A\",\"fields\":\n" +
+        "[{\"name\":\"a1\", \"type\":\"null\"}, {\"name\":\"a2\", \"type\":\"boolean\"}]}},\n" +
+        "{\"name\": \"b\", \"type\": {\"type\":\"record\",\"name\":\"B\",\"fields\":\n" +
+        "[{\"name\":\"b1\", \"type\":\"string\"}, {\"name\":\"b2\", \"type\":\"float\"}, {\"name\":\"b3\", \"type\":\"double\"}]}}\n" +
+        "]}");
+    GenericDatumReader<Object> reader = new GenericDatumReader<Object>(schema);
+    Decoder decoder = DecoderFactory.get().jsonDecoder(schema, value);
+    Object o = reader.read(null, decoder);
+    Assert.assertEquals("{\"a\": {\"a1\": null, \"a2\": true}, \"b\": {\"b1\": \"h\", \"b2\": 3.14, \"b3\": 1.4}}", o.toString());
+  }
+
+  @Test
+  public void testJsonRecordOrderingWithProjection() throws IOException {
+    String value = "{\"b\": { \"b3\": 1.4, \"b2\": 3.14, \"b1\": \"h\"}, \"a\": {\"a2\":true, \"a1\": null}}";
+    Schema writerSchema = new Schema.Parser().parse("{\"type\": \"record\", \"name\": \"ab\", \"fields\": [\n" +
+        "{\"name\": \"a\", \"type\": {\"type\":\"record\",\"name\":\"A\",\"fields\":\n" +
+        "[{\"name\":\"a1\", \"type\":\"null\"}, {\"name\":\"a2\", \"type\":\"boolean\"}]}},\n" +
+        "{\"name\": \"b\", \"type\": {\"type\":\"record\",\"name\":\"B\",\"fields\":\n" +
+        "[{\"name\":\"b1\", \"type\":\"string\"}, {\"name\":\"b2\", \"type\":\"float\"}, {\"name\":\"b3\", \"type\":\"double\"}]}}\n" +
+        "]}");
+    Schema readerSchema = new Schema.Parser().parse("{\"type\": \"record\", \"name\": \"ab\", \"fields\": [\n" +
+      "{\"name\": \"a\", \"type\": {\"type\":\"record\",\"name\":\"A\",\"fields\":\n" +
+      "[{\"name\":\"a1\", \"type\":\"null\"}, {\"name\":\"a2\", \"type\":\"boolean\"}]}}\n" +
+      "]}");
+    GenericDatumReader<Object> reader = new GenericDatumReader<Object>(writerSchema, readerSchema);
+    Decoder decoder = DecoderFactory.get().jsonDecoder(writerSchema, value);
+    Object o = reader.read(null, decoder);
+    Assert.assertEquals("{\"a\": {\"a1\": null, \"a2\": true}}", o.toString());
+  }
+
+  @Test
+  public void testJsonRecordOrderingWithProjection2() throws IOException {
+    String value = "{\"b\": { \"b1\": \"h\", \"b2\": [3.14, 3.56], \"b3\": 1.4}, \"a\": {\"a2\":true, \"a1\": null}}";
+    Schema writerSchema = new Schema.Parser().parse("{\"type\": \"record\", \"name\": \"ab\", \"fields\": [\n" +
+        "{\"name\": \"a\", \"type\": {\"type\":\"record\",\"name\":\"A\",\"fields\":\n" +
+        "[{\"name\":\"a1\", \"type\":\"null\"}, {\"name\":\"a2\", \"type\":\"boolean\"}]}},\n" +
+        "{\"name\": \"b\", \"type\": {\"type\":\"record\",\"name\":\"B\",\"fields\":\n" +
+        "[{\"name\":\"b1\", \"type\":\"string\"}, {\"name\":\"b2\", \"type\":{\"type\":\"array\", \"items\":\"float\"}}, {\"name\":\"b3\", \"type\":\"double\"}]}}\n" +
+        "]}");
+    Schema readerSchema = new Schema.Parser().parse("{\"type\": \"record\", \"name\": \"ab\", \"fields\": [\n" +
+      "{\"name\": \"a\", \"type\": {\"type\":\"record\",\"name\":\"A\",\"fields\":\n" +
+      "[{\"name\":\"a1\", \"type\":\"null\"}, {\"name\":\"a2\", \"type\":\"boolean\"}]}}\n" +
+      "]}");
+    GenericDatumReader<Object> reader = new GenericDatumReader<Object>(writerSchema, readerSchema);
+    Decoder decoder = DecoderFactory.get().jsonDecoder(writerSchema, value);
+    Object o = reader.read(null, decoder);
+    Assert.assertEquals("{\"a\": {\"a1\": null, \"a2\": true}}", o.toString());
+  }
+
 }