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

svn commit: r1444584 - in /avro/trunk: ./ lang/java/tools/src/main/java/org/apache/avro/tool/ lang/java/tools/src/test/java/org/apache/avro/tool/

Author: philz
Date: Sun Feb 10 19:01:48 2013
New Revision: 1444584

URL: http://svn.apache.org/r1444584
Log:
AVRO-1248. Avro Tool to dump protocol of a remote RPC Service

Contributed by Gareth Davis

Added:
    avro/trunk/lang/java/tools/src/main/java/org/apache/avro/tool/RpcProtocolTool.java
    avro/trunk/lang/java/tools/src/test/java/org/apache/avro/tool/TestRpcProtocolTool.java
Modified:
    avro/trunk/CHANGES.txt
    avro/trunk/lang/java/tools/src/main/java/org/apache/avro/tool/Main.java

Modified: avro/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/avro/trunk/CHANGES.txt?rev=1444584&r1=1444583&r2=1444584&view=diff
==============================================================================
--- avro/trunk/CHANGES.txt (original)
+++ avro/trunk/CHANGES.txt Sun Feb 10 19:01:48 2013
@@ -4,6 +4,9 @@ Trunk (not yet released)
 
   NEW FEATURES
 
+    AVRO-1248. Avro Tool to dump protocol of a remote RPC Service
+    (Gareth Davis via philz)
+
     AVRO-1229. Add support for booleans to Trevni. (cutting)
 
   IMPROVEMENTS

Modified: avro/trunk/lang/java/tools/src/main/java/org/apache/avro/tool/Main.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/tools/src/main/java/org/apache/avro/tool/Main.java?rev=1444584&r1=1444583&r2=1444584&view=diff
==============================================================================
--- avro/trunk/lang/java/tools/src/main/java/org/apache/avro/tool/Main.java (original)
+++ avro/trunk/lang/java/tools/src/main/java/org/apache/avro/tool/Main.java Sun Feb 10 19:01:48 2013
@@ -47,6 +47,7 @@ public class Main {
         new RecodecTool(),
         new RpcReceiveTool(),
         new RpcSendTool(),
+        new RpcProtocolTool(),
         new FromTextTool(),
         new ToTextTool(),
         new TetherTool(),

Added: avro/trunk/lang/java/tools/src/main/java/org/apache/avro/tool/RpcProtocolTool.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/tools/src/main/java/org/apache/avro/tool/RpcProtocolTool.java?rev=1444584&view=auto
==============================================================================
--- avro/trunk/lang/java/tools/src/main/java/org/apache/avro/tool/RpcProtocolTool.java (added)
+++ avro/trunk/lang/java/tools/src/main/java/org/apache/avro/tool/RpcProtocolTool.java Sun Feb 10 19:01:48 2013
@@ -0,0 +1,111 @@
+/**
+ * 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.tool;
+
+import org.apache.avro.Protocol;
+import org.apache.avro.io.BinaryEncoder;
+import org.apache.avro.io.DatumReader;
+import org.apache.avro.io.DatumWriter;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.avro.io.EncoderFactory;
+import org.apache.avro.ipc.HandshakeRequest;
+import org.apache.avro.ipc.HandshakeResponse;
+import org.apache.avro.ipc.Ipc;
+import org.apache.avro.ipc.MD5;
+import org.apache.avro.ipc.Transceiver;
+import org.apache.avro.specific.SpecificDatumReader;
+import org.apache.avro.specific.SpecificDatumWriter;
+import org.apache.avro.util.ByteBufferInputStream;
+import org.apache.avro.util.ByteBufferOutputStream;
+
+import java.io.InputStream;
+import java.io.PrintStream;
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.LinkedHashMap;
+import java.util.List;
+
+/**
+ * Tool to grab the protocol from a remote running service.
+ */
+public class RpcProtocolTool implements Tool {
+
+  @Override
+  public String getName() {
+    return "rpcprotocol";
+  }
+
+  @Override
+  public String getShortDescription() {
+    return "Output the protocol of a RPC service";
+  }
+
+  @Override
+  public int run(InputStream in, PrintStream out, PrintStream err, List<String> args) throws Exception {
+
+    if (args.size() != 1) {
+      err.println("Usage: uri");
+      return 1;
+    }
+
+    URI uri = URI.create(args.get(0));
+
+    Transceiver transceiver = null;
+    try {
+      transceiver = Ipc.createTransceiver(uri);
+
+      // write an empty HandshakeRequest
+      HandshakeRequest rq = HandshakeRequest.newBuilder()
+          .setClientHash(new MD5(new byte[16]))
+          .setServerHash(new MD5(new byte[16]))
+          .setClientProtocol(null)
+          .setMeta(new LinkedHashMap<String, ByteBuffer>())
+          .build();
+
+      DatumWriter<HandshakeRequest> handshakeWriter = new SpecificDatumWriter<HandshakeRequest>(HandshakeRequest.class);
+
+      ByteBufferOutputStream byteBufferOutputStream = new ByteBufferOutputStream();
+
+      BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(byteBufferOutputStream, null);
+
+      handshakeWriter.write(rq, encoder);
+      encoder.flush();
+
+      // send it and get the response
+      List<ByteBuffer> response = transceiver.transceive(byteBufferOutputStream.getBufferList());
+
+
+      // parse the response
+      ByteBufferInputStream byteBufferInputStream = new ByteBufferInputStream(response);
+
+      DatumReader<HandshakeResponse> handshakeReader = new SpecificDatumReader<HandshakeResponse>(HandshakeResponse.class);
+
+      HandshakeResponse handshakeResponse = handshakeReader.read(null, DecoderFactory.get().binaryDecoder(byteBufferInputStream, null));
+
+      Protocol p = Protocol.parse(handshakeResponse.getServerProtocol());
+
+      // finally output the protocol
+      out.println(p.toString(true));
+
+    } finally {
+      if( transceiver != null )
+        transceiver.close();
+    }
+    return 0;
+  }
+}

Added: avro/trunk/lang/java/tools/src/test/java/org/apache/avro/tool/TestRpcProtocolTool.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/tools/src/test/java/org/apache/avro/tool/TestRpcProtocolTool.java?rev=1444584&view=auto
==============================================================================
--- avro/trunk/lang/java/tools/src/test/java/org/apache/avro/tool/TestRpcProtocolTool.java (added)
+++ avro/trunk/lang/java/tools/src/test/java/org/apache/avro/tool/TestRpcProtocolTool.java Sun Feb 10 19:01:48 2013
@@ -0,0 +1,96 @@
+/**
+ * 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.tool;
+
+import org.apache.avro.Protocol;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.PrintStream;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ *
+ */
+@RunWith(Parameterized.class)
+public class TestRpcProtocolTool {
+
+  @Parameterized.Parameters(/*name = "{0}"*/)
+  public static List<Object[]> data() {
+    return Arrays.asList( new Object[]{"http"},
+                          new Object[]{"avro"});
+  }
+
+  private RpcReceiveTool receive;
+  private Protocol simpleProtocol;
+
+  private String uriScheme ;
+
+  public TestRpcProtocolTool(String uriScheme) {
+    this.uriScheme = uriScheme;
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    String protocolFile =
+      System.getProperty("share.dir", "../../../share") + "/test/schemas/simple.avpr";
+
+    simpleProtocol = Protocol.parse(new File(protocolFile));
+
+    // start a simple server
+    ByteArrayOutputStream baos1 = new ByteArrayOutputStream();
+    PrintStream p1 = new PrintStream(baos1);
+    receive = new RpcReceiveTool();
+    receive.run1(null, p1, System.err,
+        Arrays.asList(uriScheme + "://0.0.0.0:0/",
+            protocolFile, "hello",
+            "-data", "\"Hello!\""));
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if( receive != null )
+      receive.server.close(); // force the server to finish
+  }
+
+  @Test
+  public void testRpcProtocol() throws Exception {
+
+    // run the actual test
+    ByteArrayOutputStream baos2 = new ByteArrayOutputStream();
+    PrintStream p2 = new PrintStream(baos2, true, "UTF-8");
+    RpcProtocolTool testObject = new RpcProtocolTool();
+
+    testObject.run(null, p2, System.err,
+        Arrays.asList(uriScheme + "://127.0.0.1:" + receive.server.getPort() + "/"));
+
+    p2.flush();
+
+    assertEquals("Expected the simple.avpr protocol to be echoed to standout",
+        simpleProtocol, Protocol.parse(baos2.toString("UTF-8")));
+
+  }
+}