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/02/04 21:10:26 UTC

svn commit: r906635 - in /hadoop/avro/trunk: ./ lang/java/ lang/java/src/java/org/apache/avro/tool/ lang/java/src/test/bin/ lang/java/src/test/java/org/apache/avro/tool/ share/test/interop/ share/test/interop/bin/ share/test/interop/rpc/ share/test/int...

Author: cutting
Date: Thu Feb  4 20:10:26 2010
New Revision: 906635

URL: http://svn.apache.org/viewvc?rev=906635&view=rev
Log:
AVRO-321.  Restore java RPC interop tests.

Added:
    hadoop/avro/trunk/lang/java/src/test/bin/gen_rpc_interop.sh   (with props)
    hadoop/avro/trunk/share/test/interop/
    hadoop/avro/trunk/share/test/interop/bin/
    hadoop/avro/trunk/share/test/interop/bin/test_rpc_interop.sh   (with props)
    hadoop/avro/trunk/share/test/interop/rpc/
    hadoop/avro/trunk/share/test/interop/rpc/add/
    hadoop/avro/trunk/share/test/interop/rpc/add/onePlusOne/
    hadoop/avro/trunk/share/test/interop/rpc/add/onePlusOne/request.avro   (with props)
    hadoop/avro/trunk/share/test/interop/rpc/add/onePlusOne/response.avro   (with props)
    hadoop/avro/trunk/share/test/interop/rpc/echo/
    hadoop/avro/trunk/share/test/interop/rpc/echo/foo/
    hadoop/avro/trunk/share/test/interop/rpc/echo/foo/request.avro   (with props)
    hadoop/avro/trunk/share/test/interop/rpc/echo/foo/response.avro   (with props)
    hadoop/avro/trunk/share/test/interop/rpc/hello/
    hadoop/avro/trunk/share/test/interop/rpc/hello/world/
    hadoop/avro/trunk/share/test/interop/rpc/hello/world/request.avro   (with props)
    hadoop/avro/trunk/share/test/interop/rpc/hello/world/response.avro   (with props)
Modified:
    hadoop/avro/trunk/CHANGES.txt
    hadoop/avro/trunk/build.sh
    hadoop/avro/trunk/lang/java/build.xml
    hadoop/avro/trunk/lang/java/src/java/org/apache/avro/tool/RpcReceiveTool.java
    hadoop/avro/trunk/lang/java/src/java/org/apache/avro/tool/RpcSendTool.java
    hadoop/avro/trunk/lang/java/src/java/org/apache/avro/tool/Util.java
    hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/tool/TestRpcReceiveAndSendTools.java

Modified: hadoop/avro/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/CHANGES.txt?rev=906635&r1=906634&r2=906635&view=diff
==============================================================================
--- hadoop/avro/trunk/CHANGES.txt (original)
+++ hadoop/avro/trunk/CHANGES.txt Thu Feb  4 20:10:26 2010
@@ -302,6 +302,8 @@
 
     AVRO-208. Clarify that enum symbols must be unique.  (cutting)
 
+    AVRO-321. Restore Java RPC interop tests. (cutting)
+
   OPTIMIZATIONS
 
     AVRO-172. More efficient schema processing (massie)

Modified: hadoop/avro/trunk/build.sh
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/build.sh?rev=906635&r1=906634&r2=906635&view=diff
==============================================================================
--- hadoop/avro/trunk/build.sh (original)
+++ hadoop/avro/trunk/build.sh Thu Feb  4 20:10:26 2010
@@ -58,6 +58,9 @@
 	#(cd lang/c; make interop-data-test)
 	#(cd lang/c++; make interop-data-test)
 
+	# run interop rpc tests
+	/bin/bash share/test/interop/bin/test_rpc_interop.sh
+
 	;;
 
     dist)

Modified: hadoop/avro/trunk/lang/java/build.xml
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/build.xml?rev=906635&r1=906634&r2=906635&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/build.xml (original)
+++ hadoop/avro/trunk/lang/java/build.xml Thu Feb  4 20:10:26 2010
@@ -339,6 +339,7 @@
              errorProperty="tests.failed" failureProperty="tests.failed">
         <sysproperty key="test.count" value="${test.count}"/>
         <sysproperty key="test.dir" value="@{test.dir}"/>
+        <sysproperty key="share.dir" value="${share.dir}"/>
         <sysproperty key="test.validate" value="${test.validate}"/>
         <sysproperty key="test.genavro.dir" value="${test.genavro.dir}" />
         <sysproperty key="test.genavro.mode" value="${test.genavro.mode}" />
@@ -365,7 +366,8 @@
     <test-runner files.location="${test.java.src.dir}" tests.pattern="**/${test.java.include}.java"/>
   </target>
 
-  <target name="interop-data-generate" depends="compile-test-java">
+  <target name="interop-data-generate" depends="compile-test-java"
+	  description="Generate java interop data files.">
     <mkdir dir="${top.build}/interop/data"/>
     <java classname="org.apache.avro.RandomData"
       classpathref="test.java.classpath">

Modified: hadoop/avro/trunk/lang/java/src/java/org/apache/avro/tool/RpcReceiveTool.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/java/org/apache/avro/tool/RpcReceiveTool.java?rev=906635&r1=906634&r2=906635&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/java/org/apache/avro/tool/RpcReceiveTool.java (original)
+++ hadoop/avro/trunk/lang/java/src/java/org/apache/avro/tool/RpcReceiveTool.java Thu Feb  4 20:10:26 2010
@@ -20,8 +20,14 @@
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.PrintStream;
+import java.io.File;
 import java.util.List;
 import java.util.concurrent.CountDownLatch;
+import java.net.URI;
+
+import joptsimple.OptionParser;
+import joptsimple.OptionSet;
+import joptsimple.OptionSpec;
 
 import org.apache.avro.Protocol;
 import org.apache.avro.Protocol.Message;
@@ -88,7 +94,14 @@
         throw new RuntimeException(e);
       }
       out.println();
-      latch.countDown();
+      new Thread() {
+        public void run() {
+          try {
+            Thread.sleep(1000);
+          } catch (InterruptedException e) {}
+          latch.countDown();
+        }
+      }.start();
       return response;
     }
   }
@@ -106,12 +119,26 @@
 
   int run1(InputStream in, PrintStream out, PrintStream err,
       List<String> args) throws Exception {
-    if (args.size() != 4) {
-      err.println("Expected four arguments: protocol port message_name json_response");
+    OptionParser p = new OptionParser();
+    OptionSpec<String> file =
+      p.accepts("file", "Data file containing response datum.")
+      .withRequiredArg()
+      .ofType(String.class);
+    OptionSpec<String> data =
+      p.accepts("data", "JSON-encoded response datum.")
+      .withRequiredArg()
+      .ofType(String.class);
+    OptionSet opts = p.parse(args.toArray(new String[0]));
+    args = opts.nonOptionArguments();
+
+    if (args.size() != 3) {
+      err.println("Usage: uri protocol_file message_name (-data d | -file f)");
+      p.printHelpOn(err);
       return 1;
     }
-    Protocol protocol = Protocol.parse(args.get(0));
-    int port = Integer.parseInt(args.get(1));
+
+    URI uri = new URI(args.get(0));
+    Protocol protocol = Protocol.parse(new File(args.get(1)));
     String messageName = args.get(2);
     expectedMessage = protocol.getMessages().get(messageName);
     if (expectedMessage == null) {
@@ -119,14 +146,23 @@
           messageName, protocol));
       return 1;
     }
-    String jsonData = args.get(3);
-    this.out = out;
+    if (data.value(opts) != null) {
+      this.response =
+        Util.jsonToGenericDatum(expectedMessage.getResponse(),
+                                data.value(opts));
+    } else if (file.value(opts) != null) {
+      this.response = Util.datumFromFile(expectedMessage.getResponse(),
+                                         file.value(opts));
+    } else {
+      err.println("One of -data or -file must be specified.");
+      return 1;
+    }
     
-    this.response = Util.jsonToGenericDatum(expectedMessage.getResponse(), jsonData);
+    this.out = out;
     
     latch = new CountDownLatch(1);
-    server = new HttpServer(new SinkResponder(protocol), port);
-    err.println("Listening on port " + server.getPort());
+    server = new HttpServer(new SinkResponder(protocol), uri.getPort());
+    out.println("Port: " + server.getPort());
     return 0;
   }
   

Modified: hadoop/avro/trunk/lang/java/src/java/org/apache/avro/tool/RpcSendTool.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/java/org/apache/avro/tool/RpcSendTool.java?rev=906635&r1=906634&r2=906635&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/java/org/apache/avro/tool/RpcSendTool.java (original)
+++ hadoop/avro/trunk/lang/java/src/java/org/apache/avro/tool/RpcSendTool.java Thu Feb  4 20:10:26 2010
@@ -20,9 +20,14 @@
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.PrintStream;
-import java.net.URL;
+import java.io.File;
+import java.net.URI;
 import java.util.List;
 
+import joptsimple.OptionParser;
+import joptsimple.OptionSet;
+import joptsimple.OptionSpec;
+
 import org.apache.avro.Protocol;
 import org.apache.avro.Schema;
 import org.apache.avro.Protocol.Message;
@@ -52,25 +57,45 @@
   @Override
   public int run(InputStream in, PrintStream out, PrintStream err,
       List<String> args) throws Exception {
-    if (args.size() != 5) {
-      err.println(
-          "Expected 5 arguments: protocol message_name host port json_data");
+    OptionParser p = new OptionParser();
+    OptionSpec<String> file =
+      p.accepts("file", "Data file containing request parameters.")
+      .withRequiredArg()
+      .ofType(String.class);
+    OptionSpec<String> data =
+      p.accepts("data", "JSON-encoded request parameters.")
+      .withRequiredArg()
+      .ofType(String.class);
+    OptionSet opts = p.parse(args.toArray(new String[0]));
+    args = opts.nonOptionArguments();
+
+    if (args.size() != 3) {
+      err.println("Usage: uri protocol_file message_name (-data d | -file f)");
+      p.printHelpOn(err);
       return 1;
     }
-    Protocol protocol = Protocol.parse(args.get(0));
-    String messageName = args.get(1);
+
+    URI uri = new URI(args.get(0));
+    Protocol protocol = Protocol.parse(new File(args.get(1)));
+    String messageName = args.get(2);
     Message message = protocol.getMessages().get(messageName);
     if (message == null) {
       err.println(String.format("No message named '%s' found in protocol '%s'.",
           messageName, protocol));
       return 1;
     }
-    String host = args.get(2);
-    int port = Integer.parseInt(args.get(3));
-    String jsonData = args.get(4);
     
-    Object datum = Util.jsonToGenericDatum(message.getRequest(), jsonData);
-    GenericRequestor client = makeClient(protocol, host, port);
+    Object datum;
+    if (data.value(opts) != null) {
+      datum = Util.jsonToGenericDatum(message.getRequest(), data.value(opts));
+    } else if (file.value(opts) != null) {
+      datum = Util.datumFromFile(message.getRequest(), file.value(opts));
+    } else {
+      err.println("One of -data or -file must be specified.");
+      return 1;
+    }
+
+    GenericRequestor client = makeClient(protocol, uri);
     Object response = client.request(message.getName(), datum);
     dumpJson(out, message.getResponse(), response);
     return 0;
@@ -88,10 +113,10 @@
     out.flush();
   }
 
-  private GenericRequestor makeClient(Protocol protocol, String host, int port) 
+  private GenericRequestor makeClient(Protocol protocol, URI uri) 
   throws IOException {
     HttpTransceiver transceiver = 
-      new HttpTransceiver(new URL("http", host, port, "/"));
+      new HttpTransceiver(uri.toURL());
     GenericRequestor requestor = new GenericRequestor(protocol, transceiver);
     return requestor;
   }

Modified: hadoop/avro/trunk/lang/java/src/java/org/apache/avro/tool/Util.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/java/org/apache/avro/tool/Util.java?rev=906635&r1=906634&r2=906635&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/java/org/apache/avro/tool/Util.java (original)
+++ hadoop/avro/trunk/lang/java/src/java/org/apache/avro/tool/Util.java Thu Feb  4 20:10:26 2010
@@ -26,6 +26,7 @@
 import org.apache.avro.Schema;
 import org.apache.avro.generic.GenericDatumReader;
 import org.apache.avro.io.JsonDecoder;
+import org.apache.avro.file.DataFileReader;
 
 /** Static utility methods for tools. */
 class Util {
@@ -54,4 +55,17 @@
     Object datum = reader.read(null, new JsonDecoder(schema, jsonData));
     return datum;
   }
+
+  /** Reads and returns the first datum in a data file. */
+  static Object datumFromFile(Schema schema, String file) throws IOException {
+    DataFileReader<Object> in =
+      new DataFileReader<Object>(new File(file),
+                                 new GenericDatumReader<Object>(schema));
+    try {
+      return in.next();
+    } finally {
+      in.close();
+    }
+  }
+
 }

Added: hadoop/avro/trunk/lang/java/src/test/bin/gen_rpc_interop.sh
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/test/bin/gen_rpc_interop.sh?rev=906635&view=auto
==============================================================================
--- hadoop/avro/trunk/lang/java/src/test/bin/gen_rpc_interop.sh (added)
+++ hadoop/avro/trunk/lang/java/src/test/bin/gen_rpc_interop.sh Thu Feb  4 20:10:26 2010
@@ -0,0 +1,76 @@
+#!/bin/bash
+
+# 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.
+
+set -e						  # exit on error
+
+cd `dirname "$0"`/../../..			  # connect to lang/java 
+
+VERSION=`cat ../../share/VERSION.txt`
+
+set -x						  # echo commands
+
+function write_request() {
+    messageName="$1"
+    callName="$2"
+    params="$3"
+    data="$4"
+    
+    outdir=../../share/test/interop/rpc/$messageName/$callName
+    mkdir -p $outdir
+
+    outfile=$outdir/request.avro
+
+    schema='{"type":"record","name":"'$messageName'","fields":'$params'}'
+
+    echo -n "$data" | \
+	java -jar build/avro-tools-$VERSION.jar fromjson "$schema" - > $outfile
+}
+
+function write_response() {
+    messageName="$1"
+    callName="$2"
+    schema="$3"
+    data="$4"
+    
+    outdir=../../share/test/interop/rpc/$messageName/$callName
+    mkdir -p $outdir
+
+    outfile=$outdir/response.avro
+
+    echo -n "$data" | \
+	java -jar build/avro-tools-$VERSION.jar fromjson "$schema" - > $outfile
+}
+
+write_request hello world \
+    '[{"name": "greeting", "type": "string"}]' \
+    '{"greeting": "Hello World!"}'
+
+write_response hello world '"string"' '"Hello World"'
+
+write_request echo foo \
+    '[{"name": "record", "type": {"name": "org.apache.avro.test.TestRecord", "type": "record", "fields": [ {"name": "name", "type": "string", "order": "ignore"}, {"name": "kind", "type": {"name": "Kind", "type": "enum", "symbols": ["FOO","BAR","BAZ"]}, "order": "descending"}, {"name": "hash", "type": {"name": "MD5", "type": "fixed", "size": 16}} ] }}]' \
+    '{"record": {"name": "Foo", "kind": "FOO", "hash": "0123456789012345"}}'
+
+write_response echo foo \
+    '{"name": "org.apache.avro.test.TestRecord", "type": "record", "fields": [ {"name": "name", "type": "string", "order": "ignore"}, {"name": "kind", "type": {"name": "Kind", "type": "enum", "symbols": ["FOO","BAR","BAZ"]}, "order": "descending"}, {"name": "hash", "type": {"name": "MD5", "type": "fixed", "size": 16}} ]}' \
+    '{"name": "Foo", "kind": "FOO", "hash": "0123456789012345"}'
+
+write_request add onePlusOne \
+    '[{"name": "arg1", "type": "int"}, {"name": "arg2", "type": "int"}]' \
+    '{"arg1": 1, "arg2": 1}'
+
+write_response add onePlusOne '"int"' 2

Propchange: hadoop/avro/trunk/lang/java/src/test/bin/gen_rpc_interop.sh
------------------------------------------------------------------------------
    svn:executable = *

Modified: hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/tool/TestRpcReceiveAndSendTools.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/tool/TestRpcReceiveAndSendTools.java?rev=906635&r1=906634&r2=906635&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/tool/TestRpcReceiveAndSendTools.java (original)
+++ hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/tool/TestRpcReceiveAndSendTools.java Thu Feb  4 20:10:26 2010
@@ -18,12 +18,12 @@
 package org.apache.avro.tool;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.io.ByteArrayOutputStream;
 import java.io.PrintStream;
 import java.util.Arrays;
 
-import org.apache.avro.Protocol;
 import org.junit.Test;
 
 public class TestRpcReceiveAndSendTools {
@@ -33,27 +33,26 @@
    */
   @Test
   public void testServeAndSend() throws Exception {
-    Protocol protocol = Protocol.parse("" +
-                "{\"protocol\": \"Minimal\", " +
-                "\"messages\": { \"sink\": {" +
-                "   \"request\": [{\"name\": \"a\", \"type\": \"string\"}], " +
-                "   \"response\": \"string\"} } }");
+    String protocolFile =
+      System.getProperty("share.dir") + "/test/schemas/simple.avpr";
     ByteArrayOutputStream baos1 = new ByteArrayOutputStream();
     PrintStream p1 = new PrintStream(baos1);
     RpcReceiveTool receive = new RpcReceiveTool();
     receive.run1(null, p1, System.err, 
-        Arrays.asList(protocol.toString(), "0", "sink", "\"omega\""));
-    int port = receive.server.getPort();
+                 Arrays.asList("http://0.0.0.0:0/",
+                               protocolFile, "hello",
+                               "-data", "\"Hello!\""));
     ByteArrayOutputStream baos2 = new ByteArrayOutputStream();
     PrintStream p2 = new PrintStream(baos2);
     RpcSendTool send = new RpcSendTool();
     send.run(null, p2, System.err,
-        Arrays.asList(protocol.toString(), "sink", "localhost", 
-            Integer.toString(port), "{ \"a\": \"alpha\" }"));
+             Arrays.asList("http://127.0.0.1:"+receive.server.getPort()+"/",
+                           protocolFile, "hello",  
+                           "-data", "{ \"greeting\": \"Hi!\" }"));
     receive.run2(System.err);
     
-    assertEquals("sink\t{\"a\":\"alpha\"}\n",
-        baos1.toString("UTF-8").replace("\r", ""));
-    assertEquals("\"omega\"\n", baos2.toString("UTF-8").replace("\r", ""));
+    assertTrue(baos1.toString("UTF-8").replace("\r", "")
+               .endsWith("hello\t{\"greeting\":\"Hi!\"}\n"));
+    assertEquals("\"Hello!\"\n", baos2.toString("UTF-8").replace("\r", ""));
   }
 }

Added: hadoop/avro/trunk/share/test/interop/bin/test_rpc_interop.sh
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/share/test/interop/bin/test_rpc_interop.sh?rev=906635&view=auto
==============================================================================
--- hadoop/avro/trunk/share/test/interop/bin/test_rpc_interop.sh (added)
+++ hadoop/avro/trunk/share/test/interop/bin/test_rpc_interop.sh Thu Feb  4 20:10:26 2010
@@ -0,0 +1,64 @@
+#!/bin/bash
+
+# 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.
+
+set -e						  # exit on error
+
+cd `dirname "$0"`/../../../..			  # connect to root
+
+VERSION=`cat share/VERSION.txt`
+
+#set -x						  # echo commands
+
+java_client="java -jar lang/java/build/avro-tools-$VERSION.jar rpcsend"
+java_server="java -jar lang/java/build/avro-tools-$VERSION.jar rpcreceive"
+
+clients=("$java_client")
+servers=("$java_server")
+
+proto=share/test/schemas/simple.avpr
+
+portfile=/tmp/interop_$$
+
+function cleanup() {
+    rm -rf $portfile
+    for job in `jobs -p` ; do kill $job; done
+}
+
+trap 'cleanup' EXIT
+
+for server in "${servers[@]}"
+do
+    for msgDir in share/test/interop/rpc/*
+    do
+	msg=`basename "$msgDir"`
+	for c in ${msgDir}/*
+	do
+	    echo TEST: $c
+	    for client in "${clients[@]}"
+	    do
+		$server http://0.0.0.0:0/ $proto $msg -file $c/response.avro \
+		    > $portfile &
+		sleep 1				  # wait for server to start
+		read ignore port < $portfile
+	    	$client http://127.0.0.1:$port $proto $msg -file $c/request.avro
+		wait
+	    done
+	done
+    done
+done
+
+echo RPC INTEROP TESTS PASS

Propchange: hadoop/avro/trunk/share/test/interop/bin/test_rpc_interop.sh
------------------------------------------------------------------------------
    svn:executable = *

Added: hadoop/avro/trunk/share/test/interop/rpc/add/onePlusOne/request.avro
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/share/test/interop/rpc/add/onePlusOne/request.avro?rev=906635&view=auto
==============================================================================
Binary file - no diff available.

Propchange: hadoop/avro/trunk/share/test/interop/rpc/add/onePlusOne/request.avro
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: hadoop/avro/trunk/share/test/interop/rpc/add/onePlusOne/response.avro
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/share/test/interop/rpc/add/onePlusOne/response.avro?rev=906635&view=auto
==============================================================================
Binary file - no diff available.

Propchange: hadoop/avro/trunk/share/test/interop/rpc/add/onePlusOne/response.avro
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: hadoop/avro/trunk/share/test/interop/rpc/echo/foo/request.avro
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/share/test/interop/rpc/echo/foo/request.avro?rev=906635&view=auto
==============================================================================
Binary file - no diff available.

Propchange: hadoop/avro/trunk/share/test/interop/rpc/echo/foo/request.avro
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: hadoop/avro/trunk/share/test/interop/rpc/echo/foo/response.avro
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/share/test/interop/rpc/echo/foo/response.avro?rev=906635&view=auto
==============================================================================
Binary file - no diff available.

Propchange: hadoop/avro/trunk/share/test/interop/rpc/echo/foo/response.avro
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: hadoop/avro/trunk/share/test/interop/rpc/hello/world/request.avro
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/share/test/interop/rpc/hello/world/request.avro?rev=906635&view=auto
==============================================================================
Binary file - no diff available.

Propchange: hadoop/avro/trunk/share/test/interop/rpc/hello/world/request.avro
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: hadoop/avro/trunk/share/test/interop/rpc/hello/world/response.avro
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/share/test/interop/rpc/hello/world/response.avro?rev=906635&view=auto
==============================================================================
Binary file - no diff available.

Propchange: hadoop/avro/trunk/share/test/interop/rpc/hello/world/response.avro
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream