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 2010/03/04 00:48:51 UTC

svn commit: r918783 - in /hadoop/avro/trunk: ./ lang/java/src/java/org/apache/avro/generic/ lang/java/src/java/org/apache/avro/ipc/ lang/java/src/java/org/apache/avro/specific/ lang/java/src/java/org/apache/avro/util/ lang/java/src/test/java/org/apache...

Author: cutting
Date: Wed Mar  3 23:48:50 2010
New Revision: 918783

URL: http://svn.apache.org/viewvc?rev=918783&view=rev
Log:
AVRO-433.  Fix exceptions in Java reflect RPC.

Added:
    hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/SimpleError.java
Modified:
    hadoop/avro/trunk/CHANGES.txt
    hadoop/avro/trunk/lang/java/src/java/org/apache/avro/generic/GenericRequestor.java
    hadoop/avro/trunk/lang/java/src/java/org/apache/avro/generic/GenericResponder.java
    hadoop/avro/trunk/lang/java/src/java/org/apache/avro/ipc/RPCContext.java
    hadoop/avro/trunk/lang/java/src/java/org/apache/avro/ipc/Requestor.java
    hadoop/avro/trunk/lang/java/src/java/org/apache/avro/ipc/Responder.java
    hadoop/avro/trunk/lang/java/src/java/org/apache/avro/specific/SpecificRequestor.java
    hadoop/avro/trunk/lang/java/src/java/org/apache/avro/specific/SpecificResponder.java
    hadoop/avro/trunk/lang/java/src/java/org/apache/avro/util/Utf8.java
    hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/TestProtocolGeneric.java
    hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/TestProtocolReflect.java

Modified: hadoop/avro/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/CHANGES.txt?rev=918783&r1=918782&r2=918783&view=diff
==============================================================================
--- hadoop/avro/trunk/CHANGES.txt (original)
+++ hadoop/avro/trunk/CHANGES.txt Wed Mar  3 23:48:50 2010
@@ -26,6 +26,8 @@
 
     AVRO-437. Fix some typos in docs. (Amichai Rothman via cutting)
 
+    AVRO-433. Fix exceptions in Java reflect RPC. (cutting)
+
 Avro 1.3.0 (24 February 2010)
 
   INCOMPATIBLE CHANGES

Modified: hadoop/avro/trunk/lang/java/src/java/org/apache/avro/generic/GenericRequestor.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/java/org/apache/avro/generic/GenericRequestor.java?rev=918783&r1=918782&r2=918783&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/java/org/apache/avro/generic/GenericRequestor.java (original)
+++ hadoop/avro/trunk/lang/java/src/java/org/apache/avro/generic/GenericRequestor.java Wed Mar  3 23:48:50 2010
@@ -36,6 +36,20 @@
   }
 
   @Override
+  public Object request(String messageName, Object request)
+    throws IOException {
+    try {
+      return super.request(messageName, request);
+    } catch (Exception e) {
+      if (e instanceof RuntimeException)
+        throw (RuntimeException)e;
+      if (e instanceof IOException)
+        throw (IOException)e;
+      throw new AvroRemoteException(e);
+    }
+  }
+
+  @Override
   public void writeRequest(Schema schema, Object request, Encoder out)
     throws IOException {
     new GenericDatumWriter<Object>(schema).write(request, out);

Modified: hadoop/avro/trunk/lang/java/src/java/org/apache/avro/generic/GenericResponder.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/java/org/apache/avro/generic/GenericResponder.java?rev=918783&r1=918782&r2=918783&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/java/org/apache/avro/generic/GenericResponder.java (original)
+++ hadoop/avro/trunk/lang/java/src/java/org/apache/avro/generic/GenericResponder.java Wed Mar  3 23:48:50 2010
@@ -46,9 +46,11 @@
   }
 
   @Override
-  public void writeError(Schema schema, AvroRemoteException error,
+  public void writeError(Schema schema, Object error,
                          Encoder out) throws IOException {
-    new GenericDatumWriter<Object>(schema).write(error.getValue(), out);
+    if (error instanceof AvroRemoteException)
+      error = ((AvroRemoteException)error).getValue();
+    new GenericDatumWriter<Object>(schema).write(error, out);
   }
 
 }

Modified: hadoop/avro/trunk/lang/java/src/java/org/apache/avro/ipc/RPCContext.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/java/org/apache/avro/ipc/RPCContext.java?rev=918783&r1=918782&r2=918783&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/java/org/apache/avro/ipc/RPCContext.java (original)
+++ hadoop/avro/trunk/lang/java/src/java/org/apache/avro/ipc/RPCContext.java Wed Mar  3 23:48:50 2010
@@ -37,7 +37,7 @@
   protected Map<Utf8,ByteBuffer> requestCallMeta, responseCallMeta;
   
   protected Object response;
-  protected AvroRemoteException error;
+  protected Exception error;
   private Message message;
   
   /**
@@ -124,7 +124,7 @@
     return response;
   }
   
-  void setError(AvroRemoteException error) {
+  void setError(Exception error) {
     this.response = null;
     this.error = error;
   }
@@ -135,7 +135,7 @@
    * @return the exception generated at the server, or
    * null if no such exception
    */
-  public AvroRemoteException error() {
+  public Exception error() {
     return error;
   }
   

Modified: hadoop/avro/trunk/lang/java/src/java/org/apache/avro/ipc/Requestor.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/java/org/apache/avro/ipc/Requestor.java?rev=918783&r1=918782&r2=918783&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/java/org/apache/avro/ipc/Requestor.java (original)
+++ hadoop/avro/trunk/lang/java/src/java/org/apache/avro/ipc/Requestor.java Wed Mar  3 23:48:50 2010
@@ -84,7 +84,7 @@
 
   /** Writes a request message and reads a response or error message. */
   public Object request(String messageName, Object request)
-    throws IOException {
+    throws Exception {
     BinaryDecoder in = null;
     Message m;
     RPCContext context = new RPCContext();
@@ -130,7 +130,7 @@
       return response;
       
     } else {
-      AvroRemoteException error = readError(m.getErrors(), in);
+      Exception error = readError(m.getErrors(), in);
       context.setError(error);
       for (RPCPlugin plugin : rpcMetaPlugins) {
         plugin.clientReceiveResponse(context);
@@ -225,7 +225,7 @@
     throws IOException;
 
   /** Reads an error message. */
-  public abstract AvroRemoteException readError(Schema schema, Decoder in)
+  public abstract Exception readError(Schema schema, Decoder in)
     throws IOException;
 }
 

Modified: hadoop/avro/trunk/lang/java/src/java/org/apache/avro/ipc/Responder.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/java/org/apache/avro/ipc/Responder.java?rev=918783&r1=918782&r2=918783&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/java/org/apache/avro/ipc/Responder.java (original)
+++ hadoop/avro/trunk/lang/java/src/java/org/apache/avro/ipc/Responder.java Wed Mar  3 23:48:50 2010
@@ -89,7 +89,7 @@
         new ByteBufferInputStream(buffers), null);
     ByteBufferOutputStream bbo = new ByteBufferOutputStream();
     Encoder out = new BinaryEncoder(bbo);
-    AvroRemoteException error = null;
+    Exception error = null;
     RPCContext context = new RPCContext();
     try {
       Protocol remote = handshake(in, out);
@@ -120,12 +120,8 @@
       try {
         response = respond(m, request);
         context.setResponse(response);
-      } catch (AvroRemoteException e) {
-        error = e;
-        context.setError(error);
       } catch (Exception e) {
-        LOG.warn("application error", e);
-        error = new AvroRemoteException(new Utf8(e.toString()));
+        error = e;
         context.setError(error);
       }
       
@@ -140,15 +136,14 @@
       else
         writeError(m.getErrors(), error, out);
 
-    } catch (AvroRuntimeException e) {            // system error
+    } catch (Exception e) {                       // system error
       LOG.warn("system error", e);
-      error = new AvroRemoteException(e);
-      context.setError(error);
+      context.setError(e);
       bbo = new ByteBufferOutputStream();
       out = new BinaryEncoder(bbo);
       META_WRITER.write(context.responseCallMeta(), out);
       out.writeBoolean(true);
-      writeError(Protocol.SYSTEM_ERRORS, error, out);
+      writeError(Protocol.SYSTEM_ERRORS, new Utf8(e.toString()), out);
     }
       
     return bbo.getBufferList();
@@ -194,7 +189,7 @@
 
   /** Computes the response for a message. */
   public abstract Object respond(Message message, Object request)
-    throws AvroRemoteException;
+    throws Exception;
 
   /** Reads a request message. */
   public abstract Object readRequest(Schema schema, Decoder in)
@@ -205,7 +200,7 @@
                                      Encoder out) throws IOException;
 
   /** Writes an error message. */
-  public abstract void writeError(Schema schema, AvroRemoteException error,
+  public abstract void writeError(Schema schema, Object error,
                                   Encoder out) throws IOException;
 
 }

Modified: hadoop/avro/trunk/lang/java/src/java/org/apache/avro/specific/SpecificRequestor.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/java/org/apache/avro/specific/SpecificRequestor.java?rev=918783&r1=918782&r2=918783&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/java/org/apache/avro/specific/SpecificRequestor.java (original)
+++ hadoop/avro/trunk/lang/java/src/java/org/apache/avro/specific/SpecificRequestor.java Wed Mar  3 23:48:50 2010
@@ -75,9 +75,12 @@
   }
 
   @Override
-  public AvroRemoteException readError(Schema schema, Decoder in)
+  public Exception readError(Schema schema, Decoder in)
     throws IOException {
-    return (AvroRemoteException)getDatumReader(schema).read(null, in);
+    Object value = getDatumReader(schema).read(null, in);
+    if (value instanceof Exception)
+      return (Exception)value;
+    return new AvroRemoteException(value);
   }
 
   /** Create a proxy instance whose methods invoke RPCs. */

Modified: hadoop/avro/trunk/lang/java/src/java/org/apache/avro/specific/SpecificResponder.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/java/org/apache/avro/specific/SpecificResponder.java?rev=918783&r1=918782&r2=918783&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/java/org/apache/avro/specific/SpecificResponder.java (original)
+++ hadoop/avro/trunk/lang/java/src/java/org/apache/avro/specific/SpecificResponder.java Wed Mar  3 23:48:50 2010
@@ -30,7 +30,6 @@
 import org.apache.avro.io.DatumWriter;
 import org.apache.avro.io.Decoder;
 import org.apache.avro.io.Encoder;
-import org.apache.avro.ipc.AvroRemoteException;
 import org.apache.avro.ipc.Responder;
 
 /** {@link org.apache.avro.ipc.Responder Responder} for generated interfaces.*/
@@ -77,14 +76,13 @@
   }
 
   @Override
-  public void writeError(Schema schema, AvroRemoteException error,
+  public void writeError(Schema schema, Object error,
                          Encoder out) throws IOException {
     getDatumWriter(schema).write(error, out);
   }
 
   @Override
-  public Object respond(Message message, Object request)
-    throws AvroRemoteException {
+  public Object respond(Message message, Object request) throws Exception {
     Class[] paramTypes = new Class[message.getRequest().getFields().size()];
     int i = 0;
     try {
@@ -93,10 +91,7 @@
       Method method = impl.getClass().getMethod(message.getName(), paramTypes);
       return method.invoke(impl, (Object[])request);
     } catch (InvocationTargetException e) {
-      Throwable target = e.getTargetException();
-      if (target instanceof AvroRemoteException)
-        throw (AvroRemoteException)target;
-      else throw new AvroRuntimeException(e);
+      throw (Exception)e.getTargetException();
     } catch (NoSuchMethodException e) {
       throw new AvroRuntimeException(e);
     } catch (IllegalAccessException e) {

Modified: hadoop/avro/trunk/lang/java/src/java/org/apache/avro/util/Utf8.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/java/org/apache/avro/util/Utf8.java?rev=918783&r1=918782&r2=918783&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/java/org/apache/avro/util/Utf8.java (original)
+++ hadoop/avro/trunk/lang/java/src/java/org/apache/avro/util/Utf8.java Wed Mar  3 23:48:50 2010
@@ -20,8 +20,10 @@
 import java.io.UnsupportedEncodingException;
 
 import org.apache.avro.io.BinaryData;
+import org.apache.avro.reflect.Stringable;
 
 /** A Utf8 string. */
+@Stringable
 public class Utf8 implements Comparable<Utf8> {
   private static final byte[] EMPTY = new byte[0];
 

Added: hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/SimpleError.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/SimpleError.java?rev=918783&view=auto
==============================================================================
--- hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/SimpleError.java (added)
+++ hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/SimpleError.java Wed Mar  3 23:48:50 2010
@@ -0,0 +1,22 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.avro;
+
+/** This should be a static nested class in TestProtocolReflect, but that
+ * breaks CheckStyle (http://jira.codehaus.org/browse/MPCHECKSTYLE-20). */
+public class SimpleError extends Exception {}

Modified: hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/TestProtocolGeneric.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/TestProtocolGeneric.java?rev=918783&r1=918782&r2=918783&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/TestProtocolGeneric.java (original)
+++ hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/TestProtocolGeneric.java Wed Mar  3 23:48:50 2010
@@ -24,7 +24,6 @@
 import org.apache.avro.generic.GenericRequestor;
 import org.apache.avro.generic.GenericResponder;
 import org.apache.avro.ipc.AvroRemoteException;
-import org.apache.avro.ipc.Requestor;
 import org.apache.avro.ipc.SocketServer;
 import org.apache.avro.ipc.SocketTransceiver;
 import org.apache.avro.ipc.Transceiver;
@@ -96,7 +95,7 @@
 
   protected static SocketServer server;
   protected static Transceiver client;
-  protected static Requestor requestor;
+  protected static GenericRequestor requestor;
 
   @Before
   public void testStartServer() throws Exception {
@@ -177,7 +176,7 @@
     Transceiver t
       = new SocketTransceiver(new InetSocketAddress(server.getPort()));
     try {
-      Requestor r = new GenericRequestor(protocol, t);
+      GenericRequestor r = new GenericRequestor(protocol, t);
       GenericRecord params = new GenericData.Record(message.getRequest());
       params.put("extra", Boolean.TRUE);
       params.put("greeting", new Utf8("bob"));

Modified: hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/TestProtocolReflect.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/TestProtocolReflect.java?rev=918783&r1=918782&r2=918783&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/TestProtocolReflect.java (original)
+++ hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/TestProtocolReflect.java Wed Mar  3 23:48:50 2010
@@ -29,6 +29,7 @@
 import org.junit.Test;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertNotNull;
 
 import java.net.InetSocketAddress;
 import java.util.Random;
@@ -49,6 +50,7 @@
     TestRecord echo(TestRecord record);
     int add(int arg1, int arg2);
     byte[] echoBytes(byte[] data);
+    void error() throws SimpleError;
   }
   
   public static class TestImpl implements Simple {
@@ -56,9 +58,9 @@
     public int add(int arg1, int arg2) { return arg1 + arg2; }
     public TestRecord echo(TestRecord record) { return record; }
     public byte[] echoBytes(byte[] data) { return data; }
+    public void error() throws SimpleError { throw new SimpleError(); }
   }
 
-
   protected static Server server;
   protected static Transceiver client;
   protected static Simple proxy;
@@ -101,11 +103,21 @@
     assertArrayEquals(data, echoed);
   }
 
+  @Test
+  public void testError() throws IOException {
+    SimpleError error = null;
+    try {
+      proxy.error();
+    } catch (SimpleError e) {
+      error = e;
+    }
+    assertNotNull(error);
+  }
+
   @After
   public void testStopServer() throws IOException {
     client.close();
     server.close();
   }
 
-
 }