Return-Path: X-Original-To: apmail-accumulo-commits-archive@www.apache.org Delivered-To: apmail-accumulo-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 368F610AB4 for ; Mon, 24 Feb 2014 17:33:03 +0000 (UTC) Received: (qmail 62362 invoked by uid 500); 24 Feb 2014 17:32:52 -0000 Delivered-To: apmail-accumulo-commits-archive@accumulo.apache.org Received: (qmail 62068 invoked by uid 500); 24 Feb 2014 17:32:48 -0000 Mailing-List: contact commits-help@accumulo.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@accumulo.apache.org Delivered-To: mailing list commits@accumulo.apache.org Received: (qmail 62034 invoked by uid 99); 24 Feb 2014 17:32:47 -0000 Received: from tyr.zones.apache.org (HELO tyr.zones.apache.org) (140.211.11.114) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 24 Feb 2014 17:32:47 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id 2F374903783; Mon, 24 Feb 2014 17:32:47 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: mdrob@apache.org To: commits@accumulo.apache.org Date: Mon, 24 Feb 2014 17:32:51 -0000 Message-Id: <836f179fffda41b8b5667f79f38e0d47@git.apache.org> In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [05/10] git commit: Merge branch '1.4.5-SNAPSHOT' into 1.5.1-SNAPSHOT Merge branch '1.4.5-SNAPSHOT' into 1.5.1-SNAPSHOT Conflicts: src/trace/src/main/java/org/apache/accumulo/cloudtrace/instrument/TraceProxy.java src/trace/src/test/java/org/apache/accumulo/cloudtrace/instrument/TracerTest.java Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/41ce56a5 Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/41ce56a5 Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/41ce56a5 Branch: refs/heads/1.6.0-SNAPSHOT Commit: 41ce56a51ac2f18506e996a60fba44a464f65529 Parents: a38b004 2829426 Author: Mike Drob Authored: Mon Feb 24 12:26:48 2014 -0500 Committer: Mike Drob Committed: Mon Feb 24 12:26:48 2014 -0500 ---------------------------------------------------------------------- .../accumulo/trace/instrument/TraceProxy.java | 37 ++++++++++++------- .../accumulo/trace/instrument/TracerTest.java | 38 ++++++++++++++++++++ 2 files changed, 63 insertions(+), 12 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/accumulo/blob/41ce56a5/trace/src/main/java/org/apache/accumulo/trace/instrument/TraceProxy.java ---------------------------------------------------------------------- diff --cc trace/src/main/java/org/apache/accumulo/trace/instrument/TraceProxy.java index a7eba1c,0000000..cb93210 mode 100644,000000..100644 --- a/trace/src/main/java/org/apache/accumulo/trace/instrument/TraceProxy.java +++ b/trace/src/main/java/org/apache/accumulo/trace/instrument/TraceProxy.java @@@ -1,59 -1,0 +1,72 @@@ +/* + * 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.accumulo.trace.instrument; + +import java.lang.reflect.InvocationHandler; ++import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.lang.reflect.Proxy; + ++import org.apache.log4j.Logger; ++ +public class TraceProxy { - // private static final Logger log = Logger.getLogger(TraceProxy.class); - ++ private static final Logger log = Logger.getLogger(TraceProxy.class); ++ + static final Sampler ALWAYS = new Sampler() { + @Override + public boolean next() { + return true; + } + }; - ++ + public static T trace(T instance) { + return trace(instance, ALWAYS); + } - ++ + @SuppressWarnings("unchecked") + public static T trace(final T instance, final Sampler sampler) { + InvocationHandler handler = new InvocationHandler() { + @Override + public Object invoke(Object obj, Method method, Object[] args) throws Throwable { - if (!sampler.next()) { - return method.invoke(instance, args); ++ Span span = null; ++ if (sampler.next()) { ++ span = Trace.on(method.getName()); + } - Span span = Trace.on(method.getName()); + try { + return method.invoke(instance, args); - } catch (Throwable ex) { - ex.printStackTrace(); - throw ex; ++ // Can throw RuntimeException, Error, or any checked exceptions of the method. ++ } catch (InvocationTargetException ite) { ++ Throwable cause = ite.getCause(); ++ if (cause == null) { ++ // This should never happen, but account for it anyway ++ log.error("Invocation exception during trace with null cause: ", ite); ++ throw new RuntimeException(ite); ++ } ++ throw cause; ++ } catch (IllegalAccessException e) { ++ throw new RuntimeException(e); + } finally { - span.stop(); ++ if (span != null) { ++ span.stop(); ++ } + } + } + }; + return (T) Proxy.newProxyInstance(instance.getClass().getClassLoader(), instance.getClass().getInterfaces(), handler); + } - ++ +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/41ce56a5/trace/src/test/java/org/apache/accumulo/trace/instrument/TracerTest.java ---------------------------------------------------------------------- diff --cc trace/src/test/java/org/apache/accumulo/trace/instrument/TracerTest.java index 7dbdc83,0000000..f338bd8 mode 100644,000000..100644 --- a/trace/src/test/java/org/apache/accumulo/trace/instrument/TracerTest.java +++ b/trace/src/test/java/org/apache/accumulo/trace/instrument/TracerTest.java @@@ -1,181 -1,0 +1,219 @@@ +/* + * 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.accumulo.trace.instrument; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + ++import java.io.IOException; +import java.net.ServerSocket; +import java.net.Socket; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; ++import java.util.concurrent.Callable; + +import org.apache.accumulo.trace.instrument.receivers.SpanReceiver; +import org.apache.accumulo.trace.instrument.thrift.TraceWrap; +import org.apache.accumulo.trace.thrift.TInfo; +import org.apache.accumulo.trace.thrift.TestService; +import org.apache.accumulo.trace.thrift.TestService.Iface; +import org.apache.accumulo.trace.thrift.TestService.Processor; +import org.apache.thrift.TException; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.server.TServer; +import org.apache.thrift.server.TThreadPoolServer; +import org.apache.thrift.transport.TServerSocket; +import org.apache.thrift.transport.TSocket; +import org.apache.thrift.transport.TTransport; ++import org.junit.Before; +import org.junit.Test; + + +public class TracerTest { + static class SpanStruct { + public SpanStruct(long traceId, long spanId, long parentId, long start, long stop, String description, Map data) { + super(); + this.traceId = traceId; + this.spanId = spanId; + this.parentId = parentId; + this.start = start; + this.stop = stop; + this.description = description; + this.data = data; + } + + public long traceId; + public long spanId; + public long parentId; + public long start; + public long stop; + public String description; + public Map data; + + public long millis() { + return stop - start; + } + } + + static class TestReceiver implements SpanReceiver { + public Map> traces = new HashMap>(); + + @Override + public void span(long traceId, long spanId, long parentId, long start, long stop, String description, Map data) { + SpanStruct span = new SpanStruct(traceId, spanId, parentId, start, stop, description, data); + if (!traces.containsKey(traceId)) + traces.put(traceId, new ArrayList()); + traces.get(traceId).add(span); + } + + @Override + public void flush() {} + } + + @Test + public void testTrace() throws Exception { + TestReceiver tracer = new TestReceiver(); + Tracer.getInstance().addReceiver(tracer); + + assertFalse(Trace.isTracing()); + Trace.start("nop").stop(); + assertTrue(tracer.traces.size() == 0); + assertFalse(Trace.isTracing()); + + Trace.on("nop").stop(); + assertTrue(tracer.traces.size() == 1); + assertFalse(Trace.isTracing()); + + Span start = Trace.on("testing"); + assertEquals(Trace.currentTrace(), start); + assertTrue(Trace.isTracing()); + + Trace.start("shortest trace ever"); + Trace.currentTrace().stop(); + long traceId = Trace.currentTrace().traceId(); + assertNotNull(tracer.traces.get(traceId)); + assertTrue(tracer.traces.get(traceId).size() == 1); + assertEquals("shortest trace ever", tracer.traces.get(traceId).get(0).description); + + Span pause = Trace.start("pause"); + Thread.sleep(100); + pause.stop(); + assertTrue(tracer.traces.get(traceId).size() == 2); + assertTrue(tracer.traces.get(traceId).get(1).millis() >= 100); + + Thread t = new Thread(Trace.wrap(new Runnable() { + @Override + public void run() { + assertTrue(Trace.isTracing()); + } + }), "My Task"); + t.start(); + t.join(); + + assertTrue(tracer.traces.get(traceId).size() == 3); + assertEquals("My Task", tracer.traces.get(traceId).get(2).description); + Trace.off(); + assertFalse(Trace.isTracing()); + } + + static class Service implements TestService.Iface { + @Override + public boolean checkTrace(TInfo t, String message) throws TException { + Span trace = Trace.start(message); + try { + return Trace.isTracing(); + } finally { + trace.stop(); + } + } + } + + @Test + public void testThrift() throws Exception { + TestReceiver tracer = new TestReceiver(); + Tracer.getInstance().addReceiver(tracer); + + ServerSocket socket = new ServerSocket(0); + TServerSocket transport = new TServerSocket(socket); + transport.listen(); + TThreadPoolServer.Args args = new TThreadPoolServer.Args(transport); + args.processor(new Processor(TraceWrap.service(new Service()))); + final TServer tserver = new TThreadPoolServer(args); + Thread t = new Thread() { + public void run() { + tserver.serve(); + } + }; + t.start(); + TTransport clientTransport = new TSocket(new Socket("localhost", socket.getLocalPort())); + TestService.Iface client = new TestService.Client(new TBinaryProtocol(clientTransport), new TBinaryProtocol(clientTransport)); + client = TraceWrap.client(client); + assertFalse(client.checkTrace(null, "test")); + + Span start = Trace.on("start"); + assertTrue(client.checkTrace(null, "my test")); + start.stop(); + + assertNotNull(tracer.traces.get(start.traceId())); + String traces[] = {"my test", "checkTrace", "client:checkTrace", "start"}; + assertTrue(tracer.traces.get(start.traceId()).size() == traces.length); + for (int i = 0; i < traces.length; i++) + assertEquals(traces[i], tracer.traces.get(start.traceId()).get(i).description); + + tserver.stop(); + t.join(100); + } ++ ++ Callable callable; ++ ++ @Before ++ public void setup() { ++ callable = new Callable() { ++ @Override ++ public Object call() throws IOException { ++ throw new IOException(); ++ } ++ }; ++ } ++ ++ /** ++ * Verify that exceptions propagate up through the trace wrapping with sampling enabled, instead of seeing the reflexive exceptions. ++ */ ++ @Test(expected = IOException.class) ++ public void testTracedException() throws Exception { ++ TraceProxy.trace(callable).call(); ++ } ++ ++ /** ++ * Verify that exceptions propagate up through the trace wrapping with sampling disabled, instead of seeing the reflexive exceptions. ++ */ ++ @Test(expected = IOException.class) ++ public void testUntracedException() throws Exception { ++ Sampler never = new Sampler() { ++ @Override ++ public boolean next() { ++ return false; ++ } ++ }; ++ ++ TraceProxy.trace(callable, never).call(); ++ } +}