You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2018/10/25 11:40:56 UTC

ignite git commit: IGNITE-9637: Yardstick: upload benchmarks. This closes #4796.

Repository: ignite
Updated Branches:
  refs/heads/master a18f71ce5 -> 67d4941f3


IGNITE-9637: Yardstick: upload benchmarks. This closes #4796.


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

Branch: refs/heads/master
Commit: 67d4941f349e2a4af9ae5d04bc12d5f56b94a952
Parents: a18f71c
Author: Pavel Kuznetsov <pa...@gmail.com>
Authored: Thu Oct 25 14:40:49 2018 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Oct 25 14:40:49 2018 +0300

----------------------------------------------------------------------
 .../upload/benchmark-upload-regular.properties  | 109 +++++++++++++++++++
 .../upload/AbstractNativeBenchmark.java         |  79 ++++++--------
 .../upload/AbstractUploadBenchmark.java         |  30 ++---
 .../yardstick/upload/NativePutAllBenchmark.java |  50 +++++++++
 .../yardstick/upload/NativePutBenchmark.java    |   9 +-
 .../upload/NativeSqlInsertBenchmark.java        |  44 ++++++++
 .../upload/NativeStreamerBenchmark.java         |   6 +-
 .../upload/UploadBenchmarkArguments.java        |  28 +++--
 .../yardstick/upload/model/QueryFactory.java    |  39 +++++--
 .../ignite/yardstick/upload/model/Values10.java |  22 +++-
 10 files changed, 329 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/67d4941f/modules/yardstick/config/upload/benchmark-upload-regular.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/upload/benchmark-upload-regular.properties b/modules/yardstick/config/upload/benchmark-upload-regular.properties
new file mode 100644
index 0000000..113be9f
--- /dev/null
+++ b/modules/yardstick/config/upload/benchmark-upload-regular.properties
@@ -0,0 +1,109 @@
+#
+# 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.
+#
+
+#
+# Benchmarks for data upload in inmemory mode (persistence disabled).
+#
+
+now0=`date +'%H%M%S'`
+
+# JVM options.
+JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
+
+# Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
+JVM_OPTS=${JVM_OPTS}" \
+-Xms1g \
+-Xmx8g \
+-Xloggc:./gc${now0}.log \
+-XX:+PrintGCDetails \
+-verbose:gc \
+-XX:+UseParNewGC \
+-XX:+UseConcMarkSweepGC \
+-XX:+PrintGCDateStamps \
+"
+
+#Ignite version
+ver="RELEASE-"
+
+# List of default probes.
+# Add DStatProbe or VmStatProbe if your OS supports it (e.g. if running on Linux).
+BENCHMARK_DEFAULT_PROBES=TotalTimeProbe
+
+# Packages where the specified benchmark is searched by reflection mechanism.
+BENCHMARK_PACKAGES=org.yardstickframework,org.apache.ignite.yardstick
+
+# Flag which indicates to restart the servers before every benchmark execution.
+RESTART_SERVERS=true
+
+# Probe point writer class name.
+# BENCHMARK_WRITER=
+
+# The benchmark is applicable only for 2 servers (the second server is started in client mode) and 1 driver.
+SERVER_HOSTS=localhost,localhost
+DRIVER_HOSTS=localhost
+
+# Remote username.
+# REMOTE_USER=
+
+# Number of nodes, used to wait for the specified number of nodes to start.
+nodesNum=$((`echo ${SERVER_HOSTS} | tr ',' '\n' | wc -l` + `echo ${DRIVER_HOSTS} | tr ',' '\n' | wc -l`))
+
+# Backups count.
+b=1
+
+# Warmup.
+w=0
+
+# Threads count.
+t=1
+
+# Sync mode.
+sm=FULL_SYNC
+
+# Parameters that should be the same across all the benchmarks launches.
+commonParams="\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b ${b} \
+  --warmup ${w} --operations 1 \
+  --threads ${t} --syncMode ${sm} -sn IgniteNode \
+  --upload-rows 1000000 -cl \
+  --clientNodesAfterId 0 \
+  --atomic-mode ATOMIC \
+  --idx-count 2 \
+"
+
+thinUrl="-jdbc jdbc:ignite:thin://auto.find/"
+
+# Note: Don't forget to fill thick's config with cluster addresses too.
+v2Url="-jdbc jdbc:ignite:cfg://cache=query@config/ignite-jdbc-config.xml"
+        
+
+# Run configuration which contains all benchmarks.
+# Note that each benchmark is set to run only one time, warmup parameter is set to 0 due to custom warmup operation.
+CONFIGS="\
+${commonParams} -dn NativePutAllBenchmark -ds ${ver}upload-native-putAll --upload-jdbc-batch-size 1000, \
+${commonParams} -dn NativePutBenchmark -ds ${ver}upload-native-put, \
+${commonParams} -dn NativeStreamerBenchmark -ds ${ver}upload-native-streamer --streamer-local-batch-size 1000, \
+${commonParams} -dn NativeSqlInsertBenchmark -ds ${ver}upload-native-sql-insert, \
+  \
+${commonParams} -dn CopyBenchmark ${thinUrl} -ds ${ver}upload-jdbc-thin-copy, \
+${commonParams} -dn InsertBenchmark ${thinUrl} -ds ${ver}upload-jdbc-thin-streaming --use-streaming true --streamer-ordered false --streamer-local-batch-size 1024, \
+  \
+${commonParams} -dn InsertBenchmark ${thinUrl} -ds ${ver}upload-jdbc-thin-insert, \
+${commonParams} -dn InsertBenchmark ${v2Url} -ds ${ver}upload-jdbc-v2-insert, \
+${commonParams} -dn BatchedInsertBenchmark ${thinUrl} -ds ${ver}upload-jdbc-thin-batched-insert --upload-jdbc-batch-size 1000, \
+${commonParams} -dn BatchedInsertBenchmark ${v2Url} -ds ${ver}upload-jdbc-v2-batched-insert --upload-jdbc-batch-size 1000 \
+"

http://git-wip-us.apache.org/repos/asf/ignite/blob/67d4941f/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/AbstractNativeBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/AbstractNativeBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/AbstractNativeBenchmark.java
index 01eb6d4..6f8d7cc 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/AbstractNativeBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/AbstractNativeBenchmark.java
@@ -17,12 +17,12 @@
 
 package org.apache.ignite.yardstick.upload;
 
+import java.util.List;
 import java.util.Map;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.yardstick.IgniteAbstractBenchmark;
-import org.apache.ignite.yardstick.upload.model.Values10;
+import org.apache.ignite.yardstick.upload.model.QueryFactory;
 import org.yardstickframework.BenchmarkConfiguration;
 import org.yardstickframework.BenchmarkUtils;
 
@@ -33,11 +33,11 @@ public abstract class AbstractNativeBenchmark extends IgniteAbstractBenchmark {
     /** Number of entries to be uploaded during warmup. */
     private long insertRowsCnt;
 
-    /** Name of the {@link #cache} */
-    private String cacheName;
+    /** Name of the cache for test table. */
+    protected static final String CACHE_NAME = "SQL_PUBLIC_" + QueryFactory.UPLOAD_TABLE_NAME;
 
-    /** Cache method {@link test(Map)} uploads data to */
-    private IgniteCache<Long, Values10> cache;
+    /** Facade for creating sql queries. */
+    protected QueryFactory queries;
 
     /**
      * Sets up benchmark: performs warmup on one cache and creates another for {@link #test(Map)} method.
@@ -48,77 +48,70 @@ public abstract class AbstractNativeBenchmark extends IgniteAbstractBenchmark {
     @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
         super.setUp(cfg);
 
-        cacheName = this.getClass().getSimpleName();
+        queries = new QueryFactory(args.atomicMode());
 
         insertRowsCnt = args.upload.uploadRowsCnt();
 
+        dropAndCreateTable();
+
         // Number of entries to be uploaded during test().
         long warmupRowsCnt = args.upload.warmupRowsCnt();
 
-        // warmup
-        BenchmarkUtils.println(cfg, "Starting custom warmup.");
-        String warmupCacheName = cacheName + "Warmup";
+        // warmup.
+        BenchmarkUtils.println(cfg, "Starting custom warmup. Uploading " + warmupRowsCnt + " rows.");
 
-        try (IgniteCache<Long, Values10> warmupCache = createCache(warmupCacheName)) {
-            upload(warmupCacheName, warmupRowsCnt);
-        }
-        finally {
-            ignite().destroyCache(warmupCacheName);
-        }
+        upload(warmupRowsCnt);
 
         BenchmarkUtils.println(cfg, "Custom warmup finished.");
 
-        // cache for benchmarked action
-        cache = createCache(cacheName);
+        dropAndCreateTable();
     }
 
-    private IgniteCache<Long, Values10> createCache(String name) {
-        CacheConfiguration<Long, Values10> cfg = new CacheConfiguration<>(name);
+    /**
+     * Drops test table if exists and creates empty new one.
+     */
+    private void dropAndCreateTable() {
+        executeNativeSql(QueryFactory.DROP_TABLE_IF_EXISTS);
 
-        if (args.atomicMode() != null)
-            cfg.setAtomicityMode(args.atomicMode());
+        executeNativeSql(queries.createTable());
 
-        return ignite().createCache(cfg);
+        for (int idx = 1; idx <= args.upload.indexesCount(); idx++)
+            executeNativeSql(queries.createIndex(idx));
     }
 
     /** {@inheritDoc} */
     @Override public void tearDown() throws Exception {
         try {
-            if (cache == null)
-                throw new IllegalStateException("Cache is null, probably an error during setUp or warmup");
-
-            long size = cache.sizeLong();
+            long size = ignite().cache(CACHE_NAME).sizeLong();
+            //long size = (Long)executeNativeSql(QueryFactory.COUNT).get(0).get(0);
 
             if (size != insertRowsCnt) {
-                String msg = "Incorrect cache size: [actual=" + size + ", expected=" + insertRowsCnt +"].";
+                String msg = "Incorrect cache size: [actual=" + size + ", expected=" + insertRowsCnt + "].";
 
                 BenchmarkUtils.println(cfg, "TearDown: " + msg);
 
                 throw new RuntimeException(msg);
             }
-
-            cache.close();
-
-            ignite().destroyCache(cacheName);
-
-        }
-        catch (IgniteException ex) {
-            BenchmarkUtils.println(cfg, "Could not close or destroy cache: " + ex);
-
-            throw ex;
         }
         finally {
             super.tearDown();
         }
     }
 
+    /**
+     * Executes query using native sql.
+     */
+    private List<List<?>> executeNativeSql(String qry) {
+        return ((IgniteEx)ignite()).context().query().querySqlFields(new SqlFieldsQuery(qry), false).getAll();
+    }
+
     /** {@inheritDoc} */
     @Override public boolean test(Map<Object, Object> ctx) throws Exception {
-        upload(cacheName, insertRowsCnt);
+        upload(insertRowsCnt);
 
         return true;
     }
 
-    /** Uploads {@param insertsCnt} to cache with name {@param cacheName} using java api. */
-    protected abstract void upload(String cacheName, long insertsCnt);
+    /** Uploads {@param insertsCnt} to test cache/table using java api. */
+    protected abstract void upload(long insertsCnt);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/67d4941f/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/AbstractUploadBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/AbstractUploadBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/AbstractUploadBenchmark.java
index 6d9c84d..7270a2d 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/AbstractUploadBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/AbstractUploadBenchmark.java
@@ -29,9 +29,7 @@ import org.yardstickframework.BenchmarkConfiguration;
 import org.yardstickframework.BenchmarkUtils;
 
 /**
- * Base class for upload benchmarks.
- * Designed to run test method one single time.
- * Introduces custom warmup operation.
+ * Base class for upload benchmarks. Designed to run test method one single time. Introduces custom warmup operation.
  */
 public abstract class AbstractUploadBenchmark extends AbstractJdbcBenchmark {
     /** Total inserts size. */
@@ -84,9 +82,8 @@ public abstract class AbstractUploadBenchmark extends AbstractJdbcBenchmark {
     }
 
     /**
-     * Method to warm up Benchmark server. <br/>
-     * In upload benchmarks we need warmup action
-     * and real test action to be separated.
+     * Method to warm up Benchmark server. <br/> In upload benchmarks we need warmup action and real test action to be
+     * separated.
      */
     protected abstract void warmup(Connection warmupConn) throws Exception;
 
@@ -100,8 +97,8 @@ public abstract class AbstractUploadBenchmark extends AbstractJdbcBenchmark {
     }
 
     /**
-     * Uploads data using this special connection, that may have additional
-     * url parameters, such as {@code streaming=true}.
+     * Uploads data using this special connection, that may have additional url parameters, such as {@code
+     * streaming=true}.
      */
     protected abstract void upload(Connection uploadConn) throws Exception;
 
@@ -110,7 +107,6 @@ public abstract class AbstractUploadBenchmark extends AbstractJdbcBenchmark {
         if (args.upload.disableWal())
             executeUpdate(QueryFactory.TURN_OFF_WAL);
 
-
         try (Connection uploadConn = uploadConnection()) {
             if (args.upload.useStreaming())
                 executeUpdateOn(uploadConn, queries.turnOnStreaming(args.upload));
@@ -121,7 +117,6 @@ public abstract class AbstractUploadBenchmark extends AbstractJdbcBenchmark {
                 executeUpdateOn(uploadConn, QueryFactory.TURN_OFF_STREAMING);
         }
 
-
         if (args.upload.disableWal())
             executeUpdate(QueryFactory.TURN_ON_WAL);
 
@@ -137,13 +132,20 @@ public abstract class AbstractUploadBenchmark extends AbstractJdbcBenchmark {
         BenchmarkUtils.println(cfg, "Creating table with schema: " + queries.createTable());
 
         executeUpdate(queries.createTable());
+
+        int idxCnt = args.upload.indexesCount();
+
+        BenchmarkUtils.println("Creating " + idxCnt + " indexes.");
+
+        for (int i = 1; i <= idxCnt; i++)
+            executeUpdate(queries.createIndex(i));
     }
 
     /**
      * Retrieves records count in the test table.
      */
     public long count() throws SQLException {
-        try(PreparedStatement cnt = conn.get().prepareStatement(QueryFactory.COUNT)){
+        try (PreparedStatement cnt = conn.get().prepareStatement(QueryFactory.COUNT)) {
             try (ResultSet rs = cnt.executeQuery()) {
                 rs.next();
 
@@ -187,14 +189,14 @@ public abstract class AbstractUploadBenchmark extends AbstractJdbcBenchmark {
      * Facility method to perform updates on any connection.
      */
     private static int executeUpdateOn(Connection c, String updQry) throws SQLException {
-        try(PreparedStatement update = c.prepareStatement(updQry)){
+        try (PreparedStatement update = c.prepareStatement(updQry)) {
             return update.executeUpdate();
         }
     }
 
     /**
-     *  Creates new connection only for upload purpose.
-     *  This connection is special, since it may have additional jdbc url parameters.
+     * Creates new connection only for upload purpose. This connection is special, since it may have additional jdbc url
+     * parameters.
      */
     private Connection uploadConnection() throws SQLException {
         String urlParams = "";

http://git-wip-us.apache.org/repos/asf/ignite/blob/67d4941f/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/NativePutAllBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/NativePutAllBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/NativePutAllBenchmark.java
new file mode 100644
index 0000000..854df8c
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/NativePutAllBenchmark.java
@@ -0,0 +1,50 @@
+/*
+ * 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.upload;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.yardstick.upload.model.Values10;
+
+/**
+ * Benchmark that performs single upload of number of entries using {@link IgniteCache#putAll(Map)}.
+ */
+public class NativePutAllBenchmark extends AbstractNativeBenchmark {
+    /**
+     * Uploads randomly generated data using simple put.
+     *
+     * @param insertsCnt - how many entries should be uploaded.
+     */
+    @Override protected void upload(long insertsCnt) {
+        IgniteCache<Object, Object> c = ignite().cache(CACHE_NAME);
+
+        long batchSize = args.upload.jdbcBatchSize();
+        Map<Long, Values10> batch = new HashMap<>((int)batchSize);
+
+        for (long id = 1; id <= insertsCnt; id++) {
+            batch.put(id, new Values10());
+
+            if (id % batchSize == 0 || id == insertsCnt) {
+                c.putAll(batch);
+
+                batch.clear();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/67d4941f/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/NativePutBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/NativePutBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/NativePutBenchmark.java
index 60fbbda..4230c4f 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/NativePutBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/NativePutBenchmark.java
@@ -21,19 +21,18 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.yardstick.upload.model.Values10;
 
 /**
- * Benchmark that inserts single upload of number of entries using {@link IgniteCache#put(Object, Object)}.
+ * Benchmark performs single upload of number of entries using {@link IgniteCache#put(Object, Object)}.
  */
 public class NativePutBenchmark extends AbstractNativeBenchmark {
     /**
      * Uploads randomly generated data using simple put.
      *
-     * @param cacheName - name of the cache.
      * @param insertsCnt - how many entries should be uploaded.
      */
-    @Override protected void upload(String cacheName, long insertsCnt) {
-        IgniteCache<Object, Object> c = ignite().cache(cacheName);
+    @Override protected void upload(long insertsCnt) {
+        IgniteCache<Object, Object> c = ignite().cache(CACHE_NAME);
 
         for (long id = 1; id <= insertsCnt; id++)
-            c.put(id, new Values10() );
+            c.put(id, new Values10());
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/67d4941f/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/NativeSqlInsertBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/NativeSqlInsertBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/NativeSqlInsertBenchmark.java
new file mode 100644
index 0000000..03b1168
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/NativeSqlInsertBenchmark.java
@@ -0,0 +1,44 @@
+/*
+ * 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.upload;
+
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.yardstick.upload.model.Values10;
+
+/**
+ * Upload benchmark that uses native sql INSERT operation for data uploading.
+ */
+public class NativeSqlInsertBenchmark extends AbstractNativeBenchmark {
+    /**
+     * Performs data upload using native sql and SqlFieldsQuery.
+     *
+     * @param insertsCnt how many rows to upload.
+     */
+    @Override protected void upload(long insertsCnt) {
+        IgniteCache<Object, Object> c = ignite().cache(CACHE_NAME);
+
+        SqlFieldsQuery ins = new SqlFieldsQuery(queries.insert());
+
+        for (long i = 1; i <= insertsCnt; i++) {
+            Object[] args = new Values10().toArgs(i);
+
+            c.query(ins.setArgs(args)).getAll();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/67d4941f/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/NativeStreamerBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/NativeStreamerBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/NativeStreamerBenchmark.java
index e334e1e..ccfc545 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/NativeStreamerBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/NativeStreamerBenchmark.java
@@ -29,11 +29,11 @@ public class NativeStreamerBenchmark extends AbstractNativeBenchmark {
     /**
      * Uploads randomly generated entries to specified cache.
      *
-     * @param cacheName - name of the cache.
      * @param insertsCnt - how many entries should be uploaded.
      */
-    @Override protected void upload(String cacheName, long insertsCnt) {
-        try (IgniteDataStreamer<Long, Values10> streamer = ignite().dataStreamer(cacheName)) {
+    @SuppressWarnings("ConstantConditions")
+    @Override protected void upload(long insertsCnt) {
+        try (IgniteDataStreamer<Long, Values10> streamer = ignite().dataStreamer(CACHE_NAME)) {
             if (args.upload.streamerPerNodeBufferSize() != null)
                 streamer.perNodeBufferSize(args.upload.streamerPerNodeBufferSize());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/67d4941f/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/UploadBenchmarkArguments.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/UploadBenchmarkArguments.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/UploadBenchmarkArguments.java
index 7a1b116..e201e59 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/UploadBenchmarkArguments.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/UploadBenchmarkArguments.java
@@ -44,8 +44,8 @@ public class UploadBenchmarkArguments implements StreamerParams {
     /**
      * Parameters for JDBC connection, that only uploads data.
      *
-     * We can't just pass entire params string, due to yardstick, which relies on bash,
-     * has some troubles with escaping ampersand character.
+     * We can't just pass entire params string, due to yardstick, which relies on bash, has some troubles with escaping
+     * ampersand character.
      */
     @Parameter(names = {"--sql-jdbc-params"},
         variableArity = true,
@@ -76,7 +76,7 @@ public class UploadBenchmarkArguments implements StreamerParams {
 
     @Parameter(names = {"--streamer-ordered"}, arity = 1,
         description = "Streamer benchmarks only: set streamer ordered flag.")
-    private boolean streamerOrdered = true;
+    private boolean streamerOrdered = false;
 
     /** How many rows to upload during warmup. */
     @Parameter(names = {"--upload-warmup-rows"})
@@ -90,11 +90,15 @@ public class UploadBenchmarkArguments implements StreamerParams {
     @Parameter(names = {"--upload-jdbc-batch-size"})
     private long jdbcBatchSize = -1;
 
-    /** Turn on streaming during upload */
-    @Parameter(names={"--use-streaming"}, arity = 1,
+    /** Turn on streaming during upload. */
+    @Parameter(names = {"--use-streaming"}, arity = 1,
         description = "Upload data in insert benchmarks in streaming mode")
     private boolean useStreaming = false;
 
+    /** Number of secondary indexes to create before upload. Values can be from 0 up to 10. */
+    @Parameter(names = {"--idx-count"})
+    private int idxCnt = 0;
+
     /**
      * @return Switch wal.
      */
@@ -127,10 +131,9 @@ public class UploadBenchmarkArguments implements StreamerParams {
     }
 
     /**
-     * How many entries to collect before sending to java streamer api in either way:
-     * passing map to {@link IgniteDataStreamer#addData(Map)},
-     * or set STREAMING sql command parameter. <br/>
-     * If set to 1, {@link IgniteDataStreamer#addData(Object, Object)} method will be used.
+     * How many entries to collect before sending to java streamer api in either way: passing map to {@link
+     * IgniteDataStreamer#addData(Map)}, or set STREAMING sql command parameter. <br/> If set to 1, {@link
+     * IgniteDataStreamer#addData(Object, Object)} method will be used.
      */
     @Override @Nullable public Integer streamerLocalBatchSize() {
         return streamerLocBatchSize;
@@ -184,6 +187,13 @@ public class UploadBenchmarkArguments implements StreamerParams {
         return useStreaming;
     }
 
+    /**
+     * See {@link #idxCnt}.
+     */
+    public int indexesCount() {
+        return idxCnt;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return GridToStringBuilder.toString(UploadBenchmarkArguments.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/67d4941f/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/model/QueryFactory.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/model/QueryFactory.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/model/QueryFactory.java
index 6c98d1a..542ab10 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/model/QueryFactory.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/model/QueryFactory.java
@@ -25,11 +25,13 @@ import org.apache.ignite.yardstick.upload.StreamerParams;
 import org.jetbrains.annotations.Nullable;
 
 /**
- * Factory that hides all test data details:
- * what query to use to create table
- * or what random arguments to set in prepared statement.
+ * Factory that hides all test data details: what query to use to create table or what random arguments to set in
+ * prepared statement.
  */
 public class QueryFactory {
+    /** Name of the table upload data to. */
+    public static final String UPLOAD_TABLE_NAME = "TEST_UPLOAD";
+
     /** Query to drop table if it exists. */
     public static final String DROP_TABLE_IF_EXISTS = "DROP TABLE IF EXISTS test_upload;";
 
@@ -77,10 +79,12 @@ public class QueryFactory {
 
         create.append(')');
 
+        StringBuilder withClause = new StringBuilder("key_type=Long, value_type=Values10");
+
         if (tabAtomicMode != null)
-            create.append(" WITH \"ATOMICITY=").append(tabAtomicMode.name()).append('\"');
+            withClause.append(", ATOMICITY=").append(tabAtomicMode.name());
 
-        create.append(';');
+        create.append(" WITH \"").append(withClause).append("\";");
 
         return create.toString();
     }
@@ -121,8 +125,8 @@ public class QueryFactory {
     }
 
     /**
-     * Creates string - comma-separated attributes of test table, surrounded with braces
-     * Is used as a part of sql statement.
+     * Creates string - comma-separated attributes of test table, surrounded with braces Is used as a part of sql
+     * statement.
      *
      * @return attributes list of test table as part of sql statement.
      */
@@ -163,8 +167,7 @@ public class QueryFactory {
     }
 
     /**
-     * Generates CSV line containing specified id and random values.
-     * This line corresponds 1 row of the test table,
+     * Generates CSV line containing specified id and random values. This line corresponds 1 row of the test table,
      * which will be inserted in the end.
      *
      * @param id key in the test table.
@@ -212,8 +215,24 @@ public class QueryFactory {
         if (p.streamerPerNodeBufferSize() != null)
             cmd.append(" PER_NODE_BUFFER_SIZE ").append(p.streamerPerNodeBufferSize());
 
-        cmd.append(" ORDERED ").append(p.streamerOrdered() ? "ON" : "OFF");
+        if (p.streamerOrdered())
+            cmd.append(" ORDERED");
 
         return cmd.append(';').toString();
     }
+
+    /**
+     * @param valIdx index of value to create index on. Should in range [1..10].
+     * @return query that creates index on n-th value field.
+     */
+    public String createIndex(int valIdx) {
+        if (valIdx > 10 || valIdx < 1)
+            throw new IllegalArgumentException("Incorrect value index [" + valIdx + "]." +
+                " Value index should be in range [1..10].");
+
+        String colName = "val_" + valIdx;
+        String idxName = colName + "_idx";
+
+        return "CREATE INDEX " + idxName + " ON test_upload (" + colName + ");";
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/67d4941f/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/model/Values10.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/model/Values10.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/model/Values10.java
index f0bdee0..7767a81 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/model/Values10.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/model/Values10.java
@@ -18,10 +18,10 @@
 package org.apache.ignite.yardstick.upload.model;
 
 import java.util.concurrent.ThreadLocalRandom;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
 
 /**
- * Describes data model.
- * Matches data model, defined in {@link QueryFactory#createTable()}
+ * Describes value_type for data model, defined in {@link QueryFactory#createTable()}.
  */
 public class Values10 {
     /** */
@@ -55,7 +55,7 @@ public class Values10 {
     final long val10;
 
     /** Creates new object with randomly initialized fields */
-    public Values10(){
+    public Values10() {
         ThreadLocalRandom rnd = ThreadLocalRandom.current();
 
         val1 = String.valueOf(rnd.nextLong());
@@ -73,4 +73,20 @@ public class Values10 {
         val9 = String.valueOf(rnd.nextLong());
         val10 = rnd.nextLong();
     }
+
+    public Object[] toArgs(long id) {
+        return new Object[] {id, val1, val2, val3, val4, val5, val6, val7, val8, val9, val10};
+    }
+
+    /**
+     * @param valIdx index of field (value).
+     * @return name of the field.
+     */
+    public static String fieldName(int valIdx) {
+        if (valIdx > 10 || valIdx < 1)
+            throw new IllegalArgumentException("Incorrect value index [" + valIdx + "]." +
+                " Value index should be in range [1..10].");
+
+        return "val" + valIdx;
+    }
 }