You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by pt...@apache.org on 2017/01/25 11:39:23 UTC

[1/6] ignite git commit: IGNITE-4598: Hadoop: implemented raw comparator for BytesWritable key type. This closes #1457.

Repository: ignite
Updated Branches:
  refs/heads/ignite-2.0 e219bad22 -> 490e9a138


IGNITE-4598: Hadoop: implemented raw comparator for BytesWritable key type. This closes #1457.


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

Branch: refs/heads/ignite-2.0
Commit: d4d5976dd354e05f6ac5fa2e2faf1ac66f3b7dec
Parents: 28d66db
Author: devozerov <vo...@gridgain.com>
Authored: Tue Jan 24 16:45:59 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Tue Jan 24 16:45:59 2017 +0300

----------------------------------------------------------------------
 .../io/BytesWritablePartiallyRawComparator.java | 51 +++++++++++++++
 .../hadoop/io/TextPartiallyRawComparator.java   | 68 +-------------------
 .../processors/hadoop/impl/HadoopUtils.java     | 66 +++++++++++++++++++
 .../hadoop/impl/v2/HadoopV2TaskContext.java     | 13 +++-
 4 files changed, 129 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d4d5976d/modules/hadoop/src/main/java/org/apache/ignite/hadoop/io/BytesWritablePartiallyRawComparator.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/io/BytesWritablePartiallyRawComparator.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/io/BytesWritablePartiallyRawComparator.java
new file mode 100644
index 0000000..da9240b
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/io/BytesWritablePartiallyRawComparator.java
@@ -0,0 +1,51 @@
+/*
+ * 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.hadoop.io;
+
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils;
+import org.apache.ignite.internal.processors.hadoop.io.OffheapRawMemory;
+import org.apache.ignite.internal.processors.hadoop.io.PartiallyOffheapRawComparatorEx;
+
+/**
+ * Partial raw comparator for {@link BytesWritable} data type.
+ * <p>
+ * Implementation is borrowed from {@code org.apache.hadoop.io.FastByteComparisons} and adopted to Ignite
+ * infrastructure.
+ */
+public class BytesWritablePartiallyRawComparator implements PartiallyRawComparator<BytesWritable>,
+    PartiallyOffheapRawComparatorEx<BytesWritable> {
+    /** Length bytes. */
+    private static final int LEN_BYTES = 4;
+
+    /** {@inheritDoc} */
+    @Override public int compare(BytesWritable val1, RawMemory val2Buf) {
+        if (val2Buf instanceof OffheapRawMemory) {
+            OffheapRawMemory val2Buf0 = (OffheapRawMemory)val2Buf;
+
+            return compare(val1, val2Buf0.pointer(), val2Buf0.length());
+        }
+        else
+            throw new UnsupportedOperationException("Text can be compared only with offheap memory.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compare(BytesWritable val1, long val2Ptr, int val2Len) {
+        return HadoopUtils.compareBytes(val1.getBytes(), val1.getLength(), val2Ptr + LEN_BYTES, val2Len - LEN_BYTES);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/d4d5976d/modules/hadoop/src/main/java/org/apache/ignite/hadoop/io/TextPartiallyRawComparator.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/io/TextPartiallyRawComparator.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/io/TextPartiallyRawComparator.java
index a2bc3d4..e82f5e4 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/io/TextPartiallyRawComparator.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/io/TextPartiallyRawComparator.java
@@ -17,10 +17,9 @@
 
 package org.apache.ignite.hadoop.io;
 
-import com.google.common.primitives.Longs;
-import com.google.common.primitives.UnsignedBytes;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableUtils;
+import org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils;
 import org.apache.ignite.internal.processors.hadoop.io.OffheapRawMemory;
 import org.apache.ignite.internal.processors.hadoop.io.PartiallyOffheapRawComparatorEx;
 import org.apache.ignite.internal.util.GridUnsafe;
@@ -47,69 +46,6 @@ public class TextPartiallyRawComparator implements PartiallyRawComparator<Text>,
     @Override public int compare(Text val1, long val2Ptr, int val2Len) {
         int len2 = WritableUtils.decodeVIntSize(GridUnsafe.getByte(val2Ptr));
 
-        return compareBytes(val1.getBytes(), val1.getLength(), val2Ptr + len2, val2Len - len2);
-    }
-
-    /**
-     * Internal comparison routine.
-     *
-     * @param buf1 Bytes 1.
-     * @param len1 Length 1.
-     * @param ptr2 Pointer 2.
-     * @param len2 Length 2.
-     * @return Result.
-     */
-    @SuppressWarnings("SuspiciousNameCombination")
-    private static int compareBytes(byte[] buf1, int len1, long ptr2, int len2) {
-        int minLength = Math.min(len1, len2);
-
-        int minWords = minLength / Longs.BYTES;
-
-        for (int i = 0; i < minWords * Longs.BYTES; i += Longs.BYTES) {
-            long lw = GridUnsafe.getLong(buf1, GridUnsafe.BYTE_ARR_OFF + i);
-            long rw = GridUnsafe.getLong(ptr2 + i);
-
-            long diff = lw ^ rw;
-
-            if (diff != 0) {
-                if (GridUnsafe.BIG_ENDIAN)
-                    return (lw + Long.MIN_VALUE) < (rw + Long.MIN_VALUE) ? -1 : 1;
-
-                // Use binary search
-                int n = 0;
-                int y;
-                int x = (int) diff;
-
-                if (x == 0) {
-                    x = (int) (diff >>> 32);
-
-                    n = 32;
-                }
-
-                y = x << 16;
-
-                if (y == 0)
-                    n += 16;
-                else
-                    x = y;
-
-                y = x << 8;
-
-                if (y == 0)
-                    n += 8;
-
-                return (int) (((lw >>> n) & 0xFFL) - ((rw >>> n) & 0xFFL));
-            }
-        }
-
-        // The epilogue to cover the last (minLength % 8) elements.
-        for (int i = minWords * Longs.BYTES; i < minLength; i++) {
-            int res = UnsignedBytes.compare(buf1[i], GridUnsafe.getByte(ptr2 + i));
-
-            if (res != 0)
-                return res;
-        }
-
-        return len1 - len2;
+        return HadoopUtils.compareBytes(val1.getBytes(), val1.getLength(), val2Ptr + len2, val2Len - len2);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d4d5976d/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopUtils.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopUtils.java
index a34388d..767e10a 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopUtils.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopUtils.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal.processors.hadoop.impl;
 
+import com.google.common.primitives.Longs;
+import com.google.common.primitives.UnsignedBytes;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Writable;
@@ -32,6 +34,7 @@ import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobStatus;
 import org.apache.ignite.internal.processors.hadoop.HadoopSplitWrapper;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo;
+import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
 import java.io.ByteArrayInputStream;
@@ -328,4 +331,67 @@ public class HadoopUtils {
             HadoopCommonUtils.restoreContextClassLoader(oldLdr);
         }
     }
+
+    /**
+     * Internal comparison routine.
+     *
+     * @param buf1 Bytes 1.
+     * @param len1 Length 1.
+     * @param ptr2 Pointer 2.
+     * @param len2 Length 2.
+     * @return Result.
+     */
+    @SuppressWarnings("SuspiciousNameCombination")
+    public static int compareBytes(byte[] buf1, int len1, long ptr2, int len2) {
+        int minLength = Math.min(len1, len2);
+
+        int minWords = minLength / Longs.BYTES;
+
+        for (int i = 0; i < minWords * Longs.BYTES; i += Longs.BYTES) {
+            long lw = GridUnsafe.getLong(buf1, GridUnsafe.BYTE_ARR_OFF + i);
+            long rw = GridUnsafe.getLong(ptr2 + i);
+
+            long diff = lw ^ rw;
+
+            if (diff != 0) {
+                if (GridUnsafe.BIG_ENDIAN)
+                    return (lw + Long.MIN_VALUE) < (rw + Long.MIN_VALUE) ? -1 : 1;
+
+                // Use binary search
+                int n = 0;
+                int y;
+                int x = (int) diff;
+
+                if (x == 0) {
+                    x = (int) (diff >>> 32);
+
+                    n = 32;
+                }
+
+                y = x << 16;
+
+                if (y == 0)
+                    n += 16;
+                else
+                    x = y;
+
+                y = x << 8;
+
+                if (y == 0)
+                    n += 8;
+
+                return (int) (((lw >>> n) & 0xFFL) - ((rw >>> n) & 0xFFL));
+            }
+        }
+
+        // The epilogue to cover the last (minLength % 8) elements.
+        for (int i = minWords * Longs.BYTES; i < minLength; i++) {
+            int res = UnsignedBytes.compare(buf1[i], GridUnsafe.getByte(ptr2 + i));
+
+            if (res != 0)
+                return res;
+        }
+
+        return len1 - len2;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/d4d5976d/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
index 8acc7aa..b8d4cac 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.ByteWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.serializer.Deserializer;
@@ -40,6 +41,7 @@ import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.hadoop.io.BytesWritablePartiallyRawComparator;
 import org.apache.ignite.hadoop.io.PartiallyRawComparator;
 import org.apache.ignite.hadoop.io.TextPartiallyRawComparator;
 import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils;
@@ -48,7 +50,6 @@ import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit;
 import org.apache.ignite.internal.processors.hadoop.HadoopJob;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobProperty;
-import org.apache.ignite.internal.processors.hadoop.HadoopMapperAwareTaskOutput;
 import org.apache.ignite.internal.processors.hadoop.HadoopPartitioner;
 import org.apache.ignite.internal.processors.hadoop.HadoopSerialization;
 import org.apache.ignite.internal.processors.hadoop.HadoopSplitWrapper;
@@ -155,6 +156,7 @@ public class HadoopV2TaskContext extends HadoopTaskContext {
 
         COMBINE_KEY_GROUPING_SUPPORTED = ok;
 
+        PARTIAL_COMPARATORS.put(ByteWritable.class.getName(), BytesWritablePartiallyRawComparator.class.getName());
         PARTIAL_COMPARATORS.put(Text.class.getName(), TextPartiallyRawComparator.class.getName());
     }
 
@@ -595,11 +597,16 @@ public class HadoopV2TaskContext extends HadoopTaskContext {
         if (clsName == null) {
             Class keyCls = conf.getMapOutputKeyClass();
 
-            if (keyCls != null) {
+            while (keyCls != null) {
                 clsName = PARTIAL_COMPARATORS.get(keyCls.getName());
 
-                if (clsName != null)
+                if (clsName != null) {
                     conf.set(HadoopJobProperty.JOB_PARTIALLY_RAW_COMPARATOR.propertyName(), clsName);
+
+                    break;
+                }
+
+                keyCls = keyCls.getSuperclass();
             }
         }
     }


[2/6] ignite git commit: IGNITE-4520 Added credential request for authentication on proxy.

Posted by pt...@apache.org.
IGNITE-4520 Added credential request for authentication on proxy.


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

Branch: refs/heads/ignite-2.0
Commit: ef04f35fad5a8e74ee3e5f7fb6085a8e101d19ef
Parents: d4d5976
Author: Andrey Novikov <an...@gridgain.com>
Authored: Wed Jan 25 13:58:57 2017 +0700
Committer: Andrey Novikov <an...@gridgain.com>
Committed: Wed Jan 25 13:58:57 2017 +0700

----------------------------------------------------------------------
 .../web-agent/bin/ignite-web-agent.bat          |  4 +-
 .../web-agent/bin/ignite-web-agent.sh           |  2 +
 .../ignite/console/agent/AgentLauncher.java     | 90 ++++++++++++++++++--
 3 files changed, 88 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ef04f35f/modules/web-console/web-agent/bin/ignite-web-agent.bat
----------------------------------------------------------------------
diff --git a/modules/web-console/web-agent/bin/ignite-web-agent.bat b/modules/web-console/web-agent/bin/ignite-web-agent.bat
index 8291b55..1f1b52d 100644
--- a/modules/web-console/web-agent/bin/ignite-web-agent.bat
+++ b/modules/web-console/web-agent/bin/ignite-web-agent.bat
@@ -60,7 +60,9 @@ if %ERRORLEVEL% equ 0 (
     if "%JVM_OPTS%" == "" set JVM_OPTS=-Xms1g -Xmx1g -server -XX:+AggressiveOpts -XX:MaxMetaspaceSize=256m
 )
 
-"%JAVA_HOME%\bin\java.exe" %JVM_OPTS% -cp "*" org.apache.ignite.console.agent.AgentLauncher  %*
+set JVM_OPTS=%JVM_OPTS% -Djava.net.useSystemProxies=true
+
+"%JAVA_HOME%\bin\java.exe" %JVM_OPTS% -cp "*" org.apache.ignite.console.agent.AgentLauncher %*
 
 set JAVA_ERRORLEVEL=%ERRORLEVEL%
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ef04f35f/modules/web-console/web-agent/bin/ignite-web-agent.sh
----------------------------------------------------------------------
diff --git a/modules/web-console/web-agent/bin/ignite-web-agent.sh b/modules/web-console/web-agent/bin/ignite-web-agent.sh
index 2e9f041..c2958fc 100644
--- a/modules/web-console/web-agent/bin/ignite-web-agent.sh
+++ b/modules/web-console/web-agent/bin/ignite-web-agent.sh
@@ -88,4 +88,6 @@ if [ -z "$JVM_OPTS" ] ; then
     fi
 fi
 
+JVM_OPTS="${JVM_OPTS} -Djava.net.useSystemProxies=true"
+
 "$JAVA" ${JVM_OPTS} -cp "*" org.apache.ignite.console.agent.AgentLauncher "$@"

http://git-wip-us.apache.org/repos/asf/ignite/blob/ef04f35f/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java
----------------------------------------------------------------------
diff --git a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java
index 0c03d77..049791f 100644
--- a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java
+++ b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java
@@ -25,11 +25,15 @@ import io.socket.client.Socket;
 import io.socket.emitter.Emitter;
 import java.io.File;
 import java.io.IOException;
+import java.net.Authenticator;
 import java.net.ConnectException;
+import java.net.PasswordAuthentication;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
+import java.net.UnknownHostException;
 import java.util.Arrays;
+import java.util.Scanner;
 import java.util.concurrent.CountDownLatch;
 import java.util.jar.Attributes;
 import java.util.jar.Manifest;
@@ -76,9 +80,6 @@ public class AgentLauncher {
     /** */
     private static final String EVENT_AGENT_CLOSE = "agent:close";
 
-    /** */
-    private static final int RECONNECT_INTERVAL = 3000;
-
     /**
      * Create a trust manager that trusts all certificates It is not using a particular keyStore
      */
@@ -121,6 +122,15 @@ public class AgentLauncher {
                     System.exit(1);
                 }
 
+                ignore = X.cause(e, UnknownHostException.class);
+
+                if (ignore != null) {
+                    log.error("Failed to establish connection to server, due to errors with DNS or missing proxy settings.");
+                    log.error("Documentation for proxy configuration can be found here: http://apacheignite.readme.io/docs/web-agent#section-proxy-configuration");
+
+                    System.exit(1);
+                }
+
                 ignore = X.cause(e, IOException.class);
 
                 if (ignore != null && "404".equals(ignore.getMessage())) {
@@ -129,6 +139,29 @@ public class AgentLauncher {
                     return;
                 }
 
+                if (ignore != null && "407".equals(ignore.getMessage())) {
+                    log.error("Failed to establish connection to server, due to proxy requires authentication.");
+
+                    String userName = System.getProperty("https.proxyUsername", System.getProperty("http.proxyUsername"));
+
+                    if (userName == null || userName.trim().isEmpty())
+                        userName = readLine("Enter proxy user name: ");
+                    else
+                        System.out.println("Read username from system properties: " + userName);
+
+                    char[] pwd = readPassword("Enter proxy password: ");
+
+                    final PasswordAuthentication pwdAuth = new PasswordAuthentication(userName, pwd);
+
+                    Authenticator.setDefault(new Authenticator() {
+                        @Override protected PasswordAuthentication getPasswordAuthentication() {
+                            return pwdAuth;
+                        }
+                    });
+
+                    return;
+                }
+
                 log.error("Connection error.", e);
             }
         }
@@ -144,6 +177,32 @@ public class AgentLauncher {
     };
 
     /**
+     * @param fmt Format string.
+     * @param args Arguments.
+     */
+    private static String readLine(String fmt, Object ... args) {
+        if (System.console() != null)
+            return System.console().readLine(fmt, args);
+
+        System.out.print(String.format(fmt, args));
+
+        return new Scanner(System.in).nextLine();
+    }
+
+    /**
+     * @param fmt Format string.
+     * @param args Arguments.
+     */
+    private static char[] readPassword(String fmt, Object ... args) {
+        if (System.console() != null)
+            return System.console().readPassword(fmt, args);
+
+        System.out.print(String.format(fmt, args));
+
+        return new Scanner(System.in).nextLine().toCharArray();
+    }
+
+    /**
      * @param args Args.
      */
     @SuppressWarnings("BusyWait")
@@ -214,9 +273,9 @@ public class AgentLauncher {
             System.out.println("Security token is required to establish connection to the web console.");
             System.out.println(String.format("It is available on the Profile page: https://%s/profile", webHost));
 
-            System.out.print("Enter security tokens separated by comma: ");
+            String tokens = String.valueOf(readPassword("Enter security tokens separated by comma: "));
 
-            cfg.tokens(Arrays.asList(System.console().readLine().trim().split(",")));
+            cfg.tokens(Arrays.asList(tokens.trim().split(",")));
         }
 
         final RestHandler restHnd = new RestHandler(cfg);
@@ -226,12 +285,29 @@ public class AgentLauncher {
 
             URI uri = URI.create(cfg.serverUri());
 
+            // Create proxy authenticator using passed properties.
+            switch (uri.getScheme()) {
+                case "http":
+                case "https":
+                    final String username = System.getProperty(uri.getScheme() + ".proxyUsername");
+                    final char[] pwd = System.getProperty(uri.getScheme() +  ".proxyPassword", "").toCharArray();
+
+                    Authenticator.setDefault(new Authenticator() {
+                        @Override protected PasswordAuthentication getPasswordAuthentication() {
+                            return new PasswordAuthentication(username, pwd);
+                        }
+                    });
+
+                    break;
+
+                default:
+                    // No-op.
+            }
+
             IO.Options opts = new IO.Options();
 
             opts.path = "/agents";
 
-            opts.reconnectionDelay = RECONNECT_INTERVAL;
-
             // Workaround for use self-signed certificate
             if (Boolean.getBoolean("trust.all")) {
                 SSLContext ctx = SSLContext.getInstance("TLS");


[5/6] ignite git commit: IGNITE-4562 .NET: Add mapping for BinaryObjectException

Posted by pt...@apache.org.
IGNITE-4562 .NET: Add mapping for BinaryObjectException

This closes #1461


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

Branch: refs/heads/ignite-2.0
Commit: 885dc32ba21d08b0f4bd6b067ad80c738173a2c4
Parents: eed9d66
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Wed Jan 25 14:36:53 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Wed Jan 25 14:36:53 2017 +0300

----------------------------------------------------------------------
 .../ignite/platform/PlatformExceptionTask.java  | 78 ++++++++++++++++++++
 .../Apache.Ignite.Core.Tests/ExceptionsTest.cs  | 38 ++++++++++
 .../Apache.Ignite.Core/Impl/ExceptionUtils.cs   |  6 +-
 3 files changed, 120 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/885dc32b/modules/core/src/test/java/org/apache/ignite/platform/PlatformExceptionTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/platform/PlatformExceptionTask.java b/modules/core/src/test/java/org/apache/ignite/platform/PlatformExceptionTask.java
new file mode 100644
index 0000000..c1ab991
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/platform/PlatformExceptionTask.java
@@ -0,0 +1,78 @@
+/*
+ * 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.platform;
+
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.cache.CacheAtomicUpdateTimeoutException;
+import org.apache.ignite.cluster.ClusterGroupEmptyException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.cluster.ClusterTopologyException;
+import org.apache.ignite.compute.*;
+import org.apache.ignite.lang.IgniteFutureCancelledException;
+import org.apache.ignite.transactions.*;
+import org.jetbrains.annotations.Nullable;
+
+import javax.cache.CacheException;
+import javax.cache.integration.CacheLoaderException;
+import javax.cache.integration.CacheWriterException;
+import javax.cache.processor.EntryProcessorException;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Task to test exception mappings.
+ */
+public class PlatformExceptionTask extends ComputeTaskAdapter<String, String> {
+    /** {@inheritDoc} */
+    @Nullable @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid,
+        @Nullable String arg) {
+        switch (arg) {
+            case "IllegalArgumentException": throw new IllegalArgumentException(arg);
+            case "IllegalStateException": throw new IllegalStateException(arg);
+            case "UnsupportedOperationException": throw new UnsupportedOperationException(arg);
+            case "IgniteException": throw new IgniteException(arg);
+            case "BinaryObjectException": throw new BinaryObjectException(arg);
+            case "ClusterGroupEmptyException": throw new ClusterGroupEmptyException(arg);
+            case "ClusterTopologyException": throw new ClusterTopologyException(arg);
+            case "ComputeExecutionRejectedException": throw new ComputeExecutionRejectedException(arg);
+            case "ComputeJobFailoverException": throw new ComputeJobFailoverException(arg);
+            case "ComputeTaskCancelledException": throw new ComputeTaskCancelledException(arg);
+            case "ComputeTaskTimeoutException": throw new ComputeTaskTimeoutException(arg);
+            case "ComputeUserUndeclaredException": throw new ComputeUserUndeclaredException(arg);
+            case "CacheException": throw new CacheException(arg);
+            case "CacheLoaderException": throw new CacheLoaderException(arg);
+            case "CacheWriterException": throw new CacheWriterException(arg);
+            case "EntryProcessorException": throw new EntryProcessorException(arg);
+            case "CacheAtomicUpdateTimeoutException": throw new CacheAtomicUpdateTimeoutException(arg);
+            case "TransactionOptimisticException": throw new TransactionOptimisticException(arg);
+            case "TransactionTimeoutException": throw new TransactionTimeoutException(arg);
+            case "TransactionRollbackException": throw new TransactionRollbackException(arg);
+            case "TransactionHeuristicException": throw new TransactionHeuristicException(arg);
+            case "TransactionDeadlockException": throw new TransactionDeadlockException(arg);
+            case "IgniteFutureCancelledException": throw new IgniteFutureCancelledException(arg);
+        }
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public String reduce(List<ComputeJobResult> results) {
+        return results.get(0).getData();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/885dc32b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs
index 052ff6f..8c23ab7 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs
@@ -28,6 +28,8 @@ namespace Apache.Ignite.Core.Tests
     using Apache.Ignite.Core.Cache;
     using Apache.Ignite.Core.Cluster;
     using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Transactions;
     using NUnit.Framework;
 
     /// <summary>
@@ -35,6 +37,9 @@ namespace Apache.Ignite.Core.Tests
     /// </summary>
     public class ExceptionsTest
     {
+        /** */
+        private const string ExceptionTask = "org.apache.ignite.platform.PlatformExceptionTask";
+
         /// <summary>
         /// Before test.
         /// </summary>
@@ -70,12 +75,45 @@ namespace Apache.Ignite.Core.Tests
             Assert.IsTrue(e.InnerException.Message.StartsWith(
                 "class org.apache.ignite.cluster.ClusterGroupEmptyException: Cluster group is empty."));
 
+            // Check all exceptions mapping.
+            var comp = grid.GetCompute();
+
+            CheckException<BinaryObjectException>(comp, "BinaryObjectException");
+            CheckException<IgniteException>(comp, "IgniteException");
+            CheckException<BinaryObjectException>(comp, "BinaryObjectException");
+            CheckException<ClusterTopologyException>(comp, "ClusterTopologyException");
+            CheckException<ComputeExecutionRejectedException>(comp, "ComputeExecutionRejectedException");
+            CheckException<ComputeJobFailoverException>(comp, "ComputeJobFailoverException");
+            CheckException<ComputeTaskCancelledException>(comp, "ComputeTaskCancelledException");
+            CheckException<ComputeTaskTimeoutException>(comp, "ComputeTaskTimeoutException");
+            CheckException<ComputeUserUndeclaredException>(comp, "ComputeUserUndeclaredException");
+            CheckException<TransactionOptimisticException>(comp, "TransactionOptimisticException");
+            CheckException<TransactionTimeoutException>(comp, "TransactionTimeoutException");
+            CheckException<TransactionRollbackException>(comp, "TransactionRollbackException");
+            CheckException<TransactionHeuristicException>(comp, "TransactionHeuristicException");
+            CheckException<TransactionDeadlockException>(comp, "TransactionDeadlockException");
+            CheckException<IgniteFutureCancelledException>(comp, "IgniteFutureCancelledException");
+
+            // Check stopped grid.
             grid.Dispose();
 
             Assert.Throws<InvalidOperationException>(() => grid.GetCache<object, object>("cache1"));
         }
 
         /// <summary>
+        /// Checks the exception.
+        /// </summary>
+        private static void CheckException<T>(ICompute comp, string name) where T : Exception
+        {
+            var ex = Assert.Throws<T>(() => comp.ExecuteJavaTask<string>(ExceptionTask, name));
+
+            var javaEx = ex.InnerException as JavaException;
+
+            Assert.IsNotNull(javaEx);
+            Assert.IsTrue(javaEx.Message.Contains("at " + ExceptionTask));
+        }
+
+        /// <summary>
         /// Tests CachePartialUpdateException keys propagation.
         /// </summary>
         [Test]

http://git-wip-us.apache.org/repos/asf/ignite/blob/885dc32b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs
index ddbdd86..64f3ccc 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs
@@ -24,6 +24,7 @@ namespace Apache.Ignite.Core.Impl
     using System.Security;
     using System.Text.RegularExpressions;
     using System.Threading;
+    using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Cache;
     using Apache.Ignite.Core.Cache.Store;
     using Apache.Ignite.Core.Cluster;
@@ -62,10 +63,10 @@ namespace Apache.Ignite.Core.Impl
             Justification = "Readability")]
         static ExceptionUtils()
         {
-            // Common Java exceptions mapped to common .Net exceptions.
+            // Common Java exceptions mapped to common .NET exceptions.
             Exs["java.lang.IllegalArgumentException"] = (i, m, e) => new ArgumentException(m, e);
             Exs["java.lang.IllegalStateException"] = (i, m, e) => new InvalidOperationException(m, e);
-            Exs["java.lang.UnsupportedOperationException"] = (i, m, e) => new NotImplementedException(m, e);
+            Exs["java.lang.UnsupportedOperationException"] = (i, m, e) => new NotSupportedException(m, e);
             Exs["java.lang.InterruptedException"] = (i, m, e) => new ThreadInterruptedException(m, e);
 
             // Generic Ignite exceptions.
@@ -73,6 +74,7 @@ namespace Apache.Ignite.Core.Impl
             Exs["org.apache.ignite.IgniteCheckedException"] = (i, m, e) => new IgniteException(m, e);
             Exs["org.apache.ignite.IgniteClientDisconnectedException"] = (i, m, e) => new ClientDisconnectedException(m, e, i.GetCluster().ClientReconnectTask);
             Exs["org.apache.ignite.internal.IgniteClientDisconnectedCheckedException"] = (i, m, e) => new ClientDisconnectedException(m, e, i.GetCluster().ClientReconnectTask);
+            Exs["org.apache.ignite.binary.BinaryObjectException"] = (i, m, e) => new BinaryObjectException(m, e);
 
             // Cluster exceptions.
             Exs["org.apache.ignite.cluster.ClusterGroupEmptyException"] = (i, m, e) => new ClusterGroupEmptyException(m, e);


[3/6] ignite git commit: IGNITE-1596 Fixed version sort.

Posted by pt...@apache.org.
IGNITE-1596 Fixed version sort.


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

Branch: refs/heads/ignite-2.0
Commit: 128ba0733692178de05eae183b97355b81715a1b
Parents: ef04f35
Author: Andrey Novikov <an...@gridgain.com>
Authored: Wed Jan 25 16:48:05 2017 +0700
Committer: Andrey Novikov <an...@gridgain.com>
Committed: Wed Jan 25 16:48:05 2017 +0700

----------------------------------------------------------------------
 modules/web-console/backend/app/agent.js | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/128ba073/modules/web-console/backend/app/agent.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/app/agent.js b/modules/web-console/backend/app/agent.js
index 791ea50..961253f 100644
--- a/modules/web-console/backend/app/agent.js
+++ b/modules/web-console/backend/app/agent.js
@@ -650,14 +650,14 @@ module.exports.factory = function(_, fs, path, JSZip, socketio, settings, mongo)
                         const bParts = b.split('.');
 
                         for (let i = 0; i < aParts.length; ++i) {
-                            if (bParts.length === i)
-                                return 1;
+                            if (aParts[i] !== bParts[i])
+                                return aParts[i] < bParts[i] ? 1 : -1;
+                        }
 
-                            if (aParts[i] === aParts[i])
-                                continue;
+                        if (aParts.length === bParts.length)
+                            return 0;
 
-                            return aParts[i] > bParts[i] ? 1 : -1;
-                        }
+                        return aParts.length < bParts.length ? 1 : -1;
                     }));
 
                     // Latest version of agent distribution.


[6/6] ignite git commit: Merge branch 'master' into ignite-2.0

Posted by pt...@apache.org.
Merge branch 'master' into ignite-2.0


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

Branch: refs/heads/ignite-2.0
Commit: 490e9a138d5ed0da505f4de7f57bb0e14c540fea
Parents: e219bad 885dc32
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Wed Jan 25 14:38:57 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Wed Jan 25 14:38:57 2017 +0300

----------------------------------------------------------------------
 DEVNOTES.txt                                    |   3 +
 assembly/dependencies-fabric-lgpl.xml           |   1 +
 assembly/dependencies-fabric.xml                |   1 +
 .../ignite/platform/PlatformExceptionTask.java  |  78 +++++++
 .../io/BytesWritablePartiallyRawComparator.java |  51 +++++
 .../hadoop/io/TextPartiallyRawComparator.java   |  68 +-----
 .../processors/hadoop/impl/HadoopUtils.java     |  66 ++++++
 .../hadoop/impl/v2/HadoopV2TaskContext.java     |  13 +-
 .../Apache.Ignite.Core.Tests/ExceptionsTest.cs  |  38 ++++
 .../Apache.Ignite.Core/Impl/ExceptionUtils.cs   |   6 +-
 modules/web-console/backend/app/agent.js        |  12 +-
 .../web-agent/bin/ignite-web-agent.bat          |   4 +-
 .../web-agent/bin/ignite-web-agent.sh           |   2 +
 .../ignite/console/agent/AgentLauncher.java     |  90 +++++++-
 modules/yardstick/DEVNOTES.txt                  |  16 ++
 modules/yardstick/README.txt                    |  10 +-
 modules/yardstick/pom-standalone.xml            | 209 +++++++++++++++++++
 modules/yardstick/pom.xml                       |  10 +-
 pom.xml                                         |  95 ++++++++-
 19 files changed, 685 insertions(+), 88 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/490e9a13/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
----------------------------------------------------------------------
diff --cc modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
index a9c0bb7,b8d4cac..6127822
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
@@@ -44,11 -46,10 +46,10 @@@ import org.apache.ignite.hadoop.io.Part
  import org.apache.ignite.hadoop.io.TextPartiallyRawComparator;
  import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils;
  import org.apache.ignite.internal.processors.hadoop.HadoopExternalSplit;
 -import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit;
 -import org.apache.ignite.internal.processors.hadoop.HadoopJob;
 +import org.apache.ignite.hadoop.HadoopInputSplit;
 +import org.apache.ignite.internal.processors.hadoop.HadoopJobEx;
  import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
  import org.apache.ignite.internal.processors.hadoop.HadoopJobProperty;
- import org.apache.ignite.internal.processors.hadoop.HadoopMapperAwareTaskOutput;
  import org.apache.ignite.internal.processors.hadoop.HadoopPartitioner;
  import org.apache.ignite.internal.processors.hadoop.HadoopSerialization;
  import org.apache.ignite.internal.processors.hadoop.HadoopSplitWrapper;


[4/6] ignite git commit: IGNITE-4212 Ignite Benchmarking Simplification and Automation

Posted by pt...@apache.org.
IGNITE-4212 Ignite Benchmarking Simplification and Automation


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

Branch: refs/heads/ignite-2.0
Commit: eed9d669353d51774afdac6d99b01ebd569b8708
Parents: 128ba07
Author: oleg-ostanin <oo...@gridgain.com>
Authored: Wed Jan 25 14:08:00 2017 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Wed Jan 25 14:08:00 2017 +0300

----------------------------------------------------------------------
 DEVNOTES.txt                          |   3 +
 assembly/dependencies-fabric-lgpl.xml |   1 +
 assembly/dependencies-fabric.xml      |   1 +
 modules/yardstick/DEVNOTES.txt        |  16 +++
 modules/yardstick/README.txt          |  10 +-
 modules/yardstick/pom-standalone.xml  | 209 +++++++++++++++++++++++++++++
 modules/yardstick/pom.xml             |  10 +-
 pom.xml                               |  95 ++++++++++++-
 8 files changed, 342 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/eed9d669/DEVNOTES.txt
----------------------------------------------------------------------
diff --git a/DEVNOTES.txt b/DEVNOTES.txt
index e920b79..6a275fc 100644
--- a/DEVNOTES.txt
+++ b/DEVNOTES.txt
@@ -12,6 +12,9 @@ With LGPL dependencies:
 With LGPL dependencies and Scala 2.10:
   mvn clean package -DskipTests -Prelease,lgpl -Dignite.edition=fabric-lgpl -Dscala-2.10
 
+With LGPL dependencies and Scala 2.10 and Ignite-Yardstick:
+  mvn clean package -DskipTests -Prelease,lgpl,yardstick -Dignite.edition=fabric-lgpl -Dscala-2.10
+
 With Apache Ignite.NET:
   Build Apache Ignite.NET as described at modules/platforms/dotnet/DEVNOTES.txt.
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/eed9d669/assembly/dependencies-fabric-lgpl.xml
----------------------------------------------------------------------
diff --git a/assembly/dependencies-fabric-lgpl.xml b/assembly/dependencies-fabric-lgpl.xml
index 82f63d8..6d547c4 100644
--- a/assembly/dependencies-fabric-lgpl.xml
+++ b/assembly/dependencies-fabric-lgpl.xml
@@ -133,6 +133,7 @@
                 <exclude>org.apache.ignite:ignite-appserver-test</exclude>
                 <exclude>org.apache.ignite:ignite-websphere-test</exclude>
                 <exclude>org.apache.ignite:ignite-cassandra</exclude>
+                <exclude>org.apache.ignite:ignite-yardstick</exclude>
             </excludes>
             <sources>
                 <includeModuleDirectory>true</includeModuleDirectory>

http://git-wip-us.apache.org/repos/asf/ignite/blob/eed9d669/assembly/dependencies-fabric.xml
----------------------------------------------------------------------
diff --git a/assembly/dependencies-fabric.xml b/assembly/dependencies-fabric.xml
index 63e6ac8..d4000d6 100644
--- a/assembly/dependencies-fabric.xml
+++ b/assembly/dependencies-fabric.xml
@@ -136,6 +136,7 @@
                 <exclude>org.apache.ignite:ignite-appserver-test</exclude>
                 <exclude>org.apache.ignite:ignite-websphere-test</exclude>
                 <exclude>org.apache.ignite:ignite-cassandra</exclude>
+                <exclude>org.apache.ignite:ignite-yardstick</exclude>
             </excludes>
             <sources>
                 <includeModuleDirectory>true</includeModuleDirectory>

http://git-wip-us.apache.org/repos/asf/ignite/blob/eed9d669/modules/yardstick/DEVNOTES.txt
----------------------------------------------------------------------
diff --git a/modules/yardstick/DEVNOTES.txt b/modules/yardstick/DEVNOTES.txt
new file mode 100644
index 0000000..6921243
--- /dev/null
+++ b/modules/yardstick/DEVNOTES.txt
@@ -0,0 +1,16 @@
+Yardstick Ignite Maven Build Instructions
+=========================================
+
+Yardstick can be build from standalone sources using following maven command:
+
+mvn clean package
+
+Artifacts can be found in /target/assembly directory.
+
+To build yardstick from Apache Ignite sources use:
+
+mvn clean package -Pyardstick -pl modules/yardstick -am -DskipTests
+
+in Apache Ignite root directory
+
+Artifacts can be found in modules/yardstick/target/assembly directory.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/eed9d669/modules/yardstick/README.txt
----------------------------------------------------------------------
diff --git a/modules/yardstick/README.txt b/modules/yardstick/README.txt
index 0b8e678..353ddd8 100644
--- a/modules/yardstick/README.txt
+++ b/modules/yardstick/README.txt
@@ -8,7 +8,7 @@ Visit Yardstick Repository (https://github.com/gridgain/yardstick) for detailed
 
 The documentation below describes configuration parameters in addition to standard Yardstick parameters.
 
-Installation
+Building from Ignite sources
 ============
 1. Create a local clone of Ignite repository
 2. Run
@@ -17,6 +17,14 @@ mvn package
 
 command for Yardstick Ignite POM
 
+Building from standalone sources
+=====================
+Run
+
+mvn package
+
+command for Yardstick Ignite POM
+
 Provided Benchmarks
 ===================
 The following benchmarks are provided:

http://git-wip-us.apache.org/repos/asf/ignite/blob/eed9d669/modules/yardstick/pom-standalone.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/pom-standalone.xml b/modules/yardstick/pom-standalone.xml
new file mode 100644
index 0000000..0bd5a0f
--- /dev/null
+++ b/modules/yardstick/pom-standalone.xml
@@ -0,0 +1,209 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<!--
+    POM file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <groupId>org.apache.ignite</groupId>
+    <artifactId>ignite-yardstick</artifactId>
+    <version>to_be_replaced_by_ignite_version</version>
+    <url>http://ignite.apache.org</url>
+
+    <properties>
+        <yardstick.version>0.8.0</yardstick.version>
+        <spring.version>4.1.0.RELEASE</spring.version>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-core</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-spring</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-indexing</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-log4j</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.yardstickframework</groupId>
+            <artifactId>yardstick</artifactId>
+            <version>${yardstick.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>com.beust</groupId>
+            <artifactId>jcommander</artifactId>
+            <version>1.32</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-core</artifactId>
+            <version>${spring.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-beans</artifactId>
+            <version>${spring.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-context</artifactId>
+            <version>${spring.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-expression</artifactId>
+            <version>${spring.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-aop</artifactId>
+            <version>${spring.version}</version>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>com.mycila</groupId>
+                <artifactId>license-maven-plugin</artifactId>
+                <version>2.8</version>
+                <configuration>
+                    <skip>true</skip>
+                </configuration>
+            </plugin>
+
+            <plugin>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <version>3.1</version>
+                <configuration>
+                    <source>1.7</source>
+                    <target>1.7</target>
+                </configuration>
+            </plugin>
+
+            <plugin>
+                <artifactId>maven-dependency-plugin</artifactId>
+                <version>2.8</version>
+                <executions>
+                    <execution>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>copy-dependencies</goal>
+                        </goals>
+                        <configuration>
+                            <outputDirectory>${basedir}/target/assembly/libs</outputDirectory>
+                            <excludeTypes>pom</excludeTypes>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>unpack</id>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>unpack</goal>
+                        </goals>
+                        <configuration>
+                            <artifactItems>
+                                <artifactItem>
+                                    <groupId>org.yardstickframework</groupId>
+                                    <artifactId>yardstick</artifactId>
+                                    <version>${yardstick.version}</version>
+                                    <type>zip</type>
+                                    <classifier>resources</classifier>
+                                    <outputDirectory>${basedir}/target/assembly</outputDirectory>
+                                </artifactItem>
+                            </artifactItems>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <version>2.4</version>
+                <configuration>
+                    <outputDirectory>${basedir}/target/assembly/libs</outputDirectory>
+                </configuration>
+            </plugin>
+
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-javadoc-plugin</artifactId>
+                <version>2.9.1</version>
+                <configuration>
+                    <skip>true</skip>
+                </configuration>
+            </plugin>
+
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-antrun-plugin</artifactId>
+                <version>1.7</version>
+                <executions>
+                    <execution>
+                        <id>copy-yardstick-cfg-ignite</id>
+                        <goals>
+                            <goal>run</goal>
+                        </goals>
+                        <phase>validate</phase>
+                        <configuration>
+                            <target>
+                                <copy todir="${basedir}/target/assembly/config">
+                                    <fileset dir="${basedir}/config" />
+                                </copy>
+                            </target>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-deploy-plugin</artifactId>
+                <version>2.8.2</version>
+                <configuration>
+                    <skip>true</skip>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/eed9d669/modules/yardstick/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/pom.xml b/modules/yardstick/pom.xml
index 970348b..4c4d138 100644
--- a/modules/yardstick/pom.xml
+++ b/modules/yardstick/pom.xml
@@ -36,6 +36,7 @@
 
     <properties>
         <yardstick.version>0.8.0</yardstick.version>
+        <spring.version>4.1.0.RELEASE</spring.version>
     </properties>
 
     <dependencies>
@@ -108,7 +109,6 @@
 
     <build>
         <plugins>
-
             <plugin>
                 <groupId>com.mycila</groupId>
                 <artifactId>license-maven-plugin</artifactId>
@@ -202,6 +202,14 @@
                     </execution>
                 </executions>
             </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-deploy-plugin</artifactId>
+                <version>2.8.2</version>
+                <configuration>
+                    <skip>true</skip>
+                </configuration>
+            </plugin>
         </plugins>
     </build>
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/eed9d669/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 4be1bcb..790cd4b 100644
--- a/pom.xml
+++ b/pom.xml
@@ -458,10 +458,103 @@
         </profile>
 
         <profile>
+            <id>yardstick</id>
+            <modules>
+                <module>modules/yardstick</module>
+            </modules>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-antrun-plugin</artifactId>
+                        <version>1.7</version>
+                        <inherited>false</inherited>
+                        <dependencies>
+                            <dependency>
+                                <groupId>org.apache.ignite</groupId>
+                                <artifactId>ignite-tools</artifactId>
+                                <version>${project.version}</version>
+                            </dependency>
+                        </dependencies>
+                        <executions>
+                            <execution>
+                                <id>release-yardstick</id>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                                <phase>prepare-package</phase>
+                                <configuration>
+                                    <target>
+                                        <mkdir dir="${basedir}/target/release-package/benchmarks" />
+
+                                        <copy todir="${basedir}/target/release-package/benchmarks/">
+                                            <fileset dir="${basedir}/modules/yardstick/target/assembly/"/>
+                                        </copy>
+
+                                        <!--todo: only required jars should be exported to /benchmarks/libs during compilation-->
+                                        <delete>
+                                            <fileset dir="${basedir}/target/release-package/benchmarks/libs/">
+                                                <include name="junit-*.jar" />
+                                                <include name="ignite-apache-license-gen-*.jar" />
+                                                <include name="hamcrest-core-*.jar" />
+                                                <include name="tools-*.jar" />
+                                            </fileset>
+                                        </delete>
+
+                                        <!--todo: config structure should be refactored to be the same at "sources" and "released sources"-->
+                                        <delete>
+                                            <fileset dir="${basedir}/target/release-package/benchmarks/config/">
+                                                <include name="*.*" />
+                                                <exclude name="benchmark.properties"/>
+                                                <exclude name="benchmark-multicast.properties"/>
+                                                <exclude name="ignite-base-config.xml"/>
+                                                <exclude name="ignite-localhost-config.xml"/>
+                                                <exclude name="ignite-multicast-config.xml"/>
+                                            </fileset>
+                                        </delete>
+
+                                        <mkdir dir="${basedir}/target/release-package/benchmarks/sources/src" />
+
+                                        <copy todir="${basedir}/target/release-package/benchmarks/sources/src/">
+                                            <fileset dir="${basedir}/modules/yardstick/src"/>
+                                        </copy>
+
+                                        <mkdir dir="${basedir}/target/release-package/benchmarks/sources/config" />
+
+                                        <copy todir="${basedir}/target/release-package/benchmarks/sources/config/">
+                                            <fileset dir="${basedir}/target/release-package/benchmarks/config"/>
+                                        </copy>
+
+                                        <copy file="${basedir}/modules/yardstick/pom-standalone.xml"
+                                              tofile="${basedir}/target/release-package/benchmarks/sources/pom.xml"/>
+
+                                        <replaceregexp byline="true">
+                                            <regexp pattern="to_be_replaced_by_ignite_version"/>
+                                            <substitution expression="${project.version}"/>
+                                            <fileset dir="${basedir}/target/release-package/benchmarks/sources/" >
+                                                <include name="pom.xml"/>
+                                            </fileset>
+                                        </replaceregexp>
+
+                                        <copy file="${basedir}/modules/yardstick/README.txt"
+                                              tofile="${basedir}/target/release-package/benchmarks/README.txt" overwrite="true">
+                                        </copy>
+
+                                        <copy file="${basedir}/modules/yardstick/DEVNOTES.txt"
+                                              tofile="${basedir}/target/release-package/benchmarks/sources/DEVNOTES.txt"/>
+                                    </target>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+
+        <profile>
             <id>benchmarks</id>
             <modules>
                 <module>modules/benchmarks</module>
-                <module>modules/yardstick</module>
             </modules>
         </profile>