You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2015/08/10 08:59:16 UTC

[14/39] incubator-ignite git commit: Added benchmarks for jdbc.

Added benchmarks for jdbc.


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/52c1dfa3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/52c1dfa3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/52c1dfa3

Branch: refs/heads/ignite-946
Commit: 52c1dfa32d86183c07c82bb3c34a03316d6fd7b5
Parents: b056a73
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Tue Aug 4 17:51:05 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Tue Aug 4 17:51:05 2015 +0300

----------------------------------------------------------------------
 .../yardstick/config/benchmark-query.properties |   3 +-
 modules/yardstick/config/ignite-base-config.xml |   2 -
 .../yardstick/IgniteBenchmarkArguments.java     |  11 ++
 .../cache/IgniteJdbcSqlQueryBenchmark.java      | 134 +++++++++++++++++++
 4 files changed, 147 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52c1dfa3/modules/yardstick/config/benchmark-query.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-query.properties b/modules/yardstick/config/benchmark-query.properties
index d6a8b9e..1a75926 100644
--- a/modules/yardstick/config/benchmark-query.properties
+++ b/modules/yardstick/config/benchmark-query.properties
@@ -63,5 +63,6 @@ CONFIGS="\
 -cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryJoinBenchmark -sn IgniteNode -ds sql-query-join-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryJoinOffHeapBenchmark -sn IgniteNode -ds sql-query-join-offheap-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryPutBenchmark -sn IgniteNode -ds sql-query-put-1-backup,\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryPutOffHeapBenchmark -sn IgniteNode -ds sql-query-put-offheap-1-backup\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryPutOffHeapBenchmark -sn IgniteNode -ds sql-query-put-offheap-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -jdbc jdbc:ignite://127.0.0.1/query -t 64 -sm PRIMARY_SYNC -dn IgniteJdbcSqlQueryBenchmark -sn IgniteNode -ds sql-query-jdbc-1-backup\
 "

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52c1dfa3/modules/yardstick/config/ignite-base-config.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/ignite-base-config.xml b/modules/yardstick/config/ignite-base-config.xml
index b2c976a..c77cc9a 100644
--- a/modules/yardstick/config/ignite-base-config.xml
+++ b/modules/yardstick/config/ignite-base-config.xml
@@ -180,8 +180,6 @@
             </list>
         </property>
 
-        <property name="connectorConfiguration"><null/></property>
-
         <property name="includeEventTypes">
             <list/>
         </property>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52c1dfa3/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
index 1562b26..5eb7060 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
@@ -106,6 +106,17 @@ public class IgniteBenchmarkArguments {
     @Parameter(names = {"-col", "--collocated"}, description = "Collocated")
     private boolean collocated;
 
+    /** */
+    @Parameter(names = {"-jdbc", "--jdbcUrl"}, description = "JDBC url")
+    private String jdbcUrl;
+
+    /**
+     * @return JDBC url.
+     */
+    public String jdbcUrl() {
+        return jdbcUrl;
+    }
+
     /**
      * @return Transaction concurrency.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/52c1dfa3/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteJdbcSqlQueryBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteJdbcSqlQueryBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteJdbcSqlQueryBenchmark.java
new file mode 100644
index 0000000..0ded2bd
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteJdbcSqlQueryBenchmark.java
@@ -0,0 +1,134 @@
+/*
+ * 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.ignite.yardstick.cache;
+
+import org.apache.ignite.*;
+import org.apache.ignite.yardstick.cache.model.Person;
+import org.yardstickframework.BenchmarkConfiguration;
+
+import javax.cache.Cache;
+import java.sql.*;
+import java.util.*;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ThreadLocalRandom;
+
+import static org.yardstickframework.BenchmarkUtils.println;
+
+/**
+ * Ignite benchmark that performs query operations.
+ */
+public class IgniteJdbcSqlQueryBenchmark extends IgniteCacheAbstractBenchmark {
+    /** Statements for closing. */
+    Set<PreparedStatement> stms = Collections.synchronizedSet(new HashSet<PreparedStatement>());
+
+    /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        println(cfg, "Populating query data...");
+
+        long start = System.nanoTime();
+
+        try (IgniteDataStreamer<Integer, Person> dataLdr = ignite().dataStreamer(cache.getName())) {
+            for (int i = 0; i < args.range() && !Thread.currentThread().isInterrupted(); i++) {
+                dataLdr.addData(i, new Person(i, "firstName" + i, "lastName" + i, i * 1000));
+
+                if (i % 100000 == 0)
+                    println(cfg, "Populated persons: " + i);
+            }
+        }
+
+        println(cfg, "Finished populating query data in " + ((System.nanoTime() - start) / 1_000_000) + " ms.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        PreparedStatement stm = (PreparedStatement)ctx.get(0);
+
+        if (stm == null) {
+            stm = createStatement();
+
+            stms.add(stm);
+
+            ctx.put(0, stm);
+        }
+
+        double salary = ThreadLocalRandom.current().nextDouble() * args.range() * 1000;
+
+        double maxSalary = salary + 1000;
+
+        stm.clearParameters();
+
+        stm.setDouble(1, salary);
+        stm.setDouble(2, maxSalary);
+
+        ResultSet rs = stm.executeQuery();
+
+        while (rs.next()) {
+            double sal = rs.getDouble("salary");
+
+            if (sal < salary || sal > maxSalary)
+                throw new Exception("Invalid person retrieved [min=" + salary + ", max=" + maxSalary + ']');
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void tearDown() throws Exception {
+        for (PreparedStatement stm : stms) {
+            try {
+                stm.getConnection().close();
+
+                stm.close();
+            }
+            catch (Exception ignore) {
+                println("Failed to close connection." + stm);
+            }
+        }
+
+        super.tearDown();
+    }
+
+    /**
+     * @return Prepared statement.
+     * @throws Exception
+     */
+    private PreparedStatement createStatement() throws Exception {
+        Class.forName("org.apache.ignite.IgniteJdbcDriver");
+
+        Connection conn = null;
+
+        try {
+            conn = DriverManager.getConnection(args.jdbcUrl());
+
+            return conn.prepareStatement("select * from Person where salary >= ? and salary <= ?");
+        }
+        catch (Exception e) {
+            if (conn != null)
+                conn.close();
+
+            throw new IgniteException("Failed to create prepare statement.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("query");
+    }
+}