You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by jc...@apache.org on 2016/12/02 04:01:56 UTC

aurora git commit: Changes to intercept and time mybatis invocations

Repository: aurora
Updated Branches:
  refs/heads/master 16e4651d5 -> 3ea0331d4


Changes to intercept and time mybatis invocations

MyBatis allows us to intercept calls within the execution of a mapped statement. This allows us to
time various mapped statements and ultimately gain more insight on the performance of the database
module.

This patch introduces an interceptor on MyBatis on `updates` and `query` mapped statements. I used
the following convention to create name for the newly collected stats:
mybatis.<<the id of the mapped statement>>

After interception the process is very similar to the one in @Timed-interceptor. SlidingStats can be
used to export interval averages, total milliseconds and the event counts.

__example stats (from ./vars.json)__
mybatis.create_tables_events 1
mybatis.create_tables_events_per_sec 0.0
mybatis.create_tables_nanos_per_event 0.0
mybatis.create_tables_nanos_total 592633784
mybatis.create_tables_nanos_total_per_sec 0.0
mybatis.org.apache.aurora.scheduler.storage.db.AttributeMapper.selectAll_events 3
mybatis.org.apache.aurora.scheduler.storage.db.AttributeMapper.selectAll_events_per_sec 0.0
mybatis.org.apache.aurora.scheduler.storage.db.AttributeMapper.selectAll_nanos_per_event 0.0
mybatis.org.apache.aurora.scheduler.storage.db.AttributeMapper.selectAll_nanos_total 2858362
mybatis.org.apache.aurora.scheduler.storage.db.AttributeMapper.selectAll_nanos_total_per_sec 0.0
mybatis.org.apache.aurora.scheduler.storage.db.AttributeMapper.select_events 333
mybatis.org.apache.aurora.scheduler.storage.db.AttributeMapper.select_events_per_sec 0.0
mybatis.org.apache.aurora.scheduler.storage.db.AttributeMapper.select_nanos_per_event 0.0
mybatis.org.apache.aurora.scheduler.storage.db.AttributeMapper.select_nanos_total 85745680
mybatis.org.apache.aurora.scheduler.storage.db.AttributeMapper.select_nanos_total_per_sec 0.0

Reviewed at https://reviews.apache.org/r/54107/


Project: http://git-wip-us.apache.org/repos/asf/aurora/repo
Commit: http://git-wip-us.apache.org/repos/asf/aurora/commit/3ea0331d
Tree: http://git-wip-us.apache.org/repos/asf/aurora/tree/3ea0331d
Diff: http://git-wip-us.apache.org/repos/asf/aurora/diff/3ea0331d

Branch: refs/heads/master
Commit: 3ea0331d4cccef7d885e76ea6795e64183fe1b8d
Parents: 16e4651
Author: Reza Motamedi <re...@gmail.com>
Authored: Thu Dec 1 22:01:35 2016 -0600
Committer: Joshua Cohen <jc...@apache.org>
Committed: Thu Dec 1 22:01:35 2016 -0600

----------------------------------------------------------------------
 build.gradle                                    |   2 +
 .../aurora/scheduler/storage/db/DbModule.java   |   8 +
 .../storage/db/InstrumentingInterceptor.java    | 139 ++++++++++++++++
 .../db/InstrumentingInterceptorTest.java        | 162 +++++++++++++++++++
 .../storage/mem/InMemTaskStoreTest.java         |  12 +-
 .../storage/mem/MemCronJobStoreTest.java        |  10 ++
 6 files changed, 332 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/3ea0331d/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index 2f23b85..7064bca 100644
--- a/build.gradle
+++ b/build.gradle
@@ -388,6 +388,8 @@ dependencies {
 
   testCompile "com.sun.jersey:jersey-client:${jerseyRev}"
   testCompile "junit:junit:${junitRev}"
+  testCompile "org.powermock:powermock-module-junit4:1.6.4"
+  testCompile "org.powermock:powermock-api-easymock:1.6.4"
 }
 
 // For normal developer builds, avoid running the often-time-consuming code quality checks.

http://git-wip-us.apache.org/repos/asf/aurora/blob/3ea0331d/src/main/java/org/apache/aurora/scheduler/storage/db/DbModule.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/DbModule.java b/src/main/java/org/apache/aurora/scheduler/storage/db/DbModule.java
index e7287ce..d1a1964 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/db/DbModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/DbModule.java
@@ -73,6 +73,10 @@ public final class DbModule extends PrivateModule {
       help = "Whether to use the experimental database-backed task store.")
   public static final Arg<Boolean> USE_DB_TASK_STORE = Arg.create(false);
 
+  @CmdLine(name = "enable_db_metrics",
+      help = "Whether to use MyBatis interceptor to measure the timing of intercepted Statements.")
+  private static final Arg<Boolean> ENABLE_DB_METRICS = Arg.create(true);
+
   @CmdLine(name = "slow_query_log_threshold",
       help = "Log all queries that take at least this long to execute.")
   private static final Arg<Amount<Long, Time>> SLOW_QUERY_LOG_THRESHOLD =
@@ -247,6 +251,10 @@ public final class DbModule extends PrivateModule {
     install(new MyBatisModule() {
       @Override
       protected void initialize() {
+        if (ENABLE_DB_METRICS.get()) {
+          addInterceptorClass(InstrumentingInterceptor.class);
+        }
+
         bindProperties(binder(), ImmutableMap.of("JDBC.schema", jdbcSchema));
         install(JdbcHelper.H2_IN_MEMORY_NAMED);
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/3ea0331d/src/main/java/org/apache/aurora/scheduler/storage/db/InstrumentingInterceptor.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/InstrumentingInterceptor.java b/src/main/java/org/apache/aurora/scheduler/storage/db/InstrumentingInterceptor.java
new file mode 100644
index 0000000..caf1d64
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/InstrumentingInterceptor.java
@@ -0,0 +1,139 @@
+/**
+ * Licensed 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.aurora.scheduler.storage.db;
+
+import java.util.Properties;
+import java.util.function.Function;
+import javax.annotation.Nonnull;
+import javax.inject.Inject;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+
+import org.apache.aurora.common.stats.SlidingStats;
+import org.apache.aurora.common.util.Clock;
+import org.apache.ibatis.cache.CacheKey;
+import org.apache.ibatis.executor.Executor;
+import org.apache.ibatis.mapping.BoundSql;
+import org.apache.ibatis.mapping.MappedStatement;
+import org.apache.ibatis.plugin.Interceptor;
+import org.apache.ibatis.plugin.Intercepts;
+import org.apache.ibatis.plugin.Invocation;
+import org.apache.ibatis.plugin.Plugin;
+import org.apache.ibatis.plugin.Signature;
+import org.apache.ibatis.session.ResultHandler;
+import org.apache.ibatis.session.RowBounds;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * A Mybatis Executor invocation interceptor that exports timing information for update and query
+ * mapped statements.
+ *
+ * Currently intercepting the following invocations:
+ * 1. update(MappedStatement ms, Object parameter)
+ * 2. query(MappedStatement ms, Object parameter, RowBounds rowBounds,
+ *      ResultHandler resultHandler, CacheKey cacheKey, BoundSql boundSql)
+ * 3. query(MappedStatement ms, Object parameter, RowBounds rowBounds,
+ *      ResultHandler resultHandler)
+ * 4. queryCursor(MappedStatement ms, Object parameter, RowBounds rowBounds)
+ *
+ * more signatures can be added from: org.apache.ibatis.executors
+ */
+@Intercepts({
+    @Signature(
+        type = Executor.class,
+        method = "update",
+        args = {MappedStatement.class, Object.class}),
+    @Signature(type = Executor.class,
+        method = "query",
+        args = {MappedStatement.class, Object.class, RowBounds.class, ResultHandler.class,
+            CacheKey.class, BoundSql.class}),
+    @Signature(type = Executor.class,
+        method = "query",
+        args = {MappedStatement.class, Object.class, RowBounds.class, ResultHandler.class}),
+    @Signature(type = Executor.class,
+        method = "queryCursor",
+        args = {MappedStatement.class, Object.class, RowBounds.class})
+    })
+public class InstrumentingInterceptor implements Interceptor {
+  private static final String INVALID_INVOCATION_METRIC_NAME = "invalid_invocations";
+  private static final String STATS_NAME_PREFIX = "mybatis.";
+  private static final Logger LOG = LoggerFactory.getLogger(InstrumentingInterceptor.class);
+  private final Clock clock;
+  private final LoadingCache<String, SlidingStats> stats;
+
+  @Inject
+  public InstrumentingInterceptor(Clock clock) {
+    this(clock, (String name) -> new SlidingStats(name, "nanos"));
+  }
+
+  @VisibleForTesting
+  public InstrumentingInterceptor(Clock clock, Function<String, SlidingStats> statsFactory) {
+    this.clock = requireNonNull(clock);
+
+    this.stats = CacheBuilder.newBuilder().build(new CacheLoader<String, SlidingStats>() {
+      @Override public SlidingStats load(String statsName) {
+        return statsFactory.apply(STATS_NAME_PREFIX + statsName);
+      }
+    });
+  }
+
+  private String generateStatsName(Invocation invocation) {
+    if (firstArgumentIsMappedStatement(invocation)) {
+      MappedStatement statement = (MappedStatement) invocation.getArgs()[0];
+      return statement.getId();
+    }
+
+    LOG.warn("Received invocation for unknown or invalid target. Invocation target: {}. "
+            + "Invocation method: {}. Using metric name '{}' instead.",
+        invocation.getTarget(),
+        invocation.getMethod(),
+        INVALID_INVOCATION_METRIC_NAME);
+    return INVALID_INVOCATION_METRIC_NAME;
+  }
+
+  private boolean firstArgumentIsMappedStatement(Invocation invocation) {
+    return invocation != null
+        && invocation.getArgs() != null
+        && invocation.getArgs()[0] instanceof MappedStatement;
+  }
+
+  @Override
+  public Object intercept(@Nonnull Invocation invocation) throws Throwable {
+    long start = clock.nowNanos();
+    try {
+      return invocation.proceed();
+    } finally {
+      String statsName = generateStatsName(invocation);
+      SlidingStats stat = stats.get(statsName);
+      stat.accumulate(clock.nowNanos() - start);
+    }
+  }
+
+  @Override
+  public Object plugin(Object target) {
+    return Plugin.wrap(target, this);
+  }
+
+  @Override
+  public void setProperties(Properties properties) {
+    // intentionally left empty as instructed in http://www.mybatis.org/mybatis-3/configuration.html
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/3ea0331d/src/test/java/org/apache/aurora/scheduler/storage/db/InstrumentingInterceptorTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/storage/db/InstrumentingInterceptorTest.java b/src/test/java/org/apache/aurora/scheduler/storage/db/InstrumentingInterceptorTest.java
new file mode 100644
index 0000000..c7d437f
--- /dev/null
+++ b/src/test/java/org/apache/aurora/scheduler/storage/db/InstrumentingInterceptorTest.java
@@ -0,0 +1,162 @@
+/**
+ * Licensed 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.aurora.scheduler.storage.db;
+
+import java.lang.reflect.Method;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.aurora.common.stats.SlidingStats;
+import org.apache.aurora.common.stats.Stats;
+import org.apache.aurora.common.testing.easymock.EasyMockTest;
+import org.apache.aurora.common.util.Clock;
+import org.apache.ibatis.mapping.MappedStatement;
+import org.apache.ibatis.plugin.Invocation;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.api.easymock.PowerMock;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.expectLastCall;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+import static junit.framework.TestCase.assertNotNull;
+
+@RunWith(PowerMockRunner.class)
+@PrepareForTest({MappedStatement.class, Method.class})
+public class InstrumentingInterceptorTest extends EasyMockTest {
+  private InstrumentingInterceptor interceptor;
+  private Invocation invocation;
+  private SlidingStats slidingStats;
+
+  private Clock mockClock;
+
+  @Before
+  public void setUp() throws Exception {
+    invocation = createMock(Invocation.class);
+    mockClock = createMock(Clock.class);
+    slidingStats = createMock(SlidingStats.class);
+    interceptor = new InstrumentingInterceptor(mockClock, name -> slidingStats);
+  }
+
+  private void expectGetArgs(Object[] args, int times) {
+    expect(invocation.getArgs()).andReturn(args).times(times);
+  }
+
+  @Test
+  public void testStatIsCreatedOnce() throws Throwable {
+    final AtomicLong factoryCallCount = new AtomicLong(0);
+    interceptor = new InstrumentingInterceptor(mockClock, name -> {
+      factoryCallCount.incrementAndGet();
+      return slidingStats;
+    });
+
+    String statName = "test";
+    MappedStatement fakeMappedStatement = createMock(MappedStatement.class);
+    Object[] args = new Object[] {fakeMappedStatement};
+
+    expect(mockClock.nowNanos()).andReturn(0L).andReturn(1000L);
+
+    expectGetArgs(args, 3);
+
+    expect(fakeMappedStatement.getId()).andReturn(statName);
+    expect(invocation.proceed()).andReturn("result");
+
+    slidingStats.accumulate(1000);
+    expectLastCall();
+
+    expect(mockClock.nowNanos()).andReturn(0L).andReturn(1000L);
+
+    expectGetArgs(args, 3);
+
+    expect(fakeMappedStatement.getId()).andReturn(statName);
+    expect(invocation.proceed()).andReturn("result");
+
+    slidingStats.accumulate(1000);
+    expectLastCall();
+
+    control.replay();
+
+    // Perform the test
+    interceptor.intercept(invocation);
+
+    assertEquals(1L, factoryCallCount.get());
+    interceptor.intercept(invocation);
+    assertEquals(1L, factoryCallCount.get());
+  }
+
+  @Test
+  public void testInterceptMarksMetrics() throws Throwable {
+    MappedStatement fakeMappedStatement = createMock(MappedStatement.class);
+    Object[] args = new Object[] {fakeMappedStatement};
+
+    expect(mockClock.nowNanos()).andReturn(0L).andReturn(1000L);
+
+    expectGetArgs(args, 3);
+
+    expect(fakeMappedStatement.getId()).andReturn("test");
+    expect(invocation.proceed()).andReturn("result");
+
+    slidingStats.accumulate(1000);
+    expectLastCall();
+
+    control.replay();
+
+    // Perform the test
+    Object res = interceptor.intercept(invocation);
+    assertEquals("result", res);
+  }
+
+  @Test
+  public void testInterceptNotAMappedStatement() throws Throwable {
+    interceptor = new InstrumentingInterceptor(mockClock);
+    Method mockMethod = PowerMock.createMock(Method.class);
+
+    Object notAMappedStatement = new Object();
+    Object[] args = new Object[] {notAMappedStatement};
+
+    expect(mockClock.nowNanos()).andReturn(0L).andReturn(1000L);
+
+    expectGetArgs(args, 2);
+
+    expect(invocation.getMethod()).andReturn(mockMethod);
+    expect(invocation.getTarget()).andReturn("test");
+    expect(invocation.proceed()).andReturn(null);
+
+    control.replay();
+
+    assertNull(Stats.getVariable("mybatis.invalid_invocations_nanos_total"));
+    assertNull(Stats.getVariable("mybatis.invalid_invocations_nanos_total_per_sec"));
+    assertNull(Stats.getVariable("mybatis.invalid_invocations_events"));
+    assertNull(Stats.getVariable("mybatis.invalid_invocations_nanos_per_event"));
+    assertNull(Stats.getVariable("mybatis.invalid_invocations_events_per_sec"));
+
+    interceptor.intercept(invocation);
+
+    // upon interception of invocation that does not have a valid MappedStatement use
+    // invalid_invocations as the name
+    assertNotNull(Stats.getVariable("mybatis.invalid_invocations_nanos_total"));
+    assertNotNull(Stats.getVariable("mybatis.invalid_invocations_nanos_total_per_sec"));
+    assertNotNull(Stats.getVariable("mybatis.invalid_invocations_events"));
+    assertNotNull(Stats.getVariable("mybatis.invalid_invocations_nanos_per_event"));
+    assertNotNull(Stats.getVariable("mybatis.invalid_invocations_events_per_sec"));
+
+    assertEquals(1000L, Stats.getVariable("mybatis.invalid_invocations_nanos_total").read());
+    assertEquals(1L, Stats.getVariable("mybatis.invalid_invocations_events").read());
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/3ea0331d/src/test/java/org/apache/aurora/scheduler/storage/mem/InMemTaskStoreTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/storage/mem/InMemTaskStoreTest.java b/src/test/java/org/apache/aurora/scheduler/storage/mem/InMemTaskStoreTest.java
index 2e560c0..02719c3 100644
--- a/src/test/java/org/apache/aurora/scheduler/storage/mem/InMemTaskStoreTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/storage/mem/InMemTaskStoreTest.java
@@ -19,16 +19,20 @@ import com.google.inject.AbstractModule;
 import com.google.inject.Module;
 import com.google.inject.util.Modules;
 
+import org.apache.aurora.common.stats.SlidingStats;
 import org.apache.aurora.common.stats.StatsProvider;
+import org.apache.aurora.common.util.Clock;
 import org.apache.aurora.scheduler.base.Tasks;
 import org.apache.aurora.scheduler.storage.AbstractTaskStoreTest;
 import org.apache.aurora.scheduler.storage.Storage.MutateWork.NoResult;
 import org.apache.aurora.scheduler.storage.TaskStore;
 import org.apache.aurora.scheduler.storage.db.DbModule;
+import org.apache.aurora.scheduler.storage.db.InstrumentingInterceptor;
 import org.apache.aurora.scheduler.testing.FakeStatsProvider;
 import org.junit.Test;
 
 import static org.apache.aurora.common.inject.Bindings.KeyFactory.PLAIN;
+import static org.easymock.EasyMock.createMock;
 import static org.junit.Assert.assertEquals;
 
 public class InMemTaskStoreTest extends AbstractTaskStoreTest {
@@ -44,14 +48,20 @@ public class InMemTaskStoreTest extends AbstractTaskStoreTest {
           @Override
           protected void configure() {
             bind(StatsProvider.class).toInstance(statsProvider);
+
+            // bindings for mybatis interceptor
+            SlidingStats slidingStats = createMock(SlidingStats.class);
+            bind(InstrumentingInterceptor.class).toInstance(new InstrumentingInterceptor(
+                Clock.SYSTEM_CLOCK, s -> slidingStats
+            ));
           }
         });
   }
 
   @Test
   public void testSecondaryIndexConsistency() {
-    // Test for regression of AURORA-1305.
     storage.write((NoResult.Quiet) storeProvider -> {
+    // Test for regression of AURORA-1305.
       TaskStore.Mutable taskStore = storeProvider.getUnsafeTaskStore();
       taskStore.saveTasks(ImmutableSet.of(TASK_A));
       taskStore.deleteTasks(Tasks.ids(TASK_A));

http://git-wip-us.apache.org/repos/asf/aurora/blob/3ea0331d/src/test/java/org/apache/aurora/scheduler/storage/mem/MemCronJobStoreTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/storage/mem/MemCronJobStoreTest.java b/src/test/java/org/apache/aurora/scheduler/storage/mem/MemCronJobStoreTest.java
index 79999e1..82e0329 100644
--- a/src/test/java/org/apache/aurora/scheduler/storage/mem/MemCronJobStoreTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/storage/mem/MemCronJobStoreTest.java
@@ -18,12 +18,16 @@ import com.google.inject.AbstractModule;
 import com.google.inject.Module;
 import com.google.inject.util.Modules;
 
+import org.apache.aurora.common.stats.SlidingStats;
 import org.apache.aurora.common.stats.StatsProvider;
+import org.apache.aurora.common.util.Clock;
 import org.apache.aurora.scheduler.storage.AbstractCronJobStoreTest;
 import org.apache.aurora.scheduler.storage.db.DbModule;
+import org.apache.aurora.scheduler.storage.db.InstrumentingInterceptor;
 import org.apache.aurora.scheduler.testing.FakeStatsProvider;
 
 import static org.apache.aurora.common.inject.Bindings.KeyFactory.PLAIN;
+import static org.easymock.EasyMock.createMock;
 
 public class MemCronJobStoreTest extends AbstractCronJobStoreTest {
   @Override
@@ -34,6 +38,12 @@ public class MemCronJobStoreTest extends AbstractCronJobStoreTest {
           @Override
           protected void configure() {
             bind(StatsProvider.class).toInstance(new FakeStatsProvider());
+
+            // bindings for mybatis interceptor
+            SlidingStats slidingStats = createMock(SlidingStats.class);
+            bind(InstrumentingInterceptor.class).toInstance(new InstrumentingInterceptor(
+                Clock.SYSTEM_CLOCK, s -> slidingStats
+            ));
           }
         });
   }